You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2022/04/06 02:44:59 UTC

[incubator-doris] 13/19: [feature](function) Support geolocation functions on vectorized engine (#8790)

This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit 4ba208f3e3b4abec54e0e21a73e2884d3c4a33a8
Author: Gabriel <ga...@gmail.com>
AuthorDate: Sun Apr 3 10:50:54 2022 +0800

    [feature](function) Support geolocation functions on vectorized engine (#8790)
---
 be/src/geo/geo_types.cpp                       |  49 ++++--
 be/src/geo/geo_types.h                         |  37 +++--
 be/src/vec/CMakeLists.txt                      |   1 +
 be/src/vec/functions/functions_geo.cpp         | 211 +++++++++++++++++++++++++
 be/src/vec/functions/functions_geo.h           |  49 ++++++
 be/src/vec/functions/simple_function_factory.h |   2 +
 be/test/vec/function/CMakeLists.txt            |   1 +
 be/test/vec/function/function_geo_test.cpp     | 151 ++++++++++++++++++
 gensrc/script/doris_builtins_functions.py      |  16 +-
 9 files changed, 484 insertions(+), 33 deletions(-)

diff --git a/be/src/geo/geo_types.cpp b/be/src/geo/geo_types.cpp
index 6c61f963e2..f44ac66559 100644
--- a/be/src/geo/geo_types.cpp
+++ b/be/src/geo/geo_types.cpp
@@ -17,9 +17,12 @@
 
 #include "geo/geo_types.h"
 
+#include <s2/s2cap.h>
 #include <s2/s2cell.h>
 #include <s2/s2earth.h>
 #include <s2/s2latlng.h>
+#include <s2/s2polygon.h>
+#include <s2/s2polyline.h>
 #include <s2/util/coding/coder.h>
 #include <s2/util/units/length-units.h>
 #include <stdio.h>
@@ -31,6 +34,18 @@
 
 namespace doris {
 
+GeoPoint::GeoPoint() : _point(new S2Point()) {}
+GeoPoint::~GeoPoint() = default;
+
+GeoLine::GeoLine() = default;
+GeoLine::~GeoLine() = default;
+
+GeoPolygon::GeoPolygon() = default;
+GeoPolygon::~GeoPolygon() = default;
+
+GeoCircle::GeoCircle() = default;
+GeoCircle::~GeoCircle() = default;
+
 void print_s2point(std::ostream& os, const S2Point& point) {
     S2LatLng coord(point);
     os << std::setprecision(12) << coord.lng().degrees() << " " << coord.lat().degrees();
@@ -206,11 +221,11 @@ GeoShape* GeoShape::from_encoded(const void* ptr, size_t size) {
 }
 
 GeoParseStatus GeoPoint::from_coord(double x, double y) {
-    return to_s2point(x, y, &_point);
+    return to_s2point(x, y, _point.get());
 }
 
 GeoParseStatus GeoPoint::from_coord(const GeoCoordinate& coord) {
-    return to_s2point(coord, &_point);
+    return to_s2point(coord, _point.get());
 }
 
 std::string GeoPoint::to_string() const {
@@ -218,33 +233,47 @@ std::string GeoPoint::to_string() const {
 }
 
 void GeoPoint::encode(std::string* buf) {
-    buf->append((const char*)&_point, sizeof(_point));
+    buf->append((const char*)_point.get(), sizeof(*_point));
 }
 
 bool GeoPoint::decode(const void* data, size_t size) {
-    if (size < sizeof(_point)) {
+    if (size < sizeof(*_point)) {
         return false;
     }
-    memcpy(&_point, data, size);
+    memcpy(_point.get(), data, size);
     return true;
 }
 
 double GeoPoint::x() const {
-    return S2LatLng(_point).lng().degrees();
+    return S2LatLng(*_point).lng().degrees();
 }
 
 double GeoPoint::y() const {
-    return S2LatLng(_point).lat().degrees();
+    return S2LatLng(*_point).lat().degrees();
 }
 
 std::string GeoPoint::as_wkt() const {
     std::stringstream ss;
     ss << "POINT (";
-    print_s2point(ss, _point);
+    print_s2point(ss, *_point);
     ss << ")";
     return ss.str();
 }
 
+bool GeoPoint::ComputeDistance(double x_lng, double x_lat, double y_lng, double y_lat,
+                               double* distance) {
+    S2LatLng x = S2LatLng::FromDegrees(x_lat, x_lng);
+    if (!x.is_valid()) {
+        return false;
+    }
+    S2LatLng y = S2LatLng::FromDegrees(y_lat, y_lng);
+    if (!y.is_valid()) {
+        return false;
+    }
+    *distance = S2Earth::ToMeters(x.GetDistance(y));
+    return true;
+}
+
 GeoParseStatus GeoLine::from_coords(const GeoCoordinateList& list) {
     return to_s2polyline(list, &_polyline);
 }
@@ -318,7 +347,7 @@ bool GeoPolygon::contains(const GeoShape* rhs) const {
     switch (rhs->type()) {
     case GEO_SHAPE_POINT: {
         const GeoPoint* point = (const GeoPoint*)rhs;
-        return _polygon->Contains(point->point());
+        return _polygon->Contains(*point->point());
     }
     case GEO_SHAPE_LINE_STRING: {
         const GeoLine* line = (const GeoLine*)rhs;
@@ -351,7 +380,7 @@ bool GeoCircle::contains(const GeoShape* rhs) const {
     switch (rhs->type()) {
     case GEO_SHAPE_POINT: {
         const GeoPoint* point = (const GeoPoint*)rhs;
-        return _cap->Contains(point->point());
+        return _cap->Contains(*point->point());
     }
     default:
         return false;
diff --git a/be/src/geo/geo_types.h b/be/src/geo/geo_types.h
index 0a717cf403..a9a2a81f5c 100644
--- a/be/src/geo/geo_types.h
+++ b/be/src/geo/geo_types.h
@@ -17,11 +17,6 @@
 
 #pragma once
 
-#include <s2/s2cap.h>
-#include <s2/s2point.h>
-#include <s2/s2polygon.h>
-#include <s2/s2polyline.h>
-
 #include <memory>
 #include <string>
 #include <vector>
@@ -29,6 +24,15 @@
 #include "geo/geo_common.h"
 #include "geo/wkt_parse_type.h"
 
+class S2Polyline;
+class S2Polygon;
+class S2Cap;
+
+template <typename T>
+class Vector3;
+typedef Vector3<double> Vector3_d;
+using S2Point = Vector3_d;
+
 namespace doris {
 
 class GeoShape {
@@ -59,15 +63,18 @@ protected:
 
 class GeoPoint : public GeoShape {
 public:
-    GeoPoint() {}
-    ~GeoPoint() override {}
+    GeoPoint();
+    ~GeoPoint() override;
 
     GeoParseStatus from_coord(double x, double y);
     GeoParseStatus from_coord(const GeoCoordinate& point);
 
     GeoShapeType type() const override { return GEO_SHAPE_POINT; }
 
-    const S2Point& point() const { return _point; }
+    const S2Point* point() const { return _point.get(); }
+
+    static bool ComputeDistance(double x_lng, double x_lat, double y_lng, double y_lat,
+                                double* distance);
 
     std::string to_string() const override;
     std::string as_wkt() const override;
@@ -80,13 +87,13 @@ protected:
     bool decode(const void* data, size_t size) override;
 
 private:
-    S2Point _point;
+    std::unique_ptr<S2Point> _point;
 };
 
 class GeoLine : public GeoShape {
 public:
-    GeoLine() {}
-    ~GeoLine() override {}
+    GeoLine();
+    ~GeoLine() override;
 
     GeoParseStatus from_coords(const GeoCoordinateList& list);
 
@@ -105,8 +112,8 @@ private:
 
 class GeoPolygon : public GeoShape {
 public:
-    GeoPolygon() {}
-    ~GeoPolygon() override {}
+    GeoPolygon();
+    ~GeoPolygon() override;
 
     GeoParseStatus from_coords(const GeoCoordinateListList& list);
 
@@ -126,8 +133,8 @@ private:
 
 class GeoCircle : public GeoShape {
 public:
-    GeoCircle() {}
-    ~GeoCircle() {}
+    GeoCircle();
+    ~GeoCircle() override;
 
     GeoParseStatus init(double lng, double lat, double radius);
 
diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt
index 3b26024388..ab319c5d71 100644
--- a/be/src/vec/CMakeLists.txt
+++ b/be/src/vec/CMakeLists.txt
@@ -131,6 +131,7 @@ set(VEC_FILES
   functions/comparison_equal_for_null.cpp
   functions/function_json.cpp
   functions/function_datetime_floor_ceil.cpp
+  functions/functions_geo.cpp
   functions/hll_cardinality.cpp
   functions/hll_empty.cpp
   functions/hll_hash.cpp
diff --git a/be/src/vec/functions/functions_geo.cpp b/be/src/vec/functions/functions_geo.cpp
new file mode 100644
index 0000000000..3b8adfc878
--- /dev/null
+++ b/be/src/vec/functions/functions_geo.cpp
@@ -0,0 +1,211 @@
+// 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.
+
+#include "vec/functions/functions_geo.h"
+
+#include "geo/geo_types.h"
+#include "gutil/strings/substitute.h"
+#include "vec/functions/simple_function_factory.h"
+
+namespace doris::vectorized {
+
+struct StPoint {
+    static constexpr auto NAME = "st_point";
+    static const size_t NUM_ARGS = 2;
+    static Status execute(Block& block, const ColumnNumbers& arguments, size_t result) {
+        DCHECK_EQ(arguments.size(), 2);
+        auto return_type = block.get_data_type(result);
+        auto column_x = block.get_by_position(arguments[0]).column->convert_to_full_column_if_const();
+        auto column_y = block.get_by_position(arguments[1]).column->convert_to_full_column_if_const();
+
+        const auto size = column_x->size();
+
+        MutableColumnPtr res = nullptr;
+        res = ColumnNullable::create(return_type->create_column(), ColumnUInt8::create());
+
+        GeoPoint point;
+        std::string buf;
+        for (int row = 0; row < size; ++row) {
+            auto cur_res = point.from_coord(column_x->operator[](row).get<Float64>(),
+                                            column_y->operator[](row).get<Float64>());
+            if (cur_res != GEO_PARSE_OK) {
+                res->insert_data(nullptr, 0);
+                continue;
+            }
+
+            buf.clear();
+            point.encode_to(&buf);
+            res->insert_data(buf.data(), buf.size());
+        }
+
+        block.replace_by_position(result, std::move(res));
+        return Status::OK();
+    }
+};
+
+struct StAsTextName {
+    static constexpr auto NAME = "st_astext";
+};
+struct StAsWktName {
+    static constexpr auto NAME = "st_aswkt";
+};
+
+template<typename FunctionName>
+struct StAsText {
+    static constexpr auto NAME = FunctionName::NAME;
+    static const size_t NUM_ARGS = 1;
+    static Status execute(Block& block, const ColumnNumbers& arguments,size_t result) {
+        DCHECK_EQ(arguments.size(), 1);
+        auto return_type = block.get_data_type(result);
+        auto input = block.get_by_position(arguments[0]).column;
+
+        auto size = input->size();
+        auto col = input->convert_to_full_column_if_const();
+
+        MutableColumnPtr res = nullptr;
+        auto null_type = std::reinterpret_pointer_cast<const DataTypeNullable>(return_type);
+        res = ColumnNullable::create(return_type->create_column(), ColumnUInt8::create());
+
+        std::unique_ptr<GeoShape> shape;
+        for (int row = 0; row < size; ++row) {
+            auto shape_value = col->get_data_at(row);
+            shape.reset(GeoShape::from_encoded(shape_value.data, shape_value.size));
+
+            if (shape == nullptr) {
+                res->insert_data(nullptr, 0);
+                continue;
+            }
+            auto wkt = shape->as_wkt();
+            res->insert_data(wkt.data(), wkt.size());
+        }
+        block.replace_by_position(result, std::move(res));
+
+        return Status::OK();
+    }
+};
+
+struct StX {
+    static constexpr auto NAME = "st_x";
+    static const size_t NUM_ARGS = 1;
+    static Status execute(Block& block, const ColumnNumbers& arguments,size_t result) {
+        DCHECK_EQ(arguments.size(), 1);
+        auto return_type = block.get_data_type(result);
+        auto input = block.get_by_position(arguments[0]).column;
+
+        auto size = input->size();
+        auto col = input->convert_to_full_column_if_const();
+
+        MutableColumnPtr res = nullptr;
+        auto null_type = std::reinterpret_pointer_cast<const DataTypeNullable>(return_type);
+        res = ColumnNullable::create(return_type->create_column(), ColumnUInt8::create());
+
+        GeoPoint point;
+        for (int row = 0; row < size; ++row) {
+            auto point_value = col->get_data_at(row);
+            auto pt = point.decode_from(point_value.data, point_value.size);
+
+            if (!pt) {
+                res->insert_data(nullptr, 0);
+                continue;
+            }
+            auto x_value = point.x();
+            res->insert_data(const_cast<const char*>((char*)(&x_value)), 0);
+        }
+        block.replace_by_position(result, std::move(res));
+
+        return Status::OK();
+    }
+};
+
+struct StY {
+    static constexpr auto NAME = "st_y";
+    static const size_t NUM_ARGS = 1;
+    static Status execute(Block& block, const ColumnNumbers& arguments,size_t result) {
+        DCHECK_EQ(arguments.size(), 1);
+        auto return_type = block.get_data_type(result);
+        auto input = block.get_by_position(arguments[0]).column;
+
+        auto size = input->size();
+        auto col = input->convert_to_full_column_if_const();
+
+        MutableColumnPtr res = nullptr;
+        auto null_type = std::reinterpret_pointer_cast<const DataTypeNullable>(return_type);
+        res = ColumnNullable::create(return_type->create_column(), ColumnUInt8::create());
+
+        GeoPoint point;
+        for (int row = 0; row < size; ++row) {
+            auto point_value = col->get_data_at(row);
+            auto pt = point.decode_from(point_value.data, point_value.size);
+
+            if (!pt) {
+                res->insert_data(nullptr, 0);
+                continue;
+            }
+            auto y_value = point.y();
+            res->insert_data(const_cast<const char*>((char*)(&y_value)), 0);
+        }
+        block.replace_by_position(result, std::move(res));
+
+        return Status::OK();
+    }
+};
+
+struct StDistanceSphere {
+    static constexpr auto NAME = "st_distance_sphere";
+    static const size_t NUM_ARGS = 4;
+    static Status execute(Block& block, const ColumnNumbers& arguments, size_t result) {
+        DCHECK_EQ(arguments.size(), 4);
+        auto return_type = block.get_data_type(result);
+        auto x_lng = block.get_by_position(arguments[0]).column->convert_to_full_column_if_const();
+        auto x_lat = block.get_by_position(arguments[1]).column->convert_to_full_column_if_const();
+        auto y_lng = block.get_by_position(arguments[2]).column->convert_to_full_column_if_const();
+        auto y_lat = block.get_by_position(arguments[3]).column->convert_to_full_column_if_const();
+
+        const auto size = x_lng->size();
+
+        MutableColumnPtr res = nullptr;
+        auto null_type = std::reinterpret_pointer_cast<const DataTypeNullable>(return_type);
+        res = ColumnNullable::create(return_type->create_column(), ColumnUInt8::create());
+
+        for (int row = 0; row < size; ++row) {
+            double distance;
+            if (!GeoPoint::ComputeDistance(x_lng->operator[](row).get<Float64>(),
+                                           x_lat->operator[](row).get<Float64>(),
+                                           y_lng->operator[](row).get<Float64>(),
+                                           y_lat->operator[](row).get<Float64>(),
+                                           &distance)) {
+                res->insert_data(nullptr, 0);
+                continue;
+            }
+            res->insert_data(const_cast<const char*>((char*) &distance), 0);
+        }
+
+        block.replace_by_position(result, std::move(res));
+        return Status::OK();
+    }
+};
+
+void register_geo_functions(SimpleFunctionFactory& factory) {
+    factory.register_function<GeoFunction<StPoint>>();
+    factory.register_function<GeoFunction<StAsText<StAsWktName>>>();
+    factory.register_function<GeoFunction<StAsText<StAsTextName>>>();
+    factory.register_function<GeoFunction<StX, DataTypeFloat64>>();
+    factory.register_function<GeoFunction<StY, DataTypeFloat64>>();
+    factory.register_function<GeoFunction<StDistanceSphere, DataTypeFloat64>>();
+}
+
+} // namespace doris::vectorized
diff --git a/be/src/vec/functions/functions_geo.h b/be/src/vec/functions/functions_geo.h
new file mode 100644
index 0000000000..91d0f9c013
--- /dev/null
+++ b/be/src/vec/functions/functions_geo.h
@@ -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.
+
+#pragma once
+
+#include "udf/udf.h"
+#include "vec/core/column_numbers.h"
+#include "vec/data_types/data_type_number.h"
+#include "vec/data_types/data_type_string.h"
+#include "vec/functions/function.h"
+
+namespace doris::vectorized {
+
+template<typename Impl, typename ReturnType = DataTypeString>
+class GeoFunction : public IFunction {
+public:
+    static constexpr auto name = Impl::NAME;
+    static FunctionPtr create() { return std::make_shared<GeoFunction<Impl, ReturnType>>(); }
+    String get_name() const override { return name; }
+    size_t get_number_of_arguments() const override { return Impl::NUM_ARGS; }
+    bool is_variadic() const override { return false; }
+
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const override {
+        return make_nullable(std::make_shared<ReturnType>());
+    }
+    bool use_default_implementation_for_nulls() const override { return true; }
+    bool use_default_implementation_for_constants() const override { return true; }
+
+    Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+                        size_t result, size_t input_rows_count) {
+        return Impl::execute(block, arguments, result);
+    }
+};
+
+} // namespace doris::vectorized
diff --git a/be/src/vec/functions/simple_function_factory.h b/be/src/vec/functions/simple_function_factory.h
index 9402bcf72d..efb07e215d 100644
--- a/be/src/vec/functions/simple_function_factory.h
+++ b/be/src/vec/functions/simple_function_factory.h
@@ -73,6 +73,7 @@ void register_function_datetime_floor_ceil(SimpleFunctionFactory& factory);
 void register_function_convert_tz(SimpleFunctionFactory& factory);
 void register_function_least_greast(SimpleFunctionFactory& factory);
 void register_function_fake(SimpleFunctionFactory& factory);
+void register_geo_functions(SimpleFunctionFactory& factory);
 
 void register_function_encryption(SimpleFunctionFactory& factory);
 void register_function_regexp_extract(SimpleFunctionFactory& factory);
@@ -200,6 +201,7 @@ public:
             register_function_encryption(instance);
             register_function_regexp_extract(instance);
             register_function_hex_variadic(instance);
+            register_geo_functions(instance);
         });
         return instance;
     }
diff --git a/be/test/vec/function/CMakeLists.txt b/be/test/vec/function/CMakeLists.txt
index c4101f551d..4b7ccfbb6c 100644
--- a/be/test/vec/function/CMakeLists.txt
+++ b/be/test/vec/function/CMakeLists.txt
@@ -29,3 +29,4 @@ ADD_BE_TEST(function_nullif_test)
 ADD_BE_TEST(function_like_test)
 ADD_BE_TEST(function_arithmetic_test)
 ADD_BE_TEST(function_json_test)
+ADD_BE_TEST(function_geo_test)
diff --git a/be/test/vec/function/function_geo_test.cpp b/be/test/vec/function/function_geo_test.cpp
new file mode 100644
index 0000000000..15925f0b96
--- /dev/null
+++ b/be/test/vec/function/function_geo_test.cpp
@@ -0,0 +1,151 @@
+// 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.
+
+#include <gtest/gtest.h>
+#include <time.h>
+
+#include <string>
+
+#include "function_test_util.h"
+#include "geo/geo_types.h"
+#include "vec/core/field.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type_nullable.h"
+#include "vec/data_types/data_type_string.h"
+
+namespace doris::vectorized {
+using namespace ut_type;
+
+TEST(function_geo_test, function_geo_st_point_test) {
+    std::string func_name = "st_point";
+    {
+        InputTypeSet input_types = {TypeIndex::Float64, TypeIndex::Float64};
+
+        GeoPoint point;
+        auto cur_res = point.from_coord(24.7, 56.7);
+        ASSERT_TRUE(cur_res == GEO_PARSE_OK);
+        std::string buf;
+        point.encode_to(&buf);
+
+        DataSet data_set = {
+                {{(double) 24.7, (double) 56.7}, buf},
+                {{Null(), (double) 5}, Null()},
+                {{(double) 5, Null()}, Null()}};
+
+        check_function<DataTypeString, true>(func_name, input_types, data_set);
+    }
+}
+
+TEST(function_geo_test, function_geo_st_as_text) {
+    std::string func_name = "st_astext";
+    {
+        InputTypeSet input_types = {TypeIndex::String};
+
+        GeoPoint point;
+        auto cur_res = point.from_coord(24.7, 56.7);
+        ASSERT_TRUE(cur_res == GEO_PARSE_OK);
+        std::string buf;
+        point.encode_to(&buf);
+
+        DataSet data_set = {
+                {{buf}, std::string("POINT (24.7 56.7)")},
+                {{Null()}, Null()}};
+
+        check_function<DataTypeString, true>(func_name, input_types, data_set);
+    }
+}
+
+TEST(function_geo_test, function_geo_st_as_wkt) {
+    std::string func_name = "st_aswkt";
+    {
+        InputTypeSet input_types = {TypeIndex::String};
+
+        GeoPoint point;
+        auto cur_res = point.from_coord(24.7, 56.7);
+        ASSERT_TRUE(cur_res == GEO_PARSE_OK);
+        std::string buf;
+        point.encode_to(&buf);
+
+        DataSet data_set = {
+                {{buf}, std::string("POINT (24.7 56.7)")},
+                {{Null()}, Null()}};
+
+        check_function<DataTypeString, true>(func_name, input_types, data_set);
+    }
+}
+
+TEST(function_geo_test, function_geo_st_x) {
+    std::string func_name = "st_x";
+    {
+        InputTypeSet input_types = {TypeIndex::String};
+
+        GeoPoint point;
+        auto cur_res = point.from_coord(24.7, 56.7);
+        ASSERT_TRUE(cur_res == GEO_PARSE_OK);
+        std::string buf;
+        point.encode_to(&buf);
+
+        DataSet data_set = {
+                {{buf}, (double) 24.7},
+                {{Null()}, Null()}};
+
+        check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
+    }
+}
+
+TEST(function_geo_test, function_geo_st_y) {
+    std::string func_name = "st_y";
+    {
+        InputTypeSet input_types = {TypeIndex::String};
+
+        GeoPoint point;
+        auto cur_res = point.from_coord(24.7, 56.7);
+        ASSERT_TRUE(cur_res == GEO_PARSE_OK);
+        std::string buf;
+        point.encode_to(&buf);
+
+        DataSet data_set = {
+                {{buf}, (double) 56.7},
+                {{Null()}, Null()}};
+
+        check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
+    }
+}
+
+TEST(function_geo_test, function_geo_st_distance_sphere) {
+    std::string func_name = "st_distance_sphere";
+    {
+        InputTypeSet input_types = {TypeIndex::Float64, TypeIndex::Float64, TypeIndex::Float64, TypeIndex::Float64};
+
+        DataSet data_set = {
+                {{(double) 116.35620117, (double) 39.939093, (double) 116.4274406433, (double) 39.9020987219}, (double) 7336.9135549995917},
+                {{(double) 116.35620117, (double) 39.939093, (double) 116.4274406433, Null()}, Null()},
+                {{(double) 116.35620117, (double) 39.939093, Null(), (double) 39.9020987219}, Null()},
+                {{(double) 116.35620117, Null(), (double) 116.4274406433, (double) 39.9020987219}, Null()},
+                {{Null(), (double) 39.939093, (double) 116.4274406433, (double) 39.9020987219}, Null()}};
+
+        check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
+    }
+}
+
+} // namespace doris::vectorized
+
+int main(int argc, char** argv) {
+    doris::CpuInfo::init();
+    ::testing::InitGoogleTest(&argc, argv);
+    return RUN_ALL_TESTS();
+}
diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py
index 3a5e8d1598..2de4da6c14 100755
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -1326,30 +1326,30 @@ visible_functions = [
 
     # geo functions
     [['ST_Point'], 'VARCHAR', ['DOUBLE', 'DOUBLE'],
-        '_ZN5doris12GeoFunctions8st_pointEPN9doris_udf15FunctionContextERKNS1_9DoubleValES6_', '', '', '', ''],
+        '_ZN5doris12GeoFunctions8st_pointEPN9doris_udf15FunctionContextERKNS1_9DoubleValES6_', '', '', 'vec', 'ALWAYS_NULLABLE'],
     [['ST_X'], 'DOUBLE', ['VARCHAR'],
         '_ZN5doris12GeoFunctions4st_xEPN9doris_udf15FunctionContextERKNS1_9StringValE',
-        '', '', '', 'ALWAYS_NULLABLE'],
+        '', '', 'vec', 'ALWAYS_NULLABLE'],
     [['ST_Y'], 'DOUBLE', ['VARCHAR'],
         '_ZN5doris12GeoFunctions4st_yEPN9doris_udf15FunctionContextERKNS1_9StringValE',
-        '', '', '', 'ALWAYS_NULLABLE'],
+        '', '', 'vec', 'ALWAYS_NULLABLE'],
     [['ST_X'], 'DOUBLE', ['STRING'],
         '_ZN5doris12GeoFunctions4st_xEPN9doris_udf15FunctionContextERKNS1_9StringValE',
-        '', '', '', 'ALWAYS_NULLABLE'],
+        '', '', 'vec', 'ALWAYS_NULLABLE'],
     [['ST_Y'], 'DOUBLE', ['STRING'],
         '_ZN5doris12GeoFunctions4st_yEPN9doris_udf15FunctionContextERKNS1_9StringValE',
-        '', '', '', 'ALWAYS_NULLABLE'],
+        '', '', 'vec', 'ALWAYS_NULLABLE'],
 
     [['ST_Distance_Sphere'], 'DOUBLE', ['DOUBLE', 'DOUBLE', 'DOUBLE', 'DOUBLE'],
         '_ZN5doris12GeoFunctions18st_distance_sphereEPN9doris_udf15FunctionContextERKNS1_9DoubleValES6_S6_S6_',
-        '', '', '', ''],
+        '', '', 'vec', 'ALWAYS_NULLABLE'],
 
     [['ST_AsText', 'ST_AsWKT'], 'VARCHAR', ['VARCHAR'],
         '_ZN5doris12GeoFunctions9st_as_wktEPN9doris_udf15FunctionContextERKNS1_9StringValE',
-        '', '', '', 'ALWAYS_NULLABLE'],
+        '', '', 'vec', 'ALWAYS_NULLABLE'],
     [['ST_AsText', 'ST_AsWKT'], 'VARCHAR', ['STRING'],
         '_ZN5doris12GeoFunctions9st_as_wktEPN9doris_udf15FunctionContextERKNS1_9StringValE',
-        '', '', '', 'ALWAYS_NULLABLE'],
+        '', '', 'vec', 'ALWAYS_NULLABLE'],
     [['ST_GeometryFromText', 'ST_GeomFromText'], 'VARCHAR', ['VARCHAR'],
         '_ZN5doris12GeoFunctions11st_from_wktEPN9doris_udf15FunctionContextERKNS1_9StringValE',
         '_ZN5doris12GeoFunctions19st_from_wkt_prepareEPN9doris_udf15FunctionContextENS2_18FunctionStateScopeE',


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