You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by ga...@apache.org on 2023/04/21 07:11:26 UTC

[doris] branch master updated: [Improve](GEO)wkb input and output are represented as hexadecimal strings And delete EWKB (#18721)

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

gabriellee pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new ec1ab1a3d2 [Improve](GEO)wkb input and output are represented as hexadecimal strings And delete EWKB (#18721)
ec1ab1a3d2 is described below

commit ec1ab1a3d209ad275e562d20553d1ac960932953
Author: Liqf <10...@users.noreply.github.com>
AuthorDate: Fri Apr 21 15:11:18 2023 +0800

    [Improve](GEO)wkb input and output are represented as hexadecimal strings And delete EWKB (#18721)
---
 be/src/geo/geo_tobinary.cpp                        |  30 ++--
 be/src/geo/geo_tobinary.h                          |   4 +-
 be/src/geo/geo_tobinary_type.h                     |   4 -
 be/src/geo/geo_types.cpp                           |  45 ++----
 be/src/geo/geo_types.h                             |   2 -
 be/src/geo/wkb_parse.cpp                           |  99 +++++++++---
 be/src/geo/wkb_parse.h                             |   4 +-
 be/src/geo/wkb_parse_ctx.h                         |   3 -
 be/src/vec/functions/functions_geo.cpp             |  97 ------------
 .../sql-functions/spatial-functions/st_asbinary.md |  38 +++--
 .../sql-functions/spatial-functions/st_asewkb.md   |  70 ---------
 .../spatial-functions/st_geometryfromewkb.md       |  93 -----------
 .../spatial-functions/st_geometryfromwkb.md        |   8 -
 docs/sidebars.json                                 |   4 +-
 .../sql-functions/spatial-functions/st_asbinary.md |  38 +++--
 .../sql-functions/spatial-functions/st_asewkb.md   |  70 ---------
 .../spatial-functions/st_geometryfromewkb.md       |  92 -----------
 .../spatial-functions/st_geometryfromwkb.md        |  17 --
 .../doris/catalog/BuiltinScalarFunctions.java      |   6 -
 .../expressions/functions/scalar/StAsEWKB.java     |  70 ---------
 .../functions/scalar/StGeomFromEWKB.java           |  70 ---------
 .../functions/scalar/StGeometryFromEWKB.java       |  70 ---------
 .../expressions/visitor/ScalarFunctionVisitor.java |  15 --
 gensrc/script/doris_builtins_functions.py          |   6 -
 .../data/nereids_function_p0/scalar_function/S.out | 175 ++++++---------------
 .../spatial_functions/test_gis_function.out        |  18 ---
 .../spatial_functions/test_gis_function.out        |  18 ---
 .../nereids_function_p0/scalar_function/S.groovy   |  10 +-
 .../spatial_functions/test_gis_function.groovy     |   8 -
 .../spatial_functions/test_gis_function.groovy     |   7 -
 30 files changed, 193 insertions(+), 998 deletions(-)

diff --git a/be/src/geo/geo_tobinary.cpp b/be/src/geo/geo_tobinary.cpp
index 12b36dd3a8..46ffc1b0f0 100644
--- a/be/src/geo/geo_tobinary.cpp
+++ b/be/src/geo/geo_tobinary.cpp
@@ -27,16 +27,24 @@
 #include "geo/wkt_parse_type.h"
 #include "geo_tobinary_type.h"
 #include "geo_types.h"
+#include "iomanip"
 
 namespace doris {
 
-bool toBinary::geo_tobinary(GeoShape* shape, bool isEwkb, std::string* result) {
+bool toBinary::geo_tobinary(GeoShape* shape, std::string* result) {
     ToBinaryContext ctx;
     std::stringstream result_stream;
     ctx.outStream = &result_stream;
-    ctx.isEwkb = isEwkb;
     if (toBinary::write(shape, &ctx)) {
-        *result = result_stream.str();
+        std::stringstream hex_stream;
+        hex_stream << std::hex << std::setfill('0');
+        result_stream.seekg(0);
+        unsigned char c;
+        while (result_stream.read(reinterpret_cast<char*>(&c), 1)) {
+            hex_stream << std::setw(2) << static_cast<int>(c);
+        }
+        //for compatibility with postgres
+        *result = "\\x" + hex_stream.str();
         return true;
     }
     return false;
@@ -61,9 +69,6 @@ bool toBinary::write(GeoShape* shape, ToBinaryContext* ctx) {
 bool toBinary::writeGeoPoint(GeoPoint* point, ToBinaryContext* ctx) {
     writeByteOrder(ctx);
     writeGeometryType(wkbType::wkbPoint, ctx);
-    if (ctx->isEwkb) {
-        writeSRID(ctx);
-    }
     GeoCoordinateList p = point->to_coords();
 
     writeCoordinateList(p, false, ctx);
@@ -73,9 +78,6 @@ bool toBinary::writeGeoPoint(GeoPoint* point, ToBinaryContext* ctx) {
 bool toBinary::writeGeoLine(GeoLine* line, ToBinaryContext* ctx) {
     writeByteOrder(ctx);
     writeGeometryType(wkbType::wkbLine, ctx);
-    if (ctx->isEwkb) {
-        writeSRID(ctx);
-    }
     GeoCoordinateList p = line->to_coords();
 
     writeCoordinateList(p, true, ctx);
@@ -85,9 +87,6 @@ bool toBinary::writeGeoLine(GeoLine* line, ToBinaryContext* ctx) {
 bool toBinary::writeGeoPolygon(doris::GeoPolygon* polygon, ToBinaryContext* ctx) {
     writeByteOrder(ctx);
     writeGeometryType(wkbType::wkbPolygon, ctx);
-    if (ctx->isEwkb) {
-        writeSRID(ctx);
-    }
     writeInt(polygon->numLoops(), ctx);
     GeoCoordinateListList* coordss = polygon->to_coords();
 
@@ -109,9 +108,6 @@ void toBinary::writeByteOrder(ToBinaryContext* ctx) {
 }
 
 void toBinary::writeGeometryType(int typeId, ToBinaryContext* ctx) {
-    if (ctx->isEwkb) {
-        typeId |= 0x20000000;
-    }
     writeInt(typeId, ctx);
 }
 
@@ -120,10 +116,6 @@ void toBinary::writeInt(int val, ToBinaryContext* ctx) {
     ctx->outStream->write(reinterpret_cast<char*>(ctx->buf), 4);
 }
 
-void toBinary::writeSRID(ToBinaryContext* ctx) {
-    writeInt(SRID, ctx);
-}
-
 void toBinary::writeCoordinateList(const GeoCoordinateList& coords, bool sized,
                                    ToBinaryContext* ctx) {
     std::size_t size = coords.list.size();
diff --git a/be/src/geo/geo_tobinary.h b/be/src/geo/geo_tobinary.h
index ad4ee9d0a4..3d4bd1dca1 100644
--- a/be/src/geo/geo_tobinary.h
+++ b/be/src/geo/geo_tobinary.h
@@ -34,7 +34,7 @@ struct GeoCoordinateList;
 
 class toBinary {
 public:
-    static bool geo_tobinary(GeoShape* shape, bool isEwkb, std::string* result);
+    static bool geo_tobinary(GeoShape* shape, std::string* result);
 
     static bool write(GeoShape* shape, ToBinaryContext* ctx);
 
@@ -51,8 +51,6 @@ private:
 
     static void writeInt(int intValue, ToBinaryContext* ctx);
 
-    static void writeSRID(ToBinaryContext* ctx);
-
     static void writeCoordinateList(const GeoCoordinateList& coords, bool sized,
                                     ToBinaryContext* ctx);
 
diff --git a/be/src/geo/geo_tobinary_type.h b/be/src/geo/geo_tobinary_type.h
index e0912eae35..294548f837 100644
--- a/be/src/geo/geo_tobinary_type.h
+++ b/be/src/geo/geo_tobinary_type.h
@@ -31,15 +31,11 @@ enum wkbType {
     wkbGeometryCollection = 7
 };
 
-const int SRID = 4326;
-
 } // namespace doris
 
 struct ToBinaryContext {
     // WKBConstants::wkbwkbXDR | WKBConstants::wkbNDR
     int byteOrder;
-    //Ewkb format:true | false
-    bool isEwkb;
 
     unsigned char buf[8];
 
diff --git a/be/src/geo/geo_types.cpp b/be/src/geo/geo_types.cpp
index 672e594273..c134dbfb6e 100644
--- a/be/src/geo/geo_types.cpp
+++ b/be/src/geo/geo_types.cpp
@@ -201,23 +201,14 @@ GeoShape* GeoShape::from_wkb(const char* data, size_t size, GeoParseStatus* stat
     std::stringstream wkb;
 
     for (int i = 0; i < size; ++i) {
+        if ((i == 1 && wkb.str() == "x") || (i == 2 && wkb.str() == "\\x")) {
+            wkb.str(std::string());
+        }
         wkb << *data;
         data++;
     }
     GeoShape* shape = nullptr;
-    *status = WkbParse::parse_wkb(wkb, false, &shape);
-    return shape;
-}
-
-GeoShape* GeoShape::from_ewkb(const char* data, size_t size, GeoParseStatus* status) {
-    std::stringstream ewkb;
-
-    for (int i = 0; i < size; ++i) {
-        ewkb << *data;
-        data++;
-    }
-    GeoShape* shape = nullptr;
-    *status = WkbParse::parse_wkb(ewkb, true, &shape);
+    *status = WkbParse::parse_wkb(wkb, &shape);
     return shape;
 }
 
@@ -274,8 +265,10 @@ GeoCoordinateList GeoLine::to_coords() const {
     GeoCoordinateList coords;
     for (int i = 0; i < GeoLine::numPoint(); ++i) {
         GeoCoordinate coord;
-        coord.x = S2LatLng(*GeoLine::getPoint(i)).lng().degrees();
-        coord.y = S2LatLng(*GeoLine::getPoint(i)).lat().degrees();
+        coord.x = std::stod(
+                absl::StrFormat("%.13f", S2LatLng::Longitude(*GeoLine::getPoint(i)).degrees()));
+        coord.y = std::stod(
+                absl::StrFormat("%.13f", S2LatLng::Latitude(*GeoLine::getPoint(i)).degrees()));
         coords.add(coord);
     }
     return coords;
@@ -288,12 +281,16 @@ GeoCoordinateListList* GeoPolygon::to_coords() const {
         S2Loop* loop = GeoPolygon::getLoop(i);
         for (int j = 0; j < loop->num_vertices(); ++j) {
             GeoCoordinate coord;
-            coord.x = S2LatLng(loop->vertex(j)).lng().degrees();
-            coord.y = S2LatLng(loop->vertex(j)).lat().degrees();
+            coord.x = std::stod(
+                    absl::StrFormat("%.13f", S2LatLng::Longitude(loop->vertex(j)).degrees()));
+            coord.y = std::stod(
+                    absl::StrFormat("%.13f", S2LatLng::Latitude(loop->vertex(j)).degrees()));
             coords->add(coord);
             if (j == loop->num_vertices() - 1) {
-                coord.x = S2LatLng(loop->vertex(0)).lng().degrees();
-                coord.y = S2LatLng(loop->vertex(0)).lat().degrees();
+                coord.x = std::stod(
+                        absl::StrFormat("%.13f", S2LatLng::Longitude(loop->vertex(0)).degrees()));
+                coord.y = std::stod(
+                        absl::StrFormat("%.13f", S2LatLng::Latitude(loop->vertex(0)).degrees()));
                 coords->add(coord);
             }
         }
@@ -597,15 +594,7 @@ bool GeoShape::ComputeArea(GeoShape* rhs, double* area, std::string square_unit)
 
 std::string GeoShape::as_binary(GeoShape* rhs) {
     std::string res;
-    if (toBinary::geo_tobinary(rhs, false, &res)) {
-        return res;
-    }
-    return res;
-}
-
-std::string GeoShape::as_ewkb(doris::GeoShape* rhs) {
-    std::string res;
-    if (toBinary::geo_tobinary(rhs, true, &res)) {
+    if (toBinary::geo_tobinary(rhs, &res)) {
         return res;
     }
     return res;
diff --git a/be/src/geo/geo_types.h b/be/src/geo/geo_types.h
index e3d16d67c1..9bba6140b8 100644
--- a/be/src/geo/geo_types.h
+++ b/be/src/geo/geo_types.h
@@ -52,7 +52,6 @@ public:
     static GeoShape* from_wkt(const char* data, size_t size, GeoParseStatus* status);
 
     static GeoShape* from_wkb(const char* data, size_t size, GeoParseStatus* status);
-    static GeoShape* from_ewkb(const char* data, size_t size, GeoParseStatus* status);
 
     void encode_to(std::string* buf);
     bool decode_from(const void* data, size_t size);
@@ -62,7 +61,6 @@ public:
     virtual bool contains(const GeoShape* rhs) const { return false; }
     virtual std::string to_string() const { return ""; }
     static std::string as_binary(GeoShape* rhs);
-    static std::string as_ewkb(GeoShape* rhs);
 
     static bool ComputeArea(GeoShape* rhs, double* angle, std::string square_unit);
 
diff --git a/be/src/geo/wkb_parse.cpp b/be/src/geo/wkb_parse.cpp
index f792dd9a25..2dafc1bbc7 100644
--- a/be/src/geo/wkb_parse.cpp
+++ b/be/src/geo/wkb_parse.cpp
@@ -17,9 +17,9 @@
 
 #include "wkb_parse.h"
 
-#include <array>
 #include <cstddef>
 #include <istream>
+#include <sstream>
 #include <vector>
 
 #include "geo/ByteOrderDataInStream.h"
@@ -30,11 +30,55 @@
 
 namespace doris {
 
-GeoParseStatus WkbParse::parse_wkb(std::istream& is, bool isEwkb, GeoShape** shape) {
+unsigned char ASCIIHexToUChar(char val) {
+    switch (val) {
+    case '0':
+        return 0;
+    case '1':
+        return 1;
+    case '2':
+        return 2;
+    case '3':
+        return 3;
+    case '4':
+        return 4;
+    case '5':
+        return 5;
+    case '6':
+        return 6;
+    case '7':
+        return 7;
+    case '8':
+        return 8;
+    case '9':
+        return 9;
+    case 'A':
+    case 'a':
+        return 10;
+    case 'B':
+    case 'b':
+        return 11;
+    case 'C':
+    case 'c':
+        return 12;
+    case 'D':
+    case 'd':
+        return 13;
+    case 'E':
+    case 'e':
+        return 14;
+    case 'F':
+    case 'f':
+        return 15;
+    default:
+        return GEO_PARSE_WKB_SYNTAX_ERROR;
+    }
+}
+
+GeoParseStatus WkbParse::parse_wkb(std::istream& is, GeoShape** shape) {
     WkbParseContext ctx;
 
-    ctx.isEwkb = isEwkb;
-    ctx = *(WkbParse::read(is, &ctx));
+    ctx = *(WkbParse::read_hex(is, &ctx));
     if (ctx.parse_status == GEO_PARSE_OK) {
         *shape = ctx.shape;
     } else {
@@ -43,6 +87,36 @@ GeoParseStatus WkbParse::parse_wkb(std::istream& is, bool isEwkb, GeoShape** sha
     return ctx.parse_status;
 }
 
+WkbParseContext* WkbParse::read_hex(std::istream& is, WkbParseContext* ctx) {
+    // setup input/output stream
+    std::stringstream os(std::ios_base::binary | std::ios_base::in | std::ios_base::out);
+
+    while (true) {
+        const int input_high = is.get();
+        if (input_high == std::char_traits<char>::eof()) {
+            break;
+        }
+
+        const int input_low = is.get();
+        if (input_low == std::char_traits<char>::eof()) {
+            ctx->parse_status = GEO_PARSE_WKB_SYNTAX_ERROR;
+            return ctx;
+        }
+
+        const char high = static_cast<char>(input_high);
+        const char low = static_cast<char>(input_low);
+
+        const unsigned char result_high = ASCIIHexToUChar(high);
+        const unsigned char result_low = ASCIIHexToUChar(low);
+
+        const unsigned char value = static_cast<unsigned char>((result_high << 4) + result_low);
+
+        // write the value to the output stream
+        os << value;
+    }
+    return WkbParse::read(os, ctx);
+}
+
 WkbParseContext* WkbParse::read(std::istream& is, WkbParseContext* ctx) {
     is.seekg(0, std::ios::end);
     auto size = is.tellg();
@@ -76,23 +150,6 @@ GeoShape* WkbParse::readGeometry(WkbParseContext* ctx) {
 
     uint32_t geometryType = (typeInt & 0xffff) % 1000;
 
-    bool hasSRID = ((typeInt & 0x20000000) != 0);
-
-    if (ctx->isEwkb) {
-        if (!hasSRID) {
-            return nullptr;
-        }
-
-        ctx->srid = ctx->dis.readInt(); // read SRID
-        if (ctx->srid != 4326) {
-            return nullptr;
-        }
-    }
-
-    if (!ctx->isEwkb && hasSRID) {
-        return nullptr;
-    }
-
     GeoShape* shape;
 
     switch (geometryType) {
diff --git a/be/src/geo/wkb_parse.h b/be/src/geo/wkb_parse.h
index 966e290733..f062a5ab75 100644
--- a/be/src/geo/wkb_parse.h
+++ b/be/src/geo/wkb_parse.h
@@ -35,7 +35,9 @@ class GeoPolygon;
 
 class WkbParse {
 public:
-    static GeoParseStatus parse_wkb(std::istream& is, bool isEwkb, GeoShape** shape);
+    static GeoParseStatus parse_wkb(std::istream& is, GeoShape** shape);
+
+    static WkbParseContext* read_hex(std::istream& is, WkbParseContext* ctx);
 
     static WkbParseContext* read(std::istream& is, WkbParseContext* ctx);
 
diff --git a/be/src/geo/wkb_parse_ctx.h b/be/src/geo/wkb_parse_ctx.h
index 5c7d52a884..eb47d7fc1b 100644
--- a/be/src/geo/wkb_parse_ctx.h
+++ b/be/src/geo/wkb_parse_ctx.h
@@ -31,9 +31,6 @@ struct WkbParseContext {
 
     std::array<double, 2> ordValues;
 
-    //Ewkb format:true | false
-    bool isEwkb;
-
     int srid;
 
     doris::GeoShape* shape = nullptr;
diff --git a/be/src/vec/functions/functions_geo.cpp b/be/src/vec/functions/functions_geo.cpp
index 1b3f34cf7d..28e8b16fd1 100644
--- a/be/src/vec/functions/functions_geo.cpp
+++ b/be/src/vec/functions/functions_geo.cpp
@@ -629,56 +629,6 @@ struct StGeoFromWkb {
     }
 };
 
-struct StGeometryFromEWKB {
-    static constexpr auto NAME = "st_geometryfromewkb";
-    static constexpr GeoShapeType shape_type = GEO_SHAPE_ANY;
-};
-
-struct StGeomFromEWKB {
-    static constexpr auto NAME = "st_geomfromewkb";
-    static constexpr GeoShapeType shape_type = GEO_SHAPE_ANY;
-};
-
-template <typename Impl>
-struct StGeoFromEwkb {
-    static constexpr auto NEED_CONTEXT = true;
-    static constexpr auto NAME = Impl::NAME;
-    static const size_t NUM_ARGS = 1;
-    static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
-                          size_t result) {
-        DCHECK_EQ(arguments.size(), 1);
-        auto return_type = block.get_data_type(result);
-        auto& geo = block.get_by_position(arguments[0]).column;
-
-        const auto size = geo->size();
-        MutableColumnPtr res = return_type->create_column();
-
-        GeoParseStatus status;
-        std::string buf;
-        for (int row = 0; row < size; ++row) {
-            auto value = geo->get_data_at(row);
-            std::unique_ptr<GeoShape> shape(GeoShape::from_ewkb(value.data, value.size, &status));
-            if (shape == nullptr || status != GEO_PARSE_OK) {
-                res->insert_data(nullptr, 0);
-                continue;
-            }
-            buf.clear();
-            shape->encode_to(&buf);
-            res->insert_data(buf.data(), buf.size());
-        }
-        block.replace_by_position(result, std::move(res));
-        return Status::OK();
-    }
-
-    static Status open(FunctionContext* context, FunctionContext::FunctionStateScope scope) {
-        return Status::OK();
-    }
-
-    static Status close(FunctionContext* context, FunctionContext::FunctionStateScope scope) {
-        return Status::OK();
-    }
-};
-
 struct StAsBinary {
     static constexpr auto NEED_CONTEXT = true;
     static constexpr auto NAME = "st_asbinary";
@@ -723,50 +673,6 @@ struct StAsBinary {
     }
 };
 
-struct StAsEwkb {
-    static constexpr auto NEED_CONTEXT = true;
-    static constexpr auto NAME = "st_asewkb";
-    static const size_t NUM_ARGS = 1;
-    static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
-                          size_t result) {
-        DCHECK_EQ(arguments.size(), 1);
-        auto return_type = block.get_data_type(result);
-        MutableColumnPtr res = return_type->create_column();
-
-        auto col = block.get_by_position(arguments[0]).column->convert_to_full_column_if_const();
-        const auto size = col->size();
-
-        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;
-            }
-
-            std::string binary = GeoShape::as_ewkb(shape.get());
-            if (binary.empty()) {
-                res->insert_data(nullptr, 0);
-                continue;
-            }
-            res->insert_data(binary.data(), binary.size());
-        }
-
-        block.replace_by_position(result, std::move(res));
-        return Status::OK();
-    }
-
-    static Status open(FunctionContext* context, FunctionContext::FunctionStateScope scope) {
-        return Status::OK();
-    }
-
-    static Status close(FunctionContext* context, FunctionContext::FunctionStateScope scope) {
-        return Status::OK();
-    }
-};
-
 void register_function_geo(SimpleFunctionFactory& factory) {
     factory.register_function<GeoFunction<StPoint>>();
     factory.register_function<GeoFunction<StAsText<StAsWktName>>>();
@@ -790,10 +696,7 @@ void register_function_geo(SimpleFunctionFactory& factory) {
     factory.register_function<GeoFunction<StAreaSquareKm, DataTypeFloat64>>();
     factory.register_function<GeoFunction<StGeoFromWkb<StGeometryFromWKB>>>();
     factory.register_function<GeoFunction<StGeoFromWkb<StGeomFromWKB>>>();
-    factory.register_function<GeoFunction<StGeoFromEwkb<StGeometryFromEWKB>>>();
-    factory.register_function<GeoFunction<StGeoFromEwkb<StGeomFromEWKB>>>();
     factory.register_function<GeoFunction<StAsBinary>>();
-    factory.register_function<GeoFunction<StAsEwkb>>();
 }
 
 } // namespace doris::vectorized
diff --git a/docs/en/docs/sql-manual/sql-functions/spatial-functions/st_asbinary.md b/docs/en/docs/sql-manual/sql-functions/spatial-functions/st_asbinary.md
index 8b0519b5b4..f4212a82b0 100644
--- a/docs/en/docs/sql-manual/sql-functions/spatial-functions/st_asbinary.md
+++ b/docs/en/docs/sql-manual/sql-functions/spatial-functions/st_asbinary.md
@@ -40,30 +40,28 @@ Currently supported geometric figures are: Point, LineString, Polygon.
 
 ```
 mysql> select ST_AsBinary(st_point(24.7, 56.7));
-+-----------------------------------+
-| st_asbinary(st_point(24.7, 56.7)) |
-+-----------------------------------+
-|    33333�8@�����YL@                   |
-+-----------------------------------+
-1 row in set (0.04 sec)
++----------------------------------------------+
+| st_asbinary(st_point(24.7, 56.7))            |
++----------------------------------------------+
+| \x01010000003333333333b338409a99999999594c40 |
++----------------------------------------------+
+1 row in set (0.01 sec)
 
 mysql> select ST_AsBinary(ST_GeometryFromText("LINESTRING (1 1, 2 2)"));
-+-----------------------------------------------------------+
-| st_asbinary(st_geometryfromtext('LINESTRING (1 1, 2 2)')) |
-+-----------------------------------------------------------+
-|       �������?      �?       @�������?                                |
-+-----------------------------------------------------------+
-1 row in set (0.02 sec)
++--------------------------------------------------------------------------------------+
+| st_asbinary(st_geometryfromtext('LINESTRING (1 1, 2 2)'))                            |
++--------------------------------------------------------------------------------------+
+| \x010200000002000000000000000000f03f000000000000f03f00000000000000400000000000000040 |
++--------------------------------------------------------------------------------------+
+1 row in set (0.04 sec)
 
 mysql> select ST_AsBinary(ST_Polygon("POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))"));
-+----------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-| st_asbinary(st_polygon('POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))')) |
-+----------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-|          �8
-             毆\@.���6@B�! �\@0Ie�9�6@��-�\@��6�4�6@ޒ���\@·g    2�6@,�̆\@{1��6@�8
-                                                                                毆\@.���6@                                                                                          |
-+----------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-1 row in set (0.01 sec)
++------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
+| st_asbinary(st_polygon('POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))'))                                                         |
++------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
+| \x01030000000100000006000000f3380ce6af865c402d05a4fd0f8c364041ef8d2100865c403049658a398c3640b9fb1c1f2d865c409d9b36e334883640de921cb0ab865c40cf876709328a36402cefaa07cc865c407b319413ed8a3640f3380ce6af865c402d05a4fd0f8c3640 |
++------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
+1 row in set (0.02 sec)
 
 ```
 ### keywords
diff --git a/docs/en/docs/sql-manual/sql-functions/spatial-functions/st_asewkb.md b/docs/en/docs/sql-manual/sql-functions/spatial-functions/st_asewkb.md
deleted file mode 100644
index 20bc1518b3..0000000000
--- a/docs/en/docs/sql-manual/sql-functions/spatial-functions/st_asewkb.md
+++ /dev/null
@@ -1,70 +0,0 @@
----
-{
-    "title": "ST_AsEWKB",
-    "language": "en"
-}
----
-
-<!-- 
-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.
--->
-
-## ST_AsEWKB
-
-### Syntax
-
-`VARCHAR ST_AsEWKB(GEOMETRY geo)`
-
-### Description
-
-Converting a geometric figure into a EWKB (Extended Well-known binary) representation
-
-Currently supported geometric figures are: Point, LineString, Polygon.
-
-### example
-
-```
-mysql> select ST_AsEWKB(st_point(24.7, 56.7));
-+---------------------------------+
-| st_asewkb(st_point(24.7, 56.7)) |
-+---------------------------------+
-|    �  33333�8@�����YL@              |
-+---------------------------------+
-1 row in set (0.02 sec)
-
-mysql> select ST_AsEWKB(ST_GeometryFromText("LINESTRING (1 1, 2 2)"));
-+---------------------------------------------------------+
-| st_asewkb(st_geometryfromtext('LINESTRING (1 1, 2 2)')) |
-+---------------------------------------------------------+
-|    �     �������?      �?       @�������?                           |
-+---------------------------------------------------------+
-1 row in set (0.01 sec)
-
-mysql> select ST_AsEWKB(ST_Polygon("POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))"));
-+--------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-| st_asewkb(st_polygon('POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))')) |
-+--------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-|    �        �8
-                毆\@.���6@B�! �\@0Ie�9�6@��-�\@��6�4�6@ޒ���\@·g 2�6@,�̆\@{1��6@�8
-                                                                                毆\@.���6@                                                                                     |
-+--------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-1 row in set (0.01 sec)
-
-```
-### keywords
-ST_ASWEKB,ST,ASEWKB
diff --git a/docs/en/docs/sql-manual/sql-functions/spatial-functions/st_geometryfromewkb.md b/docs/en/docs/sql-manual/sql-functions/spatial-functions/st_geometryfromewkb.md
deleted file mode 100644
index 2c7e59325d..0000000000
--- a/docs/en/docs/sql-manual/sql-functions/spatial-functions/st_geometryfromewkb.md
+++ /dev/null
@@ -1,93 +0,0 @@
----
-{
-    "title": "ST_GeometryFromEWKB,ST_GeomFromEWKB",
-    "language": "en"
-}
----
-
-<!-- 
-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.
--->
-
-## ST_GeometryFromEWKB,ST_GeomFromEWKB
-
-### Syntax
-
-`GEOMETRY ST_GeometryFromEWKB(VARCHAR EWKB)`
-
-### Description
-
-Converting a EWKB (Extended Well-known binary) into a corresponding memory geometry
-
-Since the GIS function of doris is implemented based on the WGS84 coordinate system, it can only parse EWKB data with SRID 4326, and return NULL for EWKB data with SRID other than 4326.
-
-### example
-
-```
-mysql> select ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(ST_Point(24.7, 56.7))));
-+-----------------------------------------------------------------+
-| st_astext(st_geometryfromewkb(st_asewkb(st_point(24.7, 56.7)))) |
-+-----------------------------------------------------------------+
-| POINT (24.7 56.7)                                               |
-+-----------------------------------------------------------------+
-1 row in set (0.04 sec)
-
-mysql> select ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(ST_GeometryFromText("LINESTRING (1 1, 2 2)"))));
-+-----------------------------------------------------------------------------------------+
-| st_astext(st_geometryfromewkb(st_asewkb(st_geometryfromtext('LINESTRING (1 1, 2 2)')))) |
-+-----------------------------------------------------------------------------------------+
-| LINESTRING (1 1, 2 2)                                                                   |
-+-----------------------------------------------------------------------------------------+
-1 row in set (0.11 sec)
-
-mysql> select ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(ST_Polygon("POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))"))));
-+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-| st_astext(st_geometryfromewkb(st_asewkb(st_polygon('POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))')))) |
-+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-| POLYGON ((114.104486 22.547119, 114.093758 22.547753, 114.096504 22.532057, 114.104229 22.539826, 114.106203 22.54268, 114.104486 22.547119))                                                      |
-+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-1 row in set (0.03 sec)
-
-mysql> select ST_AsText(ST_GeomFromEWKB(ST_AsEWKB(ST_GeometryFromText("LINESTRING (1 1, 2 2)"))));
-+-------------------------------------------------------------------------------------+
-| st_astext(st_geomfromewkb(st_asewkb(st_geometryfromtext('LINESTRING (1 1, 2 2)')))) |
-+-------------------------------------------------------------------------------------+
-| LINESTRING (1 1, 2 2)                                                               |
-+-------------------------------------------------------------------------------------+
-1 row in set (0.02 sec)
-
-mysql> select ST_AsText(ST_GeomFromEWKB(ST_AsEWKB(ST_Polygon("POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))"))));
-+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-| st_astext(st_geomfromewkb(st_asewkb(st_polygon('POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))')))) |
-+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-| POLYGON ((114.104486 22.547119, 114.093758 22.547753, 114.096504 22.532057, 114.104229 22.539826, 114.106203 22.54268, 114.104486 22.547119))                                                  |
-+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-1 row in set (0.03 sec)
-
-//Parsing WKB data returns NULL.
-mysql> select ST_AsText(ST_GeometryFromEWKB(ST_AsBinary(ST_GeometryFromText("LINESTRING (1 1, 2 2)"))));
-+-------------------------------------------------------------------------------------------+
-| st_astext(st_geometryfromewkb(st_asbinary(st_geometryfromtext('LINESTRING (1 1, 2 2)')))) |
-+-------------------------------------------------------------------------------------------+
-| NULL                                                                                      |
-+-------------------------------------------------------------------------------------------+
-1 row in set (0.02 sec)
-
-```
-### keywords
-ST_GEOMETRYFROMEWKB,ST_GEOMFROMEWKB,ST,GEOMETRYFROMEWKB,GEOMFROMEWKB,EWKB
diff --git a/docs/en/docs/sql-manual/sql-functions/spatial-functions/st_geometryfromwkb.md b/docs/en/docs/sql-manual/sql-functions/spatial-functions/st_geometryfromwkb.md
index a0dd4f5dfb..b780da7b80 100644
--- a/docs/en/docs/sql-manual/sql-functions/spatial-functions/st_geometryfromwkb.md
+++ b/docs/en/docs/sql-manual/sql-functions/spatial-functions/st_geometryfromwkb.md
@@ -61,14 +61,6 @@ mysql> select ST_AsText(ST_GeometryFromWKB(ST_AsBinary(ST_GeometryFromText("LINE
 +------------------------------------------------------------------------------------------+
 1 row in set (0.06 sec)
 
-mysql> select ST_AsText(ST_GeomFromWKB(ST_AsBinary(ST_GeometryFromText("LINESTRING (1 1, 2 2)"))));
-+--------------------------------------------------------------------------------------+
-| st_astext(st_geomfromwkb(st_asbinary(st_geometryfromtext('LINESTRING (1 1, 2 2)')))) |
-+--------------------------------------------------------------------------------------+
-| LINESTRING (1 1, 2 2)                                                                |
-+--------------------------------------------------------------------------------------+
-1 row in set (0.02 sec)
-
 mysql> select ST_AsText(ST_GeometryFromWKB(ST_AsBinary(ST_Polygon("POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))"))));
 +-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
 | st_astext(st_geometryfromwkb(st_asbinary(st_polygon('POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))')))) |
diff --git a/docs/sidebars.json b/docs/sidebars.json
index ffda90e7e2..ae99b00e33 100644
--- a/docs/sidebars.json
+++ b/docs/sidebars.json
@@ -398,9 +398,7 @@
                                 "sql-manual/sql-functions/spatial-functions/st_geometryfromtext",
                                 "sql-manual/sql-functions/spatial-functions/st_linefromtext",
                                 "sql-manual/sql-functions/spatial-functions/st_asbinary",
-                                "sql-manual/sql-functions/spatial-functions/st_asewkb",
-                                "sql-manual/sql-functions/spatial-functions/st_geometryfromwkb",
-                                "sql-manual/sql-functions/spatial-functions/st_geometryfromewkb"
+                                "sql-manual/sql-functions/spatial-functions/st_geometryfromwkb"
                             ]
                         },
                         {
diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/spatial-functions/st_asbinary.md b/docs/zh-CN/docs/sql-manual/sql-functions/spatial-functions/st_asbinary.md
index aea3dc692a..1ad8f29323 100644
--- a/docs/zh-CN/docs/sql-manual/sql-functions/spatial-functions/st_asbinary.md
+++ b/docs/zh-CN/docs/sql-manual/sql-functions/spatial-functions/st_asbinary.md
@@ -40,30 +40,28 @@ under the License.
 
 ```
 mysql> select ST_AsBinary(st_point(24.7, 56.7));
-+-----------------------------------+
-| st_asbinary(st_point(24.7, 56.7)) |
-+-----------------------------------+
-|    33333�8@�����YL@                   |
-+-----------------------------------+
-1 row in set (0.04 sec)
++----------------------------------------------+
+| st_asbinary(st_point(24.7, 56.7))            |
++----------------------------------------------+
+| \x01010000003333333333b338409a99999999594c40 |
++----------------------------------------------+
+1 row in set (0.01 sec)
 
 mysql> select ST_AsBinary(ST_GeometryFromText("LINESTRING (1 1, 2 2)"));
-+-----------------------------------------------------------+
-| st_asbinary(st_geometryfromtext('LINESTRING (1 1, 2 2)')) |
-+-----------------------------------------------------------+
-|       �������?      �?       @�������?                                |
-+-----------------------------------------------------------+
-1 row in set (0.02 sec)
++--------------------------------------------------------------------------------------+
+| st_asbinary(st_geometryfromtext('LINESTRING (1 1, 2 2)'))                            |
++--------------------------------------------------------------------------------------+
+| \x010200000002000000000000000000f03f000000000000f03f00000000000000400000000000000040 |
++--------------------------------------------------------------------------------------+
+1 row in set (0.04 sec)
 
 mysql> select ST_AsBinary(ST_Polygon("POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))"));
-+----------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-| st_asbinary(st_polygon('POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))')) |
-+----------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-|          �8
-             毆\@.���6@B�! �\@0Ie�9�6@��-�\@��6�4�6@ޒ���\@·g    2�6@,�̆\@{1��6@�8
-                                                                                毆\@.���6@                                                                                          |
-+----------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-1 row in set (0.01 sec)
++------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
+| st_asbinary(st_polygon('POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))'))                                                         |
++------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
+| \x01030000000100000006000000f3380ce6af865c402d05a4fd0f8c364041ef8d2100865c403049658a398c3640b9fb1c1f2d865c409d9b36e334883640de921cb0ab865c40cf876709328a36402cefaa07cc865c407b319413ed8a3640f3380ce6af865c402d05a4fd0f8c3640 |
++------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
+1 row in set (0.02 sec)
 
 ```
 ### keywords
diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/spatial-functions/st_asewkb.md b/docs/zh-CN/docs/sql-manual/sql-functions/spatial-functions/st_asewkb.md
deleted file mode 100644
index 9068b8b48a..0000000000
--- a/docs/zh-CN/docs/sql-manual/sql-functions/spatial-functions/st_asewkb.md
+++ /dev/null
@@ -1,70 +0,0 @@
----
-{
-    "title": "ST_AsEWKB",
-    "language": "zh-CN"
-}
----
-
-<!-- 
-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.
--->
-
-## ST_AsEWKB
-
-### Syntax
-
-`VARCHAR ST_AsEWKB(GEOMETRY geo)`
-
-### Description
-
-将一个几何图形转化为一个扩展 WKB(Extended Well-known binary)的表示形式。
-
-目前支持对几何图形是:Point, LineString, Polygon。
-
-### example
-
-```
-mysql> select ST_AsEWKB(st_point(24.7, 56.7));
-+---------------------------------+
-| st_asewkb(st_point(24.7, 56.7)) |
-+---------------------------------+
-|    �  33333�8@�����YL@              |
-+---------------------------------+
-1 row in set (0.02 sec)
-
-mysql> select ST_AsEWKB(ST_GeometryFromText("LINESTRING (1 1, 2 2)"));
-+---------------------------------------------------------+
-| st_asewkb(st_geometryfromtext('LINESTRING (1 1, 2 2)')) |
-+---------------------------------------------------------+
-|    �     �������?      �?       @�������?                           |
-+---------------------------------------------------------+
-1 row in set (0.01 sec)
-
-mysql> select ST_AsEWKB(ST_Polygon("POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))"));
-+--------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-| st_asewkb(st_polygon('POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))')) |
-+--------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-|    �        �8
-                毆\@.���6@B�! �\@0Ie�9�6@��-�\@��6�4�6@ޒ���\@·g 2�6@,�̆\@{1��6@�8
-                                                                                毆\@.���6@                                                                                     |
-+--------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-1 row in set (0.01 sec)
-
-```
-### keywords
-ST_ASWEKB,ST,ASEWKB
diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/spatial-functions/st_geometryfromewkb.md b/docs/zh-CN/docs/sql-manual/sql-functions/spatial-functions/st_geometryfromewkb.md
deleted file mode 100644
index 0c9966797d..0000000000
--- a/docs/zh-CN/docs/sql-manual/sql-functions/spatial-functions/st_geometryfromewkb.md
+++ /dev/null
@@ -1,92 +0,0 @@
----
-{
-    "title": "ST_GeometryFromEWKB,ST_GeomFromEWKB",
-    "language": "zh-CN"
-}
----
-
-<!-- 
-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.
--->
-
-## ST_GeometryFromEWKB,ST_GeomFromEWKB
-
-### Syntax
-
-`GEOMETRY ST_GeometryFromEWKB(VARCHAR EWKB)`
-
-### Description
-
-将一个扩展 WKB(Extended Well-known binary)转化为对应的内存的几何形式。
-由于doris的地理函数基于 WGS84 坐标系实现的,因此只能解析SRID为4326的EWKB数据,对于SRID不为4326的EWKB数据返回 NULL。
-
-### example
-
-```
-mysql> select ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(ST_Point(24.7, 56.7))));
-+-----------------------------------------------------------------+
-| st_astext(st_geometryfromewkb(st_asewkb(st_point(24.7, 56.7)))) |
-+-----------------------------------------------------------------+
-| POINT (24.7 56.7)                                               |
-+-----------------------------------------------------------------+
-1 row in set (0.04 sec)
-
-mysql> select ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(ST_GeometryFromText("LINESTRING (1 1, 2 2)"))));
-+-----------------------------------------------------------------------------------------+
-| st_astext(st_geometryfromewkb(st_asewkb(st_geometryfromtext('LINESTRING (1 1, 2 2)')))) |
-+-----------------------------------------------------------------------------------------+
-| LINESTRING (1 1, 2 2)                                                                   |
-+-----------------------------------------------------------------------------------------+
-1 row in set (0.11 sec)
-
-mysql> select ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(ST_Polygon("POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))"))));
-+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-| st_astext(st_geometryfromewkb(st_asewkb(st_polygon('POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))')))) |
-+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-| POLYGON ((114.104486 22.547119, 114.093758 22.547753, 114.096504 22.532057, 114.104229 22.539826, 114.106203 22.54268, 114.104486 22.547119))                                                      |
-+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-1 row in set (0.03 sec)
-
-mysql> select ST_AsText(ST_GeomFromEWKB(ST_AsEWKB(ST_GeometryFromText("LINESTRING (1 1, 2 2)"))));
-+-------------------------------------------------------------------------------------+
-| st_astext(st_geomfromewkb(st_asewkb(st_geometryfromtext('LINESTRING (1 1, 2 2)')))) |
-+-------------------------------------------------------------------------------------+
-| LINESTRING (1 1, 2 2)                                                               |
-+-------------------------------------------------------------------------------------+
-1 row in set (0.02 sec)
-
-mysql> select ST_AsText(ST_GeomFromEWKB(ST_AsEWKB(ST_Polygon("POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))"))));
-+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-| st_astext(st_geomfromewkb(st_asewkb(st_polygon('POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))')))) |
-+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-| POLYGON ((114.104486 22.547119, 114.093758 22.547753, 114.096504 22.532057, 114.104229 22.539826, 114.106203 22.54268, 114.104486 22.547119))                                                  |
-+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-1 row in set (0.03 sec)
-
-//解析WKB数据返回NULL
-mysql> select ST_AsText(ST_GeometryFromEWKB(ST_AsBinary(ST_GeometryFromText("LINESTRING (1 1, 2 2)"))));
-+-------------------------------------------------------------------------------------------+
-| st_astext(st_geometryfromewkb(st_asbinary(st_geometryfromtext('LINESTRING (1 1, 2 2)')))) |
-+-------------------------------------------------------------------------------------------+
-| NULL                                                                                      |
-+-------------------------------------------------------------------------------------------+
-1 row in set (0.02 sec)
-
-```
-### keywords
-ST_GEOMETRYFROMEWKB,ST_GEOMFROMEWKB,ST,GEOMETRYFROMEWKB,GEOMFROMEWKB,EWKB
diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/spatial-functions/st_geometryfromwkb.md b/docs/zh-CN/docs/sql-manual/sql-functions/spatial-functions/st_geometryfromwkb.md
index c5cdac4abf..6242adab13 100644
--- a/docs/zh-CN/docs/sql-manual/sql-functions/spatial-functions/st_geometryfromwkb.md
+++ b/docs/zh-CN/docs/sql-manual/sql-functions/spatial-functions/st_geometryfromwkb.md
@@ -61,14 +61,6 @@ mysql> select ST_AsText(ST_GeometryFromWKB(ST_AsBinary(ST_GeometryFromText("LINE
 +------------------------------------------------------------------------------------------+
 1 row in set (0.06 sec)
 
-mysql> select ST_AsText(ST_GeomFromWKB(ST_AsBinary(ST_GeometryFromText("LINESTRING (1 1, 2 2)"))));
-+--------------------------------------------------------------------------------------+
-| st_astext(st_geomfromwkb(st_asbinary(st_geometryfromtext('LINESTRING (1 1, 2 2)')))) |
-+--------------------------------------------------------------------------------------+
-| LINESTRING (1 1, 2 2)                                                                |
-+--------------------------------------------------------------------------------------+
-1 row in set (0.02 sec)
-
 mysql> select ST_AsText(ST_GeometryFromWKB(ST_AsBinary(ST_Polygon("POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))"))));
 +-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
 | st_astext(st_geometryfromwkb(st_asbinary(st_polygon('POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))')))) |
@@ -85,15 +77,6 @@ mysql> select ST_AsText(ST_GeomFromWKB(ST_AsBinary(ST_Polygon("POLYGON ((114.104
 +-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
 1 row in set (0.03 sec)
 
-//解析EWKB数据返回NULL
-mysql> select ST_AsText(ST_GeometryFromWKB(ST_AsEWKB(ST_Point(24.7, 56.7))));
-+----------------------------------------------------------------+
-| st_astext(st_geometryfromwkb(st_asewkb(st_point(24.7, 56.7)))) |
-+----------------------------------------------------------------+
-| NULL                                                           |
-+----------------------------------------------------------------+
-1 row in set (0.02 sec)
-
 ```
 ### keywords
 ST_GEOMETRYFROMWKB,ST_GEOMFROMWKB,ST,GEOMETRYFROMWKB,GEOMFROMWKB,WKB
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
index 85ca44dec2..58692e451d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
@@ -273,16 +273,13 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.StAngleSphere
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StAreaSquareKm;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StAreaSquareMeters;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StAsBinary;
-import org.apache.doris.nereids.trees.expressions.functions.scalar.StAsEWKB;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StAstext;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StAswkt;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StAzimuth;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StCircle;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StContains;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StDistanceSphere;
-import org.apache.doris.nereids.trees.expressions.functions.scalar.StGeomFromEWKB;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StGeomFromWKB;
-import org.apache.doris.nereids.trees.expressions.functions.scalar.StGeometryFromEWKB;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StGeometryFromWKB;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StGeometryfromtext;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StGeomfromtext;
@@ -603,7 +600,6 @@ public class BuiltinScalarFunctions implements FunctionHelper {
             scalar(SplitPart.class, "split_part"),
             scalar(Sqrt.class, "sqrt"),
             scalar(StAsBinary.class, "st_asbinary"),
-            scalar(StAsEWKB.class, "st_asewkb"),
             scalar(StAstext.class, "st_astext"),
             scalar(StAswkt.class, "st_aswkt"),
             scalar(StCircle.class, "st_circle"),
@@ -614,10 +610,8 @@ public class BuiltinScalarFunctions implements FunctionHelper {
             scalar(StAzimuth.class, "st_azimuth"),
             scalar(StAreaSquareMeters.class, "st_area_square_meters"),
             scalar(StAreaSquareKm.class, "st_area_square_km"),
-            scalar(StGeometryFromEWKB.class, "st_geometryfromewkb"),
             scalar(StGeometryfromtext.class, "st_geometryfromtext"),
             scalar(StGeometryFromWKB.class, "st_geometryfromwkb"),
-            scalar(StGeomFromEWKB.class, "st_geomfromewkb"),
             scalar(StGeomfromtext.class, "st_geomfromtext"),
             scalar(StGeomFromWKB.class, "st_geomfromwkb"),
             scalar(StLinefromtext.class, "st_linefromtext"),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/StAsEWKB.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/StAsEWKB.java
deleted file mode 100644
index 55991e2f26..0000000000
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/StAsEWKB.java
+++ /dev/null
@@ -1,70 +0,0 @@
-// 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.doris.nereids.trees.expressions.functions.scalar;
-
-import org.apache.doris.catalog.FunctionSignature;
-import org.apache.doris.nereids.trees.expressions.Expression;
-import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
-import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
-import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
-import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
-import org.apache.doris.nereids.types.StringType;
-import org.apache.doris.nereids.types.VarcharType;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
-
-/**
- * ScalarFunction 'st_asewkb'. This class is generated by GenerateFunction.
- */
-public class StAsEWKB extends ScalarFunction
-        implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
-
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(VarcharType.SYSTEM_DEFAULT),
-            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(StringType.INSTANCE)
-    );
-
-    /**
-     * constructor with 1 argument.
-     */
-    public StAsEWKB(Expression arg) {
-        super("st_asewkb", arg);
-    }
-
-    /**
-     * withChildren.
-     */
-    @Override
-    public StAsEWKB withChildren(List<Expression> children) {
-        Preconditions.checkArgument(children.size() == 1);
-        return new StAsEWKB(children.get(0));
-    }
-
-    @Override
-    public List<FunctionSignature> getSignatures() {
-        return SIGNATURES;
-    }
-
-    @Override
-    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
-        return visitor.visitStAsEWKB(this, context);
-    }
-}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/StGeomFromEWKB.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/StGeomFromEWKB.java
deleted file mode 100644
index d61010436b..0000000000
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/StGeomFromEWKB.java
+++ /dev/null
@@ -1,70 +0,0 @@
-// 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.doris.nereids.trees.expressions.functions.scalar;
-
-import org.apache.doris.catalog.FunctionSignature;
-import org.apache.doris.nereids.trees.expressions.Expression;
-import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
-import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
-import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
-import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
-import org.apache.doris.nereids.types.StringType;
-import org.apache.doris.nereids.types.VarcharType;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
-
-/**
- * ScalarFunction 'st_geomfromewkb'. This class is generated by GenerateFunction.
- */
-public class StGeomFromEWKB extends ScalarFunction
-        implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
-
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(VarcharType.SYSTEM_DEFAULT),
-            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(StringType.INSTANCE)
-    );
-
-    /**
-     * constructor with 1 argument.
-     */
-    public StGeomFromEWKB(Expression arg) {
-        super("st_geomfromewkb", arg);
-    }
-
-    /**
-     * withChildren.
-     */
-    @Override
-    public StGeomFromEWKB withChildren(List<Expression> children) {
-        Preconditions.checkArgument(children.size() == 1);
-        return new StGeomFromEWKB(children.get(0));
-    }
-
-    @Override
-    public List<FunctionSignature> getSignatures() {
-        return SIGNATURES;
-    }
-
-    @Override
-    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
-        return visitor.visitStGeomfromewkb(this, context);
-    }
-}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/StGeometryFromEWKB.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/StGeometryFromEWKB.java
deleted file mode 100644
index 493ddb67e8..0000000000
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/StGeometryFromEWKB.java
+++ /dev/null
@@ -1,70 +0,0 @@
-// 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.doris.nereids.trees.expressions.functions.scalar;
-
-import org.apache.doris.catalog.FunctionSignature;
-import org.apache.doris.nereids.trees.expressions.Expression;
-import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
-import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
-import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
-import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
-import org.apache.doris.nereids.types.StringType;
-import org.apache.doris.nereids.types.VarcharType;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
-
-/**
- * ScalarFunction 'st_geometryfromewkb'. This class is generated by GenerateFunction.
- */
-public class StGeometryFromEWKB extends ScalarFunction
-        implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
-
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(VarcharType.SYSTEM_DEFAULT),
-            FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(StringType.INSTANCE)
-    );
-
-    /**
-     * constructor with 1 argument.
-     */
-    public StGeometryFromEWKB(Expression arg) {
-        super("st_geometryfromewkb", arg);
-    }
-
-    /**
-     * withChildren.
-     */
-    @Override
-    public StGeometryFromEWKB withChildren(List<Expression> children) {
-        Preconditions.checkArgument(children.size() == 1);
-        return new StGeometryFromEWKB(children.get(0));
-    }
-
-    @Override
-    public List<FunctionSignature> getSignatures() {
-        return SIGNATURES;
-    }
-
-    @Override
-    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
-        return visitor.visitStGeometryfromewkb(this, context);
-    }
-}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
index 07176f51d8..7eb28b1e08 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
@@ -275,16 +275,13 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.StAngleSphere
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StAreaSquareKm;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StAreaSquareMeters;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StAsBinary;
-import org.apache.doris.nereids.trees.expressions.functions.scalar.StAsEWKB;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StAstext;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StAswkt;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StAzimuth;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StCircle;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StContains;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StDistanceSphere;
-import org.apache.doris.nereids.trees.expressions.functions.scalar.StGeomFromEWKB;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StGeomFromWKB;
-import org.apache.doris.nereids.trees.expressions.functions.scalar.StGeometryFromEWKB;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StGeometryFromWKB;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StGeometryfromtext;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StGeomfromtext;
@@ -1446,26 +1443,14 @@ public interface ScalarFunctionVisitor<R, C> {
         return visitScalarFunction(stGeometryfromwkb, context);
     }
 
-    default R visitStGeometryfromewkb(StGeometryFromEWKB stGeometryfromewkb, C context) {
-        return visitScalarFunction(stGeometryfromewkb, context);
-    }
-
     default R visitStGeomfromwkb(StGeomFromWKB stGeomfromwkb, C context) {
         return visitScalarFunction(stGeomfromwkb, context);
     }
 
-    default R visitStGeomfromewkb(StGeomFromEWKB stGeomfromewkb, C context) {
-        return visitScalarFunction(stGeomfromewkb, context);
-    }
-
     default R visitStAsBinary(StAsBinary stAsBinary, C context) {
         return visitScalarFunction(stAsBinary, context);
     }
 
-    default R visitStAsEWKB(StAsEWKB stAsEWKB, C context) {
-        return visitScalarFunction(stAsEWKB, context);
-    }
-
     default R visitStartsWith(StartsWith startsWith, C context) {
         return visitScalarFunction(startsWith, context);
     }
diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py
index e4c7948234..fae68def16 100644
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -1781,15 +1781,9 @@ visible_functions = [
     [['ST_GeometryFromWkb', 'ST_GeomFromWkb'], 'VARCHAR', ['VARCHAR'], 'ALWAYS_NULLABLE'],
     [['ST_GeometryFromWkb', 'ST_GeomFromWkb'], 'VARCHAR', ['STRING'], 'ALWAYS_NULLABLE'],
 
-    [['ST_GeometryFromEwkb', 'ST_GeomFromEwkb'], 'VARCHAR', ['VARCHAR'], 'ALWAYS_NULLABLE'],
-    [['ST_GeometryFromEwkb', 'ST_GeomFromEwkb'], 'VARCHAR', ['STRING'], 'ALWAYS_NULLABLE'],
-
     [['ST_AsBinary'], 'VARCHAR', ['VARCHAR'], 'ALWAYS_NULLABLE'],
     [['ST_AsBinary'], 'VARCHAR', ['STRING'], 'ALWAYS_NULLABLE'],
 
-    [['ST_AsEwkb'], 'VARCHAR', ['VARCHAR'], 'ALWAYS_NULLABLE'],
-    [['ST_AsEwkb'], 'VARCHAR', ['STRING'], 'ALWAYS_NULLABLE'],
-
     [['ST_LineFromText', 'ST_LineStringFromText'], 'VARCHAR', ['VARCHAR'], 'ALWAYS_NULLABLE'],
     [['ST_LineFromText', 'ST_LineStringFromText'], 'VARCHAR', ['STRING'], 'ALWAYS_NULLABLE'],
 
diff --git a/regression-test/data/nereids_function_p0/scalar_function/S.out b/regression-test/data/nereids_function_p0/scalar_function/S.out
index 36444ceb21..a6380e0379 100644
--- a/regression-test/data/nereids_function_p0/scalar_function/S.out
+++ b/regression-test/data/nereids_function_p0/scalar_function/S.out
@@ -1452,7 +1452,7 @@ true
 -- !sql_st_angle_Varchar_Varchar --
 \N
 0.29640805436890805
-0.5845576630671012
+0.584557663067101
 2.2514343365321805
 2.2985768978250287
 2.3435868570715472
@@ -1461,13 +1461,13 @@ true
 2.449573624637989
 3.0241913973486003
 4.933732381376881
-6.0197433678512455
+6.019743367851245
 6.186722861065897
 
 -- !sql_st_angle_Varchar_Varchar_notnull --
 \N
 0.29640805436890805
-0.5845576630671012
+0.584557663067101
 2.2514343365321805
 2.2985768978250287
 2.3435868570715472
@@ -1476,7 +1476,7 @@ true
 2.449573624637989
 3.0241913973486003
 4.933732381376881
-6.0197433678512455
+6.019743367851245
 6.186722861065897
 
 -- !sql_st_azimuth_Varchar_Varchar --
@@ -2064,7 +2064,7 @@ POLYGON ((4 4, 45 4, 45 45, 4 45, 4 4))
 \N
 -74.35620117
 -46.35620117
-4.999999999999999
+5.0
 16.35620117
 43.35620117
 47.35620117
@@ -2078,7 +2078,7 @@ POLYGON ((4 4, 45 4, 45 45, 4 45, 4 4))
 -- !sql_st_x_Varchar_notnull --
 -74.35620117
 -46.35620117
-4.999999999999999
+5.0
 16.35620117
 43.35620117
 47.35620117
@@ -2093,7 +2093,7 @@ POLYGON ((4 4, 45 4, 45 45, 4 45, 4 4))
 \N
 -74.35620117
 -46.35620117
-4.999999999999999
+5.0
 16.35620117
 43.35620117
 47.35620117
@@ -2107,7 +2107,7 @@ POLYGON ((4 4, 45 4, 45 45, 4 45, 4 4))
 -- !sql_st_x_String_notnull --
 -74.35620117
 -46.35620117
-4.999999999999999
+5.0
 16.35620117
 43.35620117
 47.35620117
@@ -2121,89 +2121,89 @@ POLYGON ((4 4, 45 4, 45 45, 4 45, 4 4))
 -- !sql_st_y_Varchar --
 \N
 -39.939093
-4.999999999999999
+5.0
 19.939093
-26.939092999999996
+26.939093
 35.939093
 36.939093
 39.939093
 39.939093
 47.939093
-49.93909299999999
+49.939093
 59.939093
 79.939093
 
 -- !sql_st_y_Varchar_notnull --
 -39.939093
-4.999999999999999
+5.0
 19.939093
-26.939092999999996
+26.939093
 35.939093
 36.939093
 39.939093
 39.939093
 47.939093
-49.93909299999999
+49.939093
 59.939093
 79.939093
 
 -- !sql_st_y_String --
 \N
 -39.939093
-4.999999999999999
+5.0
 19.939093
-26.939092999999996
+26.939093
 35.939093
 36.939093
 39.939093
 39.939093
 47.939093
-49.93909299999999
+49.939093
 59.939093
 79.939093
 
 -- !sql_st_y_String_notnull --
 -39.939093
-4.999999999999999
+5.0
 19.939093
-26.939092999999996
+26.939093
 35.939093
 36.939093
 39.939093
 39.939093
 47.939093
-49.93909299999999
+49.939093
 59.939093
 79.939093
 
 -- !sql_st_asbinary_Varchar --
 \N
-POINT (-46.35620117 39.939093)
-POINT (-74.35620117 79.939093)
-POINT (126.35620117 -39.939093)
-POINT (16.35620117 19.939093)
-POINT (43.35620117 35.939093)
-POINT (47.35620117 26.939093)
-POINT (5 5)
-POINT (90.35620117 39.939093)
-POINT (90.35620117 47.939093)
-POINT (90.35620117 49.939093)
-POINT (90.35620117 59.939093)
-POINT (98.35620117 36.939093)
+\\x010100000000000000000014400000000000001440
+\\x010100000035f9fbff972d47c086730d3334f84340
+\\x010100000035f9fbff97ad454086730d3334f84140
+\\x010100000035f9fbff97ad47400ce71a6668f03a40
+\\x01010000006af2f7ff2f5b30400ce71a6668f03340
+\\x01010000009bfcfdffcb9652c0c3b986191afc5340
+\\x01010000009bfcfdffcb96564086730d3334f84340
+\\x01010000009bfcfdffcb96564086730d3334f84740
+\\x01010000009bfcfdffcb96564086730d3334f84840
+\\x01010000009bfcfdffcb96564086730d3334f84d40
+\\x01010000009bfcfdffcb96584086730d3334784240
+\\x01010000009bfcfdffcb965f4086730d3334f843c0
 
 -- !sql_st_asbinary_Varchar_notnull --
-POINT (-46.35620117 39.939093)
-POINT (-74.35620117 79.939093)
-POINT (126.35620117 -39.939093)
-POINT (16.35620117 19.939093)
-POINT (43.35620117 35.939093)
-POINT (47.35620117 26.939093)
-POINT (5 5)
-POINT (90.35620117 39.939093)
-POINT (90.35620117 47.939093)
-POINT (90.35620117 49.939093)
-POINT (90.35620117 59.939093)
-POINT (98.35620117 36.939093)
+\\x010100000000000000000014400000000000001440
+\\x010100000035f9fbff972d47c086730d3334f84340
+\\x010100000035f9fbff97ad454086730d3334f84140
+\\x010100000035f9fbff97ad47400ce71a6668f03a40
+\\x01010000006af2f7ff2f5b30400ce71a6668f03340
+\\x01010000009bfcfdffcb9652c0c3b986191afc5340
+\\x01010000009bfcfdffcb96564086730d3334f84340
+\\x01010000009bfcfdffcb96564086730d3334f84740
+\\x01010000009bfcfdffcb96564086730d3334f84840
+\\x01010000009bfcfdffcb96564086730d3334f84d40
+\\x01010000009bfcfdffcb96584086730d3334784240
+\\x01010000009bfcfdffcb965f4086730d3334f843c0
 
 -- !sql_st_geometryfromwkb_Varchar --
 \N
@@ -2263,93 +2263,6 @@ POLYGON ((1 1, 8 0, 48 34, 4 10, 1 1))
 POLYGON ((4 1, 10 4, 9 4, 1 1, 4 1))
 POLYGON ((4 4, 45 4, 45 45, 4 45, 4 4))
 
--- !sql_st_asewkb_Varchar --
-\N
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-
--- !sql_st_asewkb_Varchar_notnull --
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-LINESTRING (1 1, 2 2)
-
--- !sql_st_geometryfromewkb_Varchar --
-\N
-POLYGON ((0 0, 10 0, 10 10, 0 10, 0 0))
-POLYGON ((0 0, 12 0, 10 16, 1 10, 0 0))
-POLYGON ((0 0, 38 4, 38 37, 4 1, 0 0))
-POLYGON ((1 1, 11 4, 42 44, 4 18, 1 1))
-POLYGON ((1 1, 16 1, 16 16, 1 16, 1 1))
-POLYGON ((1 1, 34 1, 34 34, 1 34, 1 1))
-POLYGON ((1 1, 4 1, 4 4, 1 4, 1 1))
-POLYGON ((1 1, 5 0, 10 4, 4 3, 1 1))
-POLYGON ((1 1, 56 0, 67 89, 4 32, 1 1))
-POLYGON ((1 1, 8 0, 48 34, 4 10, 1 1))
-POLYGON ((4 1, 10 4, 9 4, 1 1, 4 1))
-POLYGON ((4 4, 45 4, 45 45, 4 45, 4 4))
-
--- !sql_st_geometryfromewkb_Varchar_notnull --
-POLYGON ((0 0, 10 0, 10 10, 0 10, 0 0))
-POLYGON ((0 0, 12 0, 10 16, 1 10, 0 0))
-POLYGON ((0 0, 38 4, 38 37, 4 1, 0 0))
-POLYGON ((1 1, 11 4, 42 44, 4 18, 1 1))
-POLYGON ((1 1, 16 1, 16 16, 1 16, 1 1))
-POLYGON ((1 1, 34 1, 34 34, 1 34, 1 1))
-POLYGON ((1 1, 4 1, 4 4, 1 4, 1 1))
-POLYGON ((1 1, 5 0, 10 4, 4 3, 1 1))
-POLYGON ((1 1, 56 0, 67 89, 4 32, 1 1))
-POLYGON ((1 1, 8 0, 48 34, 4 10, 1 1))
-POLYGON ((4 1, 10 4, 9 4, 1 1, 4 1))
-POLYGON ((4 4, 45 4, 45 45, 4 45, 4 4))
-
--- !sql_st_geomfromewkb_Varchar --
-\N
-POLYGON ((0 0, 10 0, 10 10, 0 10, 0 0))
-POLYGON ((0 0, 12 0, 10 16, 1 10, 0 0))
-POLYGON ((0 0, 38 4, 38 37, 4 1, 0 0))
-POLYGON ((1 1, 11 4, 42 44, 4 18, 1 1))
-POLYGON ((1 1, 16 1, 16 16, 1 16, 1 1))
-POLYGON ((1 1, 34 1, 34 34, 1 34, 1 1))
-POLYGON ((1 1, 4 1, 4 4, 1 4, 1 1))
-POLYGON ((1 1, 5 0, 10 4, 4 3, 1 1))
-POLYGON ((1 1, 56 0, 67 89, 4 32, 1 1))
-POLYGON ((1 1, 8 0, 48 34, 4 10, 1 1))
-POLYGON ((4 1, 10 4, 9 4, 1 1, 4 1))
-POLYGON ((4 4, 45 4, 45 45, 4 45, 4 4))
-
--- !sql_st_geomfromewkb_Varchar_notnull --
-POLYGON ((0 0, 10 0, 10 10, 0 10, 0 0))
-POLYGON ((0 0, 12 0, 10 16, 1 10, 0 0))
-POLYGON ((0 0, 38 4, 38 37, 4 1, 0 0))
-POLYGON ((1 1, 11 4, 42 44, 4 18, 1 1))
-POLYGON ((1 1, 16 1, 16 16, 1 16, 1 1))
-POLYGON ((1 1, 34 1, 34 34, 1 34, 1 1))
-POLYGON ((1 1, 4 1, 4 4, 1 4, 1 1))
-POLYGON ((1 1, 5 0, 10 4, 4 3, 1 1))
-POLYGON ((1 1, 56 0, 67 89, 4 32, 1 1))
-POLYGON ((1 1, 8 0, 48 34, 4 10, 1 1))
-POLYGON ((4 1, 10 4, 9 4, 1 1, 4 1))
-POLYGON ((4 4, 45 4, 45 45, 4 45, 4 4))
-
 -- !sql_starts_with_Varchar_Varchar --
 true
 true
diff --git a/regression-test/data/nereids_p0/sql_functions/spatial_functions/test_gis_function.out b/regression-test/data/nereids_p0/sql_functions/spatial_functions/test_gis_function.out
index 0f1e87fa78..db1b1ffcae 100644
--- a/regression-test/data/nereids_p0/sql_functions/spatial_functions/test_gis_function.out
+++ b/regression-test/data/nereids_p0/sql_functions/spatial_functions/test_gis_function.out
@@ -113,21 +113,3 @@ LINESTRING (1 1, 2 2)
 -- !sql --
 POLYGON ((114.104486 22.547119, 114.093758 22.547753, 114.096504 22.532057, 114.104229 22.539826, 114.106203 22.54268, 114.104486 22.547119))
 
--- !sql --
-POINT (24.7 56.7)
-
--- !sql --
-LINESTRING (1 1, 2 2)
-
--- !sql --
-POLYGON ((114.104486 22.547119, 114.093758 22.547753, 114.096504 22.532057, 114.104229 22.539826, 114.106203 22.54268, 114.104486 22.547119))
-
--- !sql --
-POINT (24.7 56.7)
-
--- !sql --
-LINESTRING (1 1, 2 2)
-
--- !sql --
-POLYGON ((114.104486 22.547119, 114.093758 22.547753, 114.096504 22.532057, 114.104229 22.539826, 114.106203 22.54268, 114.104486 22.547119))
-
diff --git a/regression-test/data/query_p0/sql_functions/spatial_functions/test_gis_function.out b/regression-test/data/query_p0/sql_functions/spatial_functions/test_gis_function.out
index 0f1e87fa78..db1b1ffcae 100644
--- a/regression-test/data/query_p0/sql_functions/spatial_functions/test_gis_function.out
+++ b/regression-test/data/query_p0/sql_functions/spatial_functions/test_gis_function.out
@@ -113,21 +113,3 @@ LINESTRING (1 1, 2 2)
 -- !sql --
 POLYGON ((114.104486 22.547119, 114.093758 22.547753, 114.096504 22.532057, 114.104229 22.539826, 114.106203 22.54268, 114.104486 22.547119))
 
--- !sql --
-POINT (24.7 56.7)
-
--- !sql --
-LINESTRING (1 1, 2 2)
-
--- !sql --
-POLYGON ((114.104486 22.547119, 114.093758 22.547753, 114.096504 22.532057, 114.104229 22.539826, 114.106203 22.54268, 114.104486 22.547119))
-
--- !sql --
-POINT (24.7 56.7)
-
--- !sql --
-LINESTRING (1 1, 2 2)
-
--- !sql --
-POLYGON ((114.104486 22.547119, 114.093758 22.547753, 114.096504 22.532057, 114.104229 22.539826, 114.106203 22.54268, 114.104486 22.547119))
-
diff --git a/regression-test/suites/nereids_function_p0/scalar_function/S.groovy b/regression-test/suites/nereids_function_p0/scalar_function/S.groovy
index c04793374f..ea6cc5c817 100644
--- a/regression-test/suites/nereids_function_p0/scalar_function/S.groovy
+++ b/regression-test/suites/nereids_function_p0/scalar_function/S.groovy
@@ -244,18 +244,12 @@ suite("nereids_scalar_fn_S") {
 	qt_sql_st_y_Varchar_notnull "select st_y(st_point(x_lng, x_lat)) from fn_test_not_nullable order by 1"
 	qt_sql_st_y_String "select st_y(st_point(x_lng, x_lat)) from fn_test order by 1"
 	qt_sql_st_y_String_notnull "select st_y(st_point(x_lng, x_lat)) from fn_test_not_nullable order by 1"
-	qt_sql_st_asbinary_Varchar "select ST_AsText(ST_GeometryFromWKB(ST_AsBinary(st_point(x_lng, x_lat)))) from fn_test order by 1"
-    qt_sql_st_asbinary_Varchar_notnull "select ST_AsText(ST_GeometryFromWKB(ST_AsBinary(st_point(x_lng, x_lat)))) from fn_test_not_nullable order by 1"
+	qt_sql_st_asbinary_Varchar "select ST_AsBinary(st_point(x_lng, x_lat)) from fn_test order by 1"
+    qt_sql_st_asbinary_Varchar_notnull "select ST_AsBinary(st_point(x_lng, x_lat)) from fn_test_not_nullable order by 1"
 	qt_sql_st_geometryfromwkb_Varchar "select ST_AsText(ST_GeometryFromWKB(ST_AsBinary(st_polyfromtext(polygon_wkt)))) from fn_test order by 1"
     qt_sql_st_geometryfromwkb_Varchar_notnull "select ST_AsText(ST_GeometryFromWKB(ST_AsBinary(st_polyfromtext(polygon_wkt)))) from fn_test_not_nullable order by 1"
 	qt_sql_st_geomfromwkb_Varchar "select ST_AsText(ST_GeomFromWKB(ST_AsBinary(st_polyfromtext(polygon_wkt)))) from fn_test order by 1"
     qt_sql_st_geomfromwkb_Varchar_notnull "select ST_AsText(ST_GeomFromWKB(ST_AsBinary(st_polyfromtext(polygon_wkt)))) from fn_test_not_nullable order by 1"
-	qt_sql_st_asewkb_Varchar "select ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(st_linestringfromtext(linestring_wkt)))) from fn_test order by 1"
-    qt_sql_st_asewkb_Varchar_notnull "select ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(st_linestringfromtext(linestring_wkt)))) from fn_test_not_nullable order by 1"
-	qt_sql_st_geometryfromewkb_Varchar "select ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(st_polyfromtext(polygon_wkt)))) from fn_test order by 1"
-    qt_sql_st_geometryfromewkb_Varchar_notnull "select ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(st_polyfromtext(polygon_wkt)))) from fn_test_not_nullable order by 1"
-	qt_sql_st_geomfromewkb_Varchar "select ST_AsText(ST_GeomFromEWKB(ST_AsEWKB(st_polyfromtext(polygon_wkt)))) from fn_test order by 1"
-    qt_sql_st_geomfromewkb_Varchar_notnull "select ST_AsText(ST_GeomFromEWKB(ST_AsEWKB(st_polyfromtext(polygon_wkt)))) from fn_test_not_nullable order by 1"
 
 	qt_sql_starts_with_Varchar_Varchar "select starts_with(kvchrs1, kvchrs1) from fn_test order by kvchrs1, kvchrs1"
 	qt_sql_starts_with_Varchar_Varchar_notnull "select starts_with(kvchrs1, kvchrs1) from fn_test_not_nullable order by kvchrs1, kvchrs1"
diff --git a/regression-test/suites/nereids_p0/sql_functions/spatial_functions/test_gis_function.groovy b/regression-test/suites/nereids_p0/sql_functions/spatial_functions/test_gis_function.groovy
index 645e01ba41..d50ac65719 100644
--- a/regression-test/suites/nereids_p0/sql_functions/spatial_functions/test_gis_function.groovy
+++ b/regression-test/suites/nereids_p0/sql_functions/spatial_functions/test_gis_function.groovy
@@ -71,12 +71,4 @@ suite("test_gis_function") {
     qt_sql "SELECT ST_AsText(ST_GeomFromWKB(ST_AsBinary(ST_GeometryFromText(\"LINESTRING (1 1, 2 2)\"))));"
     qt_sql "SELECT ST_AsText(ST_GeomFromWKB(ST_AsBinary(ST_Polygon(\"POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))\"))));"
 
-    qt_sql "SELECT ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(ST_Point(24.7, 56.7))));"
-    qt_sql "SELECT ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(ST_GeometryFromText(\"LINESTRING (1 1, 2 2)\"))));"
-    qt_sql "SELECT ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(ST_Polygon(\"POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))\"))));"
-
-    qt_sql "SELECT ST_AsText(ST_GeomFromEWKB(ST_AsEWKB(ST_Point(24.7, 56.7))));"
-    qt_sql "SELECT ST_AsText(ST_GeomFromEWKB(ST_AsEWKB(ST_GeometryFromText(\"LINESTRING (1 1, 2 2)\"))));"
-    qt_sql "SELECT ST_AsText(ST_GeomFromEWKB(ST_AsEWKB(ST_Polygon(\"POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))\"))));"
-
 }
diff --git a/regression-test/suites/query_p0/sql_functions/spatial_functions/test_gis_function.groovy b/regression-test/suites/query_p0/sql_functions/spatial_functions/test_gis_function.groovy
index 5e38c6e8bd..e98e11ba7e 100644
--- a/regression-test/suites/query_p0/sql_functions/spatial_functions/test_gis_function.groovy
+++ b/regression-test/suites/query_p0/sql_functions/spatial_functions/test_gis_function.groovy
@@ -70,11 +70,4 @@ suite("test_gis_function") {
     qt_sql "SELECT ST_AsText(ST_GeomFromWKB(ST_AsBinary(ST_GeometryFromText(\"LINESTRING (1 1, 2 2)\"))));"
     qt_sql "SELECT ST_AsText(ST_GeomFromWKB(ST_AsBinary(ST_Polygon(\"POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))\"))));"
 
-    qt_sql "SELECT ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(ST_Point(24.7, 56.7))));"
-    qt_sql "SELECT ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(ST_GeometryFromText(\"LINESTRING (1 1, 2 2)\"))));"
-    qt_sql "SELECT ST_AsText(ST_GeometryFromEWKB(ST_AsEWKB(ST_Polygon(\"POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))\"))));"
-
-    qt_sql "SELECT ST_AsText(ST_GeomFromEWKB(ST_AsEWKB(ST_Point(24.7, 56.7))));"
-    qt_sql "SELECT ST_AsText(ST_GeomFromEWKB(ST_AsEWKB(ST_GeometryFromText(\"LINESTRING (1 1, 2 2)\"))));"
-    qt_sql "SELECT ST_AsText(ST_GeomFromEWKB(ST_AsEWKB(ST_Polygon(\"POLYGON ((114.104486 22.547119,114.093758 22.547753,114.096504 22.532057,114.104229 22.539826,114.106203 22.542680,114.104486 22.547119))\"))));"
 }


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