You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by cs...@apache.org on 2023/02/07 20:46:03 UTC

[impala] branch master updated: IMPALA-11745: Add Hive's ESRI geospatial functions as builtins

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 1d05381b7 IMPALA-11745: Add Hive's ESRI geospatial functions as builtins
1d05381b7 is described below

commit 1d05381b7b791bde5820572b3e7a4b2b5db1db73
Author: Peter Rozsa <pr...@cloudera.com>
AuthorDate: Fri Jan 6 16:19:53 2023 +0100

    IMPALA-11745: Add Hive's ESRI geospatial functions as builtins
    
    This change adds geospatial functions from Hive's ESRI library
    as builtin UDFs. Plain Hive UDFs are imported without changes,
    but the generic and varargs functions are handled differently;
    generic functions are added with all of the combinations of
    their parameters (cartesian product of the parameters), and
    varargs functions are unfolded as an nth parameter simple
    function. The varargs function wrappers are generated at build
    time and they can be configured in
    gen_geospatial_udf_wrappers.py. These additional steps are
    required because of the limitations in Impala's UDF Executor
    (lack of varargs support and only partial generics support)
    which could be further improved; in this case, the additional
    wrapping/mapping steps could be removed.
    
    Changes regarding function handling/creating are sourced from
    https://gerrit.cloudera.org/c/19177
    
    A new backend flag was added to turn this feature on/off
    as "geospatial_library". The default value is "NONE" which
    means no geospatial function gets registered
    as builtin, "HIVE_ESRI" value enables this implementation.
    
    The ESRI geospatial implementation for Hive currently only
    available in Hive 4, but CDP Hive backported it to Hive 3,
    therefore for Apache Hive this feature is disabled
    regardless of the "geospatial_library" flag.
    
    Known limitations:
     - ST_MultiLineString, ST_MultiPolygon only works
       with the WKT overload
     - ST_Polygon supports a maximum of 6 pairs of coordinates
     - ST_MultiPoint, ST_LineString supports a maximum of 7
       pairs of coordinates
     - ST_ConvexHull, ST_Union supports a maximum of 6 geoms
    
    These limits can be increased in gen_geospatial_udf_wrappers.py
    
    Tests:
     - test_geospatial_udfs.py added based on
       https://github.com/Esri/spatial-framework-for-hadoop
    
    Co-Authored-by: Csaba Ringhofer <cs...@cloudera.com>
    
    Change-Id: If0ca02a70b4ba244778c9db6d14df4423072b225
    Reviewed-on: http://gerrit.cloudera.org:8080/19425
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/common/global-flags.cc                      |   11 +
 be/src/exprs/hive-udf-call.cc                      |    9 +-
 be/src/util/backend-gflag-util.cc                  |    7 +
 bin/start-impala-cluster.py                        |    8 +
 common/function-registry/CMakeLists.txt            |   28 +
 common/function-registry/gen_builtins_catalog.py   |   22 +-
 .../gen_geospatial_udf_wrappers.py                 |  161 ++
 common/thrift/BackendGflags.thrift                 |    9 +
 .../impala/compat/HiveEsriGeospatialBuiltins.java} |   23 +-
 .../impala/compat/HiveEsriGeospatialBuiltins.java  |  244 ++
 .../java/org/apache/impala/catalog/BuiltinsDb.java |    9 +-
 .../impala/catalog/CatalogServiceCatalog.java      |    6 +-
 .../org/apache/impala/catalog/ScalarFunction.java  |    4 +-
 ...BinaryToBinaryHiveLegacyFunctionExtractor.java} |   32 +-
 .../hive/executor/HiveGenericJavaFunction.java     |   29 +-
 .../impala/hive/executor/HiveJavaFunction.java     |   12 +-
 .../hive/executor/HiveJavaFunctionFactory.java     |   10 +-
 .../hive/executor/HiveJavaFunctionFactoryImpl.java |   40 +-
 .../hive/executor/HiveLegacyFunctionExtractor.java |   98 +
 .../hive/executor/HiveLegacyJavaFunction.java      |   70 +-
 .../impala/hive/executor/ImpalaDoubleWritable.java |    5 +
 .../org/apache/impala/service/BackendConfig.java   |    6 +-
 .../apache/impala/service/CatalogOpExecutor.java   |    8 +-
 .../java/org/apache/impala/service/JniCatalog.java |    3 +-
 .../hive/executor/HiveLegacyJavaFunctionTest.java  |    3 +-
 .../hive/executor/TestHiveJavaFunctionFactory.java |   14 +-
 java/CMakeLists.txt                                |    2 +-
 java/shaded-deps/hive-exec/pom.xml                 |    1 +
 testdata/datasets/README                           |    2 +-
 .../queries/QueryTest/geospatial-esri.test         | 2718 ++++++++++++++++++++
 tests/common/environ.py                            |    2 +
 tests/common/skip.py                               |    8 +-
 tests/custom_cluster/test_geospatial_library.py    |   44 +
 .../query_test/test_geospatial_functions.py        |   18 +-
 34 files changed, 3476 insertions(+), 190 deletions(-)

diff --git a/be/src/common/global-flags.cc b/be/src/common/global-flags.cc
index c427ed595..d54cb4612 100644
--- a/be/src/common/global-flags.cc
+++ b/be/src/common/global-flags.cc
@@ -375,6 +375,17 @@ DEFINE_bool(pull_table_types_and_comments, false,
     "catalogd-only flag. Required if users want GET_TABLES requests return correct table "
     "types or comments.");
 
+// TGeospatialLibrary's values are mapped here as constants
+static const string geo_lib_none = "NONE";
+static const string geo_lib_hive_esri = "HIVE_ESRI";
+
+static const string geo_lib_help_msg =
+    "Specifies which implementation of "
+    "geospatial functions should be included as builtins. Possible values: [\""
+    + geo_lib_none + "\", \"" + geo_lib_hive_esri + "\"]";
+
+DEFINE_string(geospatial_library, geo_lib_none, geo_lib_help_msg.c_str());
+
 // ++========================++
 // || Startup flag graveyard ||
 // ++========================++
diff --git a/be/src/exprs/hive-udf-call.cc b/be/src/exprs/hive-udf-call.cc
index b5e21c880..f8b5583e6 100644
--- a/be/src/exprs/hive-udf-call.cc
+++ b/be/src/exprs/hive-udf-call.cc
@@ -192,8 +192,13 @@ Status HiveUdfCall::OpenEvaluator(FunctionContext::FunctionStateScope scope,
     // Scoped handle for libCache entry.
     LibCacheEntryHandle handle;
     string local_location;
-    RETURN_IF_ERROR(LibCache::instance()->GetLocalPath(fn_.hdfs_location,
-        LibCache::TYPE_JAR, fn_.last_modified_time, &handle, &local_location));
+    // Hive UDFs added as builtins are assumed to be always available in the classpath,
+    // therefore no LibCache invocation is needed.
+    bool builtin_udf = fn_.hdfs_location.empty();
+    if (!builtin_udf) {
+      RETURN_IF_ERROR(LibCache::instance()->GetLocalPath(fn_.hdfs_location,
+          LibCache::TYPE_JAR, fn_.last_modified_time, &handle, &local_location));
+    }
     THiveUdfExecutorCtorParams ctor_params;
     ctor_params.fn = fn_;
     ctor_params.local_location = local_location;
diff --git a/be/src/util/backend-gflag-util.cc b/be/src/util/backend-gflag-util.cc
index a2bf8bae7..1e8b9683f 100644
--- a/be/src/util/backend-gflag-util.cc
+++ b/be/src/util/backend-gflag-util.cc
@@ -102,6 +102,7 @@ DECLARE_bool(auto_check_compaction);
 DECLARE_bool(enable_sync_to_latest_event_on_ddls);
 DECLARE_bool(pull_table_types_and_comments);
 DECLARE_bool(enable_reload_events);
+DECLARE_string(geospatial_library);
 
 // HS2 SAML2.0 configuration
 // Defined here because TAG_FLAG caused issues in global-flags.cc
@@ -337,6 +338,12 @@ Status PopulateThriftBackendGflags(TBackendGflags& cfg) {
       FLAGS_use_hms_column_order_for_hbase_tables);
   cfg.__set_ignored_dir_prefix_list(FLAGS_ignored_dir_prefix_list);
   cfg.__set_enable_reload_events(FLAGS_enable_reload_events);
+  if (FLAGS_geospatial_library == to_string(TGeospatialLibrary::NONE)) {
+    cfg.__set_geospatial_library(TGeospatialLibrary::NONE);
+  } else {
+    DCHECK_EQ(FLAGS_geospatial_library, to_string(TGeospatialLibrary::HIVE_ESRI));
+    cfg.__set_geospatial_library(TGeospatialLibrary::HIVE_ESRI);
+  }
   return Status::OK();
 }
 
diff --git a/bin/start-impala-cluster.py b/bin/start-impala-cluster.py
index 99d901f4d..d4295a165 100755
--- a/bin/start-impala-cluster.py
+++ b/bin/start-impala-cluster.py
@@ -141,6 +141,10 @@ parser.add_option("--enable_admission_service", dest="enable_admission_service",
 parser.add_option("--enable_external_fe_support", dest="enable_external_fe_support",
                   action="store_true", default=False,
                   help="If true, impalads will start with the external_fe_port defined.")
+parser.add_option("--geospatial_library", dest="geospatial_library",
+                  action="store", default="HIVE_ESRI",
+                  help="Sets which implementation of geospatial libraries should be "
+                  "initialized")
 
 # For testing: list of comma-separated delays, in milliseconds, that delay impalad catalog
 # replica initialization. The ith delay is applied to the ith impalad.
@@ -423,6 +427,10 @@ def build_impalad_arg_lists(cluster_size, num_coordinators, use_exclusive_coordi
       args = "{args} -admission_service_host={host}".format(
           args=args, host=admissiond_host)
 
+    if "geospatial_library" not in args:
+      args = "{args} -geospatial_library={geospatial_library}".format(
+          args=args, geospatial_library=options.geospatial_library)
+
     # Appended at the end so they can override previous args.
     if i < len(per_impalad_args):
       args = "{args} {per_impalad_args}".format(
diff --git a/common/function-registry/CMakeLists.txt b/common/function-registry/CMakeLists.txt
index 23ae52463..0c9274f1c 100644
--- a/common/function-registry/CMakeLists.txt
+++ b/common/function-registry/CMakeLists.txt
@@ -23,12 +23,25 @@ set(CODE_GEN_OUTPUT
   ${FE_OUTPUT_DIR}/builtins/ScalarBuiltins.java
 )
 
+set(GEOSPATIAL_CODE_GEN_OUTPUT
+  ${FE_OUTPUT_DIR}/builtins/ST_ConvexHull_Wrapper.java
+  ${FE_OUTPUT_DIR}/builtins/ST_LineString_Wrapper.java
+  ${FE_OUTPUT_DIR}/builtins/ST_MultiPoint_Wrapper.java
+  ${FE_OUTPUT_DIR}/builtins/ST_Polygon_Wrapper.java
+  ${FE_OUTPUT_DIR}/builtins/ST_Union_Wrapper.java
+)
+
 # Source python files
 set(FUNCTION_REGISTRY_INPUT
   gen_builtins_catalog.py
   impala_functions.py
 )
 
+set(GEOSPATIAL_WRAPPER_INPUT
+  gen_builtins_catalog.py
+  gen_geospatial_udf_wrappers.py
+)
+
 # Run the python scripts
 add_custom_command(
   OUTPUT ${CODE_GEN_OUTPUT}
@@ -38,5 +51,20 @@ add_custom_command(
   VERBATIM
 )
 
+add_custom_command(
+  OUTPUT ${GEOSPATIAL_CODE_GEN_OUTPUT}
+  COMMAND ./gen_geospatial_udf_wrappers.py
+  DEPENDS ${GEOSPATIAL_WRAPPER_INPUT}
+  COMMENT "Generating UDF wrappers for geospatial builtins."
+  VERBATIM
+)
+
 add_custom_target(function-registry ALL DEPENDS ${CODE_GEN_OUTPUT})
 
+if( $ENV{USE_APACHE_HIVE} STREQUAL "false")
+  add_custom_target(geospatial-udf-wrappers ALL DEPENDS ${GEOSPATIAL_CODE_GEN_OUTPUT})
+  add_dependencies(geospatial-udf-wrappers function-registry)
+else()
+  # Empty target to skip wrapper generation
+  add_custom_target(geospatial-udf-wrappers)
+endif()
diff --git a/common/function-registry/gen_builtins_catalog.py b/common/function-registry/gen_builtins_catalog.py
index 6f1cc22e3..6d472f5e3 100755
--- a/common/function-registry/gen_builtins_catalog.py
+++ b/common/function-registry/gen_builtins_catalog.py
@@ -20,9 +20,7 @@
 # This script generates the FE calls to populate the builtins.
 # To add a builtin, add an entry to impala_functions.py.
 
-import sys
 import os
-from string import Template
 import impala_functions
 
 java_registry_preamble = '\
@@ -65,6 +63,7 @@ FE_PATH = os.path.expandvars(
 #   - fn_name, ret_type, args, symbol, sql_names
 meta_data_entries = []
 
+
 # Read in the function and add it to the meta_data_entries map
 def add_function(fn_meta_data, user_visible):
   assert 4 <= len(fn_meta_data) <= 6, \
@@ -81,6 +80,7 @@ def add_function(fn_meta_data, user_visible):
   entry["user_visible"] = user_visible
   meta_data_entries.append(entry)
 
+
 def generate_fe_entry(entry, name):
   java_output = ""
   java_output += "\"" + name + "\""
@@ -109,6 +109,7 @@ def generate_fe_entry(entry, name):
     java_output += ", Type." + arg
   return java_output
 
+
 # Generates the FE builtins init file that registers all the builtins.
 def generate_fe_registry_init(filename):
   java_registry_file = open(filename, "w")
@@ -123,14 +124,15 @@ def generate_fe_registry_init(filename):
   java_registry_file.write(java_registry_epilogue)
   java_registry_file.close()
 
-# Read the function metadata inputs
-for function in impala_functions.visible_functions:
-  add_function(function, True)
-for function in impala_functions.invisible_functions:
-  add_function(function, False)
 
-if not os.path.exists(FE_PATH):
-  os.makedirs(FE_PATH)
+if __name__ == "__main__":
+  # Read the function metadata inputs
+  for function in impala_functions.visible_functions:
+    add_function(function, True)
+  for function in impala_functions.invisible_functions:
+    add_function(function, False)
 
-generate_fe_registry_init(FE_PATH + "ScalarBuiltins.java")
+  if not os.path.exists(FE_PATH):
+    os.makedirs(FE_PATH)
 
+  generate_fe_registry_init(FE_PATH + "ScalarBuiltins.java")
diff --git a/common/function-registry/gen_geospatial_udf_wrappers.py b/common/function-registry/gen_geospatial_udf_wrappers.py
new file mode 100755
index 000000000..73f85d78f
--- /dev/null
+++ b/common/function-registry/gen_geospatial_udf_wrappers.py
@@ -0,0 +1,161 @@
+#!/usr/bin/env python
+#
+# 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.
+
+
+# Java wrapper class generator for Hive ESRI varargs UDFs to bridge the varargs support
+# in Impala. A generated class is extending the original UDF and adding wrapper
+# 'evaluate' methods projecting the varargs method as an n parameter method.
+
+import os
+from gen_builtins_catalog import FE_PATH
+
+LICENSE = """
+// 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.
+
+// This is a generated file, DO NOT EDIT.
+// To add new functions, see the generator at
+// common/function-registry/gen_geospatial_builtins_wrappers.py"""
+
+
+ARGUMENT_EXCEPTION = "UDFArgumentException"
+ARGUMENT_LENGTH_EXCEPTION = "UDFArgumentLengthException"
+UDF_PACKAGE = "org.apache.hadoop.hive.ql.udf.esri"
+DOUBLE_TYPE = "org.apache.hadoop.hive.serde2.io.DoubleWritable"
+BYTE_TYPE = "org.apache.hadoop.io.BytesWritable"
+
+
+class Wrapper():
+
+  METHOD_FORMAT = ("""  public {return_type} evaluate({parameter_list}) {exception_clause}{{
+    return super.evaluate({argument_list});
+  }}""")
+
+  CLASS_FORMAT = """public class {class_name} extends {base_class} {{
+{methods}
+}}"""
+
+  FILE_FORMAT = """{license}
+
+package {package};
+
+{wrapper_class}"""
+
+  EXCEPTION_CLAUSE_FORMAT = "throws org.apache.hadoop.hive.ql.exec.%s"
+
+  def __init__(self, original_class, parameter_type, parameter_range, throws=None):
+    self.original_class = original_class
+    self.parameter_type = parameter_type
+    self.throws = throws
+    self.parameter_range = parameter_range
+    # Return type is always BytesWritable with the current cases
+    self.return_type = BYTE_TYPE
+
+  def generate_parameter(self, num):
+    return "{parameter_type} arg{num}".format(
+      parameter_type=self.parameter_type, num=num
+    )
+
+  def generate_argument(self, num):
+    return "arg%d" % num
+
+  def generate_argument_list(self, num):
+    arguments = list()
+    for i in range(num):
+      arguments.append(self.generate_argument(i))
+    return ", ".join(arguments)
+
+  def generate_parameter_list(self, num):
+    parameters = list()
+    for i in range(num):
+      parameters.append(self.generate_parameter(i))
+    return ", ".join(parameters)
+
+  def generate_method(self, num):
+    exception_clause = ""
+    if self.throws:
+      exception_clause = self.EXCEPTION_CLAUSE_FORMAT % self.throws
+    return self.METHOD_FORMAT.format(
+      return_type=self.return_type,
+      parameter_list=self.generate_parameter_list(num),
+      exception_clause=exception_clause,
+      argument_list=self.generate_argument_list(num),
+    )
+
+  def generate_methods(self):
+    methods = list()
+
+    for i in self.parameter_range:
+      methods.append(self.generate_method(i))
+
+    return "\n\n".join(methods)
+
+  def generate_wrapper_class(self):
+    return self.CLASS_FORMAT.format(
+      class_name=self.generate_wrapper_name(),
+      base_class=self.original_class,
+      methods=self.generate_methods()
+    )
+
+  def generate_file(self):
+    return self.FILE_FORMAT.format(
+      license=LICENSE,
+      package="org.apache.impala.builtins",
+      wrapper_class=self.generate_wrapper_class()
+    )
+
+  def generate_wrapper_name(self):
+    return "%s_Wrapper" % self.original_class.split('.').pop()
+
+  def get_filename(self):
+    return "%s.java" % self.generate_wrapper_name()
+
+
+WRAPPERS = [Wrapper("%s.ST_ConvexHull" % UDF_PACKAGE, BYTE_TYPE, range(2, 9, 1)),
+            Wrapper("%s.ST_LineString" % UDF_PACKAGE, DOUBLE_TYPE, range(2, 15, 2),
+              ARGUMENT_EXCEPTION),
+            Wrapper("%s.ST_MultiPoint" % UDF_PACKAGE, DOUBLE_TYPE, range(2, 15, 2),
+              ARGUMENT_LENGTH_EXCEPTION),
+            Wrapper("%s.ST_Polygon" % UDF_PACKAGE, DOUBLE_TYPE, range(6, 15, 2),
+              ARGUMENT_LENGTH_EXCEPTION),
+            Wrapper("%s.ST_Union" % UDF_PACKAGE, BYTE_TYPE, range(2, 15, 1))]
+
+if __name__ == "__main__":
+  if not os.path.exists(FE_PATH):
+    os.makedirs(FE_PATH)
+
+  for wrapper_config in WRAPPERS:
+    path = os.path.join(FE_PATH, wrapper_config.get_filename())
+    wrapper_class_file = open(path, "w")
+    wrapper_class_file.write(wrapper_config.generate_file())
+    wrapper_class_file.close()
diff --git a/common/thrift/BackendGflags.thrift b/common/thrift/BackendGflags.thrift
index bc09752f7..809c4f88a 100644
--- a/common/thrift/BackendGflags.thrift
+++ b/common/thrift/BackendGflags.thrift
@@ -23,6 +23,13 @@ enum TReservedWordsVersion {
   IMPALA_3_0 = 1
 }
 
+
+// Options for Geospatial function library support
+enum TGeospatialLibrary{
+  NONE,
+  HIVE_ESRI
+}
+
 // Used to pass gflags from backend to frontend, JniCatalog and JniFrontend
 // Attributes without comments correspond to gflags
 struct TBackendGflags {
@@ -231,4 +238,6 @@ struct TBackendGflags {
   103: required string ignored_dir_prefix_list
 
   104: required bool enable_reload_events
+
+  105: required TGeospatialLibrary geospatial_library
 }
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/ImpalaDoubleWritable.java b/fe/src/compat-apache-hive-3/java/org/apache/impala/compat/HiveEsriGeospatialBuiltins.java
similarity index 59%
copy from fe/src/main/java/org/apache/impala/hive/executor/ImpalaDoubleWritable.java
copy to fe/src/compat-apache-hive-3/java/org/apache/impala/compat/HiveEsriGeospatialBuiltins.java
index 2a0bd164a..b0b80762c 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/ImpalaDoubleWritable.java
+++ b/fe/src/compat-apache-hive-3/java/org/apache/impala/compat/HiveEsriGeospatialBuiltins.java
@@ -15,24 +15,15 @@
 // specific language governing permissions and limitations
 // under the License.
 
-package org.apache.impala.hive.executor;
+package org.apache.impala.compat;
 
-import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.impala.catalog.Db;
 
-import org.apache.impala.util.UnsafeUtil;
+public class HiveEsriGeospatialBuiltins {
+  /**
+   * Apache Hive 3 does not include geospatial functions
+   */
+  public static void initBuiltins(Db db) {
 
-@SuppressWarnings("restriction")
-public class ImpalaDoubleWritable extends DoubleWritable {
-  // Ptr (to native heap) where the value should be read from and written to.
-  private final long ptr_;
-
-  public ImpalaDoubleWritable(long ptr) {
-    ptr_ = ptr;
   }
-
-  @Override
-  public double get() { return UnsafeUtil.UNSAFE.getDouble(ptr_); }
-
-  @Override
-  public void set(double v) { UnsafeUtil.UNSAFE.putDouble(ptr_, v); }
 }
diff --git a/fe/src/compat-hive-3/java/org/apache/impala/compat/HiveEsriGeospatialBuiltins.java b/fe/src/compat-hive-3/java/org/apache/impala/compat/HiveEsriGeospatialBuiltins.java
new file mode 100644
index 000000000..8ce149942
--- /dev/null
+++ b/fe/src/compat-hive-3/java/org/apache/impala/compat/HiveEsriGeospatialBuiltins.java
@@ -0,0 +1,244 @@
+// 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.impala.compat;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Collections;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hive.ql.exec.UDF;
+import org.apache.hadoop.hive.ql.udf.esri.*;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.impala.builtins.ST_ConvexHull_Wrapper;
+import org.apache.impala.builtins.ST_LineString_Wrapper;
+import org.apache.impala.builtins.ST_MultiPoint_Wrapper;
+import org.apache.impala.builtins.ST_Polygon_Wrapper;
+import org.apache.impala.builtins.ST_Union_Wrapper;
+import org.apache.impala.catalog.BuiltinsDb;
+import org.apache.impala.catalog.CatalogException;
+import org.apache.impala.catalog.Db;
+import org.apache.impala.catalog.Function;
+import org.apache.impala.catalog.ScalarFunction;
+import org.apache.impala.catalog.Type;
+import org.apache.impala.hive.executor.BinaryToBinaryHiveLegacyFunctionExtractor;
+import org.apache.impala.hive.executor.HiveJavaFunction;
+import org.apache.impala.hive.executor.HiveLegacyJavaFunction;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.impala.analysis.FunctionName;
+import org.apache.impala.thrift.TFunctionBinaryType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HiveEsriGeospatialBuiltins {
+  /**
+   * Initializes Hive's ESRI geospatial UDFs as builtins.
+   */
+  public static void initBuiltins(Db db) {
+    addLegacyUDFs(db);
+    addGenericUDFs(db);
+    addVarargsUDFs(db);
+    addWorkaroundForStSetSrid(db);
+  }
+
+  private static void addLegacyUDFs(Db db) {
+    List<UDF> legacyUDFs = Arrays.asList(new ST_Area(), new ST_AsBinary(),
+        new ST_AsGeoJson(), new ST_AsJson(), new ST_AsShape(), new ST_AsText(),
+        new ST_Boundary(), new ST_Buffer(), new ST_Centroid(), new ST_CoordDim(),
+        new ST_Difference(), new ST_Dimension(), new ST_Distance(), new ST_EndPoint(),
+        new ST_Envelope(), new ST_EnvIntersects(), new ST_ExteriorRing(),
+        new ST_GeodesicLengthWGS84(), new ST_GeomCollection(), new ST_GeometryN(),
+        new ST_GeometryType(), new ST_GeomFromShape(), new ST_GeomFromText(),
+        new ST_GeomFromWKB(), new ST_InteriorRingN(), new ST_Intersection(),
+        new ST_Is3D(), new ST_IsClosed(), new ST_IsEmpty(), new ST_IsMeasured(),
+        new ST_IsRing(), new ST_IsSimple(), new ST_Length(), new ST_LineFromWKB(),
+        new ST_M(), new ST_MaxM(), new ST_MaxX(), new ST_MaxY(), new ST_MaxZ(),
+        new ST_MinM(), new ST_MinX(), new ST_MinY(), new ST_MinZ(), new ST_MLineFromWKB(),
+        new ST_MPointFromWKB(), new ST_MPolyFromWKB(), new ST_NumGeometries(),
+        new ST_NumInteriorRing(), new ST_NumPoints(), new ST_Point(),
+        new ST_PointFromWKB(), new ST_PointN(), new ST_PointZ(), new ST_PolyFromWKB(),
+        new ST_Relate(), new ST_SRID(), new ST_StartPoint(), new ST_SymmetricDiff(),
+        new ST_X(), new ST_Y(), new ST_Z());
+
+    for (UDF udf : legacyUDFs) {
+      for (Function fn : extractFromLegacyHiveBuiltin(udf, db.getName())) {
+        db.addBuiltin(fn);
+      }
+    }
+  }
+
+  private static void addGenericUDFs(Db db) {
+    List<ScalarFunction> genericUDFs = new ArrayList<>();
+
+    List<Set<Type>> stBinArguments =
+        ImmutableList.of(ImmutableSet.of(Type.DOUBLE, Type.BIGINT),
+            ImmutableSet.of(Type.STRING, Type.BINARY));
+    List<Set<Type>> stBinEnvelopeArguments =
+        ImmutableList.of(ImmutableSet.of(Type.DOUBLE, Type.BIGINT),
+            ImmutableSet.of(Type.STRING, Type.BINARY, Type.BIGINT));
+
+    genericUDFs.addAll(
+        createMappedGenericUDFs(stBinArguments, Type.BIGINT, ST_Bin.class));
+    genericUDFs.addAll(createMappedGenericUDFs(
+        stBinEnvelopeArguments, Type.BINARY, ST_BinEnvelope.class));
+    genericUDFs.add(createScalarFunction(
+        ST_GeomFromGeoJson.class, Type.BINARY, new Type[] {Type.STRING}));
+    genericUDFs.add(createScalarFunction(
+        ST_GeomFromJson.class, Type.BINARY, new Type[] {Type.STRING}));
+    genericUDFs.add(createScalarFunction(
+        ST_MultiPolygon.class, Type.BINARY, new Type[] {Type.STRING}));
+    genericUDFs.add(createScalarFunction(
+        ST_MultiLineString.class, Type.BINARY, new Type[] {Type.STRING}));
+
+    createRelationalGenericUDFs(genericUDFs);
+
+    for (ScalarFunction function : genericUDFs) {
+      db.addBuiltin(function);
+    }
+  }
+
+  private static void createRelationalGenericUDFs(List<ScalarFunction> genericUDFs) {
+    List<GenericUDF> relationalUDFs = Arrays.asList(new ST_Contains(), new ST_Crosses(),
+        new ST_Disjoint(), new ST_Equals(), new ST_Intersects(), new ST_Overlaps(),
+        new ST_Touches(), new ST_Within());
+
+    List<Set<Type>> relationalUDFArguments =
+        ImmutableList.of(ImmutableSet.of(Type.STRING, Type.BINARY),
+            ImmutableSet.of(Type.STRING, Type.BINARY));
+
+    for (GenericUDF relationalUDF : relationalUDFs) {
+      genericUDFs.addAll(createMappedGenericUDFs(
+          relationalUDFArguments, Type.BOOLEAN, relationalUDF.getClass()));
+    }
+  }
+
+  private static void addVarargsUDFs(Db db) {
+    List<ScalarFunction> varargsUDFs = new ArrayList<>();
+    varargsUDFs.addAll(
+        extractFunctions(ST_Union_Wrapper.class, ST_Union.class, db.getName()));
+    varargsUDFs.addAll(
+        extractFunctions(ST_Polygon_Wrapper.class, ST_Polygon.class, db.getName()));
+    varargsUDFs.addAll(
+        extractFunctions(ST_LineString_Wrapper.class, ST_LineString.class, db.getName()));
+    varargsUDFs.addAll(
+        extractFunctions(ST_MultiPoint_Wrapper.class, ST_MultiPoint.class, db.getName()));
+    varargsUDFs.addAll(
+        extractFunctions(ST_ConvexHull_Wrapper.class, ST_ConvexHull.class, db.getName()));
+
+    for (ScalarFunction function : varargsUDFs) {
+      db.addBuiltin(function);
+    }
+  }
+
+  private static List<ScalarFunction> extractFromLegacyHiveBuiltin(
+      UDF udf, String dbName) {
+    return extractFunctions(udf.getClass(), udf.getClass(), dbName);
+  }
+
+  private static List<ScalarFunction> extractFunctions(
+      Class<?> udfClass, Class<?> signatureClass, String dbName) {
+    // The function has the same name as the signature class name
+    String fnName = signatureClass.getSimpleName().toLowerCase();
+    // The symbol name is coming from the UDF class which contains the functions
+    String symbolName = udfClass.getName();
+    org.apache.hadoop.hive.metastore.api.Function hiveFunction =
+        HiveJavaFunction.createHiveFunction(fnName, dbName, symbolName, null);
+    try {
+      return new HiveLegacyJavaFunction(udfClass, hiveFunction, null, null)
+          .extract(new BinaryToBinaryHiveLegacyFunctionExtractor());
+    } catch (CatalogException ex) {
+      // It is a fatal error if we fail to load a builtin function.
+      Preconditions.checkState(false, ex.getMessage());
+      return Collections.emptyList();
+    }
+  }
+
+  private static ScalarFunction createScalarFunction(
+      Class<?> udf, String name, Type returnType, Type[] arguments) {
+    ScalarFunction function = new ScalarFunction(
+        new FunctionName(BuiltinsDb.NAME, name), arguments, returnType, false);
+    function.setSymbolName(udf.getName());
+    function.setUserVisible(true);
+    function.setHasVarArgs(false);
+    function.setBinaryType(TFunctionBinaryType.JAVA);
+    function.setIsPersistent(true);
+    return function;
+  }
+
+  private static ScalarFunction createScalarFunction(
+      Class<?> udf, Type returnType, Type[] arguments) {
+    return createScalarFunction(
+        udf, udf.getSimpleName().toLowerCase(), returnType, arguments);
+  }
+
+  private static List<ScalarFunction> createMappedGenericUDFs(
+      List<Set<Type>> listOfArgumentOptions, Type returnType, Class<?> genericUDF) {
+    return Sets.cartesianProduct(listOfArgumentOptions)
+        .stream()
+        .map(types -> {
+          Type[] arguments = types.toArray(new Type[0]);
+          return createScalarFunction(genericUDF, returnType, arguments);
+        })
+        .collect(Collectors.toList());
+  }
+
+  /*
+    TODO: IMPALA-11854: A workaround must be applied for ST_SetSRID UDF because the
+    GeometryUtils.setWKID method assumes that the incoming geomref's buffer can
+    be modified through the array returned by ImpalaBytesWritable.getBytes.
+   */
+  private static void addWorkaroundForStSetSrid(Db db) {
+    db.addBuiltin(
+        createScalarFunction(ST_SetSRID_Wrapper.class, ST_SetSRID.class.getSimpleName(),
+            Type.BINARY, new Type[] {Type.BINARY, Type.INT}));
+  }
+
+  public static class ST_SetSRID_Wrapper extends ST_SetSRID {
+    private static final Logger LOG = LoggerFactory.getLogger(ST_SetSRID_Wrapper.class);
+
+    @Override
+    public BytesWritable evaluate(BytesWritable geomref, IntWritable wkwrap) {
+      if (geomref != null && geomref.getLength() != 0) {
+        if (wkwrap != null) {
+          int wkid = wkwrap.get();
+          if (GeometryUtils.getWKID(geomref) != wkid) {
+            ByteBuffer bb = ByteBuffer.allocate(geomref.getLength());
+            bb.putInt(wkid);
+            bb.put(Arrays.copyOfRange(geomref.getBytes(), 4, geomref.getLength()));
+            return new BytesWritable(bb.array());
+          }
+        }
+        return geomref;
+      } else {
+        LogUtils.Log_ArgumentsNull(LOG);
+        return null;
+      }
+    }
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java b/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
index 16abcb377..b32e6acf3 100644
--- a/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
+++ b/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
@@ -33,11 +33,13 @@ import org.apache.impala.analysis.InPredicate;
 import org.apache.impala.analysis.IsNullPredicate;
 import org.apache.impala.analysis.LikePredicate;
 import org.apache.impala.builtins.ScalarBuiltins;
-import org.apache.impala.catalog.AggregateFunction;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.base.Preconditions;
+import org.apache.impala.compat.HiveEsriGeospatialBuiltins;
+import org.apache.impala.service.BackendConfig;
+import org.apache.impala.thrift.TGeospatialLibrary;
 
 public class BuiltinsDb extends Db {
   // Size in bytes of AvgState used for integer, floating point, and timestamp avg().
@@ -111,6 +113,11 @@ public class BuiltinsDb extends Db {
     IsNullPredicate.initBuiltins(this);
     LikePredicate.initBuiltins(this);
     ScalarBuiltins.initBuiltins(this);
+
+    if (BackendConfig.INSTANCE.getGeospatialLibrary().equals(
+            TGeospatialLibrary.HIVE_ESRI)) {
+      HiveEsriGeospatialBuiltins.initBuiltins(this);
+    }
   }
 
   private static final String BUILTINS_DB_COMMENT = "System database for Impala builtin functions";
diff --git a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
index 0a8e70c4c..c87cda9a2 100644
--- a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
@@ -1757,9 +1757,9 @@ public class CatalogServiceCatalog extends Catalog {
     LOG.info("Loading Java functions for database: " + db.getName());
     for (org.apache.hadoop.hive.metastore.api.Function function: functions) {
       try {
-        HiveJavaFunctionFactoryImpl factory = new HiveJavaFunctionFactoryImpl();
-        HiveJavaFunction javaFunction =
-            factory.create(localLibraryPath_, function);
+        HiveJavaFunctionFactoryImpl factory =
+            new HiveJavaFunctionFactoryImpl(localLibraryPath_);
+        HiveJavaFunction javaFunction = factory.create(function);
         for (Function fn: javaFunction.extract()) {
           db.addFunction(fn);
           fn.setCatalogVersion(incrementAndGetCatalogVersion());
diff --git a/fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java b/fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
index 5ad3f9845..b8967eb4f 100644
--- a/fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
+++ b/fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
@@ -48,8 +48,8 @@ public class ScalarFunction extends Function {
     super(fnName, argTypes, retType, hasVarArgs);
   }
 
-  public ScalarFunction(FunctionName fnName, Type argTypes[], Type retType,
-      boolean hasVarArgs) {
+  public ScalarFunction(
+      FunctionName fnName, Type[] argTypes, Type retType, boolean hasVarArgs) {
     super(fnName, argTypes, retType, hasVarArgs);
   }
 
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/ImpalaDoubleWritable.java b/fe/src/main/java/org/apache/impala/hive/executor/BinaryToBinaryHiveLegacyFunctionExtractor.java
similarity index 56%
copy from fe/src/main/java/org/apache/impala/hive/executor/ImpalaDoubleWritable.java
copy to fe/src/main/java/org/apache/impala/hive/executor/BinaryToBinaryHiveLegacyFunctionExtractor.java
index 2a0bd164a..5a36d0063 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/ImpalaDoubleWritable.java
+++ b/fe/src/main/java/org/apache/impala/hive/executor/BinaryToBinaryHiveLegacyFunctionExtractor.java
@@ -17,22 +17,22 @@
 
 package org.apache.impala.hive.executor;
 
-import org.apache.hadoop.hive.serde2.io.DoubleWritable;
-
-import org.apache.impala.util.UnsafeUtil;
-
-@SuppressWarnings("restriction")
-public class ImpalaDoubleWritable extends DoubleWritable {
-  // Ptr (to native heap) where the value should be read from and written to.
-  private final long ptr_;
-
-  public ImpalaDoubleWritable(long ptr) {
-    ptr_ = ptr;
-  }
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.impala.catalog.CatalogException;
+import org.apache.impala.catalog.PrimitiveType;
+import org.apache.impala.catalog.ScalarType;
+import org.apache.impala.common.ImpalaException;
 
+public class BinaryToBinaryHiveLegacyFunctionExtractor
+    extends HiveLegacyFunctionExtractor {
   @Override
-  public double get() { return UnsafeUtil.UNSAFE.getDouble(ptr_); }
-
-  @Override
-  public void set(double v) { UnsafeUtil.UNSAFE.putDouble(ptr_, v); }
+  protected ScalarType resolveType(
+      Class<?> type, java.util.function.Function<JavaUdfDataType, String> errorHandler)
+      throws ImpalaException {
+    if (type == BytesWritable.class) {
+      return ScalarType.createType(PrimitiveType.BINARY);
+    } else {
+      return super.resolveType(type, errorHandler);
+    }
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/HiveGenericJavaFunction.java b/fe/src/main/java/org/apache/impala/hive/executor/HiveGenericJavaFunction.java
index 80a4bfdb5..d067790cb 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/HiveGenericJavaFunction.java
+++ b/fe/src/main/java/org/apache/impala/hive/executor/HiveGenericJavaFunction.java
@@ -17,48 +17,22 @@
 
 package org.apache.impala.hive.executor;
 
-import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.api.Function;
-import org.apache.hadoop.hive.metastore.api.FunctionType;
-import org.apache.hadoop.hive.metastore.api.PrincipalType;
-import org.apache.hadoop.hive.metastore.api.ResourceType;
-import org.apache.hadoop.hive.metastore.api.ResourceUri;
-import org.apache.hadoop.hive.ql.exec.FunctionUtils.UDFClassType;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
-import org.apache.impala.analysis.FunctionName;
-import org.apache.impala.analysis.HdfsUri;
 import org.apache.impala.catalog.CatalogException;
 import org.apache.impala.catalog.ScalarFunction;
-import org.apache.impala.catalog.PrimitiveType;
-import org.apache.impala.catalog.ScalarType;
 import org.apache.impala.catalog.Type;
-import org.apache.impala.common.AnalysisException;
-import org.apache.impala.common.FileSystemUtil;
-import org.apache.impala.service.BackendConfig;
-import org.apache.impala.thrift.TFunction;
-import org.apache.impala.thrift.TFunctionBinaryType;
 
-import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.net.URL;
-import java.net.URLClassLoader;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
-import java.util.UUID;
 
 import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
 
 import org.apache.log4j.Logger;
 
@@ -111,7 +85,8 @@ public class HiveGenericJavaFunction implements HiveJavaFunction {
    * handle it.
    */
   @Override
-  public List<ScalarFunction> extract() throws CatalogException {
+  public List<ScalarFunction> extract(HiveLegacyFunctionExtractor extractor)
+      throws CatalogException {
     // Return blank list because extraction cannot be done.
     return new ArrayList<>();
   }
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/HiveJavaFunction.java b/fe/src/main/java/org/apache/impala/hive/executor/HiveJavaFunction.java
index f8946dabc..b5b246a32 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/HiveJavaFunction.java
+++ b/fe/src/main/java/org/apache/impala/hive/executor/HiveJavaFunction.java
@@ -41,7 +41,12 @@ public interface HiveJavaFunction {
    * Extract all the supported ScalarFunction objects from the Hive Java
    * function.
    */
-  public List<ScalarFunction> extract() throws CatalogException;
+  public default List<ScalarFunction> extract() throws CatalogException {
+      return extract(new HiveLegacyFunctionExtractor());
+  }
+
+  public List<ScalarFunction> extract(HiveLegacyFunctionExtractor extractor)
+      throws CatalogException;
 
   /**
    * Get the Hive "Function" object declared by the Hive metastore API.
@@ -61,4 +66,9 @@ public interface HiveJavaFunction {
         (int) (System.currentTimeMillis() / 1000),
         FunctionType.JAVA, resources);
   }
+  public static Function createHiveFunction(
+      String fnName, String dbName, String symbolName, List<ResourceUri> resources) {
+    return new Function(fnName, dbName, symbolName, "", PrincipalType.USER,
+        (int) (System.currentTimeMillis() / 1000), FunctionType.JAVA, resources);
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/HiveJavaFunctionFactory.java b/fe/src/main/java/org/apache/impala/hive/executor/HiveJavaFunctionFactory.java
index 159a732fc..6ab80d27a 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/HiveJavaFunctionFactory.java
+++ b/fe/src/main/java/org/apache/impala/hive/executor/HiveJavaFunctionFactory.java
@@ -30,12 +30,10 @@ public interface HiveJavaFunctionFactory {
    * The local path contains a directory on the local file system to which the
    * jar file on hdfs can be copied.
    */
-  public HiveJavaFunction create(String localLibPath, Function hiveFn,
-      Type retType, Type[] paramTypes) throws CatalogException;
+  public HiveJavaFunction create(Function hiveFn, Type retType, Type[] paramTypes)
+      throws CatalogException;
 
-  public HiveJavaFunction create(String localLibPath,
-      ScalarFunction fn) throws CatalogException;
+  public HiveJavaFunction create(ScalarFunction fn) throws CatalogException;
 
-  public HiveJavaFunction create(String localLibPath, Function hiveFn)
-      throws CatalogException;
+  public HiveJavaFunction create(Function hiveFn) throws CatalogException;
 }
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/HiveJavaFunctionFactoryImpl.java b/fe/src/main/java/org/apache/impala/hive/executor/HiveJavaFunctionFactoryImpl.java
index ec36f3ff4..ba04f002c 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/HiveJavaFunctionFactoryImpl.java
+++ b/fe/src/main/java/org/apache/impala/hive/executor/HiveJavaFunctionFactoryImpl.java
@@ -16,7 +16,6 @@
 // under the License.
 
 package org.apache.impala.hive.executor;
-
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.FunctionType;
 import org.apache.hadoop.hive.metastore.api.ResourceType;
@@ -24,28 +23,29 @@ import org.apache.hadoop.hive.metastore.api.ResourceUri;
 import org.apache.impala.catalog.CatalogException;
 import org.apache.impala.catalog.ScalarFunction;
 import org.apache.impala.catalog.Type;
-
 import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-
 import java.util.ArrayList;
 import java.util.List;
-
 /**
  * Factory class to create the HiveJavaFunction instance.
  */
 public class HiveJavaFunctionFactoryImpl implements HiveJavaFunctionFactory {
+  private final String localLibPath_;
+  public HiveJavaFunctionFactoryImpl(String localLibPath) {
+    localLibPath_ = localLibPath;
+  }
+
   /**
    * The local path contains a directory on the local file system to which the
    * jar file on hdfs can be copied.
    */
-  public HiveJavaFunction create(String localLibPath, Function hiveFn,
-      Type retType, Type[] paramTypes) throws CatalogException {
+  public HiveJavaFunction create(Function hiveFn, Type retType, Type[] paramTypes)
+      throws CatalogException {
     checkValidFunction(hiveFn);
     String jarUri = hiveFn.getResourceUris().get(0).getUri();
     String fnName = hiveFn.getDbName() + "." + hiveFn.getFunctionName();
-    try (HiveUdfLoader javaClass
-        = HiveUdfLoader.createWithLocalPath(localLibPath, hiveFn)) {
+    try (HiveUdfLoader javaClass =
+             HiveUdfLoader.createWithLocalPath(localLibPath_, hiveFn)) {
       switch (javaClass.getUDFClassType()) {
         case UDF:
           return new HiveLegacyJavaFunction(javaClass.getUDFClass(), hiveFn, retType,
@@ -61,20 +61,15 @@ public class HiveJavaFunctionFactoryImpl implements HiveJavaFunctionFactory {
     }
   }
 
-  public HiveJavaFunction create(String localLibPath,
-      ScalarFunction fn) throws CatalogException {
+  public HiveJavaFunction create(ScalarFunction fn) throws CatalogException {
     if (fn.hasVarArgs()) {
       throw new CatalogException("Variable arguments not supported in Hive UDFs.");
     }
-    return create(localLibPath, HiveJavaFunction.toHiveFunction((ScalarFunction) fn),
-        fn.getReturnType(), fn.getArgs());
+    return create(HiveJavaFunction.toHiveFunction(fn), fn.getReturnType(), fn.getArgs());
   }
-
-  public HiveJavaFunction create(String localLibPath, Function hiveFn)
-      throws CatalogException {
-    return create(localLibPath, hiveFn, null, null);
+  public HiveJavaFunction create(Function hiveFn) throws CatalogException {
+    return create(hiveFn, null, null);
   }
-
   /**
    * Checks if the Hive function 'fn' is Impala compatible. A function is Impala
    * compatible iff
@@ -87,7 +82,7 @@ public class HiveJavaFunctionFactoryImpl implements HiveJavaFunctionFactory {
    * Returns true if compatible and false otherwise. In case of incompatible
    * functions 'incompatMsg' has the reason for the incompatibility.
    * */
-   private void checkValidFunction(Function fn) throws CatalogException {
+  private void checkValidFunction(Function fn) throws CatalogException {
     String errorPrefix = "Skipping load of incompatible function: " +
         fn.getFunctionName() + ". ";
     if (fn.getFunctionType() != FunctionType.JAVA) {
@@ -109,10 +104,9 @@ public class HiveJavaFunctionFactoryImpl implements HiveJavaFunctionFactory {
           + "Jars for dependencies. (" + Joiner.on(",").join(resourceUris) + ") ");
     }
     if (fn.getResourceUris().get(0).getResourceType() != ResourceType.JAR) {
-      throw new CatalogException(errorPrefix + "Function binary type: " +
-        fn.getResourceUris().get(0).getResourceType().name()
-        + " is not supported. Only " + ResourceType.JAR.name()
-        + " type is supported.");
+      throw new CatalogException(errorPrefix + "Function binary type: "
+          + fn.getResourceUris().get(0).getResourceType().name()
+          + " is not supported. Only " + ResourceType.JAR.name() + " type is supported.");
     }
   }
 }
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/HiveLegacyFunctionExtractor.java b/fe/src/main/java/org/apache/impala/hive/executor/HiveLegacyFunctionExtractor.java
new file mode 100644
index 000000000..5a2f09183
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/hive/executor/HiveLegacyFunctionExtractor.java
@@ -0,0 +1,98 @@
+// 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.impala.hive.executor;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.ResourceUri;
+import org.apache.impala.analysis.FunctionName;
+import org.apache.impala.analysis.HdfsUri;
+import org.apache.impala.catalog.CatalogException;
+import org.apache.impala.catalog.PrimitiveType;
+import org.apache.impala.catalog.ScalarFunction;
+import org.apache.impala.catalog.ScalarType;
+import org.apache.impala.catalog.Type;
+import org.apache.impala.common.ImpalaException;
+import org.apache.impala.thrift.TFunctionBinaryType;
+import org.apache.log4j.Logger;
+
+public class HiveLegacyFunctionExtractor {
+  private static final Logger LOG = Logger.getLogger(HiveLegacyFunctionExtractor.class);
+
+  public ScalarFunction extract(Function function, Method method) {
+    Class<?> returnType = method.getReturnType();
+    Class<?>[] fnArgs = method.getParameterTypes();
+    String className = function.getClassName();
+
+    List<Type> fnArgsList;
+    ScalarType retType;
+    try {
+      fnArgsList = resolveParameterTypes(Arrays.asList(fnArgs));
+      retType = resolveReturnType(returnType);
+    } catch (ImpalaException exception) {
+      LOG.debug("Processing " + className + ": " + exception.getMessage());
+      return null;
+    }
+
+    HdfsUri location = null;
+    List<ResourceUri> functionResources = function.getResourceUris();
+    if (functionResources != null) {
+      ResourceUri jarLocation = function.getResourceUris().get(0);
+      location = new HdfsUri(jarLocation.getUri());
+    }
+
+    ScalarFunction fn = new ScalarFunction(
+        new FunctionName(function.getDbName(), function.getFunctionName()), fnArgsList,
+        retType, location, className, null, null);
+    fn.setHasVarArgs(false);
+    fn.setBinaryType(TFunctionBinaryType.JAVA);
+    fn.setIsPersistent(true);
+    return fn;
+  }
+
+  private List<Type> resolveParameterTypes(List<Class<?>> arguments)
+      throws ImpalaException {
+    List<Type> resolvedTypes = new ArrayList<>();
+    for (Class<?> argument : arguments) {
+      resolvedTypes.add(resolveParameterType(argument));
+    }
+    return resolvedTypes;
+  }
+
+  private ScalarType resolveParameterType(Class<?> param) throws ImpalaException {
+    return resolveType(param, type -> "Param type " + type + " not supported");
+  }
+
+  private ScalarType resolveReturnType(Class<?> returnType) throws ImpalaException {
+    return resolveType(returnType, type -> "Return type " + type + " not supported");
+  }
+
+  protected ScalarType resolveType(
+      Class<?> type, java.util.function.Function<JavaUdfDataType, String> errorHandler)
+      throws ImpalaException {
+    JavaUdfDataType javaRetType = JavaUdfDataType.getType(type);
+    if (javaRetType == JavaUdfDataType.INVALID_TYPE) {
+      throw new CatalogException(errorHandler.apply(javaRetType));
+    }
+    return ScalarType.createType(
+        PrimitiveType.fromThrift(javaRetType.getPrimitiveType()));
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/HiveLegacyJavaFunction.java b/fe/src/main/java/org/apache/impala/hive/executor/HiveLegacyJavaFunction.java
index 3de5add1b..f796d445a 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/HiveLegacyJavaFunction.java
+++ b/fe/src/main/java/org/apache/impala/hive/executor/HiveLegacyJavaFunction.java
@@ -19,14 +19,10 @@ package org.apache.impala.hive.executor;
 
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.ql.exec.UDF;
-import org.apache.impala.analysis.FunctionName;
-import org.apache.impala.analysis.HdfsUri;
 import org.apache.impala.catalog.CatalogException;
-import org.apache.impala.catalog.PrimitiveType;
 import org.apache.impala.catalog.ScalarFunction;
 import org.apache.impala.catalog.ScalarType;
 import org.apache.impala.catalog.Type;
-import org.apache.impala.thrift.TFunctionBinaryType;
 
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
@@ -38,7 +34,8 @@ import java.util.Set;
 
 import com.google.common.base.Joiner;
 
-import org.apache.log4j.Logger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * HiveLegacyJavaFunction generates the instance of the UDF object given
@@ -48,7 +45,7 @@ import org.apache.log4j.Logger;
  * Java reflection.
  */
 public class HiveLegacyJavaFunction implements HiveJavaFunction {
-  private static final Logger LOG = Logger.getLogger(HiveLegacyJavaFunction.class);
+  private static final Logger LOG = LoggerFactory.getLogger(HiveLegacyJavaFunction.class);
 
   // By convention, the function in the class must be called evaluate()
   private static final String UDF_FUNCTION_NAME = "evaluate";
@@ -136,31 +133,30 @@ public class HiveLegacyJavaFunction implements HiveJavaFunction {
    * class referred to by the given Java function. This method copies the UDF Jar
    * referenced in the function definition to a temporary file in localLibraryPath_ and
    * loads it into the jvm. Then we scan all the methods in the class using reflection and
-   * extract those methods and create corresponding Impala functions. Currently Impala
+   * extract those methods and create corresponding Impala functions. Currently, Impala
    * supports only "JAR" files for symbols and also a single Jar containing all the
    * dependent classes rather than a set of Jar files.
    */
   @Override
-  public List<ScalarFunction> extract() throws CatalogException {
+  public List<ScalarFunction> extract(HiveLegacyFunctionExtractor extractor)
+      throws CatalogException {
     Set<String> addedSignatures = new HashSet<>();
     List<ScalarFunction> result = new ArrayList<>();
-    String jarUri = hiveFn_.getResourceUris().get(0).getUri();
     // Load each method in the UDF class and create the corresponding Impala Function
     // object.
     try {
       for (Method m: UDF_.getClass().getMethods()) {
         if (m.getName().equals(UDF_FUNCTION_NAME)) {
-          ScalarFunction fn = fromHiveFunction(hiveFn_.getDbName(),
-              hiveFn_.getFunctionName(), hiveFn_.getClassName(),
-              m.getParameterTypes(), m.getReturnType(), jarUri);
+          ScalarFunction fn = extractor.extract(hiveFn_, m);
           if (fn != null) {
             if (!addedSignatures.contains(fn.signatureString())) {
               result.add(fn);
               addedSignatures.add(fn.signatureString());
             }
           } else {
-            LOG.warn("Ignoring incompatible method: " + m.toString() + " during load of "
-                + "Hive UDF:" + hiveFn_.getFunctionName() + " from " + UDF_.getClass());
+            LOG.warn(
+                "Ignoring incompatible method: {} during load of Hive UDF: {} from {}", m,
+                hiveFn_.getFunctionName(), UDF_.getClass());
           }
         }
       }
@@ -173,7 +169,8 @@ public class HiveLegacyJavaFunction implements HiveJavaFunction {
           ":  " + t);
     }
     if (result.isEmpty()) {
-      throw new CatalogException("No compatible function signatures found.");
+      throw new CatalogException(
+          "No compatible signatures found for function: " + hiveFn_.getFunctionName());
     }
     return result;
   }
@@ -237,47 +234,4 @@ public class HiveLegacyJavaFunction implements HiveJavaFunction {
       .append(Joiner.on("\n    ").join(signatures));
     return sb.toString();
   }
-
-  /**
-   * Creates a Function object based on following inputs.
-   * @param dbName Name of fn's database
-   * @param fnName Name of the function
-   * @param fnClass Function symbol name
-   * @param fnArgs List of Class objects corresponding to the args of evaluate method
-   * @param fnRetType Class corresponding to the return type of the evaluate method
-   * @param hdfsUri URI of the jar holding the udf class.
-   * @return Function object corresponding to the hive udf if the parameters are
-   *         compatible, null otherwise.
-   */
-  private ScalarFunction fromHiveFunction(String dbName, String fnName, String fnClass,
-      Class<?>[] fnArgs, Class<?> fnRetType, String hdfsUri) {
-    // Check if the return type and the method arguments are supported.
-    // Currently we only support certain primitive types.
-    JavaUdfDataType javaRetType = JavaUdfDataType.getType(fnRetType);
-    if (javaRetType == JavaUdfDataType.INVALID_TYPE) {
-      LOG.debug("Processing " + fnClass + ", return type " + fnRetType +
-          " not supported.");
-      return null;
-    }
-    List<Type> fnArgsList = new ArrayList<>();
-    for (Class<?> argClass: fnArgs) {
-      JavaUdfDataType javaUdfType = JavaUdfDataType.getType(argClass);
-      if (javaUdfType == JavaUdfDataType.INVALID_TYPE) {
-        LOG.debug("Processing " + fnClass + ", param type " + argClass +
-            " not supported.");
-        return null;
-      }
-      fnArgsList.add(ScalarType.createType(
-          PrimitiveType.fromThrift(javaUdfType.getPrimitiveType())));
-    }
-    ScalarType retType = ScalarType.createType(
-        PrimitiveType.fromThrift(javaRetType.getPrimitiveType()));
-    ScalarFunction fn = new ScalarFunction(new FunctionName(dbName, fnName), fnArgsList,
-        retType, new HdfsUri(hdfsUri), fnClass, null, null);
-    // We do not support varargs for Java UDFs, and neither does Hive.
-    fn.setHasVarArgs(false);
-    fn.setBinaryType(TFunctionBinaryType.JAVA);
-    fn.setIsPersistent(true);
-    return fn;
-  }
 }
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/ImpalaDoubleWritable.java b/fe/src/main/java/org/apache/impala/hive/executor/ImpalaDoubleWritable.java
index 2a0bd164a..55ed41345 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/ImpalaDoubleWritable.java
+++ b/fe/src/main/java/org/apache/impala/hive/executor/ImpalaDoubleWritable.java
@@ -35,4 +35,9 @@ public class ImpalaDoubleWritable extends DoubleWritable {
 
   @Override
   public void set(double v) { UnsafeUtil.UNSAFE.putDouble(ptr_, v); }
+
+  @Override
+  public String toString() {
+    return Double.toString(get());
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/service/BackendConfig.java b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
index 2346dfa49..d3d06b880 100644
--- a/fe/src/main/java/org/apache/impala/service/BackendConfig.java
+++ b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
@@ -23,8 +23,8 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.impala.analysis.SqlScanner;
-import org.apache.impala.common.PrintUtils;
 import org.apache.impala.thrift.TBackendGflags;
+import org.apache.impala.thrift.TGeospatialLibrary;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -376,4 +376,8 @@ public class BackendConfig {
   public String getIgnoredDirPrefixList() {
     return backendCfg_.ignored_dir_prefix_list;
   }
+
+  public TGeospatialLibrary getGeospatialLibrary() {
+    return backendCfg_.geospatial_library;
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
old mode 100755
new mode 100644
index fb1b82afc..21c2ac28f
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -2207,11 +2207,9 @@ public class CatalogOpExecutor {
     }
     boolean isPersistentJavaFn =
         (fn.getBinaryType() == TFunctionBinaryType.JAVA) && fn.isPersistent();
-    HiveJavaFunction hiveJavaFunction = (fn.getBinaryType() == TFunctionBinaryType.JAVA)
-        ? hiveJavaFuncFactory_.create(
-            BackendConfig.INSTANCE.getBackendCfg().local_library_path,
-            (ScalarFunction) fn)
-        : null;
+    HiveJavaFunction hiveJavaFunction = (fn.getBinaryType() == TFunctionBinaryType.JAVA) ?
+        hiveJavaFuncFactory_.create((ScalarFunction) fn) :
+        null;
     Db db = catalog_.getDb(fn.dbName());
     if (db == null) {
       throw new CatalogException("Database: " + fn.dbName() + " does not exist.");
diff --git a/fe/src/main/java/org/apache/impala/service/JniCatalog.java b/fe/src/main/java/org/apache/impala/service/JniCatalog.java
index c8e74d63e..9b7abff12 100644
--- a/fe/src/main/java/org/apache/impala/service/JniCatalog.java
+++ b/fe/src/main/java/org/apache/impala/service/JniCatalog.java
@@ -155,7 +155,8 @@ public class JniCatalog {
     authzManager_ = authzFactory.newAuthorizationManager(catalog_);
     catalog_.setAuthzManager(authzManager_);
     catalogOpExecutor_ = new CatalogOpExecutor(catalog_, authzConfig, authzManager_,
-        new HiveJavaFunctionFactoryImpl());
+        new HiveJavaFunctionFactoryImpl(
+            BackendConfig.INSTANCE.getBackendCfg().local_library_path));
     MetastoreEventFactory eventFactory =
         new EventFactoryForSyncToLatestEvent(catalogOpExecutor_);
     catalog_.setEventFactoryForSyncToLatestEvent(eventFactory);
diff --git a/fe/src/test/java/org/apache/impala/hive/executor/HiveLegacyJavaFunctionTest.java b/fe/src/test/java/org/apache/impala/hive/executor/HiveLegacyJavaFunctionTest.java
index 4341094f9..8d925938f 100644
--- a/fe/src/test/java/org/apache/impala/hive/executor/HiveLegacyJavaFunctionTest.java
+++ b/fe/src/test/java/org/apache/impala/hive/executor/HiveLegacyJavaFunctionTest.java
@@ -144,11 +144,12 @@ public class HiveLegacyJavaFunctionTest {
   public void testExtractFailNoEvaluateMethods() {
     ExpectedFunctions expectedFuncs = new ExpectedFunctions();
     Class<?> udfClass = DummyUDF.class;
+    String expectedFunctionName = (udfClass.getSimpleName() + FN).toLowerCase();
     try {
       testScalar(udfClass, expectedFuncs);
     } catch (CatalogException e) {
       Assert.assertTrue(e.getMessage().contains(
-          "No compatible function signatures found."));
+          "No compatible signatures found for function: " + expectedFunctionName));
       return;
     }
     Assert.fail("Extraction should not have succeeded.");
diff --git a/fe/src/test/java/org/apache/impala/hive/executor/TestHiveJavaFunctionFactory.java b/fe/src/test/java/org/apache/impala/hive/executor/TestHiveJavaFunctionFactory.java
index 05e2f5a13..412b1472c 100644
--- a/fe/src/test/java/org/apache/impala/hive/executor/TestHiveJavaFunctionFactory.java
+++ b/fe/src/test/java/org/apache/impala/hive/executor/TestHiveJavaFunctionFactory.java
@@ -27,9 +27,9 @@ import java.util.List;
 
 public class TestHiveJavaFunctionFactory implements HiveJavaFunctionFactory {
   public static class TestHiveJavaFunction implements HiveJavaFunction {
-
     @Override
-    public List<ScalarFunction> extract() throws CatalogException {
+    public List<ScalarFunction> extract(HiveLegacyFunctionExtractor extractor)
+        throws CatalogException {
       return new ArrayList<>();
     }
 
@@ -38,18 +38,16 @@ public class TestHiveJavaFunctionFactory implements HiveJavaFunctionFactory {
     }
   }
 
-  public HiveJavaFunction create(String localLibPath, Function hiveFn,
-      Type retType, Type[] paramTypes) throws CatalogException {
+  public HiveJavaFunction create(Function hiveFn, Type retType, Type[] paramTypes)
+      throws CatalogException {
     return new TestHiveJavaFunction();
   }
 
-  public HiveJavaFunction create(String localLibPath,
-      ScalarFunction fn) throws CatalogException {
+  public HiveJavaFunction create(ScalarFunction fn) throws CatalogException {
     return new TestHiveJavaFunction();
   }
 
-  public HiveJavaFunction create(String localLibPath, Function hiveFn)
-      throws CatalogException {
+  public HiveJavaFunction create(Function hiveFn) throws CatalogException {
     return new TestHiveJavaFunction();
   }
 }
diff --git a/java/CMakeLists.txt b/java/CMakeLists.txt
index 7fcef3312..d26c4614e 100644
--- a/java/CMakeLists.txt
+++ b/java/CMakeLists.txt
@@ -19,6 +19,6 @@ add_custom_target(validate_java_pom_versions ALL
   COMMAND $ENV{IMPALA_HOME}/bin/validate-java-pom-versions.sh
 )
 
-add_custom_target(java ALL DEPENDS gen-deps function-registry validate_java_pom_versions
+add_custom_target(java ALL DEPENDS gen-deps function-registry geospatial-udf-wrappers validate_java_pom_versions
   COMMAND $ENV{IMPALA_HOME}/bin/mvn-quiet.sh -B install -DskipTests
 )
diff --git a/java/shaded-deps/hive-exec/pom.xml b/java/shaded-deps/hive-exec/pom.xml
index 6769d396a..c718f40ac 100644
--- a/java/shaded-deps/hive-exec/pom.xml
+++ b/java/shaded-deps/hive-exec/pom.xml
@@ -112,6 +112,7 @@ the same dependencies
                 <include>org/apache/hive/common/HiveVersionAnnotation.class</include>
                 <include>org/apache/hadoop/hive/ql/ErrorMsg.class</include>
                 <include>com/google/**</include>
+                <include>com/esri/**</include>
                 <!-- IMPALA-10261: Some versions of Hive shade guava, so include
                  the shaded path as well -->
                 <include>org/apache/hive/com/google/**</include>
diff --git a/testdata/datasets/README b/testdata/datasets/README
index 0757c1ae2..fe9ea7364 100644
--- a/testdata/datasets/README
+++ b/testdata/datasets/README
@@ -18,7 +18,7 @@ The schema template SQL files have the following format:
   file format/compression combination needs to have a unique name, so all the
   statements are pameterized on table name.
   The template file is read in by the 'generate_schema_statements.py' script to
-  to generate all the schema for the Imapla benchmark tests.
+  to generate all the schema for the Impala benchmark tests.
 
   Each table is defined as a new section in the file with the following format:
 
diff --git a/testdata/workloads/functional-query/queries/QueryTest/geospatial-esri.test b/testdata/workloads/functional-query/queries/QueryTest/geospatial-esri.test
new file mode 100644
index 000000000..75357d021
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/geospatial-esri.test
@@ -0,0 +1,2718 @@
+=====
+---- QUERY
+select ST_Area(ST_BinEnvelope(1.0, ST_Bin(1.0, ST_Point(0, 0))));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_AsText(ST_BinEnvelope(1.0, ST_Bin(1.0, ST_Point(0, 0))));
+---- TYPES
+STRING
+---- RESULTS
+'POLYGON ((-0.5 -0.5, 0.5 -0.5, 0.5 0.5, -0.5 0.5, -0.5 -0.5))'
+=====
+---- QUERY
+select ST_GeometryType(ST_Point(0, 0));
+---- RESULTS
+'ST_POINT'
+====
+---- QUERY
+select ST_GeometryType(ST_Point('point (10.02 20.01)'));
+---- RESULTS
+'ST_POINT'
+====
+---- QUERY
+select ST_GeometryType(ST_Point('point z (10.02 20.01 2)'));
+---- RESULTS
+'ST_POINT'
+====
+---- QUERY
+select ST_GeometryType(ST_MultiPoint('multipoint ((1 2))'));
+---- RESULTS
+'ST_MULTIPOINT'
+====
+---- QUERY
+select ST_GeometryType(ST_Linestring(10,10, 20,20));
+---- RESULTS
+'ST_LINESTRING'
+====
+---- QUERY
+select ST_GeometryType(ST_Linestring('linestring (10 10, 20 20)'));
+---- RESULTS
+'ST_LINESTRING'
+====
+---- QUERY
+select ST_GeometryType(ST_Linestring('linestring z (10 10 2, 20 20 4)'));
+---- RESULTS
+'ST_LINESTRING'
+====
+---- QUERY
+select ST_GeometryType(ST_GeomFromText('polygon ((0 0, 0 10, 10 0, 0 0))'));
+---- RESULTS
+'ST_POLYGON'
+====
+---- QUERY
+select ST_GeometryType(ST_Polygon('polygon ((0 0, 0 10, 10 0, 0 0))'));
+---- RESULTS
+'ST_POLYGON'
+====
+---- QUERY
+select ST_GeometryType(ST_Polygon(1,1, 1,4, 4,1));
+---- RESULTS
+'ST_POLYGON'
+====
+---- QUERY
+select ST_GeometryType(ST_Polygon(1,1, 4,1, 1,4));
+---- RESULTS
+'ST_POLYGON'
+====
+---- QUERY
+select ST_GeometryType(ST_Polygon(1,1, 1,4, 4,1, 1,1));
+---- RESULTS
+'ST_POLYGON'
+====
+---- QUERY
+select ST_GeometryType(ST_Polygon(1,1, 4,1, 1,4, 1,1));
+---- RESULTS
+'ST_POLYGON'
+====
+---- QUERY
+select ST_GeometryType(ST_GeomFromGeoJson('{"type":"Point", "coordinates":[1.2, 2.4]}'));
+---- RESULTS
+'ST_POINT'
+====
+---- QUERY
+select ST_X(ST_GeomFromGeoJson('{"type":"Point", "coordinates":[1.2, 2.4]}'));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.2
+====
+---- QUERY
+select ST_Y(ST_GeomFromGeoJson('{"type":"Point", "coordinates":[1.2, 2.4]}'));
+---- TYPES
+DOUBLE
+---- RESULTS
+2.4
+====
+---- QUERY
+select ST_MinY(ST_GeomFromGeoJson('{"type":"LineString", "coordinates":[[1,2], [3,4]]}'));
+---- TYPES
+DOUBLE
+---- RESULTS
+2.0
+====
+---- QUERY
+select ST_GeometryType(ST_GeomFromGeoJson(ST_AsGeoJson(ST_Point(1.2, 2.4))));
+---- RESULTS
+'ST_POINT'
+====
+---- QUERY
+select ST_GeometryType(ST_GeomFromGeoJson(ST_AsGeoJson(ST_LineString(1,2, 3,4))));
+---- RESULTS
+'ST_LINESTRING'
+====
+---- QUERY
+select ST_NumPoints(ST_Polygon(1.5,2.5, 3.0,2.2, 2.2,1.1));
+---- RESULTS
+4
+====
+---- QUERY
+select ST_NumPoints(ST_Polygon(1.5,2.5, 3.0,2.2, 2.2,1.1, 1.5, 2.5));
+---- RESULTS
+4
+====
+---- QUERY
+select ST_NumPoints(ST_Polygon(0.1,2.2, 3.0,2.2, 2.2,1.1, 0.1, 2.2));
+---- RESULTS
+4
+====
+---- QUERY
+select ST_NumPoints(ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- RESULTS
+5
+====
+---- QUERY
+select ST_NumPoints(ST_Polygon(1,1, 1,4, 4,4, 4,1, 1,1));
+---- RESULTS
+5
+====
+---- QUERY
+select ST_IsEmpty(ST_Point(0,0));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsSimple(ST_Point(0,0));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_IsSimple(ST_LineString(1.5,2.5, 3.0,2.2));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_IsSimple(ST_LineString(0.,0., 1.,1., 0.,1., 1.,0.));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsSimple(ST_LineString(0.,0., 1.,1., 2.,2., 2.,0., 1.,1., 0.,2.));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsSimple(ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_IsSimple(ST_LineString(10,10, 20,20, 20,30, 10,30, 10,20, 20,10));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsSimple(ST_LineString(0.,0., 1.,0., 1.,1., 0.,2., 2.,2., 1.,1., 2.,0.));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsSimple(ST_MultiPoint(0,0, 2,2));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Dimension(ST_Point(0,0));
+---- RESULTS
+0
+====
+---- QUERY
+select ST_Dimension(ST_LineString(1.5,2.5, 3.0,2.2));
+---- RESULTS
+1
+====
+---- QUERY
+select ST_Dimension(ST_Polygon(1.5,2.5, 3.0,2.2, 2.2,1.1));
+---- RESULTS
+2
+====
+---- QUERY
+select ST_Dimension(ST_Polygon(1.5,2.5, 3.0,2.2, 2.2,1.1, 1.5,2.5));
+---- RESULTS
+2
+====
+---- QUERY
+select ST_Dimension(ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1));
+---- RESULTS
+2
+====
+---- QUERY
+select ST_Dimension(ST_MultiPoint(0,0, 2,2));
+---- RESULTS
+0
+====
+---- QUERY
+select ST_X(ST_Point(1,2));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_Y(ST_Point(1,2));
+---- TYPES
+DOUBLE
+---- RESULTS
+2.0
+====
+---- QUERY
+select ST_MinX(ST_Point(1,2));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_MinY(ST_Point(1,2));
+---- TYPES
+DOUBLE
+---- RESULTS
+2.0
+====
+---- QUERY
+select ST_MaxX(ST_Point(1,2));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_MaxY(ST_Point(1,2));
+---- TYPES
+DOUBLE
+---- RESULTS
+2.0
+====
+---- QUERY
+select ST_MinX(ST_LineString(1.5,2.5, 3.0,2.2));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.5
+====
+---- QUERY
+select ST_MinY(ST_LineString(1.5,2.5, 3.0,2.2));
+---- TYPES
+DOUBLE
+---- RESULTS
+2.2
+====
+---- QUERY
+select ST_MaxX(ST_LineString(1.5,2.5, 3.0,2.2));
+---- TYPES
+DOUBLE
+---- RESULTS
+3.0
+====
+---- QUERY
+select ST_MaxY(ST_LineString(1.5,2.5, 3.0,2.2));
+---- TYPES
+DOUBLE
+---- RESULTS
+2.5
+====
+---- QUERY
+select ST_MinX(ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_MinY(ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_MaxX(ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1));
+---- TYPES
+DOUBLE
+---- RESULTS
+4.0
+====
+---- QUERY
+select ST_MaxY(ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1));
+---- TYPES
+DOUBLE
+---- RESULTS
+4.0
+====
+---- QUERY
+select ST_MaxX(ST_MultiPoint(0,0, 2,2));
+---- TYPES
+DOUBLE
+---- RESULTS
+2.0
+====
+---- QUERY
+select ST_MaxY(ST_MultiPoint(0,0, 2,2));
+---- TYPES
+DOUBLE
+---- RESULTS
+2.0
+====
+---- QUERY
+select ST_MinX(ST_MultiPoint(0,0, 2,2));
+---- TYPES
+DOUBLE
+---- RESULTS
+0.0
+====
+---- QUERY
+select ST_MinY(ST_MultiPoint(0,0, 2,2));
+---- TYPES
+DOUBLE
+---- RESULTS
+0.0
+====
+---- QUERY
+select ST_Length(ST_LineString(1.5,2.5, 3.0,2.2));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.5297058540778354
+====
+---- QUERY
+select ST_Length(ST_LineString(0.0,0.0, 3.0,4.0));
+---- TYPES
+DOUBLE
+---- RESULTS
+5.0
+====
+---- QUERY
+select ST_Length(ST_SetSRID(ST_LineString(0.0,0.0, 3.0,4.0), 0));
+---- TYPES
+DOUBLE
+---- RESULTS
+5.0
+====
+---- QUERY
+select ST_Area(ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+DOUBLE
+---- RESULTS
+9.0
+====
+---- QUERY
+select ST_Area(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0), (1 1, 1 5, 5 1, 1 1))'));
+---- TYPES
+DOUBLE
+---- RESULTS
+24.0
+====
+---- QUERY
+select ST_GeodesicLengthWGS84(ST_GeomFromText('MultiLineString((0 80, 0.03 80.04))', 4326));
+---- TYPES
+DOUBLE
+---- RESULTS
+4503.988488226892
+====
+---- QUERY
+select ST_GeodesicLengthWGS84(ST_GeomFromText('LineString(0 0, 0.03 0.04)', 4326));
+---- TYPES
+DOUBLE
+---- RESULTS
+5542.156362735362
+====
+---- QUERY
+select ST_GeodesicLengthWGS84(ST_GeomFromText('MultiLineString((0 0, 0.03 0.04))', 4326));
+---- TYPES
+DOUBLE
+---- RESULTS
+5542.156362735362
+====
+---- QUERY
+select ST_GeodesicLengthWGS84(ST_GeomFromText('MultiLineString((0 80, 0.03 80.04))', 4326));
+---- TYPES
+DOUBLE
+---- RESULTS
+4503.988488226892
+====
+---- QUERY
+select ST_IsClosed(ST_LineString(0.,0., 3.,4.));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsClosed(ST_LineString(0.,0., 3.,4., 0.,4., 0.,0.));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_IsClosed(ST_MultiLineString('multilinestring ((0 0, 3 4, 2 2), (6 2, 7 8))'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsClosed(ST_MultiLineString('multilinestring ((0 0, 3 4, 2 2, 0 0), (6 2, 7 8))'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsClosed(ST_MultiLineString('multilinestring ((0 0, 3 4, 2 2, 0 0), (6 2, 7 5, 6 8, 6 2))'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Is3D(ST_Point(0., 3.));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Is3D(ST_PointZ(0., 3., 1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Is3D(ST_Point('pointzm (0. 3. 1. 2.)'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Is3D(ST_LineString(0.,0., 3.,4., 0.,4., 0.,0.));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Is3D(ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Is3D(ST_GeomFromText('linestring z (10 10 2, 20 20 4)'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Z(ST_PointZ(0., 3., 1));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_Z(ST_Point('pointzm (0. 3. 1. 2.)'));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_Z(ST_GeomFromText('linestring z (10 10 2, 20 20 4)'));
+---- RESULTS
+NULL
+====
+---- QUERY
+select ST_MinZ(ST_PointZ(0., 3., 1));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_MinZ(ST_GeomFromText('linestring z (10 10 2, 20 20 4)'));
+---- TYPES
+DOUBLE
+---- RESULTS
+2.0
+====
+---- QUERY
+select ST_MinZ(ST_MultiPoint('multipoint z((0 0 1), (2 2 3))'));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_MinZ(ST_GeomFromText('polygon z ((0 0 2, 8 0 4, 0 8 3, 0 0 2))'));
+---- TYPES
+DOUBLE
+---- RESULTS
+2.0
+====
+---- QUERY
+select ST_MaxZ(ST_PointZ(0., 3., 1));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_MaxZ(ST_GeomFromText('linestring z (10 10 2, 20 20 4)'));
+---- TYPES
+DOUBLE
+---- RESULTS
+4.0
+====
+---- QUERY
+select ST_MaxZ(ST_GeomFromText('polygon z ((0 0 2, 8 0 4, 0 8 3, 0 0 2))'));
+---- TYPES
+DOUBLE
+---- RESULTS
+4.0
+====
+---- QUERY
+select ST_MaxZ(ST_MultiPoint('multipoint z((0 0 1), (2 2 3))'));
+---- TYPES
+DOUBLE
+---- RESULTS
+3.0
+====
+---- QUERY
+select ST_IsMeasured(ST_Point(0., 3.));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsMeasured(ST_Point('point m(0. 3. 1)'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_IsMeasured(ST_Point('pointzm (0. 3. 1. 2.)'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_IsMeasured(ST_LineString(0.,0., 3.,4., 0.,4., 0.,0.));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsMeasured(ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsMeasured(ST_GeomFromText('linestring m (10 10 2, 20 20 4)'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_M(ST_Point('point m(0. 3. 1)'));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_M(ST_Point('pointzm (0. 3. 1. 2.)'));
+---- TYPES
+DOUBLE
+---- RESULTS
+2.0
+====
+---- QUERY
+select ST_M(ST_GeomFromText('linestring m (10 10 2, 20 20 4)'));
+---- RESULTS
+NULL
+====
+---- QUERY
+select ST_MinM(ST_Point('point m(0. 3. 1)'));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_MinM(ST_GeomFromText('linestring m (10 10 2, 20 20 4)'));
+---- TYPES
+DOUBLE
+---- RESULTS
+2.0
+====
+---- QUERY
+select ST_MinM(ST_GeomFromText('polygon m ((0 0 5, 8 0 4, 0 8 3, 0 0 5))'));
+---- TYPES
+DOUBLE
+---- RESULTS
+3.0
+====
+---- QUERY
+select ST_MinM(ST_MultiPoint('multipoint m((0 0 1), (2 2 3))'));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_MaxM(ST_Point('point m(0. 3. 1)'));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_MaxM(ST_GeomFromText('linestring m (10 10 2, 20 20 4)'));
+---- TYPES
+DOUBLE
+---- RESULTS
+4.0
+====
+---- QUERY
+select ST_MaxM(ST_GeomFromText('polygon m ((0 0 5, 8 0 4, 0 8 3, 0 0 5))'));
+---- TYPES
+DOUBLE
+---- RESULTS
+5.0
+====
+---- QUERY
+select ST_MaxM(ST_MultiPoint('multipoint m((0 0 1), (2 2 3))'));
+---- TYPES
+DOUBLE
+---- RESULTS
+3.0
+====
+---- QUERY
+select ST_CoordDim(ST_Point(0., 3.));
+---- RESULTS
+2
+====
+---- QUERY
+select ST_CoordDim(ST_PointZ(0., 3., 1));
+---- RESULTS
+3
+====
+---- QUERY
+select ST_CoordDim(ST_Point('point m(0. 3. 1)'));
+---- RESULTS
+3
+====
+---- QUERY
+select ST_CoordDim(ST_Point('pointzm (0. 3. 1. 2.)'));
+---- RESULTS
+4
+====
+---- QUERY
+select ST_CoordDim(ST_LineString(0.,0., 3.,4., 0.,4., 0.,0.));
+---- RESULTS
+2
+====
+---- QUERY
+select ST_CoordDim(ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1));
+---- RESULTS
+2
+====
+---- QUERY
+select ST_CoordDim(ST_GeomFromText('linestring z (10 10 2, 20 20 4)'));
+---- RESULTS
+3
+====
+---- QUERY
+select ST_CoordDim(ST_GeomFromText('linestring m (10 10 2, 20 20 4)'));
+---- RESULTS
+3
+====
+---- QUERY
+select ST_NumPoints(ST_Point('point empty'));
+---- RESULTS
+0
+====
+---- QUERY
+select ST_NumPoints(ST_Point(0., 3.));
+---- RESULTS
+1
+====
+---- QUERY
+select ST_NumPoints(ST_PointZ(0., 3., 1));
+---- RESULTS
+1
+====
+---- QUERY
+select ST_NumPoints(ST_LineString(0.,0., 3.,4.));
+---- RESULTS
+2
+====
+---- QUERY
+select ST_NumPoints(ST_LineString(0.,0., 3.,4., 0.,4., 0.,0.));
+---- RESULTS
+4
+====
+---- QUERY
+select ST_NumPoints(ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- RESULTS
+5
+====
+---- QUERY
+select ST_NumPoints(ST_Polygon(1,1, 1,4, 4,4, 4,1, 1,1));
+---- RESULTS
+5
+====
+---- QUERY
+select ST_NumPoints(ST_GeomFromText('multilinestring ((2 4, 10 10), (20 20, 7 8))'));
+---- RESULTS
+4
+====
+---- QUERY
+select ST_NumPoints(ST_GeomFromText('multipoint ((10 40), (40 30), (20 20), (30 10))', 0));
+---- RESULTS
+4
+====
+---- QUERY
+select ST_Contains(ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1), ST_Point(2, 3));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Contains(ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1), ST_Point(8, 8));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Within(ST_Point(2, 3), ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Within(ST_Point(8, 8), ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Touches(ST_Point(1, 3), ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Touches(ST_Point(8, 8), ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Overlaps(st_linestring(0,2, 2,3), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Overlaps(st_linestring(2,0, 2,3), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Overlaps(st_linestring(8,7, 7,8), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Overlaps(ST_Polygon(1,1, 1,4, 4,4, 4,1), st_linestring(2,0, 2,3));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Overlaps(st_polygon(2,0, 2,3, 3,0), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Overlaps(st_polygon(2,0, 2,1, 3,1), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Overlaps(st_linestring(2,0, 2,3), ST_linestring(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Overlaps(st_linestring(8,7, 7,8), ST_linestring(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Overlaps(st_linestring(0,0, 1,2, 1,5), ST_linestring(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Overlaps(st_point(1,1), ST_Point(1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Overlaps(st_point(2,0), ST_Point(1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Touches(st_polygon(2,0, 2,1, 3,1), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Crosses(st_linestring(0,2, 2,3), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Crosses(st_linestring(2,0, 2,3), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Crosses(st_linestring(8,7, 7,8), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Crosses(ST_Polygon(1,1, 1,4, 4,4, 4,1), st_linestring(2,0, 2,3));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Crosses(st_polygon(2,0, 2,3, 3,0), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Crosses(st_polygon(2,0, 2,1, 3,1), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Crosses(st_linestring(2,0, 2,3), ST_linestring(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Crosses(st_linestring(8,7, 7,8), ST_linestring(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Crosses(st_linestring(0,0, 1,2, 1,5), ST_linestring(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Crosses(st_point(1,1), ST_Point(1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Crosses(st_point(2,0), ST_Point(1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Crosses(ST_Linestring(0,0, 1,1), ST_Linestring(1,0, 0,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Crosses(ST_Linestring(1,0, 1,2), ST_Linestring(0,1, 2,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Crosses(ST_Linestring(0,0, 0,1), ST_Linestring(0,0, 1,0));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Crosses(ST_Linestring(0,0, 0,2), ST_Linestring(0,1, 1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_linestring('linestring(0 0, 0 1)'), ST_linestring('linestring(0 0, 1 0)'), '*0*******');
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_linestring('linestring(0 0, 0 1)'), ST_linestring('linestring(0 0, 1 0)'), '***0*****');
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_linestring('linestring(0 0, 0 1)'), ST_linestring('linestring(0 0, 1 0)'), '****0****');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_linestring('linestring(0 0, 0 1)'), ST_linestring('linestring(0 0, 1 0)'), 'FF*F0****');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_linestring('linestring(0 0, 0 2)'), ST_linestring('linestring(0 1, 1 1)'), '*0*******');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_linestring('linestring(0 0, 0 2)'), ST_linestring('linestring(0 1, 1 1)'), 'F0*FF****');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_linestring('linestring(0 0, 0 2)'), ST_linestring('linestring(0 1, 1 1)'), '***0*****');
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_linestring('linestring(0 0, 0 1)'), ST_linestring('linestring(0 0, 1 0)'), 'T********');
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_linestring('linestring(0 0, 0 2)'), ST_linestring('linestring(0 1, 1 1)'), 'T********');
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Equals(st_point(1,1), ST_Point(1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Equals(st_point(2,0), ST_Point(1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Equals(st_linestring(0,0, 1,2), ST_linestring(0,0, 1,2));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Equals(st_linestring(0,0, 1,2), ST_linestring(1,2, 0,0));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Equals(st_linestring(0,0, 1,2, 1,5), ST_linestring(1,1, 1,4, 4,4));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Equals(st_polygon(2,0, 2,1, 3,1), ST_Polygon(2,0, 2,1, 3,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Equals(st_polygon(2,0, 2,1, 3,1), ST_Polygon(3,1, 2,0, 2,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Equals(st_polygon(2,0, 8,1, 3,1), ST_Polygon(3,1, 2,0, 2,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Intersects(st_linestring(0,2, 2,3), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Intersects(st_linestring(2,0, 2,3), ST_Polygon(1,1, 4,1, 4,4, 1,4));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Intersects(ST_LineString(8,7, 7,8), ST_Polygon(1,1, 4,1, 4,4, 1,4));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Intersects(ST_Polygon(1,1, 1,4, 4,4, 4,1), st_linestring(2,0, 2,3));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Intersects(st_polygon(2,0, 2,3, 3,0), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Intersects(st_polygon(2,0, 2,1, 3,1), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Intersects(st_linestring(2,0, 2,3), ST_linestring(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Intersects(st_linestring(8,7, 7,8), ST_linestring(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Intersects(st_linestring(0,0, 1,2, 1,5), ST_linestring(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Intersects(st_linestring(0,0, 1,1), ST_linestring(0,1, 1,0));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Intersects(ST_Linestring(1,0, 1,2), ST_Linestring(0,1, 2,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Intersects(st_linestring(0,0, 0,1), ST_linestring(0,0, 1,0));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Intersects(st_linestring(0,0, 0,2), ST_linestring(0,1, 1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Intersects(st_point(1,1), ST_Point(1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Intersects(st_point(2,0), ST_Point(1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Touches(st_polygon(2,0, 2,1, 3,1), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Intersects(st_polygon(2,0, 2,1, 3,1), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Disjoint(st_polygon(2,0, 2,1, 3,1), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Disjoint(st_point(1,1), ST_Point(1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Disjoint(st_point(2,0), ST_Point(1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_EnvIntersects(ST_LineString(0,0, 1,1), ST_LineString(1,3, 2,2));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_EnvIntersects(ST_LineString(0,0, 2,2), ST_LineString(1,0, 3,2));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_EnvIntersects(st_linestring(0,2, 2,3), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_EnvIntersects(st_linestring(2,0, 2,3), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_EnvIntersects(st_linestring(8,7, 7,8), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_EnvIntersects(ST_Polygon(1,1, 1,4, 4,4, 4,1), st_linestring(2,0, 2,3));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_EnvIntersects(st_polygon(2,0, 2,3, 3,0), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_EnvIntersects(st_polygon(2,0, 2,1, 3,1), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_EnvIntersects(st_linestring(2,0, 2,3), ST_linestring(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_EnvIntersects(st_linestring(8,7, 7,8), ST_linestring(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_EnvIntersects(st_linestring(0,0, 1,2, 1,5), ST_linestring(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_EnvIntersects(st_linestring(0,0, 1,1), ST_linestring(0,1, 1,0));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_EnvIntersects(st_point(1,1), ST_Point(1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_EnvIntersects(st_point(2,0), ST_Point(1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Equals(ST_Envelope(ST_LineString(0,0, 2,2)), ST_Polygon(0,0, 2,0, 2,2, 0,2));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select st_AsText(ST_Envelope(ST_Polygon(1,1, 4,1, 4,4, 1,4)));
+---- RESULTS
+'POLYGON ((1 1, 4 1, 4 4, 1 4, 1 1))'
+====
+---- QUERY
+select st_AsText(ST_Envelope(st_polygon(2,0, 2,3, 3,0)));
+---- RESULTS
+'POLYGON ((2 0, 3 0, 3 3, 2 3, 2 0))'
+====
+---- QUERY
+select ST_Relate(st_point(2,0), ST_Point(1,1), "TFFF0FFF2");
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_point(2,0), ST_Point(1,1), "****T****");
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_point(1,1), ST_Point(1,1), "TFFF0FFF2");
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_point(1,1), ST_Point(1,1), "****T****");
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_point(1,1), ST_Point(1,1), "T********");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_polygon(2,0, 2,1, 3,1), ST_Polygon(1,1, 1,4, 4,4, 4,1), "T********");
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_polygon(2,0, 2,1, 3,1), ST_Polygon(1,1, 1,4, 4,4, 4,1), "****T****");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_polygon(2,0, 2,1, 3,1), ST_Polygon(1,1, 1,4, 4,4, 4,1), "F***1****");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_polygon(2,0, 2,3, 3,1), ST_Polygon(1,1, 1,4, 4,4, 4,1), "2***0****");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_GeomFromText('polygon((2 0, 2 1, 3 1))'), St_GeomFromText('polygon((1 1, 1 4, 4 4, 4 1))'), "F***1****");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_GeomFromText('polygon((2 0, 2 3, 3 1))'), St_GeomFromText('Polygon((1 1, 1 4, 4 4, 4 1))'), "2***0****");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_GeomFromText('polygon((2 0, 2 1, 3 1, 2 0))'), St_GeomFromText('polygon((1 1, 1 4, 4 4, 4 1, 1 1))'), "FF*F1****");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_GeomFromText('polygon((2 0, 2 1, 3 1, 2 0))'), St_GeomFromText('polygon((1 1, 1 4, 4 4, 4 1, 1 1))'), "FF*F2****");
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_linestring(0,0, 3,3), ST_linestring(1,1, 4,4), "T********");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_linestring(0,0, 3,3), ST_linestring(1,1, 4,4), "*T*******");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_linestring(0,0, 3,3), ST_linestring(1,1, 4,4), "***T*****");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_linestring(0,0, 3,3), ST_linestring(1,1, 4,4), "****T****");
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_linestring(2,0, 2,3), ST_linestring(1,1, 1,4, 4,4, 4,1), "T********");
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_linestring(2,0, 2,3), ST_linestring(1,1, 1,4, 4,4, 4,1), "*T*******");
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_linestring(2,0, 2,3), ST_linestring(1,1, 1,4, 4,4, 4,1), "***T*****");
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_linestring(2,0, 2,3), ST_linestring(1,1, 1,4, 4,4, 4,1), "****T****");
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(st_linestring(2,0, 2,3), ST_linestring(1,1, 1,4, 4,4, 4,1), "**T******");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_linestring(2,0, 2,3), ST_linestring(1,1, 1,4, 4,4, 4,1), "*****T***");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_linestring(2,0, 2,3), ST_linestring(1,1, 1,4, 4,4, 4,1), "******T**");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_linestring(2,0, 2,3), ST_linestring(1,1, 1,4, 4,4, 4,1), "*******T*");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(st_linestring(2,0, 2,3), ST_linestring(1,1, 1,4, 4,4, 4,1), "********T");
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0))'), ST_Polygon('polygon ((1 1, 1 5, 5 1, 1 1))'), '2********');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0))'), ST_Polygon('polygon ((1 1, 1 5, 5 1, 1 1))'), '*F*******');
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0))'), ST_Polygon('polygon ((1 1, 1 5, 5 1, 1 1))'), '*1*******');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0))'), ST_Polygon('polygon ((1 1, 1 5, 5 1, 1 1))'), '**2******');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0))'), ST_Polygon('polygon ((1 1, 1 5, 5 1, 1 1))'), '***F*****');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0))'), ST_Polygon('polygon ((1 1, 1 5, 5 1, 1 1))'), '****F****');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0))'), ST_Polygon('polygon ((1 1, 1 5, 5 1, 1 1))'), '*****1***');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0))'), ST_Polygon('polygon ((1 1, 1 5, 5 1, 1 1))'), '******F**');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0))'), ST_Polygon('polygon ((1 1, 1 5, 5 1, 1 1))'), '*******F*');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0))'), ST_Polygon('polygon ((1 1, 1 5, 5 1, 1 1))'), '********2');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0))'), ST_Polygon('polygon ((1 1, 1 5, 5 1, 1 1))'), '2F2FF1FF2');
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0))'), ST_Polygon('polygon ((1 1, 1 5, 5 1, 1 1))'), '212FF1FF2');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_LineString('linestring(0 2, 2 3)'), ST_Polygon('polygon((1 1, 4 1, 4 4, 1 4, 1 1))'), '**1******');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_LineString('linestring(0 2, 2 3)'), ST_Polygon('polygon((1 1, 4 1, 4 4, 1 4, 1 1))'), '**2******');
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(ST_LineString('linestring(0 2, 2 3)'), ST_Polygon('polygon((1 1, 4 1, 4 4, 1 4, 1 1))'), '******1**');
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(ST_LineString('linestring(0 2, 2 3)'), ST_Polygon('polygon((1 1, 4 1, 4 4, 1 4, 1 1))'), '******2**');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon((1 1, 4 1, 4 4, 1 4, 1 1))'), ST_LineString('linestring(0 2, 2 3)'), '**1******');
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon((1 1, 4 1, 4 4, 1 4, 1 1))'), ST_LineString('linestring(0 2, 2 3)'), '**2******');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon((1 1, 4 1, 4 4, 1 4, 1 1))'), ST_LineString('linestring(0 2, 2 3)'), '******1**');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon((1 1, 4 1, 4 4, 1 4, 1 1))'), ST_LineString('linestring(0 2, 2 3)'), '******2**');
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon((1 0, 3 0, 1 2, 1 0))'), ST_Polygon('polygon((0 1, 2 1, 0 3, 0 1))'), '212111212');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon((3 0, 3 3, 0 3, 3 0))'), ST_Polygon('polygon((2 2, 5 2, 2 5, 2 2))'), '212101212');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon((2 0, 2 2, 0 2, 2 0))'), ST_Polygon('polygon((1 1, 3 1, 1 3, 1 1))'), '212101212');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon((0 0, 2 0, 0 2, 0 0))'), ST_Polygon('polygon((0 0, 1 0, 0 1, 0 0))'), '212F11FF2');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon((0 0, 3 0, 0 3, 0 0))'), ST_Polygon('polygon((1 1, 2 1, 1 2, 1 1))'), '212F11FF2');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon((0 0, 2 0, 0 2, 0 0))'), ST_Polygon('polygon((1 1, 1 0, 0 1, 1 1))'), '212F01FF2');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon((0 0, 1 0, 0 1, 0 0))'), ST_Polygon('polygon((0 0, 1 0, 0 1, 0 0))'), '2FFF1FFF2');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Relate(ST_Polygon('polygon((0 0, 3 0, 0 3, 0 0))'), ST_Polygon('polygon((2 2, 2 0, 3 0, 3 3, 0 3, 0 2, 2 2))'), '212111212');
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Distance(ST_Point(0.0,0.0), ST_Point(3.0,4.0));
+---- TYPES
+DOUBLE
+---- RESULTS
+5.0
+====
+---- QUERY
+select ST_Distance(ST_LineString(0,0, 1,1), ST_LineString(2,1, 3,0));
+---- TYPES
+DOUBLE
+---- RESULTS
+1.0
+====
+---- QUERY
+select ST_Area(ST_Buffer(ST_GeomFromText('polygon ((0 0, 3 0, 3 2, 5 2, 5 5, 2 5, 2 3, 0 3, 0 0))'), -1));
+---- TYPES
+DOUBLE
+---- RESULTS
+2.0
+====
+---- QUERY
+select ST_GeometryType(ST_Buffer(ST_GeomFromText('polygon ((0 0, 3 0, 3 2, 5 2, 5 5, 2 5, 2 3, 0 3, 0 0))'), -1));
+---- RESULTS
+'ST_MULTIPOLYGON'
+====
+---- QUERY
+select ST_GeometryType(ST_Buffer(ST_GeomFromText('point (0 0)'), 1));
+---- RESULTS
+'ST_POLYGON'
+====
+---- QUERY
+select ST_Area(ST_Buffer(ST_GeomFromText('point (0 0)'), 1));
+---- TYPES
+DOUBLE
+---- RESULTS
+3.13935020305
+====
+---- QUERY
+select ST_IsEmpty(ST_Intersection(st_linestring(0,2, 2,3), ST_Polygon(1,1, 1,4, 4,4, 4,1)));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsEmpty(ST_Intersection(st_linestring(2,0, 2,3), ST_Polygon(1,1, 1,4, 4,4, 4,1)));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsEmpty(ST_Intersection(st_linestring(8,7, 7,8), ST_Polygon(1,1, 1,4, 4,4, 4,1)));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_IsEmpty(ST_Intersection(ST_Polygon(1,1, 1,4, 4,4, 4,1), st_linestring(2,0, 2,3)));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsEmpty(ST_Intersection(st_polygon(2,0, 2,3, 3,0), ST_Polygon(1,1, 1,4, 4,4, 4,1)));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsEmpty(ST_Intersection(st_linestring(2,0, 2,3), ST_linestring(1,1, 1,4, 4,4, 4,1)));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_IsEmpty(ST_Intersection(st_linestring(8,7, 7,8), ST_linestring(1,1, 1,4, 4,4, 4,1)));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_IsEmpty(ST_Intersection(st_linestring(0,0, 1,2, 1,5), ST_linestring(1,1, 1,4, 4,4, 4,1)));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsEmpty(ST_Intersection(st_linestring(0,0, 1,1), ST_linestring(2,2, 4,4)));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_IsEmpty(ST_Intersection(ST_LineString(0,2, 0,4), ST_Point(3,3)));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_IsEmpty(ST_Intersection(st_point(1,1), ST_Point(1,1)));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+select ST_IsEmpty(ST_Intersection(ST_Point(2,0), ST_Point(1,1)));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Dimension(ST_Intersection(st_linestring(0,2, 2,3), ST_Polygon(1,1, 1,4, 4,4, 4,1)));
+---- RESULTS
+1
+====
+---- QUERY
+select ST_Dimension(ST_Intersection(st_linestring(2,0, 2,3), ST_Polygon(1,1, 1,4, 4,4, 4,1)));
+---- RESULTS
+1
+====
+---- QUERY
+select ST_Dimension(ST_Intersection(ST_Polygon(1,1, 1,4, 4,4, 4,1), st_linestring(2,0, 2,3)));
+---- RESULTS
+1
+====
+---- QUERY
+select ST_Dimension(ST_Intersection(st_polygon(2,0, 2,3, 3,0), ST_Polygon(1,1, 1,4, 4,4, 4,1)));
+---- RESULTS
+2
+====
+---- QUERY
+select ST_Dimension(ST_Intersection(st_point(1,1), ST_Point(1,1)));
+---- RESULTS
+0
+====
+---- QUERY
+select ST_Equals(ST_Intersection(ST_Polygon(1,0, 3,0, 1,2), ST_Polygon(0,1, 2,1, 0,3)), ST_Polygon(1,1, 2,1, 1,2));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+SELECT ST_AsText(ST_SymmetricDiff(ST_Point('point(0 0)'), ST_Point('point(2 2)')));
+---- RESULTS
+'MULTIPOINT ((0 0), (2 2))'
+====
+---- QUERY
+SELECT ST_AsText(ST_SymmetricDiff(ST_MultiPoint('multipoint((0 0))'), ST_MultiPoint('multipoint((2 2))')));
+---- RESULTS
+'MULTIPOINT ((0 0), (2 2))'
+====
+---- QUERY
+SELECT ST_Equals(ST_SymmetricDiff(ST_LineString('linestring(0 2, 2 2)'), ST_LineString('linestring(1 2, 3 2)')), ST_GeomFromText('multilinestring((0 2, 1 2), (2 2, 3 2))'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+SELECT ST_Equals(ST_SymmetricDiff(ST_Polygon('polygon((0 0, 2 0, 2 2, 0 2, 0 0))'), ST_Polygon('polygon((1 1, 3 1, 3 3, 1 3, 1 1))')), ST_MultiPolygon('multipolygon(((0 0, 2 0, 2 1, 1 1, 1 2, 0 2, 0 0)), ((3 1, 3 3, 1 3, 1 2, 2 2, 2 1, 3 1)))'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Equals(ST_Boundary(ST_LineString(0,1, 1,0)), ST_MultiPoint('multipoint((1 0),(0 1))'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_Boundary(ST_Polygon(1,1, 4,1, 1,4)));
+---- RESULTS
+'ST_LINESTRING'
+====
+---- QUERY
+select ST_Equals(ST_Boundary(ST_Polygon(1,1, 4,1, 1,4)), ST_LineString(1,1, 4,1, 1,4, 1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_Boundary(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0), (1 1, 1 5, 5 1, 1 1))')));
+---- RESULTS
+'ST_MULTILINESTRING'
+====
+---- QUERY
+select ST_Equals(ST_Boundary(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0), (1 1, 1 5, 5 1, 1 1))')), ST_MultiLineString('multilinestring((0 0, 8 0, 0 8, 0 0), (1 1, 1 5, 5 1, 1 1))'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Equals(ST_ExteriorRing(ST_Polygon(1,1, 1,4, 4,1)), ST_LineString(1,1, 4,1, 1,4, 1,1));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_Equals(ST_ExteriorRing(ST_Polygon('polygon ((1 1, 4 1, 1 4))')), ST_LineString('linestring(1 1, 4 1, 1 4, 1 1)'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_ExteriorRing(ST_Polygon('polygon ((1 1, 4 1, 1 4))')));
+---- RESULTS
+'ST_LINESTRING'
+====
+---- QUERY
+select ST_Equals(ST_ExteriorRing(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0), (1 1, 1 5, 5 1, 1 1))')), ST_LineString('linestring (0 0, 8 0, 0 8, 0 0)'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_ExteriorRing(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0), (1 1, 1 5, 5 1, 1 1))')));
+---- RESULTS
+'ST_LINESTRING'
+====
+---- QUERY
+select ST_NumGeometries(ST_GeomFromText('multipoint (10 40, 40 30, 20 20, 30 10)', 0));
+---- RESULTS
+4
+====
+---- QUERY
+select ST_NumGeometries(ST_GeomFromText('multipoint ((10 40), (40 30))', 0));
+---- RESULTS
+2
+====
+---- QUERY
+select ST_NumGeometries(ST_GeomFromText('multilinestring ((2 4, 10 10), (20 20, 7 8))', 0));
+---- RESULTS
+2
+====
+---- QUERY
+select ST_NumGeometries(ST_GeomFromText('multilinestring ((2 4, 10 10, 20 20, 7 8))', 0));
+---- RESULTS
+1
+====
+---- QUERY
+select ST_NumGeometries(ST_GeomFromText('multipolygon (((3 3, 4 6, 5 3, 3 3)),((8 24, 1 28, 9 25, 8 24)), ((13 33, 7 36, 1 40, 10 43, 13 33)))'));
+---- RESULTS
+3
+====
+---- QUERY
+select ST_NumGeometries(ST_GeomFromText('multipolygon (((3 3, 4 6, 5 3, 3 3)))'));
+---- RESULTS
+1
+====
+---- QUERY
+select ST_NumGeometries(ST_GeomFromText('multipolygon (((0 0, 9 0, 9 9, 0 9, 0 0),(1 2, 1 7, 5 7, 1 2), (2 1, 7 5, 7 1, 2 1)))'));
+---- RESULTS
+1
+====
+---- QUERY
+select ST_GeometryType(ST_GeometryN(ST_GeomFromText('multipoint (10 40, 40 30, 20 20, 30 10)'), 3));
+---- RESULTS
+'ST_POINT'
+====
+---- QUERY
+select ST_Equals(ST_GeometryN(ST_GeomFromText('multipoint (10 40, 40 30, 20 20, 30 10)'), 3), ST_GeomFromText('point (20 20)'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_GeometryN(ST_GeomFromText('multilinestring ((2 4, 10 10), (20 20, 7 8))'), 2));
+---- RESULTS
+'ST_LINESTRING'
+====
+---- QUERY
+select ST_Equals(ST_GeometryN(ST_GeomFromText('multilinestring ((2 4, 10 10), (20 20, 7 8))'), 2), ST_GeomFromText('linestring (20 20, 7 8)'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_InteriorRingN(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0), (1 1, 1 5, 5 1, 1 1))'), 1));
+---- RESULTS
+'ST_LINESTRING'
+====
+---- QUERY
+select ST_Equals(ST_InteriorRingN(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0), (1 1, 1 5, 5 1, 1 1))'), 1), ST_LineString('linestring(1 1, 5 1, 1 5, 1 1)'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_GeomFromWKB(ST_AsBinary(ST_GeomFromText('point (10.02 20.01)'))));
+---- RESULTS
+'ST_POINT'
+====
+---- QUERY
+select ST_Equals(ST_GeomFromWKB(ST_AsBinary(ST_GeomFromText('point (10.02 20.01)'))),ST_GeomFromText('point (10.02 20.01)'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_GeomFromWKB(ST_AsBinary(ST_GeomFromText('linestring (10 10, 20 20)'))));
+---- RESULTS
+'ST_LINESTRING'
+====
+---- QUERY
+select ST_Equals(ST_GeomFromWKB(ST_AsBinary(ST_GeomFromText('linestring (10 10, 20 20)'))), ST_GeomFromText('linestring (10 10, 20 20)'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_GeomFromWKB(ST_AsBinary(ST_GeomFromText('polygon ((0 0, 0 10, 10 10, 0 0))'))));
+---- RESULTS
+'ST_POLYGON'
+====
+---- QUERY
+select ST_Equals(ST_GeomFromWKB(ST_AsBinary(ST_GeomFromText('polygon ((0 0, 0 10, 10 10, 0 0))'))), ST_GeomFromText('polygon ((0 0, 0 10, 10 10, 0 0))'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_GeomFromWKB(ST_AsBinary(ST_GeomFromText('MULTIPOINT ((10 40), (40 30), (20 20), (30 10))'))));
+---- RESULTS
+'ST_MULTIPOINT'
+====
+---- QUERY
+select ST_GeometryType(ST_PointFromWKB(ST_AsBinary(ST_GeomFromText('point (10 10)'))));
+---- RESULTS
+'ST_POINT'
+====
+---- QUERY
+select ST_Equals(ST_PointFromWKB(ST_AsBinary(ST_GeomFromText('point (10 10)'))), ST_GeomFromText('point (10 10)'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_LineFromWKB(ST_AsBinary(ST_GeomFromText('linestring (10 10, 20 20)'))));
+---- RESULTS
+'ST_LINESTRING'
+====
+---- QUERY
+select ST_Equals(ST_LineFromWKB(ST_AsBinary(ST_GeomFromText('linestring (10 10, 20 20)'))), ST_GeomFromText('linestring (10 10, 20 20)'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_PolyFromWKB(ST_AsBinary(ST_GeomFromText('polygon ((0 0, 1 0, 0 1, 0 0))'))));
+---- RESULTS
+'ST_POLYGON'
+====
+---- QUERY
+select ST_Equals(ST_PolyFromWKB(ST_AsBinary(ST_GeomFromText('polygon ((0 0, 1 0, 0 1, 0 0))'))), ST_GeomFromText('polygon ((0 0, 1 0, 0 1, 0 0))'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_MPointFromWKB(ST_AsBinary(ST_GeomFromText('multipoint ((10 10), (20 20))'))));
+---- RESULTS
+'ST_MULTIPOINT'
+====
+---- QUERY
+select ST_Equals(ST_MPointFromWKB(ST_AsBinary(ST_GeomFromText('multipoint ((10 10), (20 20))'))), ST_GeomFromText('multipoint ((10 10), (20 20))'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_MLineFromWKB(ST_AsBinary(ST_GeomFromText('multilinestring ((1 2, 2 1),(10 10, 20 20))'))));
+---- RESULTS
+'ST_MULTILINESTRING'
+====
+---- QUERY
+select ST_Equals(ST_MLineFromWKB(ST_AsBinary(ST_GeomFromText('multilinestring ((1 2, 2 1),(10 10, 20 20))'))), ST_GeomFromText('multilinestring ((1 2, 2 1),(10 10, 20 20))'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+====
+---- QUERY
+select ST_GeometryType(ST_MPolyFromWKB(ST_AsBinary(ST_GeomFromText('multipolygon (((0 0, 1 0, 0 1, 0 0)), ((2 2, 1 2, 2 1, 2 2)))'))));
+---- RESULTS
+'ST_MULTIPOLYGON'
+====
+---- QUERY
+select ST_Equals(ST_MPolyFromWKB(ST_AsBinary(ST_GeomFromText('multipolygon (((0 0, 1 0, 0 1, 0 0)), ((2 2, 1 2, 2 1, 2 2)))'))), ST_GeomFromText('multipolygon (((0 0, 1 0, 0 1)), ((2 2, 1 2, 2 1)))'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+true
+=====
+---- QUERY
+select ST_Length(ST_Linestring(1,1, 1,2, 2,2, 2,1)), ST_Length(ST_Linestring(1,1, 1,4, 4,4, 4,1)), ST_Length(ST_Linestring(1,1, 1,7, 7,7, 7,1));
+---- TYPES
+DOUBLE, DOUBLE, DOUBLE
+---- RESULTS
+3.0,9.0,18.0
+====
+---- QUERY
+select ST_Area(ST_Polygon(1,1, 1,2, 2,2, 2,1)), ST_Area(ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+DOUBLE, DOUBLE
+---- RESULTS
+1.0,9.0
+====
+---- QUERY
+select ST_Contains(ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1), ST_Point(2, 3)), ST_Contains(ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1), ST_Point(8, 8));
+---- TYPES
+BOOLEAN, BOOLEAN
+---- RESULTS
+true,false
+====
+---- QUERY
+select ST_CoordDim(ST_Point(0., 3.)), ST_CoordDim(ST_PointZ(0., 3., 1));
+---- TYPES
+INT, INT
+---- RESULTS
+2,3
+====
+---- QUERY
+select ST_Crosses(st_linestring(2,0, 2,3), ST_Polygon(1,1, 1,4, 4,4, 4,1)), ST_Crosses(st_linestring(8,7, 7,8), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN, BOOLEAN
+---- RESULTS
+true,false
+====
+---- QUERY
+select ST_Dimension(ST_Point(0,0)), ST_Dimension(ST_LineString(1.5,2.5, 3.0,2.2));
+---- TYPES
+INT, INT
+---- RESULTS
+0,1
+====
+---- QUERY
+select ST_Disjoint(st_point(1,1), ST_Point(1,1)), ST_Disjoint(st_point(2,0), ST_Point(1,1));
+---- TYPES
+BOOLEAN, BOOLEAN
+---- RESULTS
+false,true
+====
+---- QUERY
+select ST_EnvIntersects(st_point(1,1), ST_Point(1,1)), ST_EnvIntersects(st_point(2,0), ST_Point(1,1));
+---- TYPES
+BOOLEAN, BOOLEAN
+---- RESULTS
+true,false
+====
+---- QUERY
+select ST_Equals(st_point(1,1), ST_Point(1,1)), ST_Equals(st_point(2,0), ST_Point(1,1));
+---- TYPES
+BOOLEAN, BOOLEAN
+---- RESULTS
+true,false
+====
+---- QUERY
+select ST_Intersects(st_point(1,1), ST_Point(1,1)), ST_Intersects(st_point(2,0), ST_Point(1,1));
+---- TYPES
+BOOLEAN, BOOLEAN
+---- RESULTS
+true,false
+====
+---- QUERY
+select ST_Is3D(ST_Point(0., 3.)), ST_Is3D(ST_PointZ(0., 3., 1));
+---- TYPES
+BOOLEAN, BOOLEAN
+---- RESULTS
+false,true
+====
+---- QUERY
+select ST_Overlaps(st_polygon(2,0, 2,3, 3,0), ST_Polygon(1,1, 1,4, 4,4, 4,1)), ST_Overlaps(st_polygon(2,0, 2,1, 3,1), ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- TYPES
+BOOLEAN, BOOLEAN
+---- RESULTS
+true,false
+====
+---- QUERY
+select ST_Touches(ST_Point(1, 3), ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1)), ST_Touches(ST_Point(8, 8), ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1));
+---- TYPES
+BOOLEAN, BOOLEAN
+---- RESULTS
+true,false
+====
+---- QUERY
+select ST_Within(ST_Point(2, 3), ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1)), ST_Within(ST_Point(8, 8), ST_Polygon(1, 1, 1, 4, 4, 4, 4, 1));
+---- TYPES
+BOOLEAN, BOOLEAN
+---- RESULTS
+true,false
+====
+---- QUERY
+SELECT ST_Intersects(ST_GeomFromGeoJson('{"type": "LineString", "coordinates": [[2.5,2.5], [8.0,0.0]]}'), ST_GeomFromGeoJson('{"type": "LineString", "coordinates": [[1.5,1.5], [0.0,7.0]]}'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+====
+---- QUERY
+SELECT ST_Intersects(ST_GeomFromJson('{"paths":[[[2.5,2.5],[8,0]]],"spatialReference":{"wkid":4326}}'), ST_GeomFromJson('{"paths":[[[1.5,1.5],[0,7]]],"spatialReference":{"wkid":4326}}'));
+---- TYPES
+BOOLEAN
+---- RESULTS
+false
+=====
+---- QUERY
+select ST_AsJson(ST_Polygon(1.5,2.5, 3.0,2.2, 2.2,1.1));
+---- RESULTS
+'{"rings":[[[1.5,2.5],[3,2.2],[2.2,1.1],[1.5,2.5]]]}'
+====
+---- QUERY
+select ST_AsJson(ST_Polygon(1.5,2.5, 3.0,2.2, 2.2,1.1, 1.5, 2.5));
+---- RESULTS
+'{"rings":[[[1.5,2.5],[3,2.2],[2.2,1.1],[1.5,2.5]]]}'
+====
+---- QUERY
+select ST_AsJson(ST_Polygon(0.1,2.2, 3.0,2.2, 2.2,1.1, 0.1, 2.2));
+---- RESULTS
+'{"rings":[[[0.1,2.2],[3,2.2],[2.2,1.1],[0.1,2.2]]]}'
+====
+---- QUERY
+select ST_AsJson(ST_Polygon(1,1, 1,4, 4,4, 4,1));
+---- RESULTS
+'{"rings":[[[1,1],[1,4],[4,4],[4,1],[1,1]]]}'
+====
+---- QUERY
+select ST_AsText(ST_Polygon(1,1, 1,4, 4,1));
+---- RESULTS
+'POLYGON ((1 1, 4 1, 1 4, 1 1))'
+====
+---- QUERY
+select ST_AsText(ST_Polygon(1,1, 4,1, 1,4));
+---- RESULTS
+'POLYGON ((1 1, 4 1, 1 4, 1 1))'
+====
+---- QUERY
+select ST_AsText(ST_Polygon(1,1, 1,4, 4,1, 1,1));
+---- RESULTS
+'POLYGON ((1 1, 4 1, 1 4, 1 1))'
+====
+---- QUERY
+select ST_AsText(ST_Polygon(1,1, 4,1, 1,4, 1,1));
+---- RESULTS
+'POLYGON ((1 1, 4 1, 1 4, 1 1))'
+====
+---- QUERY
+select st_AsText(ST_Envelope(ST_LineString(0,0, 2,2)));
+---- RESULTS
+'POLYGON ((0 0, 2 0, 2 2, 0 2, 0 0))'
+====
+---- QUERY
+select st_AsText(ST_Envelope(ST_Polygon(1,1, 4,1, 4,4, 1,4)));
+---- RESULTS
+'POLYGON ((1 1, 4 1, 4 4, 1 4, 1 1))'
+====
+---- QUERY
+select st_AsText(ST_Envelope(st_polygon(2,0, 2,3, 3,0)));
+---- RESULTS
+'POLYGON ((2 0, 3 0, 3 3, 2 3, 2 0))'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_Point(1,1), ST_Point(1,1)));
+---- RESULTS
+'POINT (1 1)'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_LineString(0,2, 0,4), ST_Point(0,3)));
+---- RESULTS
+'POINT (0 3)'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_GeomFromText('linestring(0 2, 0 0, 2 0)'), ST_GeomFromText('linestring(0 3, 0 1, 1 0, 3 0)')));
+---- RESULTS
+'MULTILINESTRING ((1 0, 2 0), (0 2, 0 1))'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_LineString(0,2, 0,4), ST_LineString(0,2, 0,4)));
+---- RESULTS
+'LINESTRING (0 2, 0 4)'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_LineString(0,2, 0,4), ST_LineString(0,3, 0,5)));
+---- RESULTS
+'LINESTRING (0 3, 0 4)'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_LineString(0,2, 0,5), ST_LineString(1,3, 0,3, 0,4, 1,4)));
+---- RESULTS
+'LINESTRING (0 3, 0 4)'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_LineString(0,2, 2,3), ST_Polygon(1,1, 4,1, 4,4, 1,4)));
+---- RESULTS
+'LINESTRING (1 2.5, 2 3)'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_Polygon(1,1, 1,4, 4,4, 4,1), ST_LineString(1,1, 1,4)));
+---- RESULTS
+'LINESTRING (1 1, 1 4)'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_Polygon(1,1, 1,4, 4,4, 4,1), ST_LineString(1,3, 1,4, 0,4)));
+---- RESULTS
+'LINESTRING (1 3, 1 4)'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_Polygon(2,0, 2,3, 3,0), ST_Polygon(1,1, 4,1, 4,4, 1,4)));
+---- RESULTS
+'POLYGON ((2 1, 2.6666666666666665 1, 2 3, 2 1))'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_Polygon(1,0, 3,0, 1,2), ST_Polygon(0,1, 2,1, 0,3)));
+---- RESULTS
+'POLYGON ((1 1, 2 1, 1 2, 1 1))'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_Polygon('polygon((1 0, 3 0, 1 2, 1 0))'), ST_Polygon('polygon((0 1, 2 1, 0 3, 0 1))')));
+---- RESULTS
+'POLYGON ((1 1, 2 1, 1 2, 1 1))'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_Polygon('polygon((3 0, 3 3, 0 3, 3 0))'), ST_Polygon('polygon((2 2, 5 2, 2 5, 2 2))')));
+---- RESULTS
+'POLYGON ((2 2, 3 2, 3 3, 2 3, 2 2))'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_Polygon('polygon((2 0, 2 2, 0 2, 2 0))'), ST_Polygon('polygon((1 1, 3 1, 1 3, 1 1))')));
+---- RESULTS
+'POLYGON ((1 1, 2 1, 2 2, 1 2, 1 1))'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_Polygon('polygon((0 0, 2 0, 0 2, 0 0))'), ST_Polygon('polygon((0 0, 1 0, 0 1, 0 0))')));
+---- RESULTS
+'POLYGON ((0 0, 1 0, 0 1, 0 0))'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_Polygon('polygon((0 0, 3 0, 0 3, 0 0))'), ST_Polygon('polygon((1 1, 2 1, 1 2, 1 1))')));
+---- RESULTS
+'POLYGON ((1 1, 2 1, 1 2, 1 1))'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_Polygon('polygon((0 0, 2 0, 0 2, 0 0))'), ST_Polygon('polygon((1 1, 1 0, 0 1, 1 1))')));
+---- RESULTS
+'POLYGON ((1 0, 1 1, 0 1, 1 0))'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_Polygon('polygon((0 0, 1 0, 0 1, 0 0))'), ST_Polygon('polygon((0 0, 1 0, 0 1, 0 0))')));
+---- RESULTS
+'POLYGON ((0 0, 1 0, 0 1, 0 0))'
+====
+---- QUERY
+select ST_AsText(ST_Intersection(ST_Polygon('polygon((0 0, 3 0, 0 3, 0 0))'), ST_Polygon('polygon((2 2, 2 0, 3 0, 3 3, 0 3, 0 2, 2 2))')));
+---- RESULTS
+'MULTIPOLYGON (((2 0, 3 0, 2 1, 2 0)), ((0 2, 1 2, 0 3, 0 2)))'
+====
+---- QUERY
+select ST_AsText(ST_Centroid(ST_Point(2, 3)));
+---- RESULTS
+'POINT (2 3)'
+====
+---- QUERY
+select ST_AsText(ST_Centroid(ST_GeomFromText('multipoint ((0 0), (1 1), (1 -1), (6 0))')));
+---- RESULTS
+'POINT (2 0)'
+====
+---- QUERY
+select ST_AsText(ST_Centroid(ST_GeomFromText('linestring (0 0, 6 0)')));
+---- RESULTS
+'POINT (3 0)'
+====
+---- QUERY
+select ST_AsText(ST_Centroid(ST_GeomFromText('polygon ((0 0, 0 8, 8 8, 8 0, 0 0))')));
+---- RESULTS
+'POINT (4 4)'
+====
+---- QUERY
+select ST_AsText(ST_Centroid(ST_GeomFromText('polygon ((1 1, 5 1, 3 4))')));
+---- RESULTS
+'POINT (3 2)'
+====
+---- QUERY
+select ST_Area(ST_Buffer(ST_GeomFromText('point (0 0)'), 1));
+---- TYPES
+DOUBLE
+---- RESULTS
+3.139350203046865
+====
+---- QUERY
+select ST_AsText(ST_Buffer(ST_GeomFromText('polygon ((0 0, 3 0, 3 2, 5 2, 5 5, 2 5, 2 3, 0 3, 0 0))'), -1));
+---- RESULTS
+'MULTIPOLYGON (((3 3, 4 3, 4 4, 3 4, 3 3)), ((1 1, 2 1, 2 2, 1 2, 1 1)))'
+====
+---- QUERY
+select ST_AsText(ST_Centroid(ST_GeomFromText('polygon ((0 0, 3 6, 6 0, 0 0))')));
+---- RESULTS
+'POINT (3 2)'
+====
+---- QUERY
+select ST_AsText(ST_Centroid(ST_GeomFromText('polygon ((0 0, 0 8, 8 0, 0 0))')));
+---- RESULTS
+'POINT (2.6666666666666665 2.6666666666666665)'
+====
+---- QUERY
+SELECT ST_AsText(ST_Difference(ST_MultiPoint(1, 1, 1.5, 1.5, 2, 2), ST_Point(1.5, 1.5)));
+---- RESULTS
+'MULTIPOINT ((1 1), (2 2))'
+====
+---- QUERY
+SELECT ST_AsText(ST_Difference(ST_Polygon(0, 0, 0, 10, 10, 10, 10, 0), ST_Polygon(0, 0, 0, 5, 5, 5, 5, 0)));
+---- RESULTS
+'POLYGON ((5 0, 10 0, 10 10, 0 10, 0 5, 5 5, 5 0))'
+====
+---- QUERY
+SELECT ST_AsText(ST_SymmetricDiff(ST_Point('point(0 0)'), ST_Point('point(2 2)')));
+---- RESULTS
+'MULTIPOINT ((0 0), (2 2))'
+====
+---- QUERY
+SELECT ST_AsText(ST_SymmetricDiff(ST_MultiPoint('multipoint((0 0))'), ST_MultiPoint('multipoint((2 2))')));
+---- RESULTS
+'MULTIPOINT ((0 0), (2 2))'
+====
+---- QUERY
+SELECT ST_AsText(ST_SymmetricDiff(ST_LineString('linestring(0 2, 2 2)'), ST_LineString('linestring(1 2, 3 2)')));
+---- RESULTS
+'MULTILINESTRING ((0 2, 1 2), (2 2, 3 2))'
+====
+---- QUERY
+SELECT ST_AsText(ST_SymmetricDiff(ST_Polygon('polygon((0 0, 2 0, 2 2, 0 2, 0 0))'), ST_Polygon('polygon((1 1, 3 1, 3 3, 1 3, 1 1))')));
+---- RESULTS
+'MULTIPOLYGON (((0 0, 2 0, 2 1, 1 1, 1 2, 0 2, 0 0)), ((2 1, 3 1, 3 3, 1 3, 1 2, 2 2, 2 1)))'
+====
+---- QUERY
+select ST_AsText(ST_GeomFromText('MultiLineString((0 80, 0.03 80.04))', 4326));
+---- RESULTS
+'MULTILINESTRING ((0 80, 0.03 80.04))'
+====
+---- QUERY
+select ST_AsJson(ST_Intersection(ST_Linestring(0,0, 1,1), ST_Linestring(2,2, 3,3)));
+---- RESULTS
+'{"rings":[]}'
+====
+---- QUERY
+select ST_AsJson(ST_GeomFromJson('{"x":0.0,"y":0.0}'));
+---- RESULTS
+'{"x":0,"y":0}'
+====
+---- QUERY
+select ST_AsText(ST_GeomFromGeoJson('{"type":"Point", "coordinates":[1.2, 2.4]}'));
+---- RESULTS
+'POINT (1.2 2.4)'
+====
+---- QUERY
+select ST_AsJson(ST_GeomFromGeoJson('{"type":"Point", "coordinates":[1.2, 2.4]}'));
+---- RESULTS
+'{"x":1.2,"y":2.4,"spatialReference":{"wkid":4326}}'
+====
+---- QUERY
+select ST_AsText(ST_GeomFromGeoJson('{"type":"LineString", "coordinates":[[1,2], [3,4]]}'));
+---- RESULTS
+'LINESTRING (1 2, 3 4)'
+====
+---- QUERY
+select ST_AsJson(ST_GeomFromJson('{"x":1.2,"y":2.4,"spatialReference":{"wkid":4326}}'));
+---- RESULTS
+'{"x":1.2,"y":2.4,"spatialReference":{"wkid":4326}}'
+====
+---- QUERY
+select ST_AsJson(ST_GeomFromJson('{"x":1.2,"y":2.4,"spatialReference":{"wkid": 0}}'));
+---- RESULTS
+'{"x":1.2,"y":2.4}'
+====
+---- QUERY
+select ST_AsGeoJson(ST_GeomFromJson('{"x":1.2,"y":2.4,"spatialReference":{"wkid":4326}}'));
+---- RESULTS
+'{"type":"Point","coordinates":[1.2,2.4],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}'
+====
+---- QUERY
+select ST_AsGeoJson(ST_GeomFromJson('{"x":1.2,"y":2.4,"spatialReference":{"wkid": 0}}'));
+---- RESULTS
+'{"type":"Point","coordinates":[1.2,2.4],"crs":null}'
+====
+---- QUERY
+select ST_AsText(ST_GeomFromGeoJson(ST_AsGeoJson(ST_Point(1.2, 2.4))));
+---- RESULTS
+'POINT (1.2 2.4)'
+====
+---- QUERY
+select ST_AsText(ST_GeomFromGeoJson(ST_AsGeoJson(ST_LineString(1,2, 3,4))));
+---- RESULTS
+'LINESTRING (1 2, 3 4)'
+====
+---- QUERY
+select ST_AsText(ST_Boundary(ST_LineString(0,1, 1,0)));
+---- RESULTS
+'MULTIPOINT ((0 1), (1 0))'
+====
+---- QUERY
+select ST_AsText(ST_Boundary(ST_Polygon(1,1, 4,1, 1,4)));
+---- RESULTS
+'LINESTRING (1 1, 4 1, 1 4, 1 1)'
+====
+---- QUERY
+select ST_AsText(ST_Boundary(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0), (1 1, 1 5, 5 1, 1 1))')));
+---- RESULTS
+'MULTILINESTRING ((0 0, 8 0, 0 8, 0 0), (1 1, 1 5, 5 1, 1 1))'
+====
+---- QUERY
+select ST_AsJson(ST_PointN(ST_GeomFromText('multipoint ((10 40), (40 30), (20 20), (30 10))', 0), 0));
+---- RESULTS
+'{"x":10,"y":40}'
+====
+---- QUERY
+select ST_AsJson(ST_PointN(ST_GeomFromText('multipoint ((10 40), (40 30), (20 20), (30 10))', 0), 1));
+---- RESULTS
+'{"x":10,"y":40}'
+====
+---- QUERY
+select ST_AsJson(ST_PointN(ST_GeomFromText('multipoint ((10 40), (40 30), (20 20), (30 10))', 0), 2));
+---- RESULTS
+'{"x":40,"y":30}'
+====
+---- QUERY
+select ST_AsJson(ST_PointN(ST_GeomFromText('multipoint ((10 40), (40 30), (20 20), (30 10))', 0), 4));
+---- RESULTS
+'{"x":30,"y":10}'
+====
+---- QUERY
+select ST_AsJson(ST_PointN(ST_GeomFromText('linestring (10.02 20.01, 10.32 23.98, 11.92 25.64)'), 0));
+---- RESULTS
+'{"x":10.02,"y":20.01}'
+====
+---- QUERY
+select ST_AsJson(ST_PointN(ST_GeomFromText('linestring (10.02 20.01, 10.32 23.98, 11.92 25.64)'), 1));
+---- RESULTS
+'{"x":10.02,"y":20.01}'
+====
+---- QUERY
+select ST_AsJson(ST_PointN(ST_GeomFromText('linestring (10.02 20.01, 10.32 23.98, 11.92 25.64)'), 2));
+---- RESULTS
+'{"x":10.32,"y":23.98}'
+====
+---- QUERY
+select ST_AsJson(ST_PointN(ST_GeomFromText('linestring (10.02 20.01, 10.32 23.98, 11.92 25.64)'), 3));
+---- RESULTS
+'{"x":11.92,"y":25.64}'
+====
+---- QUERY
+select ST_AsText(ST_ExteriorRing(ST_Polygon(1,1, 1,4, 4,1)));
+---- RESULTS
+'LINESTRING (1 1, 4 1, 1 4, 1 1)'
+====
+---- QUERY
+select ST_AsText(ST_ExteriorRing(ST_Polygon('polygon ((1 1, 4 1, 1 4))')));
+---- RESULTS
+'LINESTRING (1 1, 4 1, 1 4, 1 1)'
+====
+---- QUERY
+select ST_AsText(ST_ExteriorRing(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0), (1 1, 1 5, 5 1, 1 1))')));
+---- RESULTS
+'LINESTRING (0 0, 8 0, 0 8, 0 0)'
+====
+---- QUERY
+select ST_AsText(ST_GeometryN(ST_GeomFromText('multipoint (10 40, 40 30, 20 20, 30 10)'), 3));
+---- RESULTS
+'POINT (20 20)'
+====
+---- QUERY
+select ST_AsText(ST_GeometryN(ST_GeomFromText('multilinestring ((2 4, 10 10), (20 20, 7 8))'), 2));
+---- RESULTS
+'LINESTRING (20 20, 7 8)'
+====
+---- QUERY
+select ST_AsJson(ST_GeometryN(ST_GeomFromText('multilinestring ((2 4, 10 10), (20 20, 7 8))'), 2));
+---- RESULTS
+'{"paths":[[[20,20],[7,8]]]}'
+====
+---- QUERY
+select ST_AsText(ST_GeometryN(ST_GeomFromText('multipolygon (((3 3, 4 6, 5 3, 3 3)),((8 24, 1 28, 9 25, 8 24)), ((13 33, 7 36, 1 40, 10 43, 13 33)))'), 1));
+---- RESULTS
+'POLYGON ((3 3, 5 3, 4 6, 3 3))'
+====
+---- QUERY
+select ST_AsText(ST_InteriorRingN(ST_Polygon('polygon ((0 0, 8 0, 0 8, 0 0), (1 1, 1 5, 5 1, 1 1))'), 1));
+---- RESULTS
+'LINESTRING (1 1, 1 5, 5 1, 1 1)'
+=====
+---- QUERY
+select ST_GeodesicLengthWGS84(ST_SetSRID(ST_LineString(0,80, 0.03, 80.04), 4326));
+---- TYPES
+DOUBLE
+---- RESULTS
+4503.988488226892
+====
+---- QUERY
+select ST_GeodesicLengthWGS84(ST_SetSRID(ST_GeomFromText('MultiLineString((0 80, 0.03 80.04))'), 4326));
+---- TYPES
+DOUBLE
+---- RESULTS
+4503.988488226892
+====
+---- QUERY
+select ST_GeodesicLengthWGS84(ST_SetSRID(ST_LineString(179.98,-80, -179.98, -80.03), 4326));
+---- TYPES
+DOUBLE
+---- RESULTS
+3438.190416575652
+====
+---- QUERY
+select ST_GeodesicLengthWGS84(ST_SetSRID(ST_LineString(179.98,80, -179.98, 80.03), 4326));
+---- TYPES
+DOUBLE
+---- RESULTS
+3438.190416575652
+====
+---- QUERY
+select ST_GeodesicLengthWGS84(ST_SetSRID(ST_LineString(179.98,-0.01, -179.98, 0.02), 4326));
+---- TYPES
+DOUBLE
+---- RESULTS
+5552.589421311623
+====
+---- QUERY
+select ST_AsText(ST_SetSRID(ST_GeomFromText('MultiLineString((0 80, 0.03 80.04))'), 4326));
+---- RESULTS
+'MULTILINESTRING ((0 80, 0.03 80.04))'
+====
\ No newline at end of file
diff --git a/tests/common/environ.py b/tests/common/environ.py
index 405572b8d..dfb9c2d98 100644
--- a/tests/common/environ.py
+++ b/tests/common/environ.py
@@ -81,6 +81,8 @@ else:
   MANAGED_WAREHOUSE_DIR = 'test-warehouse'
 EXTERNAL_WAREHOUSE_DIR = 'test-warehouse'
 
+IS_APACHE_HIVE = os.environ.get("USE_APACHE_HIVE", False) == 'true'
+
 # Resolve any symlinks in the path.
 impalad_basedir = \
     os.path.realpath(os.path.join(IMPALA_HOME, 'be/build', build_type_dir)).rstrip('/')
diff --git a/tests/common/skip.py b/tests/common/skip.py
index c8737e51a..863ed0a26 100644
--- a/tests/common/skip.py
+++ b/tests/common/skip.py
@@ -25,7 +25,8 @@ from functools import partial
 
 from tests.common.environ import (ImpalaTestClusterProperties,
                                   IS_DOCKERIZED_TEST_CLUSTER, IS_BUGGY_EL6_KERNEL,
-                                  HIVE_MAJOR_VERSION, IS_REDHAT_6_DERIVATIVE)
+                                  HIVE_MAJOR_VERSION, IS_REDHAT_6_DERIVATIVE,
+                                  IS_APACHE_HIVE)
 from tests.common.kudu_test_suite import get_kudu_master_flag
 from tests.util.filesystem_utils import (
     IS_ABFS,
@@ -250,3 +251,8 @@ class SkipIfCatalogV2:
 class SkipIfOS:
   redhat6 = pytest.mark.skipif(IS_REDHAT_6_DERIVATIVE,
                                reason="Flaky on redhat or centos 6")
+
+
+class SkipIfApacheHive():
+  feature_not_supported = pytest.mark.skipif(IS_APACHE_HIVE,
+      reason="Apache Hive does not support this feature")
diff --git a/tests/custom_cluster/test_geospatial_library.py b/tests/custom_cluster/test_geospatial_library.py
new file mode 100644
index 000000000..9d5ff6001
--- /dev/null
+++ b/tests/custom_cluster/test_geospatial_library.py
@@ -0,0 +1,44 @@
+# 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.
+import pytest
+
+from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
+from tests.common.skip import SkipIfApacheHive
+
+ST_POINT_SIGNATURE = "BINARY\tst_point(STRING)\tJAVA\ttrue"
+SHOW_FUNCTIONS = "show functions in _impala_builtins"
+
+
+class TestGeospatialLibrary(CustomClusterTestSuite):
+  """Tests the geospatial_library backend flag"""
+
+  @classmethod
+  def get_workload(cls):
+    return 'functional-query'
+
+  @CustomClusterTestSuite.with_args(start_args='--geospatial_library=NONE')
+  @SkipIfApacheHive.feature_not_supported
+  @pytest.mark.execute_serially
+  def test_disabled(self):
+    result = self.execute_query(SHOW_FUNCTIONS)
+    assert ST_POINT_SIGNATURE not in result.data
+
+  @SkipIfApacheHive.feature_not_supported
+  @pytest.mark.execute_serially
+  def test_enabled(self):
+    result = self.execute_query(SHOW_FUNCTIONS)
+    assert ST_POINT_SIGNATURE in result.data
diff --git a/java/CMakeLists.txt b/tests/query_test/test_geospatial_functions.py
similarity index 64%
copy from java/CMakeLists.txt
copy to tests/query_test/test_geospatial_functions.py
index 7fcef3312..2d8108d9a 100644
--- a/java/CMakeLists.txt
+++ b/tests/query_test/test_geospatial_functions.py
@@ -15,10 +15,16 @@
 # specific language governing permissions and limitations
 # under the License.
 
-add_custom_target(validate_java_pom_versions ALL
-  COMMAND $ENV{IMPALA_HOME}/bin/validate-java-pom-versions.sh
-)
+from tests.common.impala_test_suite import ImpalaTestSuite
+from tests.common.skip import SkipIfApacheHive
 
-add_custom_target(java ALL DEPENDS gen-deps function-registry validate_java_pom_versions
-  COMMAND $ENV{IMPALA_HOME}/bin/mvn-quiet.sh -B install -DskipTests
-)
+
+class TestGeospatialFuctions(ImpalaTestSuite):
+  """Tests the geospatial builtin functions"""
+  @classmethod
+  def get_workload(cls):
+    return 'functional-query'
+
+  @SkipIfApacheHive.feature_not_supported
+  def test_esri_geospatial_functions(self, vector):
+    self.run_test_case('QueryTest/geospatial-esri', vector)