You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2020/08/21 06:30:26 UTC

[calcite] 02/02: [CALCITE-1861] Spatial index, based on Hilbert space-filling curve

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

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

commit eab043f4ef43112c16a9f6708e6c53a15b1cfbe0
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Fri Jun 30 10:45:31 2017 -0700

    [CALCITE-1861] Spatial index, based on Hilbert space-filling curve
    
    Add SQL function Hilbert. It uses Google's uzaygezen library,
    and our wraper code is copied from LocationTech's SFCurve
    project.
    
    Rewrite calls to ST_DWithin to use a range of on a Hilbert
    column, plus the call to ST_DWithin for safety.
    
    Implement function ST_MakeEnvelope.
    
    Use constant reduction to recognize constant geometry
    expressions before we apply SpatialRules.
    
    Move interface Geom, and other classes, from GeoFunctions into
    new utility class Geometries.
    
    Geometry literals (not in the SQL parser (yet), but in
    RexNode space).
    
    Make Geom implement Comparable, so that it can be a literal
    value.
    
    Move SqlStdOperatorTables to new package, and rename to
    SqlOperatorTables.
    
    Add RelOptTestBase.Sql.withCatalogReaderFactory and
    withConformance, to make it easier to run planner tests with
    a different schema or SQL dialect.
    
    Deprecate RelReferentialConstraint.getNumColumns().
    
    In tests and examples, call ST_Point(longitude, latitude)
    because conventionally x is longitude and y is latitude. (Yes,
    there's a tension here. In map references and English
    sentences latitude comes before longitude, but in Cartesian
    geometry x comes before y.)
    
    Close apache/calcite#2111
---
 bom/build.gradle.kts                               |   1 +
 core/build.gradle.kts                              |   1 +
 .../adapter/enumerable/RexToLixTranslator.java     |   7 +
 .../apache/calcite/jdbc/JavaTypeFactoryImpl.java   |   4 +-
 .../calcite/prepare/CalciteCatalogReader.java      |  26 +-
 .../apache/calcite/prepare/CalcitePrepareImpl.java |   8 +-
 .../org/apache/calcite/prepare/PlannerImpl.java    |   4 +-
 .../calcite/rel/metadata/RelMdAllPredicates.java   |   7 +-
 .../apache/calcite/rel/rel2sql/SqlImplementor.java |  92 +++---
 .../org/apache/calcite/rel/rules/SpatialRules.java | 322 +++++++++++++++++++++
 .../calcite/rel/type/RelDataTypeFactory.java       |  11 +-
 .../java/org/apache/calcite/rex/RexBuilder.java    |   7 +
 .../java/org/apache/calcite/rex/RexLiteral.java    |   8 +
 .../main/java/org/apache/calcite/rex/RexUtil.java  |  86 ++++++
 .../org/apache/calcite/runtime/GeoFunctions.java   | 301 +++++--------------
 .../org/apache/calcite/runtime/Geometries.java     | 295 +++++++++++++++++++
 .../org/apache/calcite/runtime/HilbertCurve2D.java | 158 ++++++++++
 .../calcite/runtime/SpaceFillingCurve2D.java       | 146 ++++++++++
 .../java/org/apache/calcite/schema/Schemas.java    |   2 +
 .../main/java/org/apache/calcite/sql/SqlKind.java  |  24 ++
 .../apache/calcite/sql/fun/SqlGeoFunctions.java    |  13 +-
 .../sql/fun/SqlLibraryOperatorTableFactory.java    |  10 +-
 .../sql/type/JavaToSqlTypeConversionRules.java     |   4 +-
 .../calcite/sql/util/ChainedSqlOperatorTable.java  |  22 +-
 .../apache/calcite/sql/util/SqlOperatorTables.java |  62 ++++
 .../sql/validate/SqlUserDefinedTableFunction.java  |   2 +
 .../org/apache/calcite/util/BuiltInMethod.java     |   2 +
 .../org/apache/calcite/test/RelOptRulesTest.java   | 159 +++++++---
 .../org/apache/calcite/test/RelOptTestBase.java    |  11 +
 .../org/apache/calcite/test/SqlToRelTestBase.java  |  12 +-
 .../org/apache/calcite/test/SqlValidatorTest.java  |   2 +-
 .../org/apache/calcite/test/catalog/Fixture.java   |   2 +-
 .../calcite/test/catalog/MockCatalogReader.java    |  34 ++-
 .../test/catalog/MockCatalogReaderExtended.java    |  84 ++++--
 .../test/catalog/MockCatalogReaderSimple.java      |  11 +-
 .../java/org/apache/calcite/tools/PlannerTest.java |   4 +-
 .../org/apache/calcite/test/RelOptRulesTest.xml    | 194 ++++++++++++-
 core/src/test/resources/sql/spatial.iq             |  57 +++-
 gradle.properties                                  |   1 +
 .../org/apache/calcite/linq4j/function/Hints.java  |  40 +++
 .../calcite/piglet/PigUserDefinedFunction.java     |   2 +-
 site/_docs/reference.md                            |   2 +-
 site/_docs/spatial.md                              |  79 +++++
 43 files changed, 1900 insertions(+), 419 deletions(-)

diff --git a/bom/build.gradle.kts b/bom/build.gradle.kts
index ed4e16d..18beb0f 100644
--- a/bom/build.gradle.kts
+++ b/bom/build.gradle.kts
@@ -57,6 +57,7 @@ dependencies {
         apiv("com.google.code.findbugs:jsr305", "findbugs.jsr305")
         apiv("com.google.guava:guava")
         apiv("com.google.protobuf:protobuf-java", "protobuf")
+        apiv("com.google.uzaygezen:uzaygezen-core", "uzaygezen")
         apiv("com.h2database:h2")
         apiv("com.jayway.jsonpath:json-path")
         apiv("com.joestelmach:natty")
diff --git a/core/build.gradle.kts b/core/build.gradle.kts
index f1b9c4b..56ce9f4 100644
--- a/core/build.gradle.kts
+++ b/core/build.gradle.kts
@@ -49,6 +49,7 @@ dependencies {
     implementation("com.fasterxml.jackson.dataformat:jackson-dataformat-yaml")
     implementation("com.google.code.findbugs:jsr305"/* optional*/)
     implementation("com.google.guava:guava")
+    implementation("com.google.uzaygezen:uzaygezen-core")
     implementation("com.jayway.jsonpath:json-path")
     implementation("com.yahoo.datasketches:sketches-core")
     implementation("commons-codec:commons-codec")
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
index 74318ea..82d1858 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
@@ -47,6 +47,8 @@ import org.apache.calcite.rex.RexRangeRef;
 import org.apache.calcite.rex.RexSubQuery;
 import org.apache.calcite.rex.RexTableInputRef;
 import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.runtime.GeoFunctions;
+import org.apache.calcite.runtime.Geometries;
 import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.sql.SqlIntervalQualifier;
 import org.apache.calcite.sql.SqlOperator;
@@ -732,6 +734,11 @@ public class RexToLixTranslator implements RexVisitor<RexToLixTranslator.Result>
           Expressions.constant(
               literal.getValueAs(byte[].class),
               byte[].class));
+    case GEOMETRY:
+      final Geometries.Geom geom = literal.getValueAs(Geometries.Geom.class);
+      final String wkt = GeoFunctions.ST_AsWKT(geom);
+      return Expressions.call(null, BuiltInMethod.ST_GEOM_FROM_TEXT.method,
+          Expressions.constant(wkt));
     case SYMBOL:
       value2 = literal.getValueAs(Enum.class);
       javaClass = value2.getClass();
diff --git a/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
index 7ee7f3e..8b2c6f6 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
@@ -27,7 +27,7 @@ import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rel.type.RelRecordType;
-import org.apache.calcite.runtime.GeoFunctions;
+import org.apache.calcite.runtime.Geometries;
 import org.apache.calcite.runtime.Unit;
 import org.apache.calcite.sql.type.BasicSqlType;
 import org.apache.calcite.sql.type.IntervalSqlType;
@@ -209,7 +209,7 @@ public class JavaTypeFactoryImpl
       case VARBINARY:
         return ByteString.class;
       case GEOMETRY:
-        return GeoFunctions.Geom.class;
+        return Geometries.Geom.class;
       case SYMBOL:
         return Enum.class;
       case ANY:
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
index b1d1d20..e0f2f5c 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
@@ -19,6 +19,7 @@ package org.apache.calcite.prepare;
 import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.linq4j.function.Hints;
 import org.apache.calcite.model.ModelHandler;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.rel.type.RelDataType;
@@ -333,7 +334,7 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
             i -> function.getParameters().get(i).getName(),
             i -> function.getParameters().get(i).isOptional());
 
-    final SqlKind kind = SqlKind.OTHER_FUNCTION;
+    final SqlKind kind = kind(function);
     if (function instanceof ScalarFunction) {
       final SqlReturnTypeInference returnTypeInference =
           infer((ScalarFunction) function);
@@ -357,6 +358,23 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
     }
   }
 
+  /** Deduces the {@link org.apache.calcite.sql.SqlKind} of a user-defined
+   * function based on a {@link Hints} annotation, if present. */
+  private static SqlKind kind(org.apache.calcite.schema.Function function) {
+    if (function instanceof ScalarFunctionImpl) {
+      Hints hints =
+          ((ScalarFunctionImpl) function).method.getAnnotation(Hints.class);
+      if (hints != null) {
+        for (String hint : hints.value()) {
+          if (hint.startsWith("SqlKind:")) {
+            return SqlKind.valueOf(hint.substring("SqlKind:".length()));
+          }
+        }
+      }
+    }
+    return SqlKind.OTHER_FUNCTION;
+  }
+
   private static SqlReturnTypeInference infer(final ScalarFunction function) {
     return opBinding -> {
       final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
@@ -392,18 +410,18 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
   }
 
   public List<SqlOperator> getOperatorList() {
-    final ImmutableList.Builder<SqlOperator> b = ImmutableList.builder();
+    final ImmutableList.Builder<SqlOperator> builder = ImmutableList.builder();
     for (List<String> schemaPath : schemaPaths) {
       CalciteSchema schema =
           SqlValidatorUtil.getSchema(rootSchema, schemaPath, nameMatcher);
       if (schema != null) {
         for (String name : schema.getFunctionNames()) {
           schema.getFunctions(name, true).forEach(f ->
-              b.add(toOp(new SqlIdentifier(name, SqlParserPos.ZERO), f)));
+              builder.add(toOp(new SqlIdentifier(name, SqlParserPos.ZERO), f)));
         }
       }
     }
-    return b.build();
+    return builder.build();
   }
 
   public CalciteSchema getRootSchema() {
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index e826ed6..c435774 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -96,7 +96,7 @@ import org.apache.calcite.sql.parser.SqlParserImplFactory;
 import org.apache.calcite.sql.parser.impl.SqlParserImpl;
 import org.apache.calcite.sql.type.ExtraSqlTypes;
 import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+import org.apache.calcite.sql.util.SqlOperatorTables;
 import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql2rel.SqlRexConvertletTable;
@@ -704,8 +704,10 @@ public class CalcitePrepareImpl implements CalcitePrepare {
     final SqlOperatorTable opTab0 =
         context.config().fun(SqlOperatorTable.class,
             SqlStdOperatorTable.instance());
-    final SqlOperatorTable opTab =
-        ChainedSqlOperatorTable.of(opTab0, catalogReader);
+    final List<SqlOperatorTable> list = new ArrayList<>();
+    list.add(opTab0);
+    list.add(catalogReader);
+    final SqlOperatorTable opTab = SqlOperatorTables.chain(list);
     final JavaTypeFactory typeFactory = context.getTypeFactory();
     final CalciteConnectionConfig connectionConfig = context.config();
     final SqlValidator.Config config = SqlValidator.Config.DEFAULT
diff --git a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
index ba3426a..357b12a 100644
--- a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
@@ -47,7 +47,7 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+import org.apache.calcite.sql.util.SqlOperatorTables;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql2rel.RelDecorrelator;
 import org.apache.calcite.sql2rel.SqlRexConvertletTable;
@@ -321,7 +321,7 @@ public class PlannerImpl implements Planner, ViewExpander {
 
   private SqlValidator createSqlValidator(CalciteCatalogReader catalogReader) {
     final SqlOperatorTable opTab =
-        ChainedSqlOperatorTable.of(operatorTable, catalogReader);
+        SqlOperatorTables.chain(operatorTable, catalogReader);
     return new CalciteSqlValidator(opTab,
         catalogReader,
         typeFactory,
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdAllPredicates.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdAllPredicates.java
index 7945780..b3d88df 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdAllPredicates.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdAllPredicates.java
@@ -105,7 +105,12 @@ public class RelMdAllPredicates
   /**
    * Extract predicates for a table scan.
    */
-  public RelOptPredicateList getAllPredicates(TableScan table, RelMetadataQuery mq) {
+  public RelOptPredicateList getAllPredicates(TableScan scan, RelMetadataQuery mq) {
+    final BuiltInMetadata.AllPredicates.Handler handler =
+        scan.getTable().unwrap(BuiltInMetadata.AllPredicates.Handler.class);
+    if (handler != null) {
+      return handler.getAllPredicates(scan, mq);
+    }
     return RelOptPredicateList.EMPTY;
   }
 
diff --git a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
index 1ec9ee0..a172112 100644
--- a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
+++ b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
@@ -702,50 +702,7 @@ public abstract class SqlImplementor {
         }
 
       case LITERAL:
-        final RexLiteral literal = (RexLiteral) rex;
-        if (literal.getTypeName() == SqlTypeName.SYMBOL) {
-          final Enum symbol = (Enum) literal.getValue();
-          return SqlLiteral.createSymbol(symbol, POS);
-        }
-        switch (literal.getTypeName().getFamily()) {
-        case CHARACTER:
-          return SqlLiteral.createCharString((String) literal.getValue2(), POS);
-        case NUMERIC:
-        case EXACT_NUMERIC:
-          return SqlLiteral.createExactNumeric(
-              literal.getValueAs(BigDecimal.class).toPlainString(), POS);
-        case APPROXIMATE_NUMERIC:
-          return SqlLiteral.createApproxNumeric(
-              literal.getValueAs(BigDecimal.class).toPlainString(), POS);
-        case BOOLEAN:
-          return SqlLiteral.createBoolean(literal.getValueAs(Boolean.class),
-              POS);
-        case INTERVAL_YEAR_MONTH:
-        case INTERVAL_DAY_TIME:
-          final boolean negative = literal.getValueAs(Boolean.class);
-          return SqlLiteral.createInterval(negative ? -1 : 1,
-              literal.getValueAs(String.class),
-              literal.getType().getIntervalQualifier(), POS);
-        case DATE:
-          return SqlLiteral.createDate(literal.getValueAs(DateString.class),
-              POS);
-        case TIME:
-          return SqlLiteral.createTime(literal.getValueAs(TimeString.class),
-              literal.getType().getPrecision(), POS);
-        case TIMESTAMP:
-          return SqlLiteral.createTimestamp(
-              literal.getValueAs(TimestampString.class),
-              literal.getType().getPrecision(), POS);
-        case ANY:
-        case NULL:
-          switch (literal.getTypeName()) {
-          case NULL:
-            return SqlLiteral.createNull(POS);
-          // fall through
-          }
-        default:
-          throw new AssertionError(literal + ": " + literal.getTypeName());
-        }
+        return SqlImplementor.toSql((RexLiteral) rex);
 
       case CASE:
         final RexCall caseCall = (RexCall) rex;
@@ -1203,6 +1160,53 @@ public abstract class SqlImplementor {
     }
   }
 
+  /** Converts a {@link RexLiteral} to a {@link SqlLiteral}. */
+  public static SqlLiteral toSql(RexLiteral literal) {
+    if (literal.getTypeName() == SqlTypeName.SYMBOL) {
+      final Enum symbol = (Enum) literal.getValue();
+      return SqlLiteral.createSymbol(symbol, POS);
+    }
+    switch (literal.getTypeName().getFamily()) {
+    case CHARACTER:
+      return SqlLiteral.createCharString((String) literal.getValue2(), POS);
+    case NUMERIC:
+    case EXACT_NUMERIC:
+      return SqlLiteral.createExactNumeric(
+          literal.getValueAs(BigDecimal.class).toPlainString(), POS);
+    case APPROXIMATE_NUMERIC:
+      return SqlLiteral.createApproxNumeric(
+          literal.getValueAs(BigDecimal.class).toPlainString(), POS);
+    case BOOLEAN:
+      return SqlLiteral.createBoolean(literal.getValueAs(Boolean.class),
+          POS);
+    case INTERVAL_YEAR_MONTH:
+    case INTERVAL_DAY_TIME:
+      final boolean negative = literal.getValueAs(Boolean.class);
+      return SqlLiteral.createInterval(negative ? -1 : 1,
+          literal.getValueAs(String.class),
+          literal.getType().getIntervalQualifier(), POS);
+    case DATE:
+      return SqlLiteral.createDate(literal.getValueAs(DateString.class),
+          POS);
+    case TIME:
+      return SqlLiteral.createTime(literal.getValueAs(TimeString.class),
+          literal.getType().getPrecision(), POS);
+    case TIMESTAMP:
+      return SqlLiteral.createTimestamp(
+          literal.getValueAs(TimestampString.class),
+          literal.getType().getPrecision(), POS);
+    case ANY:
+    case NULL:
+      switch (literal.getTypeName()) {
+      case NULL:
+        return SqlLiteral.createNull(POS);
+      // fall through
+      }
+    default:
+      throw new AssertionError(literal + ": " + literal.getTypeName());
+    }
+  }
+
   /** Simple implementation of {@link Context} that cannot handle sub-queries
    * or correlations. Because it is so simple, you do not need to create a
    * {@link SqlImplementor} or {@link org.apache.calcite.tools.RelBuilder}
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SpatialRules.java b/core/src/main/java/org/apache/calcite/rel/rules/SpatialRules.java
new file mode 100644
index 0000000..f9885df
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SpatialRules.java
@@ -0,0 +1,322 @@
+/*
+ * 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.calcite.rel.rules;
+
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.runtime.GeoFunctions;
+import org.apache.calcite.runtime.Geometries;
+import org.apache.calcite.runtime.HilbertCurve2D;
+import org.apache.calcite.runtime.SpaceFillingCurve2D;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.tools.RelBuilder;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Point;
+import com.google.common.collect.ImmutableList;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+
+/**
+ * Collection of planner rules that convert
+ * calls to spatial functions into more efficient expressions.
+ *
+ * <p>The rules allow Calcite to use spatial indexes. For example the following
+ * query:
+ *
+ * <blockquote>SELECT ...
+ * FROM Restaurants AS r
+ * WHERE ST_DWithin(ST_Point(10, 20), ST_Point(r.longitude, r.latitude), 5)
+ * </blockquote>
+ *
+ * <p>is rewritten to
+ *
+ * <blockquote>SELECT ...
+ * FROM Restaurants AS r
+ * WHERE (r.h BETWEEN 100 AND 150
+ *        OR r.h BETWEEN 170 AND 185)
+ * AND ST_DWithin(ST_Point(10, 20), ST_Point(r.longitude, r.latitude), 5)
+ * </blockquote>
+ *
+ * <p>if there is the constraint
+ *
+ * <blockquote>CHECK (h = Hilbert(8, r.longitude, r.latitude))</blockquote>
+ *
+ * <p>If the {@code Restaurants} table is sorted on {@code h} then the latter
+ * query can be answered using two limited range-scans, and so is much more
+ * efficient.
+ *
+ * <p>Note that the original predicate
+ * {@code ST_DWithin(ST_Point(10, 20), ST_Point(r.longitude, r.latitude), 5)}
+ * is still present, but is evaluated after the approximate predicate has
+ * eliminated many potential matches.
+ */
+public abstract class SpatialRules {
+
+  private SpatialRules() {}
+
+  private static final RexUtil.RexFinder DWITHIN_FINDER =
+      RexUtil.find(EnumSet.of(SqlKind.ST_DWITHIN, SqlKind.ST_CONTAINS));
+
+  private static final RexUtil.RexFinder HILBERT_FINDER =
+      RexUtil.find(SqlKind.HILBERT);
+
+  public static final RelOptRule INSTANCE =
+      FilterHilbertRule.Config.DEFAULT.toRule();
+
+  /** Returns a geometry if an expression is constant, null otherwise. */
+  private static Geometries.Geom constantGeom(RexNode e) {
+    switch (e.getKind()) {
+    case CAST:
+      return constantGeom(((RexCall) e).getOperands().get(0));
+    case LITERAL:
+      return (Geometries.Geom) ((RexLiteral) e).getValue();
+    default:
+      return null;
+    }
+  }
+
+  /** Rule that converts ST_DWithin in a Filter condition into a predicate on
+   * a Hilbert curve. */
+  @SuppressWarnings("WeakerAccess")
+  public static class FilterHilbertRule
+      extends RelRule<FilterHilbertRule.Config> {
+    protected FilterHilbertRule(Config config) {
+      super(config);
+    }
+
+    @Override public void onMatch(RelOptRuleCall call) {
+      final Filter filter = call.rel(0);
+      final List<RexNode> conjunctions = new ArrayList<>();
+      RelOptUtil.decomposeConjunction(filter.getCondition(), conjunctions);
+
+      // Match a predicate
+      //   r.hilbert = hilbert(r.longitude, r.latitude)
+      // to one of the conjunctions
+      //   ST_DWithin(ST_Point(x, y), ST_Point(r.longitude, r.latitude), d)
+      // and if it matches add a new conjunction before it,
+      //   r.hilbert between h1 and h2
+      //   or r.hilbert between h3 and h4
+      // where {[h1, h2], [h3, h4]} are the ranges of the Hilbert curve
+      // intersecting the square
+      //   (r.longitude - d, r.latitude - d, r.longitude + d, r.latitude + d)
+      final RelOptPredicateList predicates =
+          call.getMetadataQuery().getAllPredicates(filter.getInput());
+      int changeCount = 0;
+      for (RexNode predicate : predicates.pulledUpPredicates) {
+        final RelBuilder builder = call.builder();
+        if (predicate.getKind() == SqlKind.EQUALS) {
+          final RexCall eqCall = (RexCall) predicate;
+          if (eqCall.operands.get(0) instanceof RexInputRef
+              && eqCall.operands.get(1).getKind() == SqlKind.HILBERT) {
+            final RexInputRef ref  = (RexInputRef) eqCall.operands.get(0);
+            final RexCall hilbert = (RexCall) eqCall.operands.get(1);
+            final RexUtil.RexFinder finder = RexUtil.find(ref);
+            if (finder.anyContain(conjunctions)) {
+              // If the condition already contains "ref", it is probable that
+              // this rule has already fired once.
+              continue;
+            }
+            for (int i = 0; i < conjunctions.size();) {
+              final List<RexNode> replacements =
+                  replaceSpatial(conjunctions.get(i), builder, ref, hilbert);
+              if (replacements != null) {
+                conjunctions.remove(i);
+                conjunctions.addAll(i, replacements);
+                i += replacements.size();
+                ++changeCount;
+              } else {
+                ++i;
+              }
+            }
+          }
+        }
+        if (changeCount > 0) {
+          call.transformTo(
+              builder.push(filter.getInput())
+                  .filter(conjunctions)
+                  .build());
+          return; // we found one useful constraint; don't look for more
+        }
+      }
+    }
+
+    /** Rewrites a spatial predicate to a predicate on a Hilbert curve.
+     *
+     * <p>Returns null if the predicate cannot be rewritten;
+     * a 1-element list (new) if the predicate can be fully rewritten;
+     * returns a 2-element list (new, original) if the new predicate allows
+     * some false positives.
+     *
+     * @param conjunction Original predicate
+     * @param builder Builder
+     * @param ref Reference to Hilbert column
+     * @param hilbert Function call that populates Hilbert column
+     *
+     * @return List containing rewritten predicate and original, or null
+     */
+    static List<RexNode> replaceSpatial(RexNode conjunction, RelBuilder builder,
+        RexInputRef ref, RexCall hilbert) {
+      final RexNode op0;
+      final RexNode op1;
+      final Geometries.Geom g0;
+      switch (conjunction.getKind()) {
+      case ST_DWITHIN:
+        final RexCall within = (RexCall) conjunction;
+        op0 = within.operands.get(0);
+        g0 = constantGeom(op0);
+        op1 = within.operands.get(1);
+        final Geometries.Geom g1 = constantGeom(op1);
+        if (RexUtil.isLiteral(within.operands.get(2), true)) {
+          final Number distance =
+              (Number) RexLiteral.value(within.operands.get(2));
+          switch (Double.compare(distance.doubleValue(), 0D)) {
+          case -1: // negative distance
+            return ImmutableList.of(builder.getRexBuilder().makeLiteral(false));
+
+          case 0: // zero distance
+            // Change "ST_DWithin(g, p, 0)" to "g = p"
+            conjunction = builder.equals(op0, op1);
+            // fall through
+
+          case 1:
+            if (g0 != null
+                && op1.getKind() == SqlKind.ST_POINT
+                && ((RexCall) op1).operands.equals(hilbert.operands)) {
+              // Add the new predicate before the existing predicate
+              // because it is cheaper to execute (albeit less selective).
+              return ImmutableList.of(
+                  hilbertPredicate(builder.getRexBuilder(), ref, g0, distance),
+                  conjunction);
+            } else if (g1 != null && op0.getKind() == SqlKind.ST_POINT
+                && ((RexCall) op0).operands.equals(hilbert.operands)) {
+              // Add the new predicate before the existing predicate
+              // because it is cheaper to execute (albeit less selective).
+              return ImmutableList.of(
+                  hilbertPredicate(builder.getRexBuilder(), ref, g1, distance),
+                  conjunction);
+            }
+            return null; // cannot rewrite
+
+          default:
+            throw new AssertionError("invalid sign: " + distance);
+          }
+        }
+        return null; // cannot rewrite
+
+      case ST_CONTAINS:
+        final RexCall contains = (RexCall) conjunction;
+        op0 = contains.operands.get(0);
+        g0 = constantGeom(op0);
+        op1 = contains.operands.get(1);
+        if (g0 != null
+            && op1.getKind() == SqlKind.ST_POINT
+            && ((RexCall) op1).operands.equals(hilbert.operands)) {
+          // Add the new predicate before the existing predicate
+          // because it is cheaper to execute (albeit less selective).
+          return ImmutableList.of(
+              hilbertPredicate(builder.getRexBuilder(), ref, g0),
+              conjunction);
+        }
+        return null; // cannot rewrite
+
+      default:
+        return null; // cannot rewrite
+      }
+    }
+
+    /** Creates a predicate on the column that contains the index on the Hilbert
+     * curve.
+     *
+     * <p>The predicate is a safe approximation. That is, it may allow some
+     * points that are not within the distance, but will never disallow a point
+     * that is within the distance.
+     *
+     * <p>Returns FALSE if the distance is negative (the ST_DWithin function
+     * would always return FALSE) and returns an {@code =} predicate if distance
+     * is 0. But usually returns a list of ranges,
+     * {@code ref BETWEEN c1 AND c2 OR ref BETWEEN c3 AND c4}. */
+    private static RexNode hilbertPredicate(RexBuilder rexBuilder,
+        RexInputRef ref, Geometries.Geom g, Number distance) {
+      if (distance.doubleValue() == 0D
+          && Geometries.type(g.g()) == Geometries.Type.POINT) {
+        final Point p = (Point) g.g();
+        final HilbertCurve2D hilbert = new HilbertCurve2D(8);
+        final long index = hilbert.toIndex(p.getX(), p.getY());
+        return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, ref,
+            rexBuilder.makeExactLiteral(BigDecimal.valueOf(index)));
+      }
+      final Geometries.Geom g2 =
+          GeoFunctions.ST_Buffer(g, distance.doubleValue());
+      return hilbertPredicate(rexBuilder, ref, g2);
+    }
+
+    private static RexNode hilbertPredicate(RexBuilder rexBuilder,
+        RexInputRef ref, Geometries.Geom g2) {
+      final Geometries.Geom g3 = GeoFunctions.ST_Envelope(g2);
+      final Envelope env = (Envelope) g3.g();
+      final HilbertCurve2D hilbert = new HilbertCurve2D(8);
+      final List<SpaceFillingCurve2D.IndexRange> ranges =
+          hilbert.toRanges(env.getXMin(), env.getYMin(), env.getXMax(),
+              env.getYMax(), new SpaceFillingCurve2D.RangeComputeHints());
+      final List<RexNode> nodes = new ArrayList<>();
+      for (SpaceFillingCurve2D.IndexRange range : ranges) {
+        final BigDecimal lowerBd = BigDecimal.valueOf(range.lower());
+        final BigDecimal upperBd = BigDecimal.valueOf(range.upper());
+        nodes.add(
+            rexBuilder.makeCall(
+                SqlStdOperatorTable.AND,
+                rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
+                    ref,
+                    rexBuilder.makeExactLiteral(lowerBd)),
+                rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
+                    ref,
+                    rexBuilder.makeExactLiteral(upperBd))));
+      }
+      return rexBuilder.makeCall(SqlStdOperatorTable.OR, nodes);
+    }
+
+    /** Rule configuration. */
+    public interface Config extends RelRule.Config {
+      Config DEFAULT = EMPTY
+          .withOperandSupplier(b ->
+              b.operand(Filter.class)
+                  .predicate(f -> DWITHIN_FINDER.inFilter(f)
+                      && !HILBERT_FINDER.inFilter(f))
+                  .anyInputs())
+          .as(Config.class);
+
+      @Override default FilterHilbertRule toRule() {
+        return new FilterHilbertRule(this);
+      }
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
index e82d3b4..f4cf074 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
@@ -421,6 +421,7 @@ public interface RelDataTypeFactory {
     private final List<RelDataType> types = new ArrayList<>();
     private StructKind kind = StructKind.FULLY_QUALIFIED;
     private final RelDataTypeFactory typeFactory;
+    private boolean nullableRecord = false;
 
     /**
      * Creates a Builder with the given type factory.
@@ -547,6 +548,12 @@ public interface RelDataTypeFactory {
       return this;
     }
 
+    /** Sets whether the record type will be nullable. */
+    public Builder nullableRecord(boolean nullableRecord) {
+      this.nullableRecord = nullableRecord;
+      return this;
+    }
+
     /**
      * Makes sure that field names are unique.
      */
@@ -564,7 +571,9 @@ public interface RelDataTypeFactory {
      * Creates a struct type with the current contents of this builder.
      */
     public RelDataType build() {
-      return typeFactory.createStructType(kind, types, names);
+      return typeFactory.createTypeWithNullability(
+          typeFactory.createStructType(kind, types, names),
+          nullableRecord);
     }
 
     /** Creates a dynamic struct type with the current contents of this
diff --git a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
index 760c1b5..676cf49 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
@@ -27,6 +27,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.runtime.FlatLists;
+import org.apache.calcite.runtime.Geometries;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlCollation;
 import org.apache.calcite.sql.SqlIntervalQualifier;
@@ -1488,6 +1489,9 @@ public class RexBuilder {
       }
       return new RexLiteral((Comparable) FlatLists.of(operands), type,
           sqlTypeName);
+    case GEOMETRY:
+      return new RexLiteral((Comparable) value, guessType(value),
+          SqlTypeName.GEOMETRY);
     case ANY:
       return makeLiteral(value, guessType(value), allowCast);
     default:
@@ -1621,6 +1625,9 @@ public class RexBuilder {
       return typeFactory.createSqlType(SqlTypeName.BINARY,
           ((ByteString) value).length());
     }
+    if (value instanceof Geometries.Geom) {
+      return typeFactory.createSqlType(SqlTypeName.GEOMETRY);
+    }
     throw new AssertionError("unknown type " + value.getClass());
   }
 
diff --git a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
index 4195966..70d23a2 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
@@ -22,6 +22,8 @@ import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.config.CalciteSystemProperty;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.GeoFunctions;
+import org.apache.calcite.runtime.Geometries;
 import org.apache.calcite.sql.SqlCollation;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
@@ -360,6 +362,8 @@ public class RexLiteral extends RexNode {
     case ROW:
     case MULTISET:
       return value instanceof List;
+    case GEOMETRY:
+      return value instanceof Geometries.Geom;
     case ANY:
       // Literal of type ANY is not legal. "CAST(2 AS ANY)" remains
       // an integer literal surrounded by a cast function.
@@ -681,6 +685,10 @@ public class RexLiteral extends RexNode {
               }
             }).toString());
         break;
+      case GEOMETRY:
+        final String wkt = GeoFunctions.ST_AsWKT((Geometries.Geom) value);
+        destination.append(wkt);
+        break;
       default:
         assert valueMatchesType(value, typeName, true);
         throw Util.needToImplement(typeName);
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index e1aff76..ec42695 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -519,6 +519,42 @@ public class RexUtil {
     return n;
   }
 
+  /** Returns a visitor that finds nodes of a given {@link SqlKind}. */
+  public static RexFinder find(final SqlKind kind) {
+    return new RexFinder() {
+      @Override public Void visitCall(RexCall call) {
+        if (call.getKind() == kind) {
+          throw Util.FoundOne.NULL;
+        }
+        return super.visitCall(call);
+      }
+    };
+  }
+
+  /** Returns a visitor that finds nodes of given {@link SqlKind}s. */
+  public static RexFinder find(final Set<SqlKind> kinds) {
+    return new RexFinder() {
+      @Override public Void visitCall(RexCall call) {
+        if (kinds.contains(call.getKind())) {
+          throw Util.FoundOne.NULL;
+        }
+        return super.visitCall(call);
+      }
+    };
+  }
+
+  /** Returns a visitor that finds a particular {@link RexInputRef}. */
+  public static RexFinder find(final RexInputRef ref) {
+    return new RexFinder() {
+      @Override public Void visitInputRef(RexInputRef inputRef) {
+        if (ref.equals(inputRef)) {
+          throw Util.FoundOne.NULL;
+        }
+        return super.visitInputRef(inputRef);
+      }
+    };
+  }
+
   /**
    * Walks over an expression and determines whether it is constant.
    */
@@ -2747,4 +2783,54 @@ public class RexUtil {
       return simplify.rexBuilder.makeCast(call.getType(), simplifiedNode, matchNullability);
     }
   }
+
+  /** Visitor that tells whether a node matching a particular description exists
+   * in a tree. */
+  public abstract static class RexFinder extends RexVisitorImpl<Void> {
+    RexFinder() {
+      super(true);
+    }
+
+    /** Returns whether a {@link Project} contains the kind of expression we
+     * seek. */
+    public boolean inProject(Project project) {
+      return anyContain(project.getProjects());
+    }
+
+    /** Returns whether a {@link Filter} contains the kind of expression we
+     * seek. */
+    public boolean inFilter(Filter filter) {
+      return contains(filter.getCondition());
+    }
+
+    /** Returns whether a {@link Join} contains kind of expression we seek. */
+    public boolean inJoin(Join join) {
+      return contains(join.getCondition());
+    }
+
+    /** Returns whether the given expression contains what this RexFinder
+     * seeks. */
+    public boolean contains(RexNode node) {
+      try {
+        node.accept(RexFinder.this);
+        return false;
+      } catch (Util.FoundOne e) {
+        return true;
+      }
+    }
+
+    /** Returns whether any of the given expressions contain what this RexFinder
+     * seeks. */
+    public boolean anyContain(Iterable<? extends RexNode> nodes) {
+      try {
+        for (RexNode node : nodes) {
+          node.accept(RexFinder.this);
+        }
+        return false;
+      } catch (Util.FoundOne e) {
+        return true;
+      }
+    }
+  }
+
 }
diff --git a/core/src/main/java/org/apache/calcite/runtime/GeoFunctions.java b/core/src/main/java/org/apache/calcite/runtime/GeoFunctions.java
index 9732a0c..c7c2019 100644
--- a/core/src/main/java/org/apache/calcite/runtime/GeoFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/GeoFunctions.java
@@ -20,19 +20,18 @@ import org.apache.calcite.linq4j.AbstractEnumerable;
 import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.linq4j.function.Deterministic;
 import org.apache.calcite.linq4j.function.Experimental;
+import org.apache.calcite.linq4j.function.Hints;
 import org.apache.calcite.linq4j.function.SemiStrict;
 import org.apache.calcite.linq4j.function.Strict;
-import org.apache.calcite.util.Util;
+import org.apache.calcite.runtime.Geometries.CapStyle;
+import org.apache.calcite.runtime.Geometries.Geom;
+import org.apache.calcite.runtime.Geometries.JoinStyle;
 
 import com.esri.core.geometry.Envelope;
 import com.esri.core.geometry.Geometry;
 import com.esri.core.geometry.GeometryEngine;
 import com.esri.core.geometry.Line;
-import com.esri.core.geometry.MapGeometry;
-import com.esri.core.geometry.Operator;
 import com.esri.core.geometry.OperatorBoundary;
-import com.esri.core.geometry.OperatorFactoryLocal;
-import com.esri.core.geometry.OperatorIntersects;
 import com.esri.core.geometry.Point;
 import com.esri.core.geometry.Polygon;
 import com.esri.core.geometry.Polyline;
@@ -41,7 +40,15 @@ import com.esri.core.geometry.WktExportFlags;
 import com.esri.core.geometry.WktImportFlags;
 
 import java.math.BigDecimal;
-import java.util.Objects;
+
+import static org.apache.calcite.runtime.Geometries.NO_SRID;
+import static org.apache.calcite.runtime.Geometries.bind;
+import static org.apache.calcite.runtime.Geometries.buffer;
+import static org.apache.calcite.runtime.Geometries.envelope;
+import static org.apache.calcite.runtime.Geometries.intersects;
+import static org.apache.calcite.runtime.Geometries.makeLine;
+import static org.apache.calcite.runtime.Geometries.point;
+import static org.apache.calcite.runtime.Geometries.todo;
 
 /**
  * Helper methods to implement Geo-spatial functions in generated code.
@@ -68,30 +75,9 @@ import java.util.Objects;
 @Strict
 @Experimental
 public class GeoFunctions {
-  private static final int NO_SRID = 0;
-  private static final SpatialReference SPATIAL_REFERENCE =
-      SpatialReference.create(4326);
 
   private GeoFunctions() {}
 
-  private static UnsupportedOperationException todo() {
-    return new UnsupportedOperationException();
-  }
-
-  protected static Geom bind(Geometry geometry, int srid) {
-    if (geometry == null) {
-      return null;
-    }
-    if (srid == NO_SRID) {
-      return new SimpleGeom(geometry);
-    }
-    return bind(geometry, SpatialReference.create(srid));
-  }
-
-  private static MapGeom bind(Geometry geometry, SpatialReference sr) {
-    return new MapGeom(new MapGeometry(geometry, sr));
-  }
-
   // Geometry conversion functions (2D and 3D) ================================
 
   public static String ST_AsText(Geom g) {
@@ -197,74 +183,77 @@ public class GeoFunctions {
     // in SqlGeoFunctions.
   }
 
-  /**  Creates a line-string from the given POINTs (or MULTIPOINTs). */
+  /** Creates a rectangular Polygon. */
+  public static Geom ST_MakeEnvelope(BigDecimal xMin, BigDecimal yMin,
+      BigDecimal xMax, BigDecimal yMax, int srid) {
+    return ST_GeomFromText("POLYGON(("
+        + xMin + " " + yMin + ", "
+        + xMin + " " + yMax + ", "
+        + xMax + " " + yMax + ", "
+        + xMax + " " + yMin + ", "
+        + xMin + " " + yMin + "))", srid);
+  }
+
+  /** Creates a rectangular Polygon. */
+  public static Geom ST_MakeEnvelope(BigDecimal xMin, BigDecimal yMin,
+      BigDecimal xMax, BigDecimal yMax) {
+    return ST_MakeEnvelope(xMin, yMin, xMax, yMax, NO_SRID);
+  }
+
+  /** Creates a line-string from the given POINTs (or MULTIPOINTs). */
+  @Hints({"SqlKind:ST_MAKE_LINE"})
   public static Geom ST_MakeLine(Geom geom1, Geom geom2) {
     return makeLine(geom1, geom2);
   }
 
+  @Hints({"SqlKind:ST_MAKE_LINE"})
   public static Geom ST_MakeLine(Geom geom1, Geom geom2, Geom geom3) {
     return makeLine(geom1, geom2, geom3);
   }
 
+  @Hints({"SqlKind:ST_MAKE_LINE"})
   public static Geom ST_MakeLine(Geom geom1, Geom geom2, Geom geom3,
       Geom geom4) {
     return makeLine(geom1, geom2, geom3, geom4);
   }
 
+  @Hints({"SqlKind:ST_MAKE_LINE"})
   public static Geom ST_MakeLine(Geom geom1, Geom geom2, Geom geom3,
       Geom geom4, Geom geom5) {
     return makeLine(geom1, geom2, geom3, geom4, geom5);
   }
 
+  @Hints({"SqlKind:ST_MAKE_LINE"})
   public static Geom ST_MakeLine(Geom geom1, Geom geom2, Geom geom3,
       Geom geom4, Geom geom5, Geom geom6) {
     return makeLine(geom1, geom2, geom3, geom4, geom5, geom6);
   }
 
-  private static Geom makeLine(Geom... geoms) {
-    final Polyline g = new Polyline();
-    Point p = null;
-    for (Geom geom : geoms) {
-      if (geom.g() instanceof Point) {
-        final Point prev = p;
-        p = (Point) geom.g();
-        if (prev != null) {
-          final Line line = new Line();
-          line.setStart(prev);
-          line.setEnd(p);
-          g.addSegment(line, false);
-        }
-      }
-    }
-    return new SimpleGeom(g);
-  }
-
-  /**  Alias for {@link #ST_Point(BigDecimal, BigDecimal)}. */
+  /** Alias for {@link #ST_Point(BigDecimal, BigDecimal)}. */
+  @Hints({"SqlKind:ST_POINT"})
   public static Geom ST_MakePoint(BigDecimal x, BigDecimal y) {
     return ST_Point(x, y);
   }
 
-  /**  Alias for {@link #ST_Point(BigDecimal, BigDecimal, BigDecimal)}. */
+  /** Alias for {@link #ST_Point(BigDecimal, BigDecimal, BigDecimal)}. */
+  @Hints({"SqlKind:ST_POINT3"})
   public static Geom ST_MakePoint(BigDecimal x, BigDecimal y, BigDecimal z) {
     return ST_Point(x, y, z);
   }
 
-  /**  Constructs a 2D point from coordinates. */
+  /** Constructs a 2D point from coordinates. */
+  @Hints({"SqlKind:ST_POINT"})
   public static Geom ST_Point(BigDecimal x, BigDecimal y) {
     // NOTE: Combine the double and BigDecimal variants of this function
     return point(x.doubleValue(), y.doubleValue());
   }
 
-  /**  Constructs a 3D point from coordinates. */
+  /** Constructs a 3D point from coordinates. */
+  @Hints({"SqlKind:ST_POINT3"})
   public static Geom ST_Point(BigDecimal x, BigDecimal y, BigDecimal z) {
     final Geometry g = new Point(x.doubleValue(), y.doubleValue(),
         z.doubleValue());
-    return new SimpleGeom(g);
-  }
-
-  private static Geom point(double x, double y) {
-    final Geometry g = new Point(x, y);
-    return new SimpleGeom(g);
+    return new Geometries.SimpleGeom(g);
   }
 
   // Geometry properties (2D and 3D) ==========================================
@@ -304,34 +293,12 @@ public class GeoFunctions {
 
   /** Returns the type of {@code geom}. */
   public static String ST_GeometryType(Geom geom) {
-    return type(geom.g()).name();
+    return Geometries.type(geom.g()).name();
   }
 
   /** Returns the OGC SFS type code of {@code geom}. */
   public static int ST_GeometryTypeCode(Geom geom) {
-    return type(geom.g()).code;
-  }
-
-  /** Returns the OGC type of a geometry. */
-  private static Type type(Geometry g) {
-    switch (g.getType()) {
-    case Point:
-      return Type.POINT;
-    case Polyline:
-      return Type.LINESTRING;
-    case Polygon:
-      return Type.POLYGON;
-    case MultiPoint:
-      return Type.MULTIPOINT;
-    case Envelope:
-      return Type.POLYGON;
-    case Line:
-      return Type.LINESTRING;
-    case Unknown:
-      return Type.Geometry;
-    default:
-      throw new AssertionError(g);
-    }
+    return Geometries.type(geom.g()).code;
   }
 
   /** Returns the minimum bounding box of {@code geom} (which may be a
@@ -341,15 +308,10 @@ public class GeoFunctions {
     return geom.wrap(env);
   }
 
-  private static Envelope envelope(Geometry g) {
-    final Envelope env = new Envelope();
-    g.queryEnvelope(env);
-    return env;
-  }
-
   // Geometry predicates ======================================================
 
   /** Returns whether {@code geom1} contains {@code geom2}. */
+  @Hints({"SqlKind:ST_CONTAINS"})
   public static boolean ST_Contains(Geom geom1, Geom geom2) {
     return GeometryEngine.contains(geom1.g(), geom2.g(), geom1.sr());
   }
@@ -397,13 +359,6 @@ public class GeoFunctions {
     return intersects(g1, g2, sr);
   }
 
-  private static boolean intersects(Geometry g1, Geometry g2,
-      SpatialReference sr) {
-    final OperatorIntersects op = (OperatorIntersects) OperatorFactoryLocal
-        .getInstance().getOperator(Operator.Type.Intersects);
-    return op.execute(g1, g2, sr, null);
-  }
-
   /** Returns whether {@code geom1} equals {@code geom2} and their coordinates
    * and component Geometries are listed in the same order. */
   public static boolean ST_OrderingEquals(Geom geom1, Geom geom2)  {
@@ -427,6 +382,7 @@ public class GeoFunctions {
 
   /** Returns whether {@code geom1} and {@code geom2} are within
    * {@code distance} of each other. */
+  @Hints({"SqlKind:ST_DWITHIN"})
   public static boolean ST_DWithin(Geom geom1, Geom geom2, double distance) {
     final double distance1 =
         GeometryEngine.distance(geom1.g(), geom2.g(), geom1.sr());
@@ -497,14 +453,6 @@ public class GeoFunctions {
         mitreLimit);
   }
 
-  private static Geom buffer(Geom geom, double bufferSize,
-      int quadSegCount, CapStyle endCapStyle, JoinStyle joinStyle,
-      float mitreLimit) {
-    Util.discard(endCapStyle + ":" + joinStyle + ":" + mitreLimit
-        + ":" + quadSegCount);
-    throw todo();
-  }
-
   /** Computes the union of {@code geom1} and {@code geom2}. */
   public static Geom ST_Union(Geom geom1, Geom geom2) {
     SpatialReference sr = geom1.sr();
@@ -534,145 +482,28 @@ public class GeoFunctions {
     return geom.transform(srid);
   }
 
-  // Inner classes ============================================================
-
-  /** How the "buffer" command terminates the end of a line. */
-  enum CapStyle {
-    ROUND, FLAT, SQUARE;
-
-    static CapStyle of(String value) {
-      switch (value) {
-      case "round":
-        return ROUND;
-      case "flat":
-      case "butt":
-        return FLAT;
-      case "square":
-        return SQUARE;
-      default:
-        throw new IllegalArgumentException("unknown endcap value: " + value);
-      }
-    }
-  }
-
-  /** How the "buffer" command decorates junctions between line segments. */
-  enum JoinStyle {
-    ROUND, MITRE, BEVEL;
-
-    static JoinStyle of(String value) {
-      switch (value) {
-      case "round":
-        return ROUND;
-      case "mitre":
-      case "miter":
-        return MITRE;
-      case "bevel":
-        return BEVEL;
-      default:
-        throw new IllegalArgumentException("unknown join value: " + value);
-      }
-    }
-  }
-
-  /** Geometry. It may or may not have a spatial reference
-   * associated with it. */
-  public interface Geom {
-    Geometry g();
-
-    SpatialReference sr();
-
-    Geom transform(int srid);
-
-    Geom wrap(Geometry g);
-  }
-
-  /** Sub-class of geometry that has no spatial reference. */
-  static class SimpleGeom implements Geom {
-    final Geometry g;
+  // Space-filling curves
 
-    SimpleGeom(Geometry g) {
-      this.g = Objects.requireNonNull(g);
-    }
-
-    @Override public String toString() {
-      return g.toString();
-    }
-
-    public Geometry g() {
-      return g;
-    }
-
-    public SpatialReference sr() {
-      return SPATIAL_REFERENCE;
-    }
-
-    public Geom transform(int srid) {
-      if (srid == SPATIAL_REFERENCE.getID()) {
-        return this;
-      }
-      return bind(g, srid);
-    }
-
-    public Geom wrap(Geometry g) {
-      return new SimpleGeom(g);
+  /** Returns the position of a point on the Hilbert curve,
+   * or null if it is not a 2-dimensional point. */
+  @Hints({"SqlKind:HILBERT"})
+  public static Long hilbert(Geom geom) {
+    final Geometry g = geom.g();
+    if (g instanceof Point) {
+      final double x = ((Point) g).getX();
+      final double y = ((Point) g).getY();
+      return new HilbertCurve2D(8).toIndex(x, y);
     }
+    return null;
   }
 
-  /** Sub-class of geometry that has a spatial reference. */
-  static class MapGeom implements Geom {
-    final MapGeometry mg;
-
-    MapGeom(MapGeometry mg) {
-      this.mg = Objects.requireNonNull(mg);
-    }
-
-    @Override public String toString() {
-      return mg.toString();
-    }
-
-    public Geometry g() {
-      return mg.getGeometry();
-    }
-
-    public SpatialReference sr() {
-      return mg.getSpatialReference();
-    }
-
-    public Geom transform(int srid) {
-      if (srid == NO_SRID) {
-        return new SimpleGeom(mg.getGeometry());
-      }
-      if (srid == mg.getSpatialReference().getID()) {
-        return this;
-      }
-      return bind(mg.getGeometry(), srid);
-    }
-
-    public Geom wrap(Geometry g) {
-      return bind(g, this.mg.getSpatialReference());
-    }
+  /** Returns the position of a point on the Hilbert curve. */
+  @Hints({"SqlKind:HILBERT"})
+  public static long hilbert(BigDecimal x, BigDecimal y) {
+    return new HilbertCurve2D(8).toIndex(x.doubleValue(), y.doubleValue());
   }
 
-  /** Geometry types, with the names and codes assigned by OGC. */
-  enum Type {
-    Geometry(0),
-    POINT(1),
-    LINESTRING(2),
-    POLYGON(3),
-    MULTIPOINT(4),
-    MULTILINESTRING(5),
-    MULTIPOLYGON(6),
-    GEOMCOLLECTION(7),
-    CURVE(13),
-    SURFACE(14),
-    POLYHEDRALSURFACE(15);
-
-    final int code;
-
-    Type(int code) {
-      this.code = code;
-    }
-  }
+  // Inner classes ============================================================
 
   /** Used at run time by the {@link #ST_MakeGrid} and
    * {@link #ST_MakeGridPoints} functions. */
@@ -732,7 +563,7 @@ public class GeoFunctions {
             polyline.addSegment(new Line(right, top, left, top), false);
             polyline.addSegment(new Line(left, top, left, bottom), false);
             polygon.add(polyline, false);
-            geom = new SimpleGeom(polygon);
+            geom = new Geometries.SimpleGeom(polygon);
           }
           return new Object[] {geom, id, x + 1, y + 1, baseX + x, baseY + y};
         }
diff --git a/core/src/main/java/org/apache/calcite/runtime/Geometries.java b/core/src/main/java/org/apache/calcite/runtime/Geometries.java
new file mode 100644
index 0000000..c8e9919
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/runtime/Geometries.java
@@ -0,0 +1,295 @@
+/*
+ * 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.calcite.runtime;
+
+import org.apache.calcite.linq4j.function.Deterministic;
+import org.apache.calcite.linq4j.function.Experimental;
+import org.apache.calcite.linq4j.function.Strict;
+import org.apache.calcite.util.Util;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.Line;
+import com.esri.core.geometry.MapGeometry;
+import com.esri.core.geometry.Operator;
+import com.esri.core.geometry.OperatorFactoryLocal;
+import com.esri.core.geometry.OperatorIntersects;
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.Polyline;
+import com.esri.core.geometry.SpatialReference;
+import com.google.common.collect.ImmutableList;
+
+import java.util.Objects;
+
+/**
+ * Utilities for geometry.
+ */
+@SuppressWarnings({"UnnecessaryUnboxing", "WeakerAccess", "unused"})
+@Deterministic
+@Strict
+@Experimental
+public class Geometries {
+  static final int NO_SRID = 0;
+  private static final SpatialReference SPATIAL_REFERENCE =
+      SpatialReference.create(4326);
+
+  private Geometries() {}
+
+  static UnsupportedOperationException todo() {
+    return new UnsupportedOperationException();
+  }
+
+  protected static Geom bind(Geometry geometry, int srid) {
+    if (geometry == null) {
+      return null;
+    }
+    if (srid == NO_SRID) {
+      return new SimpleGeom(geometry);
+    }
+    return bind(geometry, SpatialReference.create(srid));
+  }
+
+  static MapGeom bind(Geometry geometry, SpatialReference sr) {
+    return new MapGeom(new MapGeometry(geometry, sr));
+  }
+
+  static Geom makeLine(Geom... geoms) {
+    return makeLine(ImmutableList.copyOf(geoms));
+  }
+
+  public static Geom makeLine(Iterable<? extends Geom> geoms) {
+    final Polyline g = new Polyline();
+    Point p = null;
+    for (Geom geom : geoms) {
+      if (geom.g() instanceof Point) {
+        final Point prev = p;
+        p = (Point) geom.g();
+        if (prev != null) {
+          final Line line = new Line();
+          line.setStart(prev);
+          line.setEnd(p);
+          g.addSegment(line, false);
+        }
+      }
+    }
+    return new SimpleGeom(g);
+  }
+
+  static Geom point(double x, double y) {
+    final Geometry g = new Point(x, y);
+    return new SimpleGeom(g);
+  }
+
+  /** Returns the OGC type of a geometry. */
+  public static Type type(Geometry g) {
+    switch (g.getType()) {
+    case Point:
+      return Type.POINT;
+    case Polyline:
+      return Type.LINESTRING;
+    case Polygon:
+      return Type.POLYGON;
+    case MultiPoint:
+      return Type.MULTIPOINT;
+    case Envelope:
+      return Type.POLYGON;
+    case Line:
+      return Type.LINESTRING;
+    case Unknown:
+      return Type.Geometry;
+    default:
+      throw new AssertionError(g);
+    }
+  }
+
+  static Envelope envelope(Geometry g) {
+    final Envelope env = new Envelope();
+    g.queryEnvelope(env);
+    return env;
+  }
+
+  static boolean intersects(Geometry g1, Geometry g2,
+      SpatialReference sr) {
+    final OperatorIntersects op = (OperatorIntersects) OperatorFactoryLocal
+        .getInstance().getOperator(Operator.Type.Intersects);
+    return op.execute(g1, g2, sr, null);
+  }
+
+  static Geom buffer(Geom geom, double bufferSize,
+      int quadSegCount, CapStyle endCapStyle, JoinStyle joinStyle,
+      float mitreLimit) {
+    Util.discard(endCapStyle + ":" + joinStyle + ":" + mitreLimit
+        + ":" + quadSegCount);
+    throw todo();
+  }
+
+  /** How the "buffer" command terminates the end of a line. */
+  enum CapStyle {
+    ROUND, FLAT, SQUARE;
+
+    static CapStyle of(String value) {
+      switch (value) {
+      case "round":
+        return ROUND;
+      case "flat":
+      case "butt":
+        return FLAT;
+      case "square":
+        return SQUARE;
+      default:
+        throw new IllegalArgumentException("unknown endcap value: " + value);
+      }
+    }
+  }
+
+  /** How the "buffer" command decorates junctions between line segments. */
+  enum JoinStyle {
+    ROUND, MITRE, BEVEL;
+
+    static JoinStyle of(String value) {
+      switch (value) {
+      case "round":
+        return ROUND;
+      case "mitre":
+      case "miter":
+        return MITRE;
+      case "bevel":
+        return BEVEL;
+      default:
+        throw new IllegalArgumentException("unknown join value: " + value);
+      }
+    }
+  }
+
+  /** Geometry types, with the names and codes assigned by OGC. */
+  public enum Type {
+    Geometry(0),
+    POINT(1),
+    LINESTRING(2),
+    POLYGON(3),
+    MULTIPOINT(4),
+    MULTILINESTRING(5),
+    MULTIPOLYGON(6),
+    GEOMCOLLECTION(7),
+    CURVE(13),
+    SURFACE(14),
+    POLYHEDRALSURFACE(15);
+
+    final int code;
+
+    Type(int code) {
+      this.code = code;
+    }
+  }
+
+  /** Geometry. It may or may not have a spatial reference
+   * associated with it. */
+  public interface Geom extends Comparable<Geom> {
+    Geometry g();
+
+    Type type();
+
+    SpatialReference sr();
+
+    Geom transform(int srid);
+
+    Geom wrap(Geometry g);
+  }
+
+  /** Sub-class of geometry that has no spatial reference. */
+  static class SimpleGeom implements Geom {
+    final Geometry g;
+
+    SimpleGeom(Geometry g) {
+      this.g = Objects.requireNonNull(g);
+    }
+
+    @Override public String toString() {
+      return g.toString();
+    }
+
+    public int compareTo(Geom o) {
+      return toString().compareTo(o.toString());
+    }
+
+    public Geometry g() {
+      return g;
+    }
+
+    public Type type() {
+      return Geometries.type(g);
+    }
+
+    public SpatialReference sr() {
+      return SPATIAL_REFERENCE;
+    }
+
+    public Geom transform(int srid) {
+      if (srid == SPATIAL_REFERENCE.getID()) {
+        return this;
+      }
+      return bind(g, srid);
+    }
+
+    public Geom wrap(Geometry g) {
+      return new SimpleGeom(g);
+    }
+  }
+
+  /** Sub-class of geometry that has a spatial reference. */
+  static class MapGeom implements Geom {
+    final MapGeometry mg;
+
+    MapGeom(MapGeometry mg) {
+      this.mg = Objects.requireNonNull(mg);
+    }
+
+    @Override public String toString() {
+      return mg.toString();
+    }
+
+    public int compareTo(Geom o) {
+      return toString().compareTo(o.toString());
+    }
+
+    public Geometry g() {
+      return mg.getGeometry();
+    }
+
+    public Type type() {
+      return Geometries.type(mg.getGeometry());
+    }
+
+    public SpatialReference sr() {
+      return mg.getSpatialReference();
+    }
+
+    public Geom transform(int srid) {
+      if (srid == NO_SRID) {
+        return new SimpleGeom(mg.getGeometry());
+      }
+      if (srid == mg.getSpatialReference().getID()) {
+        return this;
+      }
+      return bind(mg.getGeometry(), srid);
+    }
+
+    public Geom wrap(Geometry g) {
+      return bind(g, this.mg.getSpatialReference());
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/runtime/HilbertCurve2D.java b/core/src/main/java/org/apache/calcite/runtime/HilbertCurve2D.java
new file mode 100644
index 0000000..6a32a30
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/runtime/HilbertCurve2D.java
@@ -0,0 +1,158 @@
+/*
+ * 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.calcite.runtime;
+
+import com.google.common.collect.ImmutableList;
+import com.google.uzaygezen.core.BacktrackingQueryBuilder;
+import com.google.uzaygezen.core.BitVector;
+import com.google.uzaygezen.core.BitVectorFactories;
+import com.google.uzaygezen.core.CompactHilbertCurve;
+import com.google.uzaygezen.core.FilteredIndexRange;
+import com.google.uzaygezen.core.LongContent;
+import com.google.uzaygezen.core.PlainFilterCombiner;
+import com.google.uzaygezen.core.Query;
+import com.google.uzaygezen.core.SimpleRegionInspector;
+import com.google.uzaygezen.core.ZoomingSpaceVisitorAdapter;
+import com.google.uzaygezen.core.ranges.LongRange;
+import com.google.uzaygezen.core.ranges.LongRangeHome;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * 2-dimensional Hilbert space-filling curve.
+ *
+ * <p>Includes code from
+ * <a href="https://github.com/locationtech/sfcurve">LocationTech SFCurve</a>,
+ * Copyright (c) 2015 Azavea.
+ */
+public class HilbertCurve2D implements SpaceFillingCurve2D {
+  final long precision;
+  final CompactHilbertCurve chc;
+  private final int resolution;
+
+  public HilbertCurve2D(int resolution) {
+    this.resolution = resolution;
+    precision = (long) Math.pow(2, resolution);
+    chc = new CompactHilbertCurve(new int[] {resolution, resolution});
+  }
+
+  long getNormalizedLongitude(double x) {
+    return (long) ((x + 180) * (precision - 1) / 360d);
+  }
+
+  long getNormalizedLatitude(double y) {
+    return (long) ((y + 90) * (precision - 1) / 180d);
+  }
+
+  long setNormalizedLatitude(long latNormal) {
+    if (!(latNormal >= 0 && latNormal <= precision)) {
+      throw new NumberFormatException(
+          "Normalized latitude must be greater than 0 and less than the maximum precision");
+    }
+    return (long) (latNormal * 180d / (precision - 1));
+  }
+
+  long setNormalizedLongitude(long lonNormal) {
+    if (!(lonNormal >= 0 && lonNormal <= precision)) {
+      throw new NumberFormatException(
+          "Normalized longitude must be greater than 0 and less than the maximum precision");
+    }
+    return (long) (lonNormal * 360d / (precision - 1));
+  }
+
+  public long toIndex(double x, double y) {
+    final long normX = getNormalizedLongitude(x);
+    final long normY = getNormalizedLatitude(y);
+    final BitVector[] p = {
+        BitVectorFactories.OPTIMAL.apply(resolution),
+        BitVectorFactories.OPTIMAL.apply(resolution)
+    };
+
+    p[0].copyFrom(normX);
+    p[1].copyFrom(normY);
+
+    final BitVector hilbert = BitVectorFactories.OPTIMAL.apply(resolution * 2);
+
+    chc.index(p, 0, hilbert);
+    return hilbert.toLong();
+  }
+
+  public Point toPoint(long i) {
+    final BitVector h = BitVectorFactories.OPTIMAL.apply(resolution * 2);
+    h.copyFrom(i);
+    final BitVector[] p = {
+        BitVectorFactories.OPTIMAL.apply(resolution),
+        BitVectorFactories.OPTIMAL.apply(resolution)
+    };
+
+    chc.indexInverse(h, p);
+
+    final long x = setNormalizedLongitude(p[0].toLong()) - 180;
+    final long y = setNormalizedLatitude(p[1].toLong()) - 90;
+    return new Point((double) x, (double) y);
+  }
+
+  public List<IndexRange> toRanges(double xMin, double yMin, double xMax,
+      double yMax, RangeComputeHints hints) {
+    final CompactHilbertCurve chc =
+        new CompactHilbertCurve(new int[] {resolution, resolution});
+    final List<LongRange> region = new ArrayList<>();
+
+    final long minNormalizedLongitude = getNormalizedLongitude(xMin);
+    final long minNormalizedLatitude  = getNormalizedLatitude(yMin);
+
+    final long maxNormalizedLongitude = getNormalizedLongitude(xMax);
+    final long maxNormalizedLatitude  = getNormalizedLatitude(yMax);
+
+    region.add(LongRange.of(minNormalizedLongitude, maxNormalizedLongitude));
+    region.add(LongRange.of(minNormalizedLatitude, maxNormalizedLatitude));
+
+    final LongContent zero = new LongContent(0L);
+
+    final SimpleRegionInspector<LongRange, Long, LongContent, LongRange> inspector =
+        SimpleRegionInspector.create(ImmutableList.of(region),
+            new LongContent(1L), range -> range, LongRangeHome.INSTANCE,
+            zero);
+
+    final PlainFilterCombiner<LongRange, Long, LongContent, LongRange> combiner =
+        new PlainFilterCombiner<>(LongRange.of(0, 1));
+
+    final BacktrackingQueryBuilder<LongRange, Long, LongContent, LongRange> queryBuilder =
+        BacktrackingQueryBuilder.create(inspector, combiner, Integer.MAX_VALUE,
+            true, LongRangeHome.INSTANCE, zero);
+
+    chc.accept(new ZoomingSpaceVisitorAdapter(chc, queryBuilder));
+
+    final Query<LongRange, LongRange> query = queryBuilder.get();
+
+    final List<FilteredIndexRange<LongRange, LongRange>> ranges =
+        query.getFilteredIndexRanges();
+
+    // result
+    final List<IndexRange> result = new ArrayList<>();
+
+    for (FilteredIndexRange<LongRange, LongRange> l : ranges) {
+      final LongRange range = l.getIndexRange();
+      final Long start = range.getStart();
+      final Long end = range.getEnd();
+      final boolean contained = l.isPotentialOverSelectivity();
+      result.add(0, IndexRanges.create(start, end, contained));
+    }
+    return result;
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/runtime/SpaceFillingCurve2D.java b/core/src/main/java/org/apache/calcite/runtime/SpaceFillingCurve2D.java
new file mode 100644
index 0000000..c685bb3
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/runtime/SpaceFillingCurve2D.java
@@ -0,0 +1,146 @@
+/*
+ * 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.calcite.runtime;
+
+import com.google.common.collect.Ordering;
+
+import java.util.HashMap;
+import java.util.List;
+
+/**
+ * Utilities for space-filling curves.
+ *
+ * <p>Includes code from
+ * <a href="https://github.com/locationtech/sfcurve">LocationTech SFCurve</a>,
+ * Copyright (c) 2015 Azavea.
+ */
+public interface SpaceFillingCurve2D {
+  long toIndex(double x, double y);
+  Point toPoint(long i);
+  List<IndexRange> toRanges(double xMin, double yMin, double xMax,
+      double yMax, RangeComputeHints hints);
+
+  /** Hints for the {@link SpaceFillingCurve2D#toRanges} method. */
+  class RangeComputeHints extends HashMap<String, Object> {
+  }
+
+  /** Range. */
+  interface IndexRange {
+    long lower();
+    long upper();
+    boolean contained();
+
+    IndexRangeTuple tuple();
+  }
+
+  /** Data representing a range. */
+  class IndexRangeTuple {
+    final long lower;
+    final long upper;
+    final boolean contained;
+
+    IndexRangeTuple(long lower, long upper, boolean contained) {
+      this.lower = lower;
+      this.upper = upper;
+      this.contained = contained;
+    }
+  }
+
+  /** Base class for Range implementations. */
+  abstract class AbstractRange implements IndexRange {
+    final long lower;
+    final long upper;
+
+    protected AbstractRange(long lower, long upper) {
+      this.lower = lower;
+      this.upper = upper;
+    }
+
+    public long lower() {
+      return lower;
+    }
+
+    public long upper() {
+      return upper;
+    }
+
+    public IndexRangeTuple tuple() {
+      return new IndexRangeTuple(lower, upper, contained());
+    }
+  }
+
+  /** Range that is covered. */
+  class CoveredRange extends AbstractRange {
+    CoveredRange(long lower, long upper) {
+      super(lower, upper);
+    }
+
+    public boolean contained() {
+      return true;
+    }
+
+    @Override public String toString() {
+      return "covered(" + lower + ", " + upper + ")";
+    }
+  }
+
+  /** Range that is not contained. */
+  class OverlappingRange extends AbstractRange {
+    OverlappingRange(long lower, long upper) {
+      super(lower, upper);
+    }
+
+    public boolean contained() {
+      return false;
+    }
+
+    @Override public String toString() {
+      return "overlap(" + lower + ", " + upper + ")";
+    }
+  }
+
+  /** Lexicographic ordering for {@link IndexRange}. */
+  class IndexRangeOrdering extends Ordering<IndexRange> {
+    public int compare(IndexRange x, IndexRange y) {
+      final int c1 = Long.compare(x.lower(), y.lower());
+      if (c1 != 0) {
+        return c1;
+      }
+      return Long.compare(x.upper(), y.upper());
+    }
+  }
+
+  /** Utilities for {@link IndexRange}. */
+  class IndexRanges {
+    private IndexRanges() {}
+
+    static IndexRange create(long l, long u, boolean contained) {
+      return contained ? new CoveredRange(l, u) : new OverlappingRange(l, u);
+    }
+  }
+
+  /** A 2-dimensional point. */
+  class Point {
+    final double x;
+    final double y;
+
+    Point(double x, double y) {
+      this.x = x;
+      this.y = y;
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/schema/Schemas.java b/core/src/main/java/org/apache/calcite/schema/Schemas.java
index 384a4cf..00b9239 100644
--- a/core/src/main/java/org/apache/calcite/schema/Schemas.java
+++ b/core/src/main/java/org/apache/calcite/schema/Schemas.java
@@ -515,6 +515,8 @@ public final class Schemas {
       return PathImpl.EMPTY;
     }
     if (!rootSchema.name.isEmpty()) {
+      // If path starts with the name of the root schema, ignore the first step
+      // in the path.
       Preconditions.checkState(rootSchema.name.equals(iterator.next()));
     }
     for (;;) {
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index 8921ce3..d14477c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -876,6 +876,30 @@ public enum SqlKind {
   /** {@code FOREIGN KEY} constraint. */
   FOREIGN_KEY,
 
+  // Spatial functions. They are registered as "user-defined functions" but it
+  // is convenient to have a "kind" so that we can quickly match them in planner
+  // rules.
+
+  /** The {@code ST_DWithin} geo-spatial function. */
+  ST_DWITHIN,
+
+  /** The {@code ST_Point} function. */
+  ST_POINT,
+
+  /** The {@code ST_Point} function that makes a 3D point. */
+  ST_POINT3,
+
+  /** The {@code ST_MakeLine} function that makes a line. */
+  ST_MAKE_LINE,
+
+  /** The {@code ST_Contains} function that tests whether one geometry contains
+   * another. */
+  ST_CONTAINS,
+
+  /** The {@code Hilbert} function that converts (x, y) to a position on a
+   * Hilbert space-filling curve. */
+  HILBERT,
+
   // DDL and session control statements follow. The list is not exhaustive: feel
   // free to add more.
 
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlGeoFunctions.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlGeoFunctions.java
index dd4a411..7638d42 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlGeoFunctions.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlGeoFunctions.java
@@ -23,6 +23,7 @@ import org.apache.calcite.linq4j.Linq4j;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.runtime.GeoFunctions;
+import org.apache.calcite.runtime.Geometries.Geom;
 import org.apache.calcite.schema.ScannableTable;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.Statistic;
@@ -51,18 +52,18 @@ public class SqlGeoFunctions {
 
   /** Calculates a regular grid of polygons based on {@code geom}.
    *
-   * @see GeoFunctions#ST_MakeGrid */
+   * @see GeoFunctions ST_MakeGrid */
   @SuppressWarnings({"WeakerAccess", "unused"})
-  public static ScannableTable ST_MakeGrid(final GeoFunctions.Geom geom,
+  public static ScannableTable ST_MakeGrid(final Geom geom,
       final BigDecimal deltaX, final BigDecimal deltaY) {
     return new GridTable(geom, deltaX, deltaY, false);
   }
 
   /** Calculates a regular grid of points based on {@code geom}.
    *
-   * @see GeoFunctions#ST_MakeGridPoints */
+   * @see GeoFunctions ST_MakeGridPoints */
   @SuppressWarnings({"WeakerAccess", "unused"})
-  public static ScannableTable ST_MakeGridPoints(final GeoFunctions.Geom geom,
+  public static ScannableTable ST_MakeGridPoints(final Geom geom,
       final BigDecimal deltaX, final BigDecimal deltaY) {
     return new GridTable(geom, deltaX, deltaY, true);
   }
@@ -70,12 +71,12 @@ public class SqlGeoFunctions {
   /** Returns the points or rectangles in a grid that covers a given
    * geometry. */
   public static class GridTable implements ScannableTable {
-    private final GeoFunctions.Geom geom;
+    private final Geom geom;
     private final BigDecimal deltaX;
     private final BigDecimal deltaY;
     private boolean point;
 
-    GridTable(GeoFunctions.Geom geom, BigDecimal deltaX, BigDecimal deltaY,
+    GridTable(Geom geom, BigDecimal deltaX, BigDecimal deltaY,
         boolean point) {
       this.geom = geom;
       this.deltaX = deltaX;
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperatorTableFactory.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperatorTableFactory.java
index 0444d16..c3a81a3 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperatorTableFactory.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperatorTableFactory.java
@@ -16,12 +16,10 @@
  */
 package org.apache.calcite.sql.fun;
 
-import org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.calcite.runtime.GeoFunctions;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
 import org.apache.calcite.sql.util.ListSqlOperatorTable;
+import org.apache.calcite.sql.util.SqlOperatorTables;
 import org.apache.calcite.util.Util;
 
 import com.google.common.cache.CacheBuilder;
@@ -85,9 +83,7 @@ public class SqlLibraryOperatorTableFactory {
         standard = true;
         break;
       case SPATIAL:
-        list.addAll(
-            CalciteCatalogReader.operatorTable(GeoFunctions.class.getName(),
-                SqlGeoFunctions.class.getName()).getOperatorList());
+        list.addAll(SqlOperatorTables.spatialInstance().getOperatorList());
         break;
       default:
         custom = true;
@@ -116,7 +112,7 @@ public class SqlLibraryOperatorTableFactory {
     SqlOperatorTable operatorTable = new ListSqlOperatorTable(list.build());
     if (standard) {
       operatorTable =
-          ChainedSqlOperatorTable.of(SqlStdOperatorTable.instance(),
+          SqlOperatorTables.chain(SqlStdOperatorTable.instance(),
               operatorTable);
     }
     return operatorTable;
diff --git a/core/src/main/java/org/apache/calcite/sql/type/JavaToSqlTypeConversionRules.java b/core/src/main/java/org/apache/calcite/sql/type/JavaToSqlTypeConversionRules.java
index c6e4944..ca549ff 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/JavaToSqlTypeConversionRules.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/JavaToSqlTypeConversionRules.java
@@ -17,7 +17,7 @@
 package org.apache.calcite.sql.type;
 
 import org.apache.calcite.avatica.util.ArrayImpl;
-import org.apache.calcite.runtime.GeoFunctions;
+import org.apache.calcite.runtime.Geometries;
 
 import com.google.common.collect.ImmutableMap;
 
@@ -71,7 +71,7 @@ public class JavaToSqlTypeConversionRules {
           .put(Time.class, SqlTypeName.TIME)
           .put(BigDecimal.class, SqlTypeName.DECIMAL)
 
-          .put(GeoFunctions.Geom.class, SqlTypeName.GEOMETRY)
+          .put(Geometries.Geom.class, SqlTypeName.GEOMETRY)
 
           .put(ResultSet.class, SqlTypeName.CURSOR)
           .put(ColumnList.class, SqlTypeName.COLUMN_LIST)
diff --git a/core/src/main/java/org/apache/calcite/sql/util/ChainedSqlOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/util/ChainedSqlOperatorTable.java
index a6e2356..a85d025 100644
--- a/core/src/main/java/org/apache/calcite/sql/util/ChainedSqlOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/util/ChainedSqlOperatorTable.java
@@ -31,6 +31,8 @@ import java.util.List;
 /**
  * ChainedSqlOperatorTable implements the {@link SqlOperatorTable} interface by
  * chaining together any number of underlying operator table instances.
+ *
+ * <p>To create, call {@link SqlOperatorTables#chain}.
  */
 public class ChainedSqlOperatorTable implements SqlOperatorTable {
   //~ Instance fields --------------------------------------------------------
@@ -39,27 +41,19 @@ public class ChainedSqlOperatorTable implements SqlOperatorTable {
 
   //~ Constructors -----------------------------------------------------------
 
-  /**
-   * Creates a table based on a given list.
-   */
+  @Deprecated // to be removed before 2.0
   public ChainedSqlOperatorTable(List<SqlOperatorTable> tableList) {
-    this.tableList = ImmutableList.copyOf(tableList);
+    this(ImmutableList.copyOf(tableList));
   }
 
-  /** Creates a {@code ChainedSqlOperatorTable}. */
-  public static SqlOperatorTable of(SqlOperatorTable... tables) {
-    return new ChainedSqlOperatorTable(ImmutableList.copyOf(tables));
+  /** Internal constructor; call {@link SqlOperatorTables#chain}. */
+  protected ChainedSqlOperatorTable(ImmutableList<SqlOperatorTable> tableList) {
+    this.tableList = ImmutableList.copyOf(tableList);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  /**
-   * Adds an underlying table. The order in which tables are added is
-   * significant; tables added earlier have higher lookup precedence. A table
-   * is not added if it is already on the list.
-   *
-   * @param table table to add
-   */
+  @Deprecated // to be removed before 2.0
   public void add(SqlOperatorTable table) {
     if (!tableList.contains(table)) {
       tableList.add(table);
diff --git a/core/src/main/java/org/apache/calcite/sql/util/SqlOperatorTables.java b/core/src/main/java/org/apache/calcite/sql/util/SqlOperatorTables.java
new file mode 100644
index 0000000..232d752
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/util/SqlOperatorTables.java
@@ -0,0 +1,62 @@
+/*
+ * 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.calcite.sql.util;
+
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.runtime.GeoFunctions;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlGeoFunctions;
+
+import com.google.common.base.Suppliers;
+import com.google.common.collect.ImmutableList;
+
+import java.util.function.Supplier;
+
+/**
+ * Utilities for {@link SqlOperatorTable}s.
+ */
+public class SqlOperatorTables extends ReflectiveSqlOperatorTable {
+
+  private static final Supplier<SqlOperatorTable> SPATIAL =
+      Suppliers.memoize(SqlOperatorTables::createSpatial)::get;
+
+  private static SqlOperatorTable createSpatial() {
+    return CalciteCatalogReader.operatorTable(
+        GeoFunctions.class.getName(),
+        SqlGeoFunctions.class.getName());
+  }
+
+  /** Returns the Spatial operator table, creating it if necessary. */
+  public static SqlOperatorTable spatialInstance() {
+    return SPATIAL.get();
+  }
+
+  /** Creates a composite operator table. */
+  public static SqlOperatorTable chain(Iterable<SqlOperatorTable> tables) {
+    final ImmutableList<SqlOperatorTable> list =
+        ImmutableList.copyOf(tables);
+    if (list.size() == 1) {
+      return list.get(0);
+    }
+    return new ChainedSqlOperatorTable(list);
+  }
+
+  /** Creates a composite operator table from an array of tables. */
+  public static SqlOperatorTable chain(SqlOperatorTable... tables) {
+    return chain(ImmutableList.copyOf(tables));
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
index ce5f4d9..440019d 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
@@ -27,6 +27,7 @@ import org.apache.calcite.sql.type.SqlOperandMetadata;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.util.Util;
 
 import java.lang.reflect.Type;
 import java.util.List;
@@ -50,6 +51,7 @@ public class SqlUserDefinedTableFunction extends SqlUserDefinedFunction
         operandTypeInference,
         operandTypeChecker instanceof SqlOperandMetadata
             ? (SqlOperandMetadata) operandTypeChecker : null, function);
+    Util.discard(paramTypes);
   }
 
   /** Creates a user-defined table function. */
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index 0b462f1..6dbc319 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -85,6 +85,7 @@ import org.apache.calcite.runtime.Bindable;
 import org.apache.calcite.runtime.CompressionFunctions;
 import org.apache.calcite.runtime.Enumerables;
 import org.apache.calcite.runtime.FlatLists;
+import org.apache.calcite.runtime.GeoFunctions;
 import org.apache.calcite.runtime.JsonFunctions;
 import org.apache.calcite.runtime.Matcher;
 import org.apache.calcite.runtime.Pattern;
@@ -373,6 +374,7 @@ public enum BuiltInMethod {
   IS_JSON_OBJECT(JsonFunctions.class, "isJsonObject", String.class),
   IS_JSON_ARRAY(JsonFunctions.class, "isJsonArray", String.class),
   IS_JSON_SCALAR(JsonFunctions.class, "isJsonScalar", String.class),
+  ST_GEOM_FROM_TEXT(GeoFunctions.class, "ST_GeomFromText", String.class),
   INITCAP(SqlFunctions.class, "initcap", String.class),
   SUBSTRING(SqlFunctions.class, "substring", String.class, int.class,
       int.class),
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 8ddcc50..e5106e7 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -78,6 +78,7 @@ import org.apache.calcite.rel.rules.PruneEmptyRules;
 import org.apache.calcite.rel.rules.PushProjector;
 import org.apache.calcite.rel.rules.ReduceExpressionsRule;
 import org.apache.calcite.rel.rules.ReduceExpressionsRule.ProjectReduceExpressionsRule;
+import org.apache.calcite.rel.rules.SpatialRules;
 import org.apache.calcite.rel.rules.UnionMergeRule;
 import org.apache.calcite.rel.rules.ValuesReduceRule;
 import org.apache.calcite.rel.type.RelDataType;
@@ -99,11 +100,13 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql2rel.RelDecorrelator;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
 import org.apache.calcite.test.catalog.MockCatalogReader;
+import org.apache.calcite.test.catalog.MockCatalogReaderExtended;
 import org.apache.calcite.tools.Program;
 import org.apache.calcite.tools.Programs;
 import org.apache.calcite.tools.RelBuilder;
@@ -170,9 +173,10 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
 class RelOptRulesTest extends RelOptTestBase {
   //~ Methods ----------------------------------------------------------------
 
-  private final PushProjector.ExprCondition skipItem = expr ->
-      expr instanceof RexCall
+  private static boolean skipItem(RexNode expr) {
+    return expr instanceof RexCall
           && "item".equalsIgnoreCase(((RexCall) expr).getOperator().getName());
+  }
 
   protected DiffRepository getDiffRepos() {
     return DiffRepository.lookup(RelOptRulesTest.class);
@@ -1534,7 +1538,7 @@ class RelOptRulesTest extends RelOptTestBase {
   @Test void testProjectCorrelateTransposeDynamic() {
     ProjectCorrelateTransposeRule customPCTrans =
         ProjectCorrelateTransposeRule.Config.DEFAULT
-            .withPreserveExprCondition(skipItem)
+            .withPreserveExprCondition(RelOptRulesTest::skipItem)
             .toRule();
 
     String sql = "select t1.c_nationkey, t2.a as fake_col2 "
@@ -1637,7 +1641,7 @@ class RelOptRulesTest extends RelOptTestBase {
   @Test void testProjectCorrelateTransposeWithExprCond() {
     ProjectCorrelateTransposeRule customPCTrans =
         ProjectCorrelateTransposeRule.Config.DEFAULT
-            .withPreserveExprCondition(skipItem)
+            .withPreserveExprCondition(RelOptRulesTest::skipItem)
             .toRule();
 
     final String sql = "select t1.name, t2.ename\n"
@@ -2661,40 +2665,40 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  private void checkPlanning(String query) {
-    final Tester tester1 = tester.withCatalogReaderFactory(
-        (typeFactory, caseSensitive) -> new MockCatalogReader(typeFactory, caseSensitive) {
-          @Override public MockCatalogReader init() {
-            // CREATE SCHEMA abc;
-            // CREATE TABLE a(a INT);
-            // ...
-            // CREATE TABLE j(j INT);
-            MockSchema schema = new MockSchema("SALES");
-            registerSchema(schema);
-            final RelDataType intType =
-                typeFactory.createSqlType(SqlTypeName.INTEGER);
-            for (int i = 0; i < 10; i++) {
-              String t = String.valueOf((char) ('A' + i));
-              MockTable table = MockTable.create(this, schema, t, false, 100);
-              table.addColumn(t, intType);
-              registerTable(table);
-            }
-            return this;
-          }
-          // CHECKSTYLE: IGNORE 1
-        });
+  /** Creates an environment for testing multi-join queries. */
+  private Sql multiJoin(String query) {
     HepProgram program = new HepProgramBuilder()
         .addMatchOrder(HepMatchOrder.BOTTOM_UP)
         .addRuleInstance(CoreRules.PROJECT_REMOVE)
         .addRuleInstance(CoreRules.JOIN_TO_MULTI_JOIN)
         .build();
-    sql(query).withTester(t -> tester1)
-        .with(program)
-        .check();
+    return sql(query)
+        .withCatalogReaderFactory((typeFactory, caseSensitive) ->
+            new MockCatalogReader(typeFactory, caseSensitive) {
+              @Override public MockCatalogReader init() {
+                // CREATE SCHEMA abc;
+                // CREATE TABLE a(a INT);
+                // ...
+                // CREATE TABLE j(j INT);
+                MockSchema schema = new MockSchema("SALES");
+                registerSchema(schema);
+                final RelDataType intType =
+                    typeFactory.createSqlType(SqlTypeName.INTEGER);
+                for (int i = 0; i < 10; i++) {
+                  String t = String.valueOf((char) ('A' + i));
+                  MockTable table = MockTable.create(this, schema, t, false, 100);
+                  table.addColumn(t, intType);
+                  registerTable(table);
+                }
+                return this;
+              }
+              // CHECKSTYLE: IGNORE 1
+            })
+        .with(program);
   }
 
   @Test void testConvertMultiJoinRuleOuterJoins() {
-    checkPlanning("select * from "
+    final String sql = "select * from "
         + "    (select * from "
         + "        (select * from "
         + "            (select * from A right outer join B on a = b) "
@@ -2710,25 +2714,29 @@ class RelOptRulesTest extends RelOptTestBase {
         + "        on a = e and b = f and c = g and d = h) "
         + "    inner join "
         + "    (select * from I inner join J on i = j) "
-        + "    on a = i and h = j");
+        + "    on a = i and h = j";
+    multiJoin(sql).check();
   }
 
   @Test void testConvertMultiJoinRuleOuterJoins2() {
     // in (A right join B) join C, pushing C is not allowed;
     // therefore there should be 2 MultiJoin
-    checkPlanning("select * from A right join B on a = b join C on b = c");
+    multiJoin("select * from A right join B on a = b join C on b = c")
+        .check();
   }
 
   @Test void testConvertMultiJoinRuleOuterJoins3() {
     // in (A join B) left join C, pushing C is allowed;
     // therefore there should be 1 MultiJoin
-    checkPlanning("select * from A join B on a = b left join C on b = c");
+    multiJoin("select * from A join B on a = b left join C on b = c")
+        .check();
   }
 
   @Test void testConvertMultiJoinRuleOuterJoins4() {
     // in (A join B) right join C, pushing C is not allowed;
     // therefore there should be 2 MultiJoin
-    checkPlanning("select * from A join B on a = b right join C on b = c");
+    multiJoin("select * from A join B on a = b right join C on b = c")
+        .check();
   }
 
   @Test void testPushSemiJoinPastProject() {
@@ -6099,6 +6107,87 @@ class RelOptRulesTest extends RelOptTestBase {
     diffRepos.assertEquals("planAfter", "${planAfter}", planAfter);
   }
 
+  /** Creates an environment for testing spatial queries. */
+  private Sql spatial(String sql) {
+    final HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(CoreRules.PROJECT_REDUCE_EXPRESSIONS)
+        .addRuleInstance(CoreRules.FILTER_REDUCE_EXPRESSIONS)
+        .addRuleInstance(SpatialRules.INSTANCE)
+        .build();
+    return sql(sql)
+        .withCatalogReaderFactory((typeFactory, caseSensitive) ->
+            new MockCatalogReaderExtended(typeFactory, caseSensitive).init())
+        .withConformance(SqlConformanceEnum.LENIENT)
+        .with(program);
+  }
+
+  /** Tests that a call to {@code ST_DWithin}
+   * is rewritten with an additional range predicate. */
+  @Test void testSpatialDWithinToHilbert() {
+    final String sql = "select *\n"
+        + "from GEO.Restaurants as r\n"
+        + "where ST_DWithin(ST_Point(10.0, 20.0),\n"
+        + "                 ST_Point(r.longitude, r.latitude), 10)";
+    spatial(sql).check();
+  }
+
+  /** Tests that a call to {@code ST_DWithin}
+   * is rewritten with an additional range predicate. */
+  @Test void testSpatialDWithinToHilbertZero() {
+    final String sql = "select *\n"
+        + "from GEO.Restaurants as r\n"
+        + "where ST_DWithin(ST_Point(10.0, 20.0),\n"
+        + "                 ST_Point(r.longitude, r.latitude), 0)";
+    spatial(sql).check();
+  }
+
+  @Test void testSpatialDWithinToHilbertNegative() {
+    final String sql = "select *\n"
+        + "from GEO.Restaurants as r\n"
+        + "where ST_DWithin(ST_Point(10.0, 20.0),\n"
+        + "                 ST_Point(r.longitude, r.latitude), -2)";
+    spatial(sql).check();
+  }
+
+  /** As {@link #testSpatialDWithinToHilbert()} but arguments reversed. */
+  @Test void testSpatialDWithinReversed() {
+    final String sql = "select *\n"
+        + "from GEO.Restaurants as r\n"
+        + "where ST_DWithin(ST_Point(r.longitude, r.latitude),\n"
+        + "                 ST_Point(10.0, 20.0), 6)";
+    spatial(sql).check();
+  }
+
+  /** Points within a given distance of a line. */
+  @Test void testSpatialDWithinLine() {
+    final String sql = "select *\n"
+        + "from GEO.Restaurants as r\n"
+        + "where ST_DWithin(\n"
+        + "  ST_MakeLine(ST_Point(8.0, 20.0), ST_Point(12.0, 20.0)),\n"
+        + "  ST_Point(r.longitude, r.latitude), 4)";
+    spatial(sql).check();
+  }
+
+  /** Points near a constant point, using ST_Contains and ST_Buffer. */
+  @Test void testSpatialContainsPoint() {
+    final String sql = "select *\n"
+        + "from GEO.Restaurants as r\n"
+        + "where ST_Contains(\n"
+        + "  ST_Buffer(ST_Point(10.0, 20.0), 6),\n"
+        + "  ST_Point(r.longitude, r.latitude))";
+    spatial(sql).check();
+  }
+
+  /** Constant reduction on geo-spatial expression. */
+  @Test void testSpatialReduce() {
+    final String sql = "select\n"
+        + "  ST_Buffer(ST_Point(0.0, 1.0), 2) as b\n"
+        + "from GEO.Restaurants as r";
+    spatial(sql)
+        .withProperty(Hook.REL_BUILDER_SIMPLIFY, false)
+        .check();
+  }
+
   @Test void testOversimplifiedCaseStatement() {
     String sql = "select * from emp "
         + "where MGR > 0 and "
@@ -6641,7 +6730,7 @@ class RelOptRulesTest extends RelOptTestBase {
     ProjectJoinTransposeRule projectJoinTransposeRule =
         CoreRules.PROJECT_JOIN_TRANSPOSE.config
             .withOperandFor(Project.class, Join.class)
-            .withPreserveExprCondition(skipItem)
+            .withPreserveExprCondition(RelOptRulesTest::skipItem)
             .toRule();
 
     final String sql = "select t1.c_nationkey[0], t2.c_nationkey[0]\n"
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
index 6af76d5..aeacefc 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
@@ -34,6 +34,8 @@ import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.runtime.FlatLists;
 import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.sql.test.SqlTestFactory;
+import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql2rel.RelDecorrelator;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.Closer;
@@ -240,6 +242,15 @@ abstract class RelOptTestBase extends SqlToRelTestBase {
       return withTransform(tester -> tester.withTrim(b));
     }
 
+    public Sql withCatalogReaderFactory(
+        SqlTestFactory.MockCatalogReaderFactory factory) {
+      return withTransform(tester -> tester.withCatalogReaderFactory(factory));
+    }
+
+    public Sql withConformance(final SqlConformance conformance) {
+      return withTransform(tester -> tester.withConformance(conformance));
+    }
+
     public Sql withContext(final Context context) {
       return withTransform(tester -> tester.withContext(context));
     }
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
index 1f1fa31..7c26442 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -55,6 +55,7 @@ import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.test.SqlTestFactory;
 import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
+import org.apache.calcite.sql.util.SqlOperatorTables;
 import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
@@ -691,8 +692,15 @@ public abstract class SqlToRelTestBase {
     public SqlValidator createValidator(
         SqlValidatorCatalogReader catalogReader,
         RelDataTypeFactory typeFactory) {
+      final SqlOperatorTable operatorTable = getOperatorTable();
+      final SqlConformance conformance = getConformance();
+      final List<SqlOperatorTable> list = new ArrayList<>();
+      list.add(operatorTable);
+      if (conformance.allowGeometry()) {
+        list.add(SqlOperatorTables.spatialInstance());
+      }
       return new FarragoTestValidator(
-          getOperatorTable(),
+          SqlOperatorTables.chain(list),
           catalogReader,
           typeFactory,
           SqlValidator.Config.DEFAULT
@@ -865,7 +873,7 @@ public abstract class SqlToRelTestBase {
     }
   }
 
-    /** Validator for testing. */
+  /** Validator for testing. */
   private static class FarragoTestValidator extends SqlValidatorImpl {
     FarragoTestValidator(
         SqlOperatorTable opTab,
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index bb6a82f..b9d7ed6 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -8111,7 +8111,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     sql("select ^sum(ramp(3))^ from (values (1))")
         .fails("Cannot apply 'SUM' to arguments of type 'SUM\\(<CURSOR>\\)'\\. "
             + "Supported form\\(s\\): 'SUM\\(<NUMERIC>\\)'");
-    sql("select * from (values (1)) group by ^ramp(3)^")
+    sql("select 0 from (values (1)) group by ^ramp(3)^")
         .fails("Cannot call table function here: 'RAMP'");
     sql("select count(*) from (values (1)) having ^ramp(3)^")
         .fails("HAVING clause must be a condition");
diff --git a/core/src/test/java/org/apache/calcite/test/catalog/Fixture.java b/core/src/test/java/org/apache/calcite/test/catalog/Fixture.java
index 2cf7cd8..4382372 100644
--- a/core/src/test/java/org/apache/calcite/test/catalog/Fixture.java
+++ b/core/src/test/java/org/apache/calcite/test/catalog/Fixture.java
@@ -71,7 +71,7 @@ final class Fixture extends AbstractFixture {
   final RelDataType abRecordType = typeFactory.builder()
       .add("A", varchar10Type)
       .add("B", varchar10Type)
-      .build();;
+      .build();
   final RelDataType skillRecordType = typeFactory.builder()
       .add("TYPE", varchar10Type)
       .add("DESC", varchar20Type)
diff --git a/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReader.java b/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReader.java
index 8c764c7..93dc2c2 100644
--- a/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReader.java
@@ -312,12 +312,18 @@ public abstract class MockCatalogReader extends CalciteCatalogReader {
     protected final InitializerExpressionFactory initializerFactory;
     protected final Set<String> rolledUpColumns = new HashSet<>();
 
+    /** Wrapped objects that can be obtained by calling
+     * {@link #unwrap(Class)}. Initially an immutable list, but converted to
+     * a mutable array list on first assignment. */
+    protected List<Object> wraps;
+
     public MockTable(MockCatalogReader catalogReader, String catalogName,
         String schemaName, String name, boolean stream, boolean temporal,
         double rowCount, ColumnResolver resolver,
         InitializerExpressionFactory initializerFactory) {
       this(catalogReader, ImmutableList.of(catalogName, schemaName, name),
-          stream, temporal, rowCount, resolver, initializerFactory);
+          stream, temporal, rowCount, resolver, initializerFactory,
+          ImmutableList.of());
     }
 
     public void registerRolledUpColumn(String columnName) {
@@ -327,7 +333,7 @@ public abstract class MockCatalogReader extends CalciteCatalogReader {
     private MockTable(MockCatalogReader catalogReader, List<String> names,
         boolean stream, boolean temporal, double rowCount,
         ColumnResolver resolver,
-        InitializerExpressionFactory initializerFactory) {
+        InitializerExpressionFactory initializerFactory, List<Object> wraps) {
       this.catalogReader = catalogReader;
       this.stream = stream;
       this.temporal = temporal;
@@ -335,6 +341,7 @@ public abstract class MockCatalogReader extends CalciteCatalogReader {
       this.names = names;
       this.resolver = resolver;
       this.initializerFactory = initializerFactory;
+      this.wraps = ImmutableList.copyOf(wraps);
     }
 
     /**
@@ -359,6 +366,14 @@ public abstract class MockCatalogReader extends CalciteCatalogReader {
       for (String name : monotonicColumnSet) {
         addMonotonic(name);
       }
+      this.wraps = ImmutableList.of();
+    }
+
+    void addWrap(Object wrap) {
+      if (wraps instanceof ImmutableList) {
+        wraps = new ArrayList<>(wraps);
+      }
+      wraps.add(wrap);
     }
 
     /** Implementation of AbstractModifiableTable. */
@@ -430,7 +445,7 @@ public abstract class MockCatalogReader extends CalciteCatalogReader {
 
     @Override protected RelOptTable extend(final Table extendedTable) {
       return new MockTable(catalogReader, names, stream, temporal, rowCount,
-          resolver, initializerFactory) {
+          resolver, initializerFactory, wraps) {
         @Override public RelDataType getRowType() {
           return extendedTable.getRowType(catalogReader.typeFactory);
         }
@@ -445,7 +460,7 @@ public abstract class MockCatalogReader extends CalciteCatalogReader {
     public static MockTable create(MockCatalogReader catalogReader,
         List<String> names, boolean stream, double rowCount) {
       return new MockTable(catalogReader, names, stream, false, rowCount, null,
-          NullInitializerExpressionFactory.INSTANCE);
+          NullInitializerExpressionFactory.INSTANCE, ImmutableList.of());
     }
 
     public static MockTable create(MockCatalogReader catalogReader,
@@ -481,6 +496,11 @@ public abstract class MockCatalogReader extends CalciteCatalogReader {
                 : new ModifiableTableWithCustomColumnResolving(Util.last(names));
         return clazz.cast(table);
       }
+      for (Object handler : wraps) {
+        if (clazz.isInstance(handler)) {
+          return clazz.cast(handler);
+        }
+      }
       return null;
     }
 
@@ -612,7 +632,8 @@ public abstract class MockCatalogReader extends CalciteCatalogReader {
         boolean stream, double rowCount, ColumnResolver resolver,
         InitializerExpressionFactory initializerExpressionFactory) {
       super(catalogReader, ImmutableList.of(catalogName, schemaName, name),
-          stream, false, rowCount, resolver, initializerExpressionFactory);
+          stream, false, rowCount, resolver, initializerExpressionFactory,
+          ImmutableList.of());
       this.modifiableViewTable = modifiableViewTable;
     }
 
@@ -724,7 +745,8 @@ public abstract class MockCatalogReader extends CalciteCatalogReader {
         boolean stream, double rowCount, ColumnResolver resolver,
         InitializerExpressionFactory initializerExpressionFactory) {
       super(catalogReader, ImmutableList.of(catalogName, schemaName, name),
-          stream, false, rowCount, resolver, initializerExpressionFactory);
+          stream, false, rowCount, resolver, initializerExpressionFactory,
+          ImmutableList.of());
       this.viewTable = viewTable;
     }
 
diff --git a/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReaderExtended.java b/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReaderExtended.java
index 9246113..5683def 100644
--- a/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReaderExtended.java
+++ b/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReaderExtended.java
@@ -16,14 +16,22 @@
  */
 package org.apache.calcite.test.catalog;
 
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.BuiltInMetadata;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
-import org.apache.calcite.rel.type.RelRecordType;
-import org.apache.calcite.rel.type.StructKind;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.schema.TableMacro;
 import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.SqlOperatorTables;
 
 import com.google.common.collect.ImmutableList;
 
@@ -106,7 +114,7 @@ public class MockCatalogReaderExtended extends MockCatalogReaderSimple {
         new CompoundNameColumn("F0", "C0", f.intType),
         new CompoundNameColumn("F1", "C1", f.intTypeNull));
     final List<CompoundNameColumn> extendedColumns =
-        new ArrayList<CompoundNameColumn>(columnsExtended);
+        new ArrayList<>(columnsExtended);
     extendedColumns.add(new CompoundNameColumn("F2", "C2", f.varchar20Type));
     final CompoundNameColumnResolver structExtendedTableResolver =
         new CompoundNameColumnResolver(extendedColumns, "F0");
@@ -173,21 +181,12 @@ public class MockCatalogReaderExtended extends MockCatalogReaderSimple {
     final MockTable nullableRowsTable =
         MockTable.create(this, nullableRowsSchema, "NR_T1", false, 100);
     RelDataType bigIntNotNull = typeFactory.createSqlType(SqlTypeName.BIGINT);
-    RelDataType nullableRecordType = new RelRecordType(
-        StructKind.FULLY_QUALIFIED,
-        Arrays.asList(
-            new RelDataTypeFieldImpl(
-                "NOT_NULL_FIELD",
-                0,
-                bigIntNotNull),
-            new RelDataTypeFieldImpl(
-                "NULLABLE_FIELD",
-                0,
-                typeFactory.createTypeWithNullability(bigIntNotNull, true)
-            )
-        ),
-        true
-    );
+    RelDataType nullableRecordType =
+        typeFactory.builder()
+            .nullableRecord(true)
+            .add("NOT_NULL_FIELD", bigIntNotNull)
+            .add("NULLABLE_FIELD", bigIntNotNull).nullable(true)
+            .build();
 
     nullableRowsTable.addColumn("ROW_COLUMN", nullableRecordType, false);
     nullableRowsTable.addColumn(
@@ -196,6 +195,53 @@ public class MockCatalogReaderExtended extends MockCatalogReaderSimple {
         true);
     registerTable(nullableRowsTable);
 
+    MockSchema geoSchema = new MockSchema("GEO");
+    registerSchema(geoSchema);
+    final MockTable restaurantTable =
+        MockTable.create(this, geoSchema, "RESTAURANTS", false, 100);
+    restaurantTable.addColumn("NAME", f.varchar20Type, true);
+    restaurantTable.addColumn("LATITUDE", f.intType);
+    restaurantTable.addColumn("LONGITUDE", f.intType);
+    restaurantTable.addColumn("CUISINE", f.varchar10Type);
+    restaurantTable.addColumn("HILBERT", f.bigintType);
+    restaurantTable.addMonotonic("HILBERT");
+    restaurantTable.addWrap(
+        new BuiltInMetadata.AllPredicates.Handler() {
+          public RelOptPredicateList getAllPredicates(RelNode r,
+              RelMetadataQuery mq) {
+            // Return the predicate:
+            //  r.hilbert = hilbert(r.longitude, r.latitude)
+            //
+            // (Yes, x = longitude, y = latitude. Same as ST_MakePoint.)
+            final RexBuilder rexBuilder = r.getCluster().getRexBuilder();
+            final RexInputRef refLatitude = rexBuilder.makeInputRef(r, 1);
+            final RexInputRef refLongitude = rexBuilder.makeInputRef(r, 2);
+            final RexInputRef refHilbert = rexBuilder.makeInputRef(r, 4);
+            return RelOptPredicateList.of(rexBuilder,
+                ImmutableList.of(
+                    rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+                        refHilbert,
+                        rexBuilder.makeCall(hilbertOp(),
+                            refLongitude, refLatitude))));
+          }
+
+          SqlOperator hilbertOp() {
+            for (SqlOperator op
+                : SqlOperatorTables.spatialInstance().getOperatorList()) {
+              if (op.getKind() == SqlKind.HILBERT
+                  && op.getOperandCountRange().isValidCount(2)) {
+                return op;
+              }
+            }
+            throw new AssertionError();
+          }
+
+          public MetadataDef<BuiltInMetadata.AllPredicates> getDef() {
+            return BuiltInMetadata.AllPredicates.DEF;
+          }
+        });
+    registerTable(restaurantTable);
+
     return this;
   }
 }
diff --git a/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReaderSimple.java b/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReaderSimple.java
index 6824733..1723711 100644
--- a/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReaderSimple.java
+++ b/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReaderSimple.java
@@ -42,7 +42,7 @@ import java.util.List;
  * Simple catalog reader for testing.
  */
 public class MockCatalogReaderSimple extends MockCatalogReader {
-  private final Fixture fixture;
+  private final ObjectSqlType addressType;
 
   /**
    * Creates a MockCatalogReader.
@@ -55,19 +55,20 @@ public class MockCatalogReaderSimple extends MockCatalogReader {
   public MockCatalogReaderSimple(RelDataTypeFactory typeFactory,
       boolean caseSensitive) {
     super(typeFactory, caseSensitive);
-    fixture = new Fixture(typeFactory);
+
+    addressType = new Fixture(typeFactory).addressType;
   }
 
   @Override public RelDataType getNamedType(SqlIdentifier typeName) {
-    if (typeName.equalsDeep(fixture.addressType.getSqlIdentifier(), Litmus.IGNORE)) {
-      return fixture.addressType;
+    if (typeName.equalsDeep(addressType.getSqlIdentifier(), Litmus.IGNORE)) {
+      return addressType;
     } else {
       return super.getNamedType(typeName);
     }
   }
 
   @Override public MockCatalogReader init() {
-    ObjectSqlType addressType = fixture.addressType;
+    final Fixture fixture = new Fixture(typeFactory);
 
     // Register "SALES" schema.
     MockSchema salesSchema = new MockSchema("SALES");
diff --git a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
index b3ddf40..0ff144c 100644
--- a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
@@ -71,8 +71,8 @@ import org.apache.calcite.sql.test.SqlTests;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
 import org.apache.calcite.sql.util.ListSqlOperatorTable;
+import org.apache.calcite.sql.util.SqlOperatorTables;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.test.CalciteAssert;
@@ -204,7 +204,7 @@ class PlannerTest {
   @Test void testValidateUserDefinedAggregate() throws Exception {
     final SqlStdOperatorTable stdOpTab = SqlStdOperatorTable.instance();
     SqlOperatorTable opTab =
-        ChainedSqlOperatorTable.of(stdOpTab,
+        SqlOperatorTables.chain(stdOpTab,
             new ListSqlOperatorTable(
                 ImmutableList.of(new MyCountAggFunction())));
     final SchemaPlus rootSchema = Frameworks.createRootSchema(true);
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index 86f6a44..bd5176f 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -4169,7 +4169,9 @@ LogicalProject(EXPR$0=[CAST(/($2, $3)):INTEGER NOT NULL])
     </TestCase>
     <TestCase name="testNoOversimplificationBelowIsNull">
         <Resource name="sql">
-            <![CDATA[select * from emp where ( (empno=1 and mgr=1) or (empno=null and mgr=1) ) is null]]>
+            <![CDATA[select *
+from emp
+where ( (empno=1 and mgr=1) or (empno=null and mgr=1) ) is null]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
@@ -8560,6 +8562,28 @@ LogicalProject(N=[$0])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testSpatialDWithinToHilbert">
+        <Resource name="sql">
+            <![CDATA[select *
+from GEO.Restaurants as r
+where ST_DWithin(ST_Point(10.0, 20.0),
+                 ST_Point(r.longitude, r.latitude), 10)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(NAME=[$0], LATITUDE=[$1], LONGITUDE=[$2], CUISINE=[$3], HILBERT=[$4])
+  LogicalFilter(condition=[ST_DWITHIN(ST_POINT(10.0:DECIMAL(3, 1), 20.0:DECIMAL(3, 1)), ST_POINT($2, $1), 10)])
+    LogicalTableScan(table=[[CATALOG, GEO, RESTAURANTS]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(NAME=[$0], LATITUDE=[$1], LONGITUDE=[$2], CUISINE=[$3], HILBERT=[$4])
+  LogicalFilter(condition=[AND(OR(AND(>=($4, 36556), <=($4, 36568)), AND(>=($4, 36447), <=($4, 36548)), AND(>=($4, 36442), <=($4, 36445)), AND(>=($4, 36425), <=($4, 36427)), AND(>=($4, 36421), <=($4, 36423)), AND(>=($4, 36403), <=($4, 36404)), AND(>=($4, 36392), <=($4, 36401)), AND(>=($4, 33267), <=($4, 33270)), AND(>=($4, 33231), <=($4, 33265)), AND(>=($4, 33226), <=($4, 33229)), AND(>=($4, 33209), <=($4, 33211)), AND(>=($4, 33205), <=($4, 33207)), AND(>=($4, 33187), <=($4, 33190)), AND [...]
+    LogicalTableScan(table=[[CATALOG, GEO, RESTAURANTS]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testSwapOuterJoin">
         <Resource name="sql">
             <![CDATA[select 1 from sales.dept d left outer join sales.emp e
@@ -8585,8 +8609,9 @@ LogicalProject(EXPR$0=[1])
     </TestCase>
     <TestCase name="testDistinctCountMultipleViaJoin">
         <Resource name="sql">
-            <![CDATA[select deptno, count(distinct ename), count(distinct job, ename),
-count(distinct deptno, job), sum(sal)
+            <![CDATA[select deptno, count(distinct ename),
+  count(distinct job, ename),
+  count(distinct deptno, job), sum(sal)
 from sales.emp group by deptno]]>
         </Resource>
         <Resource name="planBefore">
@@ -8618,7 +8643,8 @@ LogicalProject(DEPTNO=[$0], EXPR$1=[$3], EXPR$2=[$5], EXPR$3=[$7], EXPR$4=[$1])
     </TestCase>
     <TestCase name="testDistinctCountMultiple">
         <Resource name="sql">
-            <![CDATA[select deptno, count(distinct ename), count(distinct job)
+            <![CDATA[select deptno, count(distinct ename),
+  count(distinct job)
 from sales.emp group by deptno]]>
         </Resource>
         <Resource name="planBefore">
@@ -8661,7 +8687,8 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $2], EXPR$1=[COUNT($1) FIL
     </TestCase>
     <TestCase name="testDistinctCountMixed">
         <Resource name="sql">
-            <![CDATA[select deptno, count(distinct deptno, job) as cddj, sum(sal) as s
+            <![CDATA[select deptno, count(distinct deptno, job) as cddj,
+  sum(sal) as s
 from sales.emp group by deptno]]>
         </Resource>
         <Resource name="planBefore">
@@ -9017,7 +9044,8 @@ LogicalAggregate(group=[{}], VOLUME=[$SUM0($0)], C1_SUM_SAL=[SUM($1)])
     </TestCase>
     <TestCase name="testPushDistinctAggregateIntoJoin">
         <Resource name="sql">
-            <![CDATA[select count(distinct sal) from sales.emp join sales.dept on job = name]]>
+            <![CDATA[select count(distinct sal) from sales.emp
+ join sales.dept on job = name]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
@@ -9399,6 +9427,136 @@ LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], EXPR$2=[SUM($2
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testSpatialContainsPoint">
+        <Resource name="sql">
+            <![CDATA[select *
+from GEO.Restaurants as r
+where ST_Contains(
+  ST_Buffer(ST_Point(10.0, 20.0), 6),
+  ST_Point(r.longitude, r.latitude))]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(NAME=[$0], LATITUDE=[$1], LONGITUDE=[$2], CUISINE=[$3], HILBERT=[$4])
+  LogicalFilter(condition=[ST_CONTAINS(ST_BUFFER(ST_POINT(10.0:DECIMAL(3, 1), 20.0:DECIMAL(3, 1)), 6), ST_POINT($2, $1))])
+    LogicalTableScan(table=[[CATALOG, GEO, RESTAURANTS]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(NAME=[$0], LATITUDE=[$1], LONGITUDE=[$2], CUISINE=[$3], HILBERT=[$4])
+  LogicalFilter(condition=[AND(OR(AND(>=($4, 36496), <=($4, 36520)), AND(>=($4, 36456), <=($4, 36464)), AND(>=($4, 33252), <=($4, 33254)), AND(>=($4, 33236), <=($4, 33244)), AND(>=($4, 33164), <=($4, 33176)), AND(>=($4, 33112), <=($4, 33156)), AND(>=($4, 33092), <=($4, 33100)), AND(>=($4, 33055), <=($4, 33080)), AND(>=($4, 33050), <=($4, 33053)), AND(>=($4, 33033), <=($4, 33035))), ST_CONTAINS(MULTIPOLYGON (((16 20, 15.987153539431617 20.39241877538086, 15.948669168242859 20.783157153320 [...]
+    LogicalTableScan(table=[[CATALOG, GEO, RESTAURANTS]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testSpatialDWithinLine">
+        <Resource name="sql">
+            <![CDATA[select *
+from GEO.Restaurants as r
+where ST_DWithin(
+  ST_MakeLine(ST_Point(8.0, 20.0), ST_Point(12.0, 20.0)),
+  ST_Point(r.longitude, r.latitude), 4)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(NAME=[$0], LATITUDE=[$1], LONGITUDE=[$2], CUISINE=[$3], HILBERT=[$4])
+  LogicalFilter(condition=[ST_DWITHIN(ST_MAKELINE(ST_POINT(8.0:DECIMAL(2, 1), 20.0:DECIMAL(3, 1)), ST_POINT(12.0:DECIMAL(3, 1), 20.0:DECIMAL(3, 1))), ST_POINT($2, $1), 4)])
+    LogicalTableScan(table=[[CATALOG, GEO, RESTAURANTS]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(NAME=[$0], LATITUDE=[$1], LONGITUDE=[$2], CUISINE=[$3], HILBERT=[$4])
+  LogicalFilter(condition=[AND(OR(AND(>=($4, 36517), <=($4, 36519)), AND(>=($4, 36505), <=($4, 36507)), AND(>=($4, 36501), <=($4, 36503)), AND(>=($4, 36457), <=($4, 36459)), AND(>=($4, 33236), <=($4, 33240)), AND(>=($4, 33164), <=($4, 33176)), AND(>=($4, 33112), <=($4, 33156)), AND(>=($4, 33092), <=($4, 33100)), AND(>=($4, 33064), <=($4, 33076))), ST_DWITHIN(MULTILINESTRING ((8 20, 12 20)):GEOMETRY, ST_POINT($2, $1), 4))])
+    LogicalTableScan(table=[[CATALOG, GEO, RESTAURANTS]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testSpatialDWithinReversed">
+        <Resource name="sql">
+            <![CDATA[select *
+from GEO.Restaurants as r
+where ST_DWithin(ST_Point(r.longitude, r.latitude),
+                 ST_Point(10.0, 20.0), 6)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(NAME=[$0], LATITUDE=[$1], LONGITUDE=[$2], CUISINE=[$3], HILBERT=[$4])
+  LogicalFilter(condition=[ST_DWITHIN(ST_POINT($2, $1), ST_POINT(10.0:DECIMAL(3, 1), 20.0:DECIMAL(3, 1)), 6)])
+    LogicalTableScan(table=[[CATALOG, GEO, RESTAURANTS]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(NAME=[$0], LATITUDE=[$1], LONGITUDE=[$2], CUISINE=[$3], HILBERT=[$4])
+  LogicalFilter(condition=[AND(OR(AND(>=($4, 36496), <=($4, 36520)), AND(>=($4, 36456), <=($4, 36464)), AND(>=($4, 33252), <=($4, 33254)), AND(>=($4, 33236), <=($4, 33244)), AND(>=($4, 33164), <=($4, 33176)), AND(>=($4, 33112), <=($4, 33156)), AND(>=($4, 33092), <=($4, 33100)), AND(>=($4, 33055), <=($4, 33080)), AND(>=($4, 33050), <=($4, 33053)), AND(>=($4, 33033), <=($4, 33035))), ST_DWITHIN(ST_POINT($2, $1), POINT (10 20):GEOMETRY, 6))])
+    LogicalTableScan(table=[[CATALOG, GEO, RESTAURANTS]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testSpatialDWithinToHilbertNegative">
+        <Resource name="sql">
+            <![CDATA[select *
+from GEO.Restaurants as r
+where ST_DWithin(ST_Point(10.0, 20.0),
+                 ST_Point(r.longitude, r.latitude), -2)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(NAME=[$0], LATITUDE=[$1], LONGITUDE=[$2], CUISINE=[$3], HILBERT=[$4])
+  LogicalFilter(condition=[ST_DWITHIN(ST_POINT(10.0:DECIMAL(3, 1), 20.0:DECIMAL(3, 1)), ST_POINT($2, $1), -2)])
+    LogicalTableScan(table=[[CATALOG, GEO, RESTAURANTS]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(NAME=[$0], LATITUDE=[$1], LONGITUDE=[$2], CUISINE=[$3], HILBERT=[$4])
+  LogicalValues(tuples=[[]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testSpatialDWithinToHilbertZero">
+        <Resource name="sql">
+            <![CDATA[select *
+from GEO.Restaurants as r
+where ST_DWithin(ST_Point(10.0, 20.0),
+                 ST_Point(r.longitude, r.latitude), 0)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(NAME=[$0], LATITUDE=[$1], LONGITUDE=[$2], CUISINE=[$3], HILBERT=[$4])
+  LogicalFilter(condition=[ST_DWITHIN(ST_POINT(10.0:DECIMAL(3, 1), 20.0:DECIMAL(3, 1)), ST_POINT($2, $1), 0)])
+    LogicalTableScan(table=[[CATALOG, GEO, RESTAURANTS]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(NAME=[$0], LATITUDE=[$1], LONGITUDE=[$2], CUISINE=[$3], HILBERT=[$4])
+  LogicalFilter(condition=[AND(=($4, 33139), =(POINT (10 20), ST_POINT($2, $1)))])
+    LogicalTableScan(table=[[CATALOG, GEO, RESTAURANTS]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testSpatialReduce">
+        <Resource name="sql">
+            <![CDATA[select
+  ST_Buffer(ST_Point(0.0, 1.0), 2) as b
+from GEO.Restaurants as r]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(B=[ST_BUFFER(ST_POINT(0.0:DECIMAL(2, 1), 1.0:DECIMAL(2, 1)), 2)])
+  LogicalTableScan(table=[[CATALOG, GEO, RESTAURANTS]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(B=[CAST(MULTIPOLYGON (((2 1, 1.9957178464772056 1.1308062584602858, 1.9828897227476194 1.2610523844401027, 1.9615705608064595 1.3901806440322562, 1.9318516525781353 1.517638090205041, 1.8938602589902103 1.6428789306063227, 1.8477590650225726 1.7653668647301788, 1.7937454830653756 1.8845773804380017, 1.7320508075688765 1.9999999999999991, 1.6629392246050898 2.1111404660392035, 1.5867066805824699 2.217522858017441, 1.5036796149579543 2.318691630200137, 1.4142135623730947 2.4 [...]
+  LogicalTableScan(table=[[CATALOG, GEO, RESTAURANTS]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testAggregateRemove6">
         <Resource name="sql">
             <![CDATA[select deptno, max(sal) from sales.emp group by deptno
@@ -9472,7 +9630,8 @@ LogicalAggregate(group=[{7}])
     </TestCase>
     <TestCase name="testAggregateJoinRemove2">
         <Resource name="sql">
-            <![CDATA[select e.deptno, count(distinct e.job) from sales.emp e
+            <![CDATA[select e.deptno, count(distinct e.job)
+from sales.emp e
 left outer join sales.dept d on e.deptno = d.deptno
 group by e.deptno]]>
         </Resource>
@@ -9494,7 +9653,8 @@ LogicalAggregate(group=[{7}], EXPR$1=[COUNT(DISTINCT $2)])
     </TestCase>
     <TestCase name="testAggregateJoinRemove3">
         <Resource name="sql">
-            <![CDATA[select e.deptno, count(distinct d.name) from sales.emp e
+            <![CDATA[select e.deptno, count(distinct d.name)
+from sales.emp e
 left outer join sales.dept d on e.deptno = d.deptno
 group by e.deptno]]>
         </Resource>
@@ -9518,7 +9678,8 @@ LogicalAggregate(group=[{7}], EXPR$1=[COUNT(DISTINCT $10)])
     </TestCase>
     <TestCase name="testAggregateJoinRemove4">
         <Resource name="sql">
-            <![CDATA[select distinct d.deptno from sales.emp e
+            <![CDATA[select distinct d.deptno
+from sales.emp e
 right outer join sales.dept d on e.deptno = d.deptno]]>
         </Resource>
         <Resource name="planBefore">
@@ -9539,7 +9700,8 @@ LogicalProject(DEPTNO=[$0])
     </TestCase>
     <TestCase name="testAggregateJoinRemove5">
         <Resource name="sql">
-            <![CDATA[select d.deptno, count(distinct d.name) from sales.emp e
+            <![CDATA[select d.deptno, count(distinct d.name)
+from sales.emp e
 right outer join sales.dept d on e.deptno = d.deptno
 group by d.deptno]]>
         </Resource>
@@ -9561,7 +9723,8 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)])
     </TestCase>
     <TestCase name="testAggregateJoinRemove6">
         <Resource name="sql">
-            <![CDATA[select d.deptno, count(distinct e.job) from sales.emp e
+            <![CDATA[select d.deptno, count(distinct e.job)
+from sales.emp e
 right outer join sales.dept d on e.deptno = d.deptno
 group by d.deptno]]>
         </Resource>
@@ -12724,7 +12887,10 @@ LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[COUNT($0)], EXPR$2=[BIT_O
     </TestCase>
     <TestCase name="testProjectJoinTransposeItem">
         <Resource name="sql">
-            <![CDATA[select t1.c_nationkey[0], t2.c_nationkey[0] from sales.customer as t1 left outer join sales.customer as t2 on t1.c_nationkey[0] = t2.c_nationkey[0]]]>
+            <![CDATA[select t1.c_nationkey[0], t2.c_nationkey[0]
+from sales.customer as t1
+left outer join sales.customer as t2
+on t1.c_nationkey[0] = t2.c_nationkey[0]]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
@@ -12751,7 +12917,9 @@ LogicalProject(EXPR$0=[$1], EXPR$1=[$3])
     </TestCase>
     <TestCase name="testSimplifyItemIsNotNull">
         <Resource name="sql">
-            <![CDATA[select * from sales.customer as t1 where t1.c_nationkey[0] is not null]]>
+            <![CDATA[select *
+from sales.customer as t1
+where t1.c_nationkey[0] is not null]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
diff --git a/core/src/test/resources/sql/spatial.iq b/core/src/test/resources/sql/spatial.iq
index a9aac81..1dfc45e 100644
--- a/core/src/test/resources/sql/spatial.iq
+++ b/core/src/test/resources/sql/spatial.iq
@@ -265,7 +265,17 @@ EXPR$0
 # Not implemented
 
 # ST_MakeEnvelope(xMin, yMin, xMax, yMax  [, srid ]) Creates a rectangular Polygon
-# Not implemented
+SELECT ST_AsText(ST_MakeEnvelope(10.0, 10.0, 11.0, 11.0, 4326));
+
+EXPR$0
+MULTIPOLYGON (((10 10, 11 10, 11 11, 10 11, 10 10)))
+!ok
+
+SELECT ST_AsText(ST_MakeEnvelope(12.0, -1.0, 6.0, 4.0, 4326));
+
+EXPR$0
+MULTIPOLYGON (((12 -1, 12 4, 6 4, 6 -1, 12 -1)))
+!ok
 
 # ST_MakeGrid(geom, deltaX, deltaY) Calculates a regular grid of polygons based on *geom*
 SELECT * FROM TABLE(ST_MakeGrid(ST_Point(13.0,22.0), 10.0, 5.0));
@@ -283,12 +293,12 @@ THE_GEOM, ID, ID_COL, ID_ROW, ABS_COL, ABS_ROW
 select "name", "latitude", "longitude", p.*
 from GEO."countries" AS c,
   lateral table(
-    ST_MakeGridPoints(ST_MakePoint("latitude", "longitude"), 10.0, 10.0)) as p
+    ST_MakeGridPoints(ST_MakePoint("longitude", "latitude"), 10.0, 10.0)) as p
 ORDER BY "latitude" DESC LIMIT 3;
 name, latitude, longitude, THE_GEOM, ID, ID_COL, ID_ROW, ABS_COL, ABS_ROW
-Svalbard and Jan Mayen, 77.553604, 23.670272, {"x":75,"y":25}, 0, 1, 1, 7, 2
-Greenland, 71.706936, -42.604303, {"x":75,"y":-45}, 0, 1, 1, 7, -5
-Iceland, 64.963051, -19.020835, {"x":65,"y":-15}, 0, 1, 1, 6, -2
+Svalbard and Jan Mayen, 77.553604, 23.670272, {"x":25,"y":75}, 0, 1, 1, 2, 7
+Greenland, 71.706936, -42.604303, {"x":-45,"y":75}, 0, 1, 1, -5, 7
+Iceland, 64.963051, -19.020835, {"x":-15,"y":65}, 0, 1, 1, -2, 6
 !ok
 
 # ST_MakeLine(point1 [, point ]*) Creates a line-string from the given points (or multi-points)
@@ -341,13 +351,13 @@ EXPR$0
 1.5
 !ok
 
-select "name", ST_MakePoint("latitude", "longitude") AS p
+select "name", ST_MakePoint("longitude", "latitude") AS p
 from GEO."countries" AS c
 ORDER BY "latitude" DESC LIMIT 3;
 name, P
 U.S.Minor Outlying Islands, null
-Svalbard and Jan Mayen, {"x":77.553604,"y":23.670272}
-Greenland, {"x":71.706936,"y":-42.604303}
+Svalbard and Jan Mayen, {"x":23.670272,"y":77.553604}
+Greenland, {"x":-42.604303,"y":71.706936}
 !ok
 
 # ST_MakePolygon(lineString [, hole ]*) Creates a polygon from *lineString* with the given holes (which are required to be closed line-strings)
@@ -697,7 +707,7 @@ true
 
 # Countries within 10 degrees of London
 select "name" from GEO."countries" AS c
-where ST_Distance(ST_MakePoint(51.5, -0.12), ST_MakePoint("latitude", "longitude")) < 10;
+where ST_Distance(ST_MakePoint(-0.12, 51.5), ST_MakePoint("longitude", "latitude")) < 10;
 name
 Andorra
 Belgium
@@ -714,7 +724,7 @@ United Kingdom
 
 # Countries within 10 degrees of London, formulated a different way
 select "name" from GEO."countries" AS c
-where ST_DWithin(ST_MakePoint(51.5, -0.12), ST_MakePoint("latitude", "longitude"), 10);
+where ST_DWithin(ST_MakePoint(-0.12, 51.5), ST_MakePoint("longitude", "latitude"), 10);
 name
 Andorra
 Belgium
@@ -834,7 +844,7 @@ EXPR$0
 SELECT ST_Buffer(
  ST_GeomFromText('POINT(100 90)'),
  50, 'quad_segs=8');
-at org.apache.calcite.runtime.GeoFunctions.todo
+at org.apache.calcite.runtime.Geometries.todo
 !error GeoFunctions
 
 # ST_BUFFER(geom, bufferSize, quadSegs) variant - not implemented
@@ -842,7 +852,7 @@ at org.apache.calcite.runtime.GeoFunctions.todo
 SELECT ST_Buffer(
  ST_GeomFromText('POINT(100 90)'),
  50, 2);
-at org.apache.calcite.runtime.GeoFunctions.todo
+at org.apache.calcite.runtime.Geometries.todo
 !error GeoFunctions
 !}
 
@@ -1235,4 +1245,27 @@ WY, Yellowstone NP
 CA, Yosemite NP
 !ok
 
+# Space-filling curves.
+select x, y, hilbert(ST_Point(x, y))
+from (
+  values (0.0, 0.0),
+     (0, 1),
+     (1, 0),
+     (0, -1),
+     (10, 10),
+     (20, 20)) as t(x, y);
+X, Y, EXPR$2
+0.0, -1.0, 10921
+0.0, 0.0, 10922
+0.0, 1.0, 32767
+1.0, 0.0, 54613
+10.0, 10.0, 32973
+20.0, 20.0, 33204
+!ok
+
+values hilbert(ST_Point(20.0, 20.0));
+EXPR$0
+33204
+!ok
+
 # End spatial.iq
diff --git a/gradle.properties b/gradle.properties
index 8b2ad7e..aa8b9c1 100644
--- a/gradle.properties
+++ b/gradle.properties
@@ -134,5 +134,6 @@ spark.version=2.2.2
 sqlline.version=1.9.0
 teradata.tpcds.version=1.2
 tpch.version=0.1
+uzaygezen.version=0.2
 xalan.version=2.7.1
 xercesImpl.version=2.9.1
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Hints.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Hints.java
new file mode 100644
index 0000000..8048829
--- /dev/null
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Hints.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j.function;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+import static java.lang.annotation.ElementType.METHOD;
+import static java.lang.annotation.ElementType.TYPE;
+
+/**
+ * Annotation applied to a user-defined function that gives extra metadata
+ * about that function.
+ *
+ * <p>Examples:
+ * <ul>
+ *   <li>@Hints("SqlKind:ST_DWithin") public static void myFun()</li>
+ * </ul>
+ */
+@Target({METHOD, TYPE })
+@Retention(RetentionPolicy.RUNTIME)
+@Experimental
+public @interface Hints {
+  String[] value();
+}
diff --git a/piglet/src/main/java/org/apache/calcite/piglet/PigUserDefinedFunction.java b/piglet/src/main/java/org/apache/calcite/piglet/PigUserDefinedFunction.java
index acdb24a..720a3d9 100644
--- a/piglet/src/main/java/org/apache/calcite/piglet/PigUserDefinedFunction.java
+++ b/piglet/src/main/java/org/apache/calcite/piglet/PigUserDefinedFunction.java
@@ -33,6 +33,7 @@ import com.google.common.collect.ImmutableList;
 /** Pig user-defined function. */
 public class PigUserDefinedFunction extends SqlUserDefinedFunction {
   public final FuncSpec funcSpec;
+
   private PigUserDefinedFunction(SqlIdentifier opName,
       SqlReturnTypeInference returnTypeInference,
       SqlOperandTypeInference operandTypeInference,
@@ -58,5 +59,4 @@ public class PigUserDefinedFunction extends SqlUserDefinedFunction {
       SqlOperandMetadata operandMetadata, Function function) {
     this(name, returnTypeInference, operandMetadata, function, null);
   }
-
 }
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index d5928aa..982c797 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -2057,6 +2057,7 @@ Not implemented:
 
 | C | Operator syntax      | Description
 |:- |:-------------------- |:-----------
+| p | ST_MakeEnvelope(xMin, yMin, xMax, yMax  [, srid ]) | Creates a rectangular POLYGON
 | h | ST_MakeGrid(geom, deltaX, deltaY) | Calculates a regular grid of POLYGONs based on *geom*
 | h | ST_MakeGridPoints(geom, deltaX, deltaY) | Calculates a regular grid of points based on *geom*
 | o | ST_MakeLine(point1 [, point ]*) | Creates a line-string from the given POINTs (or MULTIPOINTs)
@@ -2069,7 +2070,6 @@ Not implemented:
 * ST_Expand(geom, distance) Expands *geom*'s envelope
 * ST_Expand(geom, deltaX, deltaY) Expands *geom*'s envelope
 * ST_MakeEllipse(point, width, height) Constructs an ellipse
-* ST_MakeEnvelope(xMin, yMin, xMax, yMax  [, srid ]) Creates a rectangular POLYGON
 * ST_MakePolygon(lineString [, hole ]*) Creates a POLYGON from *lineString* with the given holes (which are required to be closed LINESTRINGs)
 * ST_MinimumDiameter(geom) Returns the minimum diameter of *geom*
 * ST_MinimumRectangle(geom) Returns the minimum rectangle enclosing *geom*
diff --git a/site/_docs/spatial.md b/site/_docs/spatial.md
index efe2172..a74e4ce 100644
--- a/site/_docs/spatial.md
+++ b/site/_docs/spatial.md
@@ -69,6 +69,85 @@ SELECT ST_PointFromText('POINT(-71.064544 42.28787)');
 1 row selected (0.323 seconds)
 {% endhighlight %}
 
+## Query rewrites
+
+One class of rewrites uses
+[Hilbert space-filling curves](https://en.wikipedia.org/wiki/Hilbert_curve).
+Suppose that a table
+has columns `x` and `y` denoting the position of a point and also a column `h`
+denoting the distance of that point along a curve. Then a predicate involving
+distance of (x, y) from a fixed point can be translated into a predicate
+involving ranges of h.
+
+Suppose we have a table with the locations of restaurants:
+
+{% highlight sql %}
+CREATE TABLE Restaurants (
+  INT id NOT NULL PRIMARY KEY,
+  VARCHAR(30) name,
+  VARCHAR(20) cuisine,
+  INT x NOT NULL,
+  INT y NOT NULL,
+  INT h  NOT NULL DERIVED (ST_Hilbert(x, y)))
+SORT KEY (h);
+{% endhighlight %}
+
+The optimizer requires that `h` is the position on the Hilbert curve of
+point (`x`, `y`), and also requires that the table is sorted on `h`.
+The `DERIVED` and `SORT KEY` clauses in the DDL syntax are invented for the
+purposes of this example, but a clustered table with a `CHECK` constraint
+would work just as well.
+
+The query
+
+{% highlight sql %}
+SELECT *
+FROM Restaurants
+WHERE ST_DWithin(ST_Point(x, y), ST_Point(10.0, 20.0), 6)
+{% endhighlight %}
+
+can be rewritten to
+
+{% highlight sql %}
+SELECT *
+FROM Restaurants
+WHERE (h BETWEEN 36496 AND 36520
+    OR h BETWEEN 36456 AND 36464
+    OR h BETWEEN 33252 AND 33254
+    OR h BETWEEN 33236 AND 33244
+    OR h BETWEEN 33164 AND 33176
+    OR h BETWEEN 33092 AND 33100
+    OR h BETWEEN 33055 AND 33080
+    OR h BETWEEN 33050 AND 33053
+    OR h BETWEEN 33033 AND 33035)
+AND ST_DWithin(ST_Point(x, y), ST_Point(10.0, 20.0), 6)
+{% endhighlight %}
+
+The rewritten query contains a collection of ranges on `h` followed by the
+original `ST_DWithin` predicate. The range predicates are evaluated first and
+are very fast because the table is sorted on `h`.
+
+Here is the full set of transformations:
+
+| Description | Expression
+|:----------- |: ------
+| Test whether a constant rectangle (X, X2, Y, Y2) contains a point (a, b)<br/><br/>Rewrite to use Hilbert index | ST_Contains(&#8203;ST_Rectangle(&#8203;X, X2, Y, Y2), ST_Point(a, b)))<br/><br/>h BETWEEN C1 AND C2<br/>OR ...<br/>OR h BETWEEN C<sub>2k</sub> AND C<sub>2k+1</sub>
+| Test whether a constant geometry G contains a point (a, b)<br/><br/>Rewrite to use bounding box of constant geometry, which is also constant, then rewrite to Hilbert range(s) as above | ST_Contains(&#8203;ST_Envelope(&#8203;G), ST_Point(a, b))<br/><br/>ST_Contains(&#8203;ST_Rectangle(&#8203;X, X2, Y, Y2), ST_Point(a, b)))
+| Test whether a point (a, b) is within a buffer around a constant point (X, Y)<br/><br/>Special case of previous, because buffer is a constant geometry | ST_Contains(&#8203;ST_Buffer(&#8203;ST_Point(a, b), D), ST_Point(X, Y))
+| Test whether a point (a, b) is within a constant distance D of a constant point (X, Y)<br/><br/>First, convert to buffer, then use previous rewrite for constant geometry | ST_DWithin(&#8203;ST_Point(a, b), ST_Point(X, Y), D))<br/><br/>ST_Contains(&#8203;ST_Buffer(&#8203;ST_Point(&#8203;X, Y), D), ST_Point(a, b))
+| Test whether a constant point (X, Y) is within a constant distance D of a point (a, b)<br/><br/>Reverse arguments of call to <code>ST_DWithin</code>, then use previous rewrite | ST_DWithin(&#8203;ST_Point(X, Y), ST_Point(a, b), D))<br/><br/>ST_Contains(&#8203;ST_Buffer(&#8203;ST_Point(&#8203;X, Y), D), ST_Point(a, b))
+
+In the above, `a` and `b` are variables, `X`, `X2`, `Y`, `Y2`, `D` and `G` are
+constants.
+
+Many rewrites are inexact: there are some points where the predicate would
+return false but the rewritten predicate returns true.
+For example, a rewrite might convert a test whether a point is in a circle to a
+test for whether the point is in the circle's bounding square.
+These rewrites are worth performing because they are much quicker to apply,
+and often allow range scans on the Hilbert index.
+But for safety, Calcite applies the original predicate, to remove false positives.
+
 ## Acknowledgements
 
 Calcite's OpenGIS implementation uses the