You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by nk...@apache.org on 2016/04/21 17:52:08 UTC

lucene-solr:master: LUCENE-7186: add a numerically stable morton encoding method to GeoEncodingUtils

Repository: lucene-solr
Updated Branches:
  refs/heads/master dac6fe09b -> 545a14000


LUCENE-7186: add a numerically stable morton encoding method to GeoEncodingUtils


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/545a1400
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/545a1400
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/545a1400

Branch: refs/heads/master
Commit: 545a14000bf4c02f25d6a3a4a52d51326199965f
Parents: dac6fe0
Author: nknize <nk...@apache.org>
Authored: Mon Apr 11 13:53:27 2016 -0500
Committer: nknize <nk...@apache.org>
Committed: Thu Apr 21 10:14:36 2016 -0500

----------------------------------------------------------------------
 .../java/org/apache/lucene/util/BitUtil.java    |   4 +-
 .../geopoint/document/GeoPointField.java        | 110 +++++++++++++-
 .../geopoint/document/GeoPointTokenStream.java  |   4 +-
 .../geopoint/search/GeoPointInPolygonQuery.java |   1 -
 .../search/GeoPointNumericTermsEnum.java        |  17 ++-
 .../search/GeoPointPrefixTermsEnum.java         |  26 ++--
 .../GeoPointTermQueryConstantScoreWrapper.java  |   7 +-
 .../lucene/spatial/util/GeoEncodingUtils.java   | 146 -------------------
 .../lucene/spatial/util/MortonEncoder.java      | 103 +++++++++++++
 .../geopoint/search/TestGeoPointQuery.java      |   5 +-
 .../search/TestLegacyGeoPointQuery.java         |   5 +-
 .../spatial/util/TestGeoEncodingUtils.java      | 102 -------------
 .../lucene/spatial/util/TestGeoPointField.java  |  51 +++++++
 .../lucene/spatial/util/TestMortonEncoder.java  | 108 ++++++++++++++
 14 files changed, 397 insertions(+), 292 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/core/src/java/org/apache/lucene/util/BitUtil.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/BitUtil.java b/lucene/core/src/java/org/apache/lucene/util/BitUtil.java
index 90f4f17..404ce7f 100644
--- a/lucene/core/src/java/org/apache/lucene/util/BitUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/util/BitUtil.java
@@ -116,7 +116,9 @@ public final class BitUtil {
    *
    * Adapted from: http://graphics.stanford.edu/~seander/bithacks.html#InterleaveBMN
    */
-  public static long interleave(long v1, long v2) {
+  public static long interleave(int even, int odd) {
+    long v1 = 0x00000000FFFFFFFFL & even;
+    long v2 = 0x00000000FFFFFFFFL & odd;
     v1 = (v1 | (v1 << SHIFT[4])) & MAGIC[4];
     v1 = (v1 | (v1 << SHIFT[3])) & MAGIC[3];
     v1 = (v1 | (v1 << SHIFT[2])) & MAGIC[2];

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointField.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointField.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointField.java
index 8b1483a..c20ad85 100644
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointField.java
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointField.java
@@ -22,8 +22,14 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.IndexOptions;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
 import org.apache.lucene.geo.GeoUtils;
+import org.apache.lucene.util.BitUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+
+import static org.apache.lucene.spatial.util.MortonEncoder.encode;
+import static org.apache.lucene.geo.GeoUtils.MIN_LAT_INCL;
+import static org.apache.lucene.geo.GeoUtils.MIN_LON_INCL;
 
 /**
  * <p>
@@ -49,6 +55,19 @@ public final class GeoPointField extends Field {
   /** encoding step value for GeoPoint prefix terms */
   public static final int PRECISION_STEP = 9;
 
+  /** number of bits used for quantizing latitude and longitude values */
+  public static final short BITS = 31;
+  /** scaling factors to convert lat/lon into unsigned space */
+  private static final double LAT_SCALE = (0x1L<<BITS)/180.0D;
+  private static final double LON_SCALE = (0x1L<<BITS)/360.0D;
+
+  /**
+   * The maximum term length (used for <code>byte[]</code> buffer size)
+   * for encoding <code>geoEncoded</code> values.
+   * @see #geoCodedToPrefixCodedBytes(long, int, BytesRefBuilder)
+   */
+  private static final int BUF_SIZE_LONG = 28/8 + 1;
+
   /**
    * <b>Expert:</b> Optional flag to select term encoding for GeoPointField types
    */
@@ -185,7 +204,7 @@ public final class GeoPointField extends Field {
     }
 
     // set field data
-    fieldsData = GeoEncodingUtils.mortonHash(latitude, longitude);
+    fieldsData = encodeLatLon(latitude, longitude);
   }
 
   private static FieldType getFieldType(Store stored) {
@@ -232,12 +251,12 @@ public final class GeoPointField extends Field {
 
   /** access latitude value */
   public double getLat() {
-    return GeoEncodingUtils.mortonUnhashLat((long) fieldsData);
+    return decodeLatitude((long) fieldsData);
   }
 
   /** access longitude value */
   public double getLon() {
-    return GeoEncodingUtils.mortonUnhashLon((long) fieldsData);
+    return decodeLongitude((long) fieldsData);
   }
 
   @Override
@@ -246,9 +265,88 @@ public final class GeoPointField extends Field {
       return null;
     }
     StringBuilder sb = new StringBuilder();
-    sb.append(GeoEncodingUtils.mortonUnhashLat((long) fieldsData));
+    sb.append(decodeLatitude((long) fieldsData));
     sb.append(',');
-    sb.append(GeoEncodingUtils.mortonUnhashLon((long) fieldsData));
+    sb.append(decodeLongitude((long) fieldsData));
     return sb.toString();
   }
+
+  /*************************
+   * 31 bit encoding utils *
+   *************************/
+  public static long encodeLatLon(final double lat, final double lon) {
+    long result = encode(lat, lon);
+    if (result == 0xFFFFFFFFFFFFFFFFL) {
+      return result & 0xC000000000000000L;
+    }
+    return result >>> 2;
+  }
+
+  /** decode longitude value from morton encoded geo point */
+  public static final double decodeLongitude(final long hash) {
+    return unscaleLon(BitUtil.deinterleave(hash));
+  }
+
+  /** decode latitude value from morton encoded geo point */
+  public static final double decodeLatitude(final long hash) {
+    return unscaleLat(BitUtil.deinterleave(hash >>> 1));
+  }
+
+  private static final double unscaleLon(final long val) {
+    return (val / LON_SCALE) + MIN_LON_INCL;
+  }
+
+  private static final double unscaleLat(final long val) {
+    return (val / LAT_SCALE) + MIN_LAT_INCL;
+  }
+
+  /** Convert a geocoded morton long into a prefix coded geo term */
+  public static void geoCodedToPrefixCoded(long hash, int shift, BytesRefBuilder bytes) {
+    geoCodedToPrefixCodedBytes(hash, shift, bytes);
+  }
+
+  /** Convert a prefix coded geo term back into the geocoded morton long */
+  public static long prefixCodedToGeoCoded(final BytesRef val) {
+    final long result = 0L
+        | (val.bytes[val.offset+0] & 255L) << 24
+        | (val.bytes[val.offset+1] & 255L) << 16
+        | (val.bytes[val.offset+2] & 255L) << 8
+        | val.bytes[val.offset+3] & 255L;
+
+    return result << 32;
+  }
+
+  /**
+   * GeoTerms are coded using 4 prefix bytes + 1 byte to record number of prefix bits
+   *
+   * example prefix at shift 54 (yields 10 significant prefix bits):
+   *  pppppppp pp000000 00000000 00000000 00001010
+   *  (byte 1) (byte 2) (byte 3) (byte 4) (sigbits)
+   */
+  private static void geoCodedToPrefixCodedBytes(final long hash, final int shift, final BytesRefBuilder bytes) {
+    // ensure shift is 32..63
+    if (shift < 32 || shift > 63) {
+      throw new IllegalArgumentException("Illegal shift value, must be 32..63; got shift=" + shift);
+    }
+    int nChars = BUF_SIZE_LONG + 1; // one extra for the byte that contains the number of significant bits
+    bytes.setLength(nChars);
+    bytes.grow(nChars--);
+    final int sigBits = 64 - shift;
+    bytes.setByteAt(BUF_SIZE_LONG, (byte)(sigBits));
+    long sortableBits = hash;
+    sortableBits >>>= shift;
+    sortableBits <<= 32 - sigBits;
+    do {
+      bytes.setByteAt(--nChars, (byte)(sortableBits));
+      sortableBits >>>= 8;
+    } while (nChars > 0);
+  }
+
+  /** Get the prefix coded geo term shift value */
+  public static int getPrefixCodedShift(final BytesRef val) {
+    final int shift = val.bytes[val.offset + BUF_SIZE_LONG];
+    if (shift > 63 || shift < 0)
+      throw new NumberFormatException("Invalid shift value (" + shift + ") in prefixCoded bytes (is encoded value really a geo point?)");
+    return shift;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointTokenStream.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointTokenStream.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointTokenStream.java
index c374c4e..cc5d8f2 100644
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointTokenStream.java
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointTokenStream.java
@@ -29,8 +29,8 @@ import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.AttributeReflector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
 
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.geoCodedToPrefixCoded;
 import static org.apache.lucene.spatial.geopoint.document.GeoPointField.PRECISION_STEP;
 
 /**
@@ -163,7 +163,7 @@ final class GeoPointTokenStream extends TokenStream {
 
     @Override
     public BytesRef getBytesRef() {
-      GeoEncodingUtils.geoCodedToPrefixCoded(value, shift, bytes);
+      geoCodedToPrefixCoded(value, shift, bytes);
       return bytes.get();
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQuery.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQuery.java
index 26af78e..7a0a9e9 100644
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQuery.java
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQuery.java
@@ -22,7 +22,6 @@ import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.spatial.geopoint.document.GeoPointField;
 import org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
 import org.apache.lucene.geo.Rectangle;
 import org.apache.lucene.geo.Polygon;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointNumericTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointNumericTermsEnum.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointNumericTermsEnum.java
index 071ab1e..06dfec7 100644
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointNumericTermsEnum.java
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointNumericTermsEnum.java
@@ -23,11 +23,12 @@ import java.util.List;
 
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.spatial.geopoint.document.GeoPointField;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.LegacyNumericUtils;
 
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.BITS;
+
 /**
  * Decomposes a given {@link GeoPointMultiTermQuery} into a set of terms that represent the query criteria using
  * {@link org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding#NUMERIC} method defined by
@@ -46,8 +47,8 @@ final class GeoPointNumericTermsEnum extends GeoPointTermsEnum {
 
   GeoPointNumericTermsEnum(final TermsEnum tenum, final GeoPointMultiTermQuery query) {
     super(tenum, query);
-    DETAIL_LEVEL = (short)(((GeoEncodingUtils.BITS<<1)-this.maxShift)/2);
-    computeRange(0L, (short) (((GeoEncodingUtils.BITS) << 1) - 1));
+    DETAIL_LEVEL = (short)(((BITS<<1)-this.maxShift)/2);
+    computeRange(0L, (short) (((BITS) << 1) - 1));
     assert rangeBounds.isEmpty() == false;
     Collections.sort(rangeBounds);
   }
@@ -79,12 +80,12 @@ final class GeoPointNumericTermsEnum extends GeoPointTermsEnum {
    * @param res spatial res represented as a bit shift (MSB is lower res)
    */
   private void relateAndRecurse(final long start, final long end, final short res) {
-    final double minLon = GeoEncodingUtils.mortonUnhashLon(start);
-    final double minLat = GeoEncodingUtils.mortonUnhashLat(start);
-    final double maxLon = GeoEncodingUtils.mortonUnhashLon(end);
-    final double maxLat = GeoEncodingUtils.mortonUnhashLat(end);
+    final double minLon = GeoPointField.decodeLongitude(start);
+    final double minLat = GeoPointField.decodeLatitude(start);
+    final double maxLon = GeoPointField.decodeLongitude(end);
+    final double maxLat = GeoPointField.decodeLatitude(end);
 
-    final short level = (short)((GeoEncodingUtils.BITS<<1)-res>>>1);
+    final short level = (short)((BITS<<1)-res>>>1);
 
     // if cell is within and a factor of the precision step, or it crosses the edge of the shape add the range
     final boolean within = res % GeoPointField.PRECISION_STEP == 0 && relationImpl.cellWithin(minLat, maxLat, minLon, maxLon);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointPrefixTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointPrefixTermsEnum.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointPrefixTermsEnum.java
index f511392..6429b92 100644
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointPrefixTermsEnum.java
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointPrefixTermsEnum.java
@@ -21,14 +21,10 @@ import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.spatial.geopoint.document.GeoPointField;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
 
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonHash;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLat;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLon;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.geoCodedToPrefixCoded;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.prefixCodedToGeoCoded;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.getPrefixCodedShift;
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.geoCodedToPrefixCoded;
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.prefixCodedToGeoCoded;
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.getPrefixCodedShift;
 
 /**
  * Decomposes a given {@link GeoPointMultiTermQuery} into a set of terms that represent the query criteria using
@@ -58,7 +54,7 @@ final class GeoPointPrefixTermsEnum extends GeoPointTermsEnum {
 
   public GeoPointPrefixTermsEnum(final TermsEnum tenum, final GeoPointMultiTermQuery query) {
     super(tenum, query);
-    this.start = mortonHash(query.minLat, query.minLon);
+    this.start = GeoPointField.encodeLatLon(query.minLat, query.minLon);
     this.currentRange = new Range(0, shift, true);
     // start shift at maxShift value (from computeMaxShift)
     this.shift = maxShift;
@@ -90,14 +86,14 @@ final class GeoPointPrefixTermsEnum extends GeoPointTermsEnum {
   }
 
   private void nextRelation() {
-    double minLon = mortonUnhashLon(currStart);
-    double minLat = mortonUnhashLat(currStart);
+    double minLon = GeoPointField.decodeLongitude(currStart);
+    double minLat = GeoPointField.decodeLatitude(currStart);
     double maxLon;
     double maxLat;
     boolean isWithin;
     do {
-      maxLon = mortonUnhashLon(currEnd);
-      maxLat = mortonUnhashLat(currEnd);
+      maxLon = GeoPointField.decodeLongitude(currEnd);
+      maxLat = GeoPointField.decodeLatitude(currEnd);
 
       isWithin = false;
       // within or a boundary
@@ -123,8 +119,8 @@ final class GeoPointPrefixTermsEnum extends GeoPointTermsEnum {
         currEnd = currStart | (1L<<--shift) - 1;
       } else {
         advanceVariables();
-        minLon = mortonUnhashLon(currStart);
-        minLat = mortonUnhashLat(currStart);
+        minLon = GeoPointField.decodeLongitude(currStart);
+        minLat = GeoPointField.decodeLatitude(currStart);
       }
     } while(shift < 63);
   }
@@ -194,7 +190,7 @@ final class GeoPointPrefixTermsEnum extends GeoPointTermsEnum {
 
       final int comparison = term.compareTo(currentCell);
       if (comparison > 0) {
-        seek(GeoEncodingUtils.prefixCodedToGeoCoded(term), (short)(64-GeoEncodingUtils.getPrefixCodedShift(term)));
+        seek(prefixCodedToGeoCoded(term), (short)(64 - getPrefixCodedShift(term)));
         continue;
       }
       return currentCell;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
index 7e97f16..96e0bd9 100644
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
@@ -32,15 +32,12 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TwoPhaseIterator;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField;
 import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.DocIdSetBuilder;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.SparseFixedBitSet;
 
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLat;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLon;
-
-
 /**
  * Custom ConstantScoreWrapper for {@code GeoPointMultiTermQuery} that cuts over to DocValues
  * for post filtering boundary ranges. Multi-valued GeoPoint documents are supported.
@@ -140,7 +137,7 @@ final class GeoPointTermQueryConstantScoreWrapper <Q extends GeoPointMultiTermQu
               int count = sdv.count();
               for (int i = 0; i < count; i++) {
                 long hash = sdv.valueAt(i);
-                if (termsEnum.postFilter(mortonUnhashLat(hash), mortonUnhashLon(hash))) {
+                if (termsEnum.postFilter(GeoPointField.decodeLatitude(hash), GeoPointField.decodeLongitude(hash))) {
                   return true;
                 }
               }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoEncodingUtils.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoEncodingUtils.java b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoEncodingUtils.java
deleted file mode 100644
index 95aea2a..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoEncodingUtils.java
+++ /dev/null
@@ -1,146 +0,0 @@
-package org.apache.lucene.spatial.util;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import org.apache.lucene.util.BitUtil;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-
-import static org.apache.lucene.geo.GeoUtils.MIN_LON_INCL;
-import static org.apache.lucene.geo.GeoUtils.MIN_LAT_INCL;
-
-/**
- * Basic reusable geopoint encoding methods
- *
- * @lucene.experimental
- */
-public final class GeoEncodingUtils {
-  /** number of bits used for quantizing latitude and longitude values */
-  public static final short BITS = 31;
-
-  private static final double LAT_SCALE = (0x1L<<BITS)/180.0D;
-  private static final double LON_SCALE = (0x1L<<BITS)/360.0D;
-
-  /**
-   * The maximum term length (used for <code>byte[]</code> buffer size)
-   * for encoding <code>geoEncoded</code> values.
-   * @see #geoCodedToPrefixCodedBytes(long, int, BytesRefBuilder)
-   */
-  private static final int BUF_SIZE_LONG = 28/8 + 1;
-
-  // No instance:
-  private GeoEncodingUtils() {
-  }
-
-  /**
-   * encode latitude, longitude geopoint values using morton encoding method
-   * https://en.wikipedia.org/wiki/Z-order_curve
-   */
-  public static final Long mortonHash(final double lat, final double lon) {
-    return BitUtil.interleave(scaleLon(lon), scaleLat(lat));
-  }
-
-  /** decode longitude value from morton encoded geo point */
-  public static final double mortonUnhashLon(final long hash) {
-    return unscaleLon(BitUtil.deinterleave(hash));
-  }
-
-  /** decode latitude value from morton encoded geo point */
-  public static final double mortonUnhashLat(final long hash) {
-    return unscaleLat(BitUtil.deinterleave(hash >>> 1));
-  }
-
-  private static final long scaleLon(final double val) {
-    return (long) ((val-MIN_LON_INCL) * LON_SCALE);
-  }
-
-  private static final long scaleLat(final double val) {
-    return (long) ((val-MIN_LAT_INCL) * LAT_SCALE);
-  }
-
-  private static final double unscaleLon(final long val) {
-    return (val / LON_SCALE) + MIN_LON_INCL;
-  }
-
-  private static final double unscaleLat(final long val) {
-    return (val / LAT_SCALE) + MIN_LAT_INCL;
-  }
-
-  /** Convert a geocoded morton long into a prefix coded geo term */
-  public static void geoCodedToPrefixCoded(long hash, int shift, BytesRefBuilder bytes) {
-    geoCodedToPrefixCodedBytes(hash, shift, bytes);
-  }
-
-  /** Convert a prefix coded geo term back into the geocoded morton long */
-  public static long prefixCodedToGeoCoded(final BytesRef val) {
-    final long result = 0L
-        | (val.bytes[val.offset+0] & 255L) << 24
-        | (val.bytes[val.offset+1] & 255L) << 16
-        | (val.bytes[val.offset+2] & 255L) << 8
-        | val.bytes[val.offset+3] & 255L;
-
-    return result << 32;
-  }
-
-  /**
-   * GeoTerms are coded using 4 prefix bytes + 1 byte to record number of prefix bits
-   *
-   * example prefix at shift 54 (yields 10 significant prefix bits):
-   *  pppppppp pp000000 00000000 00000000 00001010
-   *  (byte 1) (byte 2) (byte 3) (byte 4) (sigbits)
-   */
-  private static void geoCodedToPrefixCodedBytes(final long hash, final int shift, final BytesRefBuilder bytes) {
-    // ensure shift is 32..63
-    if (shift < 32 || shift > 63) {
-      throw new IllegalArgumentException("Illegal shift value, must be 32..63; got shift=" + shift);
-    }
-    int nChars = BUF_SIZE_LONG + 1; // one extra for the byte that contains the number of significant bits
-    bytes.setLength(nChars);
-    bytes.grow(nChars--);
-    final int sigBits = 64 - shift;
-    bytes.setByteAt(BUF_SIZE_LONG, (byte)(sigBits));
-    long sortableBits = hash;
-    sortableBits >>>= shift;
-    sortableBits <<= 32 - sigBits;
-    do {
-      bytes.setByteAt(--nChars, (byte)(sortableBits));
-      sortableBits >>>= 8;
-    } while (nChars > 0);
-  }
-
-  /** Get the prefix coded geo term shift value */
-  public static int getPrefixCodedShift(final BytesRef val) {
-    final int shift = val.bytes[val.offset + BUF_SIZE_LONG];
-    if (shift > 63 || shift < 0)
-      throw new NumberFormatException("Invalid shift value (" + shift + ") in prefixCoded bytes (is encoded value really a geo point?)");
-    return shift;
-  }
-
-  /** Converts a long value into a bit string (useful for debugging) */
-  public static String geoTermToString(long term) {
-    StringBuilder s = new StringBuilder(64);
-    final int numberOfLeadingZeros = Long.numberOfLeadingZeros(term);
-    for (int i = 0; i < numberOfLeadingZeros; i++) {
-      s.append('0');
-    }
-    if (term != 0) {
-      s.append(Long.toBinaryString(term));
-    }
-    return s.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/util/MortonEncoder.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/util/MortonEncoder.java b/lucene/spatial/src/java/org/apache/lucene/spatial/util/MortonEncoder.java
new file mode 100644
index 0000000..2340259
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/util/MortonEncoder.java
@@ -0,0 +1,103 @@
+/*
+ * 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.lucene.spatial.util;
+
+import org.apache.lucene.geo.GeoEncodingUtils;
+import org.apache.lucene.util.BitUtil;
+
+import static org.apache.lucene.geo.GeoUtils.checkLatitude;
+import static org.apache.lucene.geo.GeoUtils.checkLongitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitudeCeil;
+import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitudeCeil;
+
+/**
+ * Quantizes lat/lon points and bit interleaves them into a binary morton code
+ * in the range of 0x00000000... : 0xFFFFFFFF...
+ * https://en.wikipedia.org/wiki/Z-order_curve
+ *
+ * This is useful for bitwise operations in raster space
+ *
+ * @lucene.experimental
+ */
+public class MortonEncoder {
+
+  private MortonEncoder() {} // no instance
+
+  /**
+   * Main encoding method to quantize lat/lon points and bit interleave them into a binary morton code
+   * in the range of 0x00000000... : 0xFFFFFFFF...
+   *
+   * @param latitude latitude value: must be within standard +/-90 coordinate bounds.
+   * @param longitude longitude value: must be within standard +/-180 coordinate bounds.
+   * @return bit interleaved encoded values as a 64-bit {@code long}
+   * @throws IllegalArgumentException if latitude or longitude is out of bounds
+   */
+  public static final long encode(double latitude, double longitude) {
+    checkLatitude(latitude);
+    checkLongitude(longitude);
+    // encode lat/lon flipping the sign bit so negative ints sort before positive ints
+    final int latEnc = encodeLatitude(latitude) ^ 0x80000000;
+    final int lonEnc = encodeLongitude(longitude) ^ 0x80000000;
+    return BitUtil.interleave(lonEnc, latEnc);
+  }
+
+  /**
+   * Quantizes lat/lon points and bit interleaves them into a sortable morton code
+   * ranging from 0x00 : 0xFF...
+   * https://en.wikipedia.org/wiki/Z-order_curve
+   * This is useful for bitwise operations in raster space
+   * @param latitude latitude value: must be within standard +/-90 coordinate bounds.
+   * @param longitude longitude value: must be within standard +/-180 coordinate bounds.
+   * @return bit interleaved encoded values as a 64-bit {@code long}
+   * @throws IllegalArgumentException if latitude or longitude is out of bounds
+   */
+  public static final long encodeCeil(double latitude, double longitude) {
+    checkLatitude(latitude);
+    checkLongitude(longitude);
+    // encode lat/lon flipping the sign bit so negative ints sort before positive ints
+    final int latEnc = encodeLatitudeCeil(latitude) ^ 0x80000000;
+    final int lonEnc = encodeLongitudeCeil(longitude) ^ 0x80000000;
+    return BitUtil.interleave(lonEnc, latEnc);
+  }
+
+  /** decode latitude value from morton encoded geo point */
+  public static final double decodeLatitude(final long hash) {
+    // decode lat/lon flipping the sign bit so negative ints sort before positive ints
+    return GeoEncodingUtils.decodeLatitude((int) BitUtil.deinterleave(hash >>> 1) ^ 0x80000000);
+  }
+
+  /** decode longitude value from morton encoded geo point */
+  public static final double decodeLongitude(final long hash) {
+    // decode lat/lon flipping the sign bit so negative ints sort before positive ints
+    return GeoEncodingUtils.decodeLongitude((int) BitUtil.deinterleave(hash) ^ 0x80000000);
+  }
+
+  /** Converts a long value into a full 64 bit string (useful for debugging) */
+  public static String geoTermToString(long term) {
+    StringBuilder s = new StringBuilder(64);
+    final int numberOfLeadingZeros = Long.numberOfLeadingZeros(term);
+    for (int i = 0; i < numberOfLeadingZeros; i++) {
+      s.append('0');
+    }
+    if (term != 0) {
+      s.append(Long.toBinaryString(term));
+    }
+    return s.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestGeoPointQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestGeoPointQuery.java b/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestGeoPointQuery.java
index ab6b646..cfd6630 100644
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestGeoPointQuery.java
+++ b/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestGeoPointQuery.java
@@ -18,7 +18,6 @@ package org.apache.lucene.spatial.geopoint.search;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
 import org.apache.lucene.geo.BaseGeoPointTestCase;
 import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.geo.Rectangle;
@@ -34,12 +33,12 @@ public class TestGeoPointQuery extends BaseGeoPointTestCase {
   
   @Override
   protected double quantizeLat(double lat) {
-    return GeoEncodingUtils.mortonUnhashLat(GeoEncodingUtils.mortonHash(lat, 0));
+    return GeoPointField.decodeLatitude(GeoPointField.encodeLatLon(lat, 0));
   }
   
   @Override
   protected double quantizeLon(double lon) {
-    return GeoEncodingUtils.mortonUnhashLon(GeoEncodingUtils.mortonHash(0, lon));
+    return GeoPointField.decodeLongitude(GeoPointField.encodeLatLon(0, lon));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestLegacyGeoPointQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestLegacyGeoPointQuery.java b/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestLegacyGeoPointQuery.java
index 8766c0e..c2f05ea 100644
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestLegacyGeoPointQuery.java
+++ b/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestLegacyGeoPointQuery.java
@@ -18,7 +18,6 @@ package org.apache.lucene.spatial.geopoint.search;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
 import org.apache.lucene.geo.BaseGeoPointTestCase;
 import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.geo.Rectangle;
@@ -36,12 +35,12 @@ public class TestLegacyGeoPointQuery extends BaseGeoPointTestCase {
   
   @Override
   protected double quantizeLat(double lat) {
-    return GeoEncodingUtils.mortonUnhashLat(GeoEncodingUtils.mortonHash(lat, 0));
+    return GeoPointField.decodeLatitude(GeoPointField.encodeLatLon(lat, 0));
   }
   
   @Override
   protected double quantizeLon(double lon) {
-    return GeoEncodingUtils.mortonUnhashLon(GeoEncodingUtils.mortonHash(0, lon));
+    return GeoPointField.decodeLongitude(GeoPointField.encodeLatLon(0, lon));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoEncodingUtils.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoEncodingUtils.java b/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoEncodingUtils.java
deleted file mode 100644
index aab0de0..0000000
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoEncodingUtils.java
+++ /dev/null
@@ -1,102 +0,0 @@
-package org.apache.lucene.spatial.util;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.LuceneTestCase;
-
-import static org.apache.lucene.geo.GeoTestUtil.nextLatitude;
-import static org.apache.lucene.geo.GeoTestUtil.nextLongitude;
-
-/**
- * Tests methods in {@link GeoEncodingUtils}
- */
-public class TestGeoEncodingUtils extends LuceneTestCase {
-  /**
-   * Tests stability of {@link GeoEncodingUtils#geoCodedToPrefixCoded}
-   */
-  public void testGeoPrefixCoding() throws Exception {
-    int numIters = atLeast(1000);
-    long hash;
-    long decodedHash;
-    BytesRefBuilder brb = new BytesRefBuilder();
-    while (numIters-- >= 0) {
-      hash = GeoEncodingUtils.mortonHash(nextLatitude(), nextLongitude());
-      for (int i=32; i<64; ++i) {
-        GeoEncodingUtils.geoCodedToPrefixCoded(hash, i, brb);
-        decodedHash = GeoEncodingUtils.prefixCodedToGeoCoded(brb.get());
-        assertEquals((hash >>> i) << i, decodedHash);
-      }
-    }
-  }
-
-  public void testMortonEncoding() throws Exception {
-    long hash = GeoEncodingUtils.mortonHash(90, 180);
-    assertEquals(180.0, GeoEncodingUtils.mortonUnhashLon(hash), 0);
-    assertEquals(90.0, GeoEncodingUtils.mortonUnhashLat(hash), 0);
-  }
-
-  public void testEncodeDecode() throws Exception {
-    int iters = atLeast(10000);
-    for(int iter=0;iter<iters;iter++) {
-      double lat = nextLatitude();
-      double lon = nextLongitude();
-
-      long enc = GeoEncodingUtils.mortonHash(lat, lon);
-      double latEnc = GeoEncodingUtils.mortonUnhashLat(enc);
-      double lonEnc = GeoEncodingUtils.mortonUnhashLon(enc);
-
-      // todo remove tolerance
-      assertEquals("lat=" + lat + " latEnc=" + latEnc + " diff=" + (lat - latEnc), lat, latEnc, 1e-6);
-      assertEquals("lon=" + lon + " lonEnc=" + lonEnc + " diff=" + (lon - lonEnc), lon, lonEnc, 1e-6);
-    }
-  }
-
-  /** make sure values always go down: this is important for edge case consistency */
-  public void testEncodeDecodeRoundsDown() throws Exception {
-    int iters = atLeast(1000);
-    for(int iter=0;iter<iters;iter++) {
-      double lat = -90 + 180.0 * random().nextDouble();
-      double lon = -180 + 360.0 * random().nextDouble();
-
-      long enc = GeoEncodingUtils.mortonHash(lat, lon);
-      double latEnc = GeoEncodingUtils.mortonUnhashLat(enc);
-      double lonEnc = GeoEncodingUtils.mortonUnhashLon(enc);
-      assertTrue(latEnc <= lat);
-      assertTrue(lonEnc <= lon);
-    }
-  }
-
-  public void testScaleUnscaleIsStable() throws Exception {
-    int iters = atLeast(1000);
-    for(int iter=0;iter<iters;iter++) {
-      double lat = nextLatitude();
-      double lon = nextLongitude();
-
-      long enc = GeoEncodingUtils.mortonHash(lat, lon);
-      double latEnc = GeoEncodingUtils.mortonUnhashLat(enc);
-      double lonEnc = GeoEncodingUtils.mortonUnhashLon(enc);
-
-      long enc2 = GeoEncodingUtils.mortonHash(lat, lon);
-      double latEnc2 = GeoEncodingUtils.mortonUnhashLat(enc2);
-      double lonEnc2 = GeoEncodingUtils.mortonUnhashLon(enc2);
-      assertEquals(latEnc, latEnc2, 0.0);
-      assertEquals(lonEnc, lonEnc2, 0.0);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoPointField.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoPointField.java b/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoPointField.java
new file mode 100644
index 0000000..567d46d
--- /dev/null
+++ b/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoPointField.java
@@ -0,0 +1,51 @@
+package org.apache.lucene.spatial.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.spatial.geopoint.document.GeoPointField;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.LuceneTestCase;
+
+import static org.apache.lucene.geo.GeoTestUtil.nextLatitude;
+import static org.apache.lucene.geo.GeoTestUtil.nextLongitude;
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.encodeLatLon;
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.geoCodedToPrefixCoded;
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.prefixCodedToGeoCoded;
+
+/**
+ * Tests encoding methods in {@link GeoPointField}
+ */
+public class TestGeoPointField extends LuceneTestCase {
+  /**
+   * Tests stability of {@link GeoPointField#geoCodedToPrefixCoded}
+   */
+  public void testGeoPrefixCoding() throws Exception {
+    int numIters = atLeast(1000);
+    long hash;
+    long decodedHash;
+    BytesRefBuilder brb = new BytesRefBuilder();
+    while (numIters-- >= 0) {
+      hash = encodeLatLon(nextLatitude(), nextLongitude());
+      for (int i=32; i<64; ++i) {
+        geoCodedToPrefixCoded(hash, i, brb);
+        decodedHash = prefixCodedToGeoCoded(brb.get());
+        assertEquals((hash >>> i) << i, decodedHash);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestMortonEncoder.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestMortonEncoder.java b/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestMortonEncoder.java
new file mode 100644
index 0000000..9096e6e
--- /dev/null
+++ b/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestMortonEncoder.java
@@ -0,0 +1,108 @@
+/*
+ * 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.lucene.spatial.util;
+
+import java.util.Random;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util.TestUtil;
+
+import static org.apache.lucene.geo.GeoEncodingUtils.decodeLatitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.decodeLongitude;
+import static org.apache.lucene.spatial.util.MortonEncoder.decodeLatitude;
+import static org.apache.lucene.spatial.util.MortonEncoder.decodeLongitude;
+import static org.apache.lucene.spatial.util.MortonEncoder.encode;
+import static org.apache.lucene.spatial.util.MortonEncoder.encodeCeil;
+
+import static org.apache.lucene.util.BitUtil.deinterleave;
+import static org.apache.lucene.util.BitUtil.interleave;
+
+/**
+ * Tests methods in {@link MortonEncoder}
+ */
+public class TestMortonEncoder extends LuceneTestCase {
+
+  public void testMortonEncoding() throws Exception {
+    final long TRANSLATE = 1L << 31;
+    final double LATITUDE_DECODE = 180.0D/(0x1L<<32);
+    final double LONGITUDE_DECODE = 360.0D/(0x1L<<32);
+    Random random = random();
+    for(int i=0; i < 10000; ++i) {
+      long encoded = random().nextLong();
+      long encodedLat = deinterleave(encoded >>> 1);
+      long encodedLon = deinterleave(encoded);
+      double expectedLat = decodeLatitude((int)(encodedLat - TRANSLATE));
+      double decodedLat = decodeLatitude(encoded);
+      double expectedLon = decodeLongitude((int)(encodedLon - TRANSLATE));
+      double decodedLon = decodeLongitude(encoded);
+      assertEquals(expectedLat, decodedLat, 0.0D);
+      assertEquals(expectedLon, decodedLon, 0.0D);
+      // should round-trip
+      assertEquals(encoded, encode(decodedLat, decodedLon));
+
+      // test within the range
+      if (encoded != 0xFFFFFFFFFFFFFFFFL) {
+        // this is the next representable value
+        // all double values between [min .. max) should encode to the current integer
+        // all double values between (min .. max] should encodeCeil to the next integer.
+        double maxLat = expectedLat + LATITUDE_DECODE;
+        encodedLat += 1;
+        assertEquals(maxLat, decodeLatitude((int)(encodedLat - TRANSLATE)), 0.0D);
+        double maxLon = expectedLon + LONGITUDE_DECODE;
+        encodedLon += 1;
+        assertEquals(maxLon, decodeLongitude((int)(encodedLon - TRANSLATE)), 0.0D);
+        long encodedNext = encode(maxLat, maxLon);
+        assertEquals(interleave((int)encodedLon, (int)encodedLat), encodedNext);
+
+        // first and last doubles in range that will be quantized
+        double minEdgeLat = Math.nextUp(expectedLat);
+        double minEdgeLon = Math.nextUp(expectedLon);
+        long encodedMinEdge = encode(minEdgeLat, minEdgeLon);
+        long encodedMinEdgeCeil = encodeCeil(minEdgeLat, minEdgeLon);
+        double maxEdgeLat = Math.nextDown(maxLat);
+        double maxEdgeLon = Math.nextDown(maxLon);
+        long encodedMaxEdge = encode(maxEdgeLat, maxEdgeLon);
+        long encodedMaxEdgeCeil = encodeCeil(maxEdgeLat, maxEdgeLon);
+
+        assertEquals(encodedLat - 1, deinterleave(encodedMinEdge >>> 1));
+        assertEquals(encodedLat, deinterleave(encodedMinEdgeCeil >>> 1));
+        assertEquals(encodedLon - 1, deinterleave(encodedMinEdge));
+        assertEquals(encodedLon, deinterleave(encodedMinEdgeCeil));
+
+        assertEquals(encodedLat - 1, deinterleave(encodedMaxEdge >>> 1));
+        assertEquals(encodedLat, deinterleave(encodedMaxEdgeCeil >>> 1));
+        assertEquals(encodedLon - 1, deinterleave(encodedMaxEdge));
+        assertEquals(encodedLon, deinterleave(encodedMaxEdgeCeil));
+
+        // check random values within the double range
+        long minBitsLat = NumericUtils.doubleToSortableLong(minEdgeLat);
+        long maxBitsLat = NumericUtils.doubleToSortableLong(maxEdgeLat);
+        long minBitsLon = NumericUtils.doubleToSortableLong(minEdgeLon);
+        long maxBitsLon = NumericUtils.doubleToSortableLong(maxEdgeLon);
+        for (int j = 0; j < 100; j++) {
+          double valueLat = NumericUtils.sortableLongToDouble(TestUtil.nextLong(random, minBitsLat, maxBitsLat));
+          double valueLon = NumericUtils.sortableLongToDouble(TestUtil.nextLong(random, minBitsLon, maxBitsLon));
+          // round down
+          assertEquals(encoded,   encode(valueLat, valueLon));
+          // round up
+          assertEquals(interleave((int)encodedLon, (int)encodedLat), encodeCeil(valueLat, valueLon));
+        }
+      }
+    }
+  }
+}