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

[1/2] lucene-solr git commit: LUCENE-7063: add tests/docs for numericutils, rename confusing methods, remove overlap with LegacyNumericUtils

Repository: lucene-solr
Updated Branches:
  refs/heads/master 44d8ee911 -> 3ffeccab7


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
index 5cc6788..253b290 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
@@ -40,6 +40,7 @@ import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
 import org.apache.lucene.spatial.util.DistanceToShapeValueSource;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.LegacyNumericUtils;
+import org.apache.lucene.util.NumericUtils;
 
 
 /**
@@ -581,7 +582,7 @@ public class BBoxStrategy extends SpatialStrategy {
 
   private Query makeNumberTermQuery(String field, double number) {
     BytesRefBuilder bytes = new BytesRefBuilder();
-    LegacyNumericUtils.longToPrefixCoded(LegacyNumericUtils.doubleToSortableLong(number), 0, bytes);
+    LegacyNumericUtils.longToPrefixCoded(NumericUtils.doubleToSortableLong(number), 0, bytes);
     return new TermQuery(new Term(field, bytes.get()));
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Geo3DPoint.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Geo3DPoint.java b/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Geo3DPoint.java
index 7541408..fbdb00d 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Geo3DPoint.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Geo3DPoint.java
@@ -81,12 +81,12 @@ public final class Geo3DPoint extends Field {
   
   /** Encode single dimension */
   public static void encodeDimension(PlanetModel planetModel, double value, byte bytes[], int offset) {
-    NumericUtils.intToBytes(Geo3DUtil.encodeValue(planetModel.getMaximumMagnitude(), value), bytes, offset);
+    NumericUtils.intToSortableBytes(Geo3DUtil.encodeValue(planetModel.getMaximumMagnitude(), value), bytes, offset);
   }
   
   /** Decode single dimension */
   public static double decodeDimension(PlanetModel planetModel, byte value[], int offset) {
-    return Geo3DUtil.decodeValueCenter(planetModel.getMaximumMagnitude(), NumericUtils.bytesToInt(value, offset));
+    return Geo3DUtil.decodeValueCenter(planetModel.getMaximumMagnitude(), NumericUtils.sortableBytesToInt(value, offset));
   }
 
   /** Returns a query matching all points inside the provided shape.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/spatial3d/src/java/org/apache/lucene/geo3d/PointInGeo3DShapeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/geo3d/PointInGeo3DShapeQuery.java b/lucene/spatial3d/src/java/org/apache/lucene/geo3d/PointInGeo3DShapeQuery.java
index 80fc15e..4d81696 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/geo3d/PointInGeo3DShapeQuery.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/geo3d/PointInGeo3DShapeQuery.java
@@ -117,12 +117,12 @@ class PointInGeo3DShapeQuery extends Query {
                              // here are inclusive, we need to extend the bounds to the largest un-quantized values that
                              // could quantize into these bounds.  The encoding (Geo3DUtil.encodeValue) does
                              // a Math.round from double to long, so e.g. 1.4 -> 1, and -1.4 -> -1:
-                             double xMin = Geo3DUtil.decodeValueMin(planetMax, NumericUtils.bytesToInt(minPackedValue, 0));
-                             double xMax = Geo3DUtil.decodeValueMax(planetMax, NumericUtils.bytesToInt(maxPackedValue, 0));
-                             double yMin = Geo3DUtil.decodeValueMin(planetMax, NumericUtils.bytesToInt(minPackedValue, 1 * Integer.BYTES));
-                             double yMax = Geo3DUtil.decodeValueMax(planetMax, NumericUtils.bytesToInt(maxPackedValue, 1 * Integer.BYTES));
-                             double zMin = Geo3DUtil.decodeValueMin(planetMax, NumericUtils.bytesToInt(minPackedValue, 2 * Integer.BYTES));
-                             double zMax = Geo3DUtil.decodeValueMax(planetMax, NumericUtils.bytesToInt(maxPackedValue, 2 * Integer.BYTES));
+                             double xMin = Geo3DUtil.decodeValueMin(planetMax, NumericUtils.sortableBytesToInt(minPackedValue, 0));
+                             double xMax = Geo3DUtil.decodeValueMax(planetMax, NumericUtils.sortableBytesToInt(maxPackedValue, 0));
+                             double yMin = Geo3DUtil.decodeValueMin(planetMax, NumericUtils.sortableBytesToInt(minPackedValue, 1 * Integer.BYTES));
+                             double yMax = Geo3DUtil.decodeValueMax(planetMax, NumericUtils.sortableBytesToInt(maxPackedValue, 1 * Integer.BYTES));
+                             double zMin = Geo3DUtil.decodeValueMin(planetMax, NumericUtils.sortableBytesToInt(minPackedValue, 2 * Integer.BYTES));
+                             double zMax = Geo3DUtil.decodeValueMax(planetMax, NumericUtils.sortableBytesToInt(maxPackedValue, 2 * Integer.BYTES));
 
                              //System.out.println("  compare: x=" + cellXMin + "-" + cellXMax + " y=" + cellYMin + "-" + cellYMax + " z=" + cellZMin + "-" + cellZMax);
                              assert xMin <= xMax;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java
index 5113bcd..16a6b6b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java
@@ -69,7 +69,7 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
     byte[] point = new byte[4];
     for(int i=0;i<20;i++) {
       Document doc = new Document();
-      NumericUtils.intToBytes(i, point, 0);
+      NumericUtils.intToSortableBytes(i, point, 0);
       doc.add(new BinaryPoint("dim", point));
       w.addDocument(doc);
     }
@@ -93,7 +93,7 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
                        }
                        public void visit(int docID, byte[] packedValue) {
                          seen.set(docID);
-                         assertEquals(docID, NumericUtils.bytesToInt(packedValue, 0));
+                         assertEquals(docID, NumericUtils.sortableBytesToInt(packedValue, 0));
                        }
                      });
     assertEquals(20, seen.cardinality());
@@ -108,7 +108,7 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
     byte[] point = new byte[4];
     for(int i=0;i<20;i++) {
       Document doc = new Document();
-      NumericUtils.intToBytes(i, point, 0);
+      NumericUtils.intToSortableBytes(i, point, 0);
       doc.add(new BinaryPoint("dim", point));
       w.addDocument(doc);
       if (i == 10) {
@@ -135,7 +135,7 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
                        }
                        public void visit(int docID, byte[] packedValue) {
                          seen.set(docID);
-                         assertEquals(docID, NumericUtils.bytesToInt(packedValue, 0));
+                         assertEquals(docID, NumericUtils.sortableBytesToInt(packedValue, 0));
                        }
                      });
     assertEquals(20, seen.cardinality());
@@ -149,7 +149,7 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
     byte[] point = new byte[4];
     for(int i=0;i<10;i++) {
       Document doc = new Document();
-      NumericUtils.intToBytes(i, point, 0);
+      NumericUtils.intToSortableBytes(i, point, 0);
       doc.add(new BinaryPoint("dim", point));
       doc.add(new NumericDocValuesField("id", i));
       doc.add(newStringField("x", "x", Field.Store.NO));
@@ -183,7 +183,7 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
                              if (liveDocs.get(docID)) {
                                seen.set(docID);
                              }
-                             assertEquals(idValues.get(docID), NumericUtils.bytesToInt(packedValue, 0));
+                             assertEquals(idValues.get(docID), NumericUtils.sortableBytesToInt(packedValue, 0));
                            }
                          });
         assertEquals(0, seen.cardinality());
@@ -349,7 +349,7 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
         for(int dim=0;dim<numDims;dim++) {
           values[dim] = randomBigInt(numBytesPerDim);
           bytes[dim] = new byte[numBytesPerDim];
-          NumericUtils.bigIntToBytes(values[dim], numBytesPerDim, bytes[dim], 0);
+          NumericUtils.bigIntToSortableBytes(values[dim], numBytesPerDim, bytes[dim], 0);
           if (VERBOSE) {
             System.out.println("    " + dim + " -> " + values[dim]);
           }
@@ -405,7 +405,7 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
               public void visit(int docID, byte[] packedValue) {
                 //System.out.println("visit check docID=" + docID);
                 for(int dim=0;dim<numDims;dim++) {
-                  BigInteger x = NumericUtils.bytesToBigInt(packedValue, dim * numBytesPerDim, numBytesPerDim);
+                  BigInteger x = NumericUtils.sortableBytesToBigInt(packedValue, dim * numBytesPerDim, numBytesPerDim);
                   if (x.compareTo(queryMin[dim]) < 0 || x.compareTo(queryMax[dim]) > 0) {
                     //System.out.println("  no");
                     return;
@@ -420,8 +420,8 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
               public Relation compare(byte[] minPacked, byte[] maxPacked) {
                 boolean crosses = false;
                 for(int dim=0;dim<numDims;dim++) {
-                  BigInteger min = NumericUtils.bytesToBigInt(minPacked, dim * numBytesPerDim, numBytesPerDim);
-                  BigInteger max = NumericUtils.bytesToBigInt(maxPacked, dim * numBytesPerDim, numBytesPerDim);
+                  BigInteger min = NumericUtils.sortableBytesToBigInt(minPacked, dim * numBytesPerDim, numBytesPerDim);
+                  BigInteger max = NumericUtils.sortableBytesToBigInt(maxPacked, dim * numBytesPerDim, numBytesPerDim);
                   assert max.compareTo(min) >= 0;
 
                   if (max.compareTo(queryMin[dim]) < 0 || min.compareTo(queryMax[dim]) > 0) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
index c1da780..50692e8 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
@@ -448,6 +448,16 @@ public final class TestUtil {
       return result;
     }
   }
+  
+  /**
+   * Returns a randomish big integer with {@code 1 .. maxBytes} storage.
+   */
+  public static BigInteger nextBigInteger(Random random, int maxBytes) {
+    int length = TestUtil.nextInt(random, 1, maxBytes);
+    byte[] buffer = new byte[length];
+    random.nextBytes(buffer);
+    return new BigInteger(buffer);
+  }
 
   public static String randomSimpleString(Random r, int maxLength) {
     return randomSimpleString(r, 0, maxLength);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParsers.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParsers.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParsers.java
index 53105bd..dad4a5b 100644
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParsers.java
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParsers.java
@@ -22,6 +22,7 @@ import java.util.Date;
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LegacyNumericUtils;
+import org.apache.lucene.util.NumericUtils;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.TrieDateField;
 import org.apache.solr.schema.TrieDoubleField;
@@ -113,14 +114,14 @@ public class AnalyticsParsers {
   public static final NumericParser FLOAT_DOC_VALUES_PARSER = new NumericParser() {
     public String parse(BytesRef bytes) throws IOException {
       try {
-        return ""+ LegacyNumericUtils.sortableIntToFloat(LegacyNumericUtils.prefixCodedToInt(bytes));
+        return ""+ NumericUtils.sortableIntToFloat(LegacyNumericUtils.prefixCodedToInt(bytes));
       } catch (NumberFormatException e) {
         throw new IOException("The byte array "+Arrays.toString(bytes.bytes)+" cannot be converted to a float.");
       }
     }
     @Override
     public String parseNum(long l) {
-      return ""+ LegacyNumericUtils.sortableIntToFloat((int) l);
+      return ""+ NumericUtils.sortableIntToFloat((int) l);
     }
   };
   
@@ -130,14 +131,14 @@ public class AnalyticsParsers {
   public static final NumericParser DOUBLE_DOC_VALUES_PARSER = new NumericParser() {
     public String parse(BytesRef bytes) throws IOException {
       try {
-        return ""+ LegacyNumericUtils.sortableLongToDouble(LegacyNumericUtils.prefixCodedToLong(bytes));
+        return ""+ NumericUtils.sortableLongToDouble(LegacyNumericUtils.prefixCodedToLong(bytes));
       } catch (NumberFormatException e) {
         throw new IOException("The byte array "+Arrays.toString(bytes.bytes)+" cannot be converted to a double.");
       }
     }
     @Override
     public String parseNum(long l) {
-      return ""+ LegacyNumericUtils.sortableLongToDouble(l);
+      return ""+ NumericUtils.sortableLongToDouble(l);
     }
   };
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/solr/core/src/java/org/apache/solr/request/IntervalFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/IntervalFacets.java b/solr/core/src/java/org/apache/solr/request/IntervalFacets.java
index e606086..fedc7fe 100644
--- a/solr/core/src/java/org/apache/solr/request/IntervalFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/IntervalFacets.java
@@ -35,7 +35,7 @@ import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.LegacyNumericUtils;
+import org.apache.lucene.util.NumericUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
@@ -554,10 +554,10 @@ public class IntervalFacets implements Iterable<FacetInterval> {
             startLimit = ((Integer) schemaField.getType().toObject(schemaField, start)).longValue();
             break;
           case FLOAT:
-            startLimit = LegacyNumericUtils.floatToSortableInt((float) schemaField.getType().toObject(schemaField, start));
+            startLimit = NumericUtils.floatToSortableInt((float) schemaField.getType().toObject(schemaField, start));
             break;
           case DOUBLE:
-            startLimit = LegacyNumericUtils.doubleToSortableLong((double) schemaField.getType().toObject(schemaField, start));
+            startLimit = NumericUtils.doubleToSortableLong((double) schemaField.getType().toObject(schemaField, start));
             break;
           default:
             throw new AssertionError();
@@ -583,10 +583,10 @@ public class IntervalFacets implements Iterable<FacetInterval> {
             endLimit = ((Integer) schemaField.getType().toObject(schemaField, end)).longValue();
             break;
           case FLOAT:
-            endLimit = LegacyNumericUtils.floatToSortableInt((float) schemaField.getType().toObject(schemaField, end));
+            endLimit = NumericUtils.floatToSortableInt((float) schemaField.getType().toObject(schemaField, end));
             break;
           case DOUBLE:
-            endLimit = LegacyNumericUtils.doubleToSortableLong((double) schemaField.getType().toObject(schemaField, end));
+            endLimit = NumericUtils.doubleToSortableLong((double) schemaField.getType().toObject(schemaField, end));
             break;
           default:
             throw new AssertionError();
@@ -753,8 +753,8 @@ public class IntervalFacets implements Iterable<FacetInterval> {
      * <li>{@link IntervalCompareResult#GREATER_THAN_END} if the value is greater than {@code endLimit}
      * <li>{@link IntervalCompareResult#LOWER_THAN_START} if the value is lower than {@code startLimit}
      * </ul>
-     * @see org.apache.lucene.util.LegacyNumericUtils#floatToSortableInt(float)
-     * @see org.apache.lucene.util.LegacyNumericUtils#doubleToSortableLong(double)
+     * @see org.apache.lucene.util.NumericUtils#floatToSortableInt(float)
+     * @see org.apache.lucene.util.NumericUtils#doubleToSortableLong(double)
      */
     public IntervalCompareResult includes(long value) {
       if (startLimit > value) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/solr/core/src/java/org/apache/solr/schema/TrieDoubleField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/TrieDoubleField.java b/solr/core/src/java/org/apache/solr/schema/TrieDoubleField.java
index f867655..b055f1a 100644
--- a/solr/core/src/java/org/apache/solr/schema/TrieDoubleField.java
+++ b/solr/core/src/java/org/apache/solr/schema/TrieDoubleField.java
@@ -30,6 +30,7 @@ import org.apache.lucene.queries.function.valuesource.SortedSetFieldSource;
 import org.apache.lucene.search.SortedSetSelector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LegacyNumericUtils;
+import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.lucene.util.mutable.MutableValueDouble;
 
@@ -82,7 +83,7 @@ public class TrieDoubleField extends TrieField implements DoubleValueFieldType {
               assert !exists(doc) : "zero bytes for doc, but exists is true";
               return 0D;
             }
-            return  LegacyNumericUtils.sortableLongToDouble(LegacyNumericUtils.prefixCodedToLong(bytes));
+            return NumericUtils.sortableLongToDouble(LegacyNumericUtils.prefixCodedToLong(bytes));
           }
 
           @Override
@@ -107,7 +108,7 @@ public class TrieDoubleField extends TrieField implements DoubleValueFieldType {
                 //mval.value = mval.exists ? doubleVal(doc) : 0.0D;
                 BytesRef bytes = view.get(doc);
                 mval.exists = (0 == bytes.length);
-                mval.value = mval.exists ? LegacyNumericUtils.sortableLongToDouble(LegacyNumericUtils.prefixCodedToLong(bytes)) : 0D;
+                mval.value = mval.exists ? NumericUtils.sortableLongToDouble(LegacyNumericUtils.prefixCodedToLong(bytes)) : 0D;
               }
             };
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/solr/core/src/java/org/apache/solr/schema/TrieField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/TrieField.java b/solr/core/src/java/org/apache/solr/schema/TrieField.java
index 251eb85..572bf88 100644
--- a/solr/core/src/java/org/apache/solr/schema/TrieField.java
+++ b/solr/core/src/java/org/apache/solr/schema/TrieField.java
@@ -51,6 +51,7 @@ import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.LegacyNumericUtils;
+import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.mutable.MutableValueDate;
 import org.apache.lucene.util.mutable.MutableValueLong;
 import org.apache.solr.common.SolrException;
@@ -351,8 +352,8 @@ public class TrieField extends PrimitiveFieldType {
       case FLOAT:
         if (matchOnly) {
           query = DocValuesRangeQuery.newLongRange(field.getName(),
-                min == null ? null : (long) LegacyNumericUtils.floatToSortableInt(Float.parseFloat(min)),
-                max == null ? null : (long) LegacyNumericUtils.floatToSortableInt(Float.parseFloat(max)),
+                min == null ? null : (long) NumericUtils.floatToSortableInt(Float.parseFloat(min)),
+                max == null ? null : (long) NumericUtils.floatToSortableInt(Float.parseFloat(max)),
                 minInclusive, maxInclusive);
         } else {
           query = LegacyNumericRangeQuery.newFloatRange(field.getName(), ps,
@@ -377,8 +378,8 @@ public class TrieField extends PrimitiveFieldType {
       case DOUBLE:
         if (matchOnly) {
           query = DocValuesRangeQuery.newLongRange(field.getName(),
-                min == null ? null : LegacyNumericUtils.doubleToSortableLong(Double.parseDouble(min)),
-                max == null ? null : LegacyNumericUtils.doubleToSortableLong(Double.parseDouble(max)),
+                min == null ? null : NumericUtils.doubleToSortableLong(Double.parseDouble(min)),
+                max == null ? null : NumericUtils.doubleToSortableLong(Double.parseDouble(max)),
                 minInclusive, maxInclusive);
         } else {
           query = LegacyNumericRangeQuery.newDoubleRange(field.getName(), ps,
@@ -439,13 +440,13 @@ public class TrieField extends PrimitiveFieldType {
           LegacyNumericUtils.intToPrefixCoded(Integer.parseInt(s), 0, result);
           break;
         case FLOAT:
-          LegacyNumericUtils.intToPrefixCoded(LegacyNumericUtils.floatToSortableInt(Float.parseFloat(s)), 0, result);
+          LegacyNumericUtils.intToPrefixCoded(NumericUtils.floatToSortableInt(Float.parseFloat(s)), 0, result);
           break;
         case LONG:
           LegacyNumericUtils.longToPrefixCoded(Long.parseLong(s), 0, result);
           break;
         case DOUBLE:
-          LegacyNumericUtils.longToPrefixCoded(LegacyNumericUtils.doubleToSortableLong(Double.parseDouble(s)), 0, result);
+          LegacyNumericUtils.longToPrefixCoded(NumericUtils.doubleToSortableLong(Double.parseDouble(s)), 0, result);
           break;
         case DATE:
           LegacyNumericUtils.longToPrefixCoded(DateFormatUtil.parseMath(null, s).getTime(), 0, result);
@@ -483,11 +484,11 @@ public class TrieField extends PrimitiveFieldType {
       case INTEGER:
         return Integer.toString( LegacyNumericUtils.prefixCodedToInt(indexedForm) );
       case FLOAT:
-        return Float.toString( LegacyNumericUtils.sortableIntToFloat(LegacyNumericUtils.prefixCodedToInt(indexedForm)) );
+        return Float.toString( NumericUtils.sortableIntToFloat(LegacyNumericUtils.prefixCodedToInt(indexedForm)) );
       case LONG:
         return Long.toString( LegacyNumericUtils.prefixCodedToLong(indexedForm) );
       case DOUBLE:
-        return Double.toString( LegacyNumericUtils.sortableLongToDouble(LegacyNumericUtils.prefixCodedToLong(indexedForm)) );
+        return Double.toString( NumericUtils.sortableLongToDouble(LegacyNumericUtils.prefixCodedToLong(indexedForm)) );
       case DATE:
         return DateFormatUtil.formatExternal(new Date(LegacyNumericUtils.prefixCodedToLong(indexedForm)));
       default:
@@ -503,13 +504,13 @@ public class TrieField extends PrimitiveFieldType {
         value = Integer.toString( LegacyNumericUtils.prefixCodedToInt(indexedForm) );
         break;
       case FLOAT:
-        value = Float.toString( LegacyNumericUtils.sortableIntToFloat(LegacyNumericUtils.prefixCodedToInt(indexedForm)) );
+        value = Float.toString( NumericUtils.sortableIntToFloat(LegacyNumericUtils.prefixCodedToInt(indexedForm)) );
         break;
       case LONG:
         value = Long.toString( LegacyNumericUtils.prefixCodedToLong(indexedForm) );
         break;
       case DOUBLE:
-        value = Double.toString( LegacyNumericUtils.sortableLongToDouble(LegacyNumericUtils.prefixCodedToLong(indexedForm)) );
+        value = Double.toString( NumericUtils.sortableLongToDouble(LegacyNumericUtils.prefixCodedToLong(indexedForm)) );
         break;
       case DATE:
         value = DateFormatUtil.formatExternal(new Date(LegacyNumericUtils.prefixCodedToLong(indexedForm)));
@@ -529,11 +530,11 @@ public class TrieField extends PrimitiveFieldType {
       case INTEGER:
         return LegacyNumericUtils.prefixCodedToInt(term);
       case FLOAT:
-        return LegacyNumericUtils.sortableIntToFloat(LegacyNumericUtils.prefixCodedToInt(term));
+        return NumericUtils.sortableIntToFloat(LegacyNumericUtils.prefixCodedToInt(term));
       case LONG:
         return LegacyNumericUtils.prefixCodedToLong(term);
       case DOUBLE:
-        return LegacyNumericUtils.sortableLongToDouble(LegacyNumericUtils.prefixCodedToLong(term));
+        return NumericUtils.sortableLongToDouble(LegacyNumericUtils.prefixCodedToLong(term));
       case DATE:
         return new Date(LegacyNumericUtils.prefixCodedToLong(term));
       default:
@@ -556,14 +557,14 @@ public class TrieField extends PrimitiveFieldType {
           LegacyNumericUtils.intToPrefixCoded(val.intValue(), 0, bytes);
           break;
         case FLOAT:
-          LegacyNumericUtils.intToPrefixCoded(LegacyNumericUtils.floatToSortableInt(val.floatValue()), 0, bytes);
+          LegacyNumericUtils.intToPrefixCoded(NumericUtils.floatToSortableInt(val.floatValue()), 0, bytes);
           break;
         case LONG: //fallthrough!
         case DATE:
           LegacyNumericUtils.longToPrefixCoded(val.longValue(), 0, bytes);
           break;
         case DOUBLE:
-          LegacyNumericUtils.longToPrefixCoded(LegacyNumericUtils.doubleToSortableLong(val.doubleValue()), 0, bytes);
+          LegacyNumericUtils.longToPrefixCoded(NumericUtils.doubleToSortableLong(val.doubleValue()), 0, bytes);
           break;
         default:
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for trie field: " + f.name());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/solr/core/src/java/org/apache/solr/schema/TrieFloatField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/TrieFloatField.java b/solr/core/src/java/org/apache/solr/schema/TrieFloatField.java
index 4156aa6..38210a3 100644
--- a/solr/core/src/java/org/apache/solr/schema/TrieFloatField.java
+++ b/solr/core/src/java/org/apache/solr/schema/TrieFloatField.java
@@ -30,6 +30,7 @@ import org.apache.lucene.queries.function.valuesource.SortedSetFieldSource;
 import org.apache.lucene.search.SortedSetSelector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LegacyNumericUtils;
+import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.lucene.util.mutable.MutableValueFloat;
 
@@ -82,7 +83,7 @@ public class TrieFloatField extends TrieField implements FloatValueFieldType {
               assert !exists(doc) : "zero bytes for doc, but exists is true";
               return 0F;
             }
-            return  LegacyNumericUtils.sortableIntToFloat(LegacyNumericUtils.prefixCodedToInt(bytes));
+            return NumericUtils.sortableIntToFloat(LegacyNumericUtils.prefixCodedToInt(bytes));
           }
 
           @Override
@@ -108,7 +109,7 @@ public class TrieFloatField extends TrieField implements FloatValueFieldType {
                 //
                 BytesRef bytes = view.get(doc);
                 mval.exists = (0 == bytes.length);
-                mval.value = mval.exists ? LegacyNumericUtils.sortableIntToFloat(LegacyNumericUtils.prefixCodedToInt(bytes)) : 0F;
+                mval.value = mval.exists ? NumericUtils.sortableIntToFloat(LegacyNumericUtils.prefixCodedToInt(bytes)) : 0F;
               }
             };
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
index 1add47a..b5abf27 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java
@@ -25,7 +25,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.search.Query;
-import org.apache.lucene.util.LegacyNumericUtils;
+import org.apache.lucene.util.NumericUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.FacetParams;
 import org.apache.solr.common.util.SimpleOrderedMap;
@@ -470,7 +470,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
 
     @Override
     public long bitsToSortableBits(long bits) {
-      return LegacyNumericUtils.sortableDoubleBits(bits);
+      return NumericUtils.sortableDoubleBits(bits);
     }
 
     public FloatCalc(final SchemaField f) { super(f); }
@@ -491,7 +491,7 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
 
     @Override
     public long bitsToSortableBits(long bits) {
-      return LegacyNumericUtils.sortableDoubleBits(bits);
+      return NumericUtils.sortableDoubleBits(bits);
     }
 
     public DoubleCalc(final SchemaField f) { super(f); }


[2/2] lucene-solr git commit: LUCENE-7063: add tests/docs for numericutils, rename confusing methods, remove overlap with LegacyNumericUtils

Posted by rm...@apache.org.
LUCENE-7063: add tests/docs for numericutils, rename confusing methods, remove overlap with LegacyNumericUtils


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

Branch: refs/heads/master
Commit: 3ffeccab7e9949d7cc1e43027d9347a8968131b2
Parents: 44d8ee9
Author: Robert Muir <rm...@apache.org>
Authored: Thu Mar 3 21:51:14 2016 -0500
Committer: Robert Muir <rm...@apache.org>
Committed: Thu Mar 3 21:52:12 2016 -0500

----------------------------------------------------------------------
 .../analysis/LegacyNumericTokenStream.java      |   5 +-
 .../org/apache/lucene/document/DoublePoint.java |   4 +-
 .../org/apache/lucene/document/FloatPoint.java  |   4 +-
 .../org/apache/lucene/document/IntPoint.java    |   4 +-
 .../org/apache/lucene/document/LongPoint.java   |   4 +-
 .../lucene/search/LegacyNumericRangeQuery.java  |  17 +-
 .../lucene/search/SortedNumericSelector.java    |   6 +-
 .../apache/lucene/util/LegacyNumericUtils.java  |  59 ---
 .../org/apache/lucene/util/NumericUtils.java    | 157 +++---
 .../apache/lucene/util/RamUsageEstimator.java   |  49 ++
 .../test/org/apache/lucene/index/TestTerms.java |  10 +-
 .../lucene/search/TestNumericRangeQuery32.java  |   3 +-
 .../lucene/search/TestNumericRangeQuery64.java  |   3 +-
 .../apache/lucene/search/TestPointQueries.java  |  14 +-
 .../search/TestSortedNumericSortField.java      |  15 +-
 .../lucene/util/TestLegacyNumericUtils.java     |  16 +-
 .../apache/lucene/util/TestNumericUtils.java    | 491 +++++++++++++++++++
 .../org/apache/lucene/util/bkd/TestBKD.java     | 105 +---
 .../apache/lucene/facet/range/DoubleRange.java  |   5 +-
 .../facet/range/DoubleRangeFacetCounts.java     |   8 +-
 .../search/join/ToParentBlockJoinSortField.java |   6 +-
 .../org/apache/lucene/index/SorterTestBase.java |   2 +-
 .../apache/lucene/document/BigIntegerPoint.java |   4 +-
 .../org/apache/lucene/document/LatLonPoint.java |  16 +-
 .../lucene/search/TestDocValuesRangeQuery.java  |   2 +-
 .../lucene/spatial/bbox/BBoxStrategy.java       |   3 +-
 .../org/apache/lucene/geo3d/Geo3DPoint.java     |   4 +-
 .../lucene/geo3d/PointInGeo3DShapeQuery.java    |  12 +-
 .../lucene/index/BasePointFormatTestCase.java   |  20 +-
 .../java/org/apache/lucene/util/TestUtil.java   |  10 +
 .../solr/analytics/util/AnalyticsParsers.java   |   9 +-
 .../org/apache/solr/request/IntervalFacets.java |  14 +-
 .../org/apache/solr/schema/TrieDoubleField.java |   5 +-
 .../java/org/apache/solr/schema/TrieField.java  |  29 +-
 .../org/apache/solr/schema/TrieFloatField.java  |   5 +-
 .../apache/solr/search/facet/FacetRange.java    |   6 +-
 36 files changed, 778 insertions(+), 348 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/java/org/apache/lucene/analysis/LegacyNumericTokenStream.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/analysis/LegacyNumericTokenStream.java b/lucene/core/src/java/org/apache/lucene/analysis/LegacyNumericTokenStream.java
index b382b10..19f7d37 100644
--- a/lucene/core/src/java/org/apache/lucene/analysis/LegacyNumericTokenStream.java
+++ b/lucene/core/src/java/org/apache/lucene/analysis/LegacyNumericTokenStream.java
@@ -30,6 +30,7 @@ import org.apache.lucene.util.AttributeReflector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.LegacyNumericUtils;
+import org.apache.lucene.util.NumericUtils;
 
 /**
  * <b>Expert:</b> This class provides a {@link TokenStream}
@@ -299,7 +300,7 @@ public final class LegacyNumericTokenStream extends TokenStream {
    * <code>new Field(name, new LegacyNumericTokenStream(precisionStep).setDoubleValue(value))</code>
    */
   public LegacyNumericTokenStream setDoubleValue(final double value) {
-    numericAtt.init(LegacyNumericUtils.doubleToSortableLong(value), valSize = 64, precisionStep, -precisionStep);
+    numericAtt.init(NumericUtils.doubleToSortableLong(value), valSize = 64, precisionStep, -precisionStep);
     return this;
   }
   
@@ -310,7 +311,7 @@ public final class LegacyNumericTokenStream extends TokenStream {
    * <code>new Field(name, new LegacyNumericTokenStream(precisionStep).setFloatValue(value))</code>
    */
   public LegacyNumericTokenStream setFloatValue(final float value) {
-    numericAtt.init(LegacyNumericUtils.floatToSortableInt(value), valSize = 32, precisionStep, -precisionStep);
+    numericAtt.init(NumericUtils.floatToSortableInt(value), valSize = 32, precisionStep, -precisionStep);
     return this;
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
index 9dbd96e..52b039f 100644
--- a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
@@ -137,12 +137,12 @@ public final class DoublePoint extends Field {
   
   /** Encode single double dimension */
   public static void encodeDimension(double value, byte dest[], int offset) {
-    NumericUtils.longToBytes(NumericUtils.doubleToSortableLong(value), dest, offset);
+    NumericUtils.longToSortableBytes(NumericUtils.doubleToSortableLong(value), dest, offset);
   }
   
   /** Decode single double dimension */
   public static double decodeDimension(byte value[], int offset) {
-    return NumericUtils.sortableLongToDouble(NumericUtils.bytesToLong(value, offset));
+    return NumericUtils.sortableLongToDouble(NumericUtils.sortableBytesToLong(value, offset));
   }
   
   // static methods for generating queries

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
index 0b82abe..cad666c 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
@@ -137,12 +137,12 @@ public final class FloatPoint extends Field {
   
   /** Encode single float dimension */
   public static void encodeDimension(float value, byte dest[], int offset) {
-    NumericUtils.intToBytes(NumericUtils.floatToSortableInt(value), dest, offset);
+    NumericUtils.intToSortableBytes(NumericUtils.floatToSortableInt(value), dest, offset);
   }
   
   /** Decode single float dimension */
   public static float decodeDimension(byte value[], int offset) {
-    return NumericUtils.sortableIntToFloat(NumericUtils.bytesToInt(value, offset));
+    return NumericUtils.sortableIntToFloat(NumericUtils.sortableBytesToInt(value, offset));
   }
   
   // static methods for generating queries

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
index effcb62..b6f5ae7 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
@@ -137,12 +137,12 @@ public final class IntPoint extends Field {
   
   /** Encode single integer dimension */
   public static void encodeDimension(int value, byte dest[], int offset) {
-    NumericUtils.intToBytes(value, dest, offset);
+    NumericUtils.intToSortableBytes(value, dest, offset);
   }
   
   /** Decode single integer dimension */
   public static int decodeDimension(byte value[], int offset) {
-    return NumericUtils.bytesToInt(value, offset);
+    return NumericUtils.sortableBytesToInt(value, offset);
   }
   
   // static methods for generating queries

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
index a1d05d1..c0672ae 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
@@ -137,12 +137,12 @@ public final class LongPoint extends Field {
   
   /** Encode single long dimension */
   public static void encodeDimension(long value, byte dest[], int offset) {
-    NumericUtils.longToBytes(value, dest, offset);
+    NumericUtils.longToSortableBytes(value, dest, offset);
   }
   
   /** Decode single long dimension */
   public static long decodeDimension(byte value[], int offset) {
-    return NumericUtils.bytesToLong(value, offset);
+    return NumericUtils.sortableBytesToLong(value, offset);
   }
   
   // static methods for generating queries

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/java/org/apache/lucene/search/LegacyNumericRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LegacyNumericRangeQuery.java b/lucene/core/src/java/org/apache/lucene/search/LegacyNumericRangeQuery.java
index 5e0506f..c61f7a8 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LegacyNumericRangeQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LegacyNumericRangeQuery.java
@@ -29,6 +29,7 @@ import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LegacyNumericUtils;
+import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.index.Term; // for javadocs
 
 /**
@@ -364,13 +365,13 @@ public final class LegacyNumericRangeQuery<T extends Number> extends MultiTermQu
 
   // used to handle float/double infinity correcty
   static final long LONG_NEGATIVE_INFINITY =
-    LegacyNumericUtils.doubleToSortableLong(Double.NEGATIVE_INFINITY);
+    NumericUtils.doubleToSortableLong(Double.NEGATIVE_INFINITY);
   static final long LONG_POSITIVE_INFINITY =
-    LegacyNumericUtils.doubleToSortableLong(Double.POSITIVE_INFINITY);
+    NumericUtils.doubleToSortableLong(Double.POSITIVE_INFINITY);
   static final int INT_NEGATIVE_INFINITY =
-    LegacyNumericUtils.floatToSortableInt(Float.NEGATIVE_INFINITY);
+    NumericUtils.floatToSortableInt(Float.NEGATIVE_INFINITY);
   static final int INT_POSITIVE_INFINITY =
-    LegacyNumericUtils.floatToSortableInt(Float.POSITIVE_INFINITY);
+    NumericUtils.floatToSortableInt(Float.POSITIVE_INFINITY);
 
   /**
    * Subclass of FilteredTermsEnum for enumerating all terms that match the
@@ -400,7 +401,7 @@ public final class LegacyNumericRangeQuery<T extends Number> extends MultiTermQu
           } else {
             assert dataType == FieldType.LegacyNumericType.DOUBLE;
             minBound = (min == null) ? LONG_NEGATIVE_INFINITY
-              : LegacyNumericUtils.doubleToSortableLong(min.doubleValue());
+              : NumericUtils.doubleToSortableLong(min.doubleValue());
           }
           if (!minInclusive && min != null) {
             if (minBound == Long.MAX_VALUE) break;
@@ -414,7 +415,7 @@ public final class LegacyNumericRangeQuery<T extends Number> extends MultiTermQu
           } else {
             assert dataType == FieldType.LegacyNumericType.DOUBLE;
             maxBound = (max == null) ? LONG_POSITIVE_INFINITY
-              : LegacyNumericUtils.doubleToSortableLong(max.doubleValue());
+              : NumericUtils.doubleToSortableLong(max.doubleValue());
           }
           if (!maxInclusive && max != null) {
             if (maxBound == Long.MIN_VALUE) break;
@@ -440,7 +441,7 @@ public final class LegacyNumericRangeQuery<T extends Number> extends MultiTermQu
           } else {
             assert dataType == FieldType.LegacyNumericType.FLOAT;
             minBound = (min == null) ? INT_NEGATIVE_INFINITY
-              : LegacyNumericUtils.floatToSortableInt(min.floatValue());
+              : NumericUtils.floatToSortableInt(min.floatValue());
           }
           if (!minInclusive && min != null) {
             if (minBound == Integer.MAX_VALUE) break;
@@ -454,7 +455,7 @@ public final class LegacyNumericRangeQuery<T extends Number> extends MultiTermQu
           } else {
             assert dataType == FieldType.LegacyNumericType.FLOAT;
             maxBound = (max == null) ? INT_POSITIVE_INFINITY
-              : LegacyNumericUtils.floatToSortableInt(max.floatValue());
+              : NumericUtils.floatToSortableInt(max.floatValue());
           }
           if (!maxInclusive && max != null) {
             if (maxBound == Integer.MIN_VALUE) break;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java b/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java
index bdf67cc..314cc4f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java
@@ -20,7 +20,7 @@ package org.apache.lucene.search;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.util.LegacyNumericUtils;
+import org.apache.lucene.util.NumericUtils;
 
 /** 
  * Selects a value from the document's list to use as the representative value 
@@ -81,14 +81,14 @@ public class SortedNumericSelector {
         return new NumericDocValues() {
           @Override
           public long get(int docID) {
-            return LegacyNumericUtils.sortableFloatBits((int) view.get(docID));
+            return NumericUtils.sortableFloatBits((int) view.get(docID));
           }
         };
       case DOUBLE:
         return new NumericDocValues() {
           @Override
           public long get(int docID) {
-            return LegacyNumericUtils.sortableDoubleBits(view.get(docID));
+            return NumericUtils.sortableDoubleBits(view.get(docID));
           }
         };
       default:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/java/org/apache/lucene/util/LegacyNumericUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/LegacyNumericUtils.java b/lucene/core/src/java/org/apache/lucene/util/LegacyNumericUtils.java
index f338d25..9a26bfa 100644
--- a/lucene/core/src/java/org/apache/lucene/util/LegacyNumericUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/util/LegacyNumericUtils.java
@@ -40,13 +40,6 @@ import org.apache.lucene.index.TermsEnum;
  * prefixed (in the first char) by the <code>shift</code> value (number of bits removed) used
  * during encoding.
  *
- * <p>To also index floating point numbers, this class supplies two methods to convert them
- * to integer values by changing their bit layout: {@link #doubleToSortableLong},
- * {@link #floatToSortableInt}. You will have no precision loss by
- * converting floating point numbers to integers and back (only that the integer form
- * is not usable). Other data types like dates can easily converted to longs or ints (e.g.
- * date to long: {@link java.util.Date#getTime}).
- *
  * <p>For easy usage, the trie algorithm is implemented for indexing inside
  * {@link org.apache.lucene.analysis.LegacyNumericTokenStream} that can index <code>int</code>, <code>long</code>,
  * <code>float</code>, and <code>double</code>. For querying,
@@ -232,58 +225,6 @@ public final class LegacyNumericUtils {
   }
 
   /**
-   * Converts a <code>double</code> value to a sortable signed <code>long</code>.
-   * The value is converted by getting their IEEE 754 floating-point &quot;double format&quot;
-   * bit layout and then some bits are swapped, to be able to compare the result as long.
-   * By this the precision is not reduced, but the value can easily used as a long.
-   * The sort order (including {@link Double#NaN}) is defined by
-   * {@link Double#compareTo}; {@code NaN} is greater than positive infinity.
-   * @see #sortableLongToDouble
-   */
-  public static long doubleToSortableLong(double val) {
-    return sortableDoubleBits(Double.doubleToLongBits(val));
-  }
-
-  /**
-   * Converts a sortable <code>long</code> back to a <code>double</code>.
-   * @see #doubleToSortableLong
-   */
-  public static double sortableLongToDouble(long val) {
-    return Double.longBitsToDouble(sortableDoubleBits(val));
-  }
-
-  /**
-   * Converts a <code>float</code> value to a sortable signed <code>int</code>.
-   * The value is converted by getting their IEEE 754 floating-point &quot;float format&quot;
-   * bit layout and then some bits are swapped, to be able to compare the result as int.
-   * By this the precision is not reduced, but the value can easily used as an int.
-   * The sort order (including {@link Float#NaN}) is defined by
-   * {@link Float#compareTo}; {@code NaN} is greater than positive infinity.
-   * @see #sortableIntToFloat
-   */
-  public static int floatToSortableInt(float val) {
-    return sortableFloatBits(Float.floatToIntBits(val));
-  }
-
-  /**
-   * Converts a sortable <code>int</code> back to a <code>float</code>.
-   * @see #floatToSortableInt
-   */
-  public static float sortableIntToFloat(int val) {
-    return Float.intBitsToFloat(sortableFloatBits(val));
-  }
-  
-  /** Converts IEEE 754 representation of a double to sortable order (or back to the original) */
-  public static long sortableDoubleBits(long bits) {
-    return bits ^ (bits >> 63) & 0x7fffffffffffffffL;
-  }
-  
-  /** Converts IEEE 754 representation of a float to sortable order (or back to the original) */
-  public static int sortableFloatBits(int bits) {
-    return bits ^ (bits >> 31) & 0x7fffffff;
-  }
-
-  /**
    * Splits a long range recursively.
    * You may implement a builder that adds clauses to a
    * {@link org.apache.lucene.search.BooleanQuery} for each call to its

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java b/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
index b237328..f6c15c0 100644
--- a/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
@@ -22,6 +22,14 @@ import java.util.Arrays;
 
 /**
  * Helper APIs to encode numeric values as sortable bytes and vice-versa.
+ * 
+ * <p>
+ * To also index floating point numbers, this class supplies two methods to convert them
+ * to integer values by changing their bit layout: {@link #doubleToSortableLong},
+ * {@link #floatToSortableInt}. You will have no precision loss by
+ * converting floating point numbers to integers and back (only that the integer form
+ * is not usable). Other data types like dates can easily converted to longs or ints (e.g.
+ * date to long: {@link java.util.Date#getTime}).
  *
  * @lucene.internal
  */
@@ -38,16 +46,16 @@ public final class NumericUtils {
    * {@link Double#compareTo}; {@code NaN} is greater than positive infinity.
    * @see #sortableLongToDouble
    */
-  public static long doubleToSortableLong(double val) {
-    return sortableDoubleBits(Double.doubleToLongBits(val));
+  public static long doubleToSortableLong(double value) {
+    return sortableDoubleBits(Double.doubleToLongBits(value));
   }
 
   /**
    * Converts a sortable <code>long</code> back to a <code>double</code>.
    * @see #doubleToSortableLong
    */
-  public static double sortableLongToDouble(long val) {
-    return Double.longBitsToDouble(sortableDoubleBits(val));
+  public static double sortableLongToDouble(long encoded) {
+    return Double.longBitsToDouble(sortableDoubleBits(encoded));
   }
 
   /**
@@ -59,16 +67,16 @@ public final class NumericUtils {
    * {@link Float#compareTo}; {@code NaN} is greater than positive infinity.
    * @see #sortableIntToFloat
    */
-  public static int floatToSortableInt(float val) {
-    return sortableFloatBits(Float.floatToIntBits(val));
+  public static int floatToSortableInt(float value) {
+    return sortableFloatBits(Float.floatToIntBits(value));
   }
 
   /**
    * Converts a sortable <code>int</code> back to a <code>float</code>.
    * @see #floatToSortableInt
    */
-  public static float sortableIntToFloat(int val) {
-    return Float.intBitsToFloat(sortableFloatBits(val));
+  public static float sortableIntToFloat(int encoded) {
+    return Float.intBitsToFloat(sortableFloatBits(encoded));
   }
   
   /** Converts IEEE 754 representation of a double to sortable order (or back to the original) */
@@ -122,83 +130,76 @@ public final class NumericUtils {
     }
   }
 
-  /** Returns true if N-dim rect A contains N-dim rect B */
-  public static boolean contains(int bytesPerDim,
-                                 byte[] minPackedA, byte[] maxPackedA,
-                                 byte[] minPackedB, byte[] maxPackedB) {
-    int dims = minPackedA.length / bytesPerDim;
-    for(int dim=0;dim<dims;dim++) {
-      int offset = dim * bytesPerDim;
-      if (StringHelper.compare(bytesPerDim, minPackedA, offset, minPackedB, offset) > 0) {
-        return false;
-      }
-      if (StringHelper.compare(bytesPerDim, maxPackedA, offset, maxPackedB, offset) < 0) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  public static void intToBytes(int x, byte[] dest, int offset) {
+  /** 
+   * Encodes an integer {@code value} such that unsigned byte order comparison
+   * is consistent with {@link Integer#compare(int, int)}
+   * @see #sortableBytesToInt(byte[], int)
+   */
+  public static void intToSortableBytes(int value, byte[] result, int offset) {
     // Flip the sign bit, so negative ints sort before positive ints correctly:
-    x ^= 0x80000000;
-    for (int i = 0; i < 4; i++) {
-      dest[offset+i] = (byte) (x >> 24-i*8);
-    }
+    value ^= 0x80000000;
+    result[offset] =   (byte) (value >> 24);
+    result[offset+1] = (byte) (value >> 16);
+    result[offset+2] = (byte) (value >>  8);
+    result[offset+3] = (byte) value;
   }
 
-  public static int bytesToInt(byte[] src, int offset) {
-    int x = 0;
-    for (int i = 0; i < 4; i++) {
-      x |= (src[offset+i] & 0xff) << (24-i*8);
-    }
+  /**
+   * Decodes an integer value previously written with {@link #intToSortableBytes}
+   * @see #intToSortableBytes(int, byte[], int)
+   */
+  public static int sortableBytesToInt(byte[] encoded, int offset) {
+    int x = ((encoded[offset] & 0xFF) << 24)   | 
+            ((encoded[offset+1] & 0xFF) << 16) |
+            ((encoded[offset+2] & 0xFF) <<  8) | 
+             (encoded[offset+3] & 0xFF);
     // Re-flip the sign bit to restore the original value:
     return x ^ 0x80000000;
   }
 
-  public static void longToBytes(long v, byte[] bytes, int offset) {
+  /** 
+   * Encodes an long {@code value} such that unsigned byte order comparison
+   * is consistent with {@link Long#compare(long, long)}
+   * @see #sortableBytesToLong(byte[], int)
+   */
+  public static void longToSortableBytes(long value, byte[] result, int offset) {
     // Flip the sign bit so negative longs sort before positive longs:
-    v ^= 0x8000000000000000L;
-    longToBytesDirect(v, bytes, offset);
+    value ^= 0x8000000000000000L;
+    result[offset] =   (byte) (value >> 56);
+    result[offset+1] = (byte) (value >> 48);
+    result[offset+2] = (byte) (value >> 40);
+    result[offset+3] = (byte) (value >> 32);
+    result[offset+4] = (byte) (value >> 24);
+    result[offset+5] = (byte) (value >> 16);
+    result[offset+6] = (byte) (value >> 8);
+    result[offset+7] = (byte) value;
   }
 
-  public static void longToBytesDirect(long v, byte[] bytes, int offset) {
-    bytes[offset] = (byte) (v >> 56);
-    bytes[offset+1] = (byte) (v >> 48);
-    bytes[offset+2] = (byte) (v >> 40);
-    bytes[offset+3] = (byte) (v >> 32);
-    bytes[offset+4] = (byte) (v >> 24);
-    bytes[offset+5] = (byte) (v >> 16);
-    bytes[offset+6] = (byte) (v >> 8);
-    bytes[offset+7] = (byte) v;
-  }
-
-  public static long bytesToLong(byte[] bytes, int offset) {
-    long v = bytesToLongDirect(bytes, offset);
+  /**
+   * Decodes a long value previously written with {@link #longToSortableBytes}
+   * @see #longToSortableBytes(long, byte[], int)
+   */
+  public static long sortableBytesToLong(byte[] encoded, int offset) {
+    long v = ((encoded[offset] & 0xFFL) << 56)   |
+             ((encoded[offset+1] & 0xFFL) << 48) |
+             ((encoded[offset+2] & 0xFFL) << 40) |
+             ((encoded[offset+3] & 0xFFL) << 32) |
+             ((encoded[offset+4] & 0xFFL) << 24) |
+             ((encoded[offset+5] & 0xFFL) << 16) |
+             ((encoded[offset+6] & 0xFFL) << 8)  |
+              (encoded[offset+7] & 0xFFL);
     // Flip the sign bit back
     v ^= 0x8000000000000000L;
     return v;
   }
 
-  public static long bytesToLongDirect(byte[] bytes, int offset) {
-    long v = ((bytes[offset] & 0xffL) << 56) |
-      ((bytes[offset+1] & 0xffL) << 48) |
-      ((bytes[offset+2] & 0xffL) << 40) |
-      ((bytes[offset+3] & 0xffL) << 32) |
-      ((bytes[offset+4] & 0xffL) << 24) |
-      ((bytes[offset+5] & 0xffL) << 16) |
-      ((bytes[offset+6] & 0xffL) << 8) |
-      (bytes[offset+7] & 0xffL);
-    return v;
-  }
-
-  public static void sortableBigIntBytes(byte[] bytes) {
-    // Flip the sign bit so negative bigints sort before positive bigints:
-    bytes[0] ^= 0x80;
-  }
-
-  public static void bigIntToBytes(BigInteger bigInt, int bigIntSize, byte[] result, int offset) {
+  /** 
+   * Encodes a BigInteger {@code value} such that unsigned byte order comparison
+   * is consistent with {@link BigInteger#compareTo(BigInteger)}. This also sign-extends
+   * the value to {@code bigIntSize} bytes if necessary: useful to create a fixed-width size.
+   * @see #sortableBytesToBigInt(byte[], int, int)
+   */
+  public static void bigIntToSortableBytes(BigInteger bigInt, int bigIntSize, byte[] result, int offset) {
     byte[] bigIntBytes = bigInt.toByteArray();
     byte[] fullBigIntBytes;
 
@@ -214,17 +215,23 @@ public final class NumericUtils {
     } else {
       throw new IllegalArgumentException("BigInteger: " + bigInt + " requires more than " + bigIntSize + " bytes storage");
     }
-    sortableBigIntBytes(fullBigIntBytes);
+    // Flip the sign bit so negative bigints sort before positive bigints:
+    fullBigIntBytes[0] ^= 0x80;
 
     System.arraycopy(fullBigIntBytes, 0, result, offset, bigIntSize);
 
-    assert bytesToBigInt(result, offset, bigIntSize).equals(bigInt): "bigInt=" + bigInt + " converted=" + bytesToBigInt(result, offset, bigIntSize);
+    assert sortableBytesToBigInt(result, offset, bigIntSize).equals(bigInt): "bigInt=" + bigInt + " converted=" + sortableBytesToBigInt(result, offset, bigIntSize);
   }
 
-  public static BigInteger bytesToBigInt(byte[] bytes, int offset, int length) {
+  /**
+   * Decodes a BigInteger value previously written with {@link #bigIntToSortableBytes}
+   * @see #bigIntToSortableBytes(BigInteger, int, byte[], int)
+   */
+  public static BigInteger sortableBytesToBigInt(byte[] encoded, int offset, int length) {
     byte[] bigIntBytes = new byte[length];
-    System.arraycopy(bytes, offset, bigIntBytes, 0, length);
-    sortableBigIntBytes(bigIntBytes);
+    System.arraycopy(encoded, offset, bigIntBytes, 0, length);
+    // Flip the sign bit back to the original
+    bigIntBytes[0] ^= 0x80;
     return new BigInteger(bigIntBytes);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/java/org/apache/lucene/util/RamUsageEstimator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/RamUsageEstimator.java b/lucene/core/src/java/org/apache/lucene/util/RamUsageEstimator.java
index 3ff5375..2765d7f 100644
--- a/lucene/core/src/java/org/apache/lucene/util/RamUsageEstimator.java
+++ b/lucene/core/src/java/org/apache/lucene/util/RamUsageEstimator.java
@@ -56,6 +56,55 @@ public final class RamUsageEstimator {
   private RamUsageEstimator() {}
 
   /** 
+   * Number of bytes used to represent a {@code boolean} in binary form
+   * @deprecated use {@code 1} instead.
+   */
+  @Deprecated
+  public final static int NUM_BYTES_BOOLEAN = 1;
+  /** 
+   * Number of bytes used to represent a {@code byte} in binary form
+   * @deprecated use {@code 1} instead.
+   */
+  @Deprecated
+  public final static int NUM_BYTES_BYTE = 1;
+  /** 
+   * Number of bytes used to represent a {@code char} in binary form
+   * @deprecated use {@link Character#BYTES} instead.
+   */
+  @Deprecated
+  public final static int NUM_BYTES_CHAR = Character.BYTES;
+  /** 
+   * Number of bytes used to represent a {@code short} in binary form
+   * @deprecated use {@link Short#BYTES} instead.
+   */
+  @Deprecated
+  public final static int NUM_BYTES_SHORT = Short.BYTES;
+  /** 
+   * Number of bytes used to represent an {@code int} in binary form
+   * @deprecated use {@link Integer#BYTES} instead.
+   */
+  @Deprecated
+  public final static int NUM_BYTES_INT = Integer.BYTES;
+  /** 
+   * Number of bytes used to represent a {@code float} in binary form
+   * @deprecated use {@link Float#BYTES} instead.
+   */
+  @Deprecated
+  public final static int NUM_BYTES_FLOAT = Float.BYTES;
+  /** 
+   * Number of bytes used to represent a {@code long} in binary form
+   * @deprecated use {@link Long#BYTES} instead.
+   */
+  @Deprecated
+  public final static int NUM_BYTES_LONG = Long.BYTES;
+  /** 
+   * Number of bytes used to represent a {@code double} in binary form
+   * @deprecated use {@link Double#BYTES} instead.
+   */
+  @Deprecated
+  public final static int NUM_BYTES_DOUBLE = Double.BYTES;
+
+  /** 
    * True, iff compressed references (oops) are enabled by this JVM 
    */
   public final static boolean COMPRESSED_REFS_ENABLED;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/test/org/apache/lucene/index/TestTerms.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTerms.java b/lucene/core/src/test/org/apache/lucene/index/TestTerms.java
index 6c41646..0ee3447 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestTerms.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestTerms.java
@@ -16,7 +16,6 @@
  */
 package org.apache.lucene.index;
 
-
 import org.apache.lucene.analysis.CannedBinaryTokenStream;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.LegacyDoubleField;
@@ -29,6 +28,7 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LegacyNumericUtils;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.TestUtil;
 
 public class TestTerms extends LuceneTestCase {
@@ -167,8 +167,8 @@ public class TestTerms extends LuceneTestCase {
     
     IndexReader r = w.getReader();
     Terms terms = MultiFields.getTerms(r, "field");
-    assertEquals(minValue, LegacyNumericUtils.sortableIntToFloat(LegacyNumericUtils.getMinInt(terms)), 0.0f);
-    assertEquals(maxValue, LegacyNumericUtils.sortableIntToFloat(LegacyNumericUtils.getMaxInt(terms)), 0.0f);
+    assertEquals(minValue, NumericUtils.sortableIntToFloat(LegacyNumericUtils.getMinInt(terms)), 0.0f);
+    assertEquals(maxValue, NumericUtils.sortableIntToFloat(LegacyNumericUtils.getMaxInt(terms)), 0.0f);
 
     r.close();
     w.close();
@@ -194,8 +194,8 @@ public class TestTerms extends LuceneTestCase {
 
     Terms terms = MultiFields.getTerms(r, "field");
 
-    assertEquals(minValue, LegacyNumericUtils.sortableLongToDouble(LegacyNumericUtils.getMinLong(terms)), 0.0);
-    assertEquals(maxValue, LegacyNumericUtils.sortableLongToDouble(LegacyNumericUtils.getMaxLong(terms)), 0.0);
+    assertEquals(minValue, NumericUtils.sortableLongToDouble(LegacyNumericUtils.getMinLong(terms)), 0.0);
+    assertEquals(maxValue, NumericUtils.sortableLongToDouble(LegacyNumericUtils.getMaxLong(terms)), 0.0);
 
     r.close();
     w.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java b/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java
index b9d5197..3a82ff3 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java
@@ -34,6 +34,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.LegacyNumericUtils;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.TestLegacyNumericUtils; // NaN arrays
 import org.apache.lucene.util.TestUtil;
 import org.junit.AfterClass;
@@ -525,7 +526,7 @@ public class TestNumericRangeQuery32 extends LuceneTestCase {
     final int lower=-1000, upper=+2000;
     
     Query tq= LegacyNumericRangeQuery.newFloatRange(field, precisionStep,
-        LegacyNumericUtils.sortableIntToFloat(lower), LegacyNumericUtils.sortableIntToFloat(upper), true, true);
+        NumericUtils.sortableIntToFloat(lower), NumericUtils.sortableIntToFloat(upper), true, true);
     TopDocs tTopDocs = searcher.search(tq, 1);
     assertEquals("Returned count of range query must be equal to inclusive range length", upper-lower+1, tTopDocs.totalHits );
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java b/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java
index 7d03fae..7f63fbc 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestNumericRangeQuery64.java
@@ -34,6 +34,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.LegacyNumericUtils;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.TestLegacyNumericUtils;
 import org.apache.lucene.util.TestUtil;
 import org.junit.AfterClass;
@@ -559,7 +560,7 @@ public class TestNumericRangeQuery64 extends LuceneTestCase {
     final long lower=-1000L, upper=+2000L;
     
     Query tq= LegacyNumericRangeQuery.newDoubleRange(field, precisionStep,
-        LegacyNumericUtils.sortableLongToDouble(lower), LegacyNumericUtils.sortableLongToDouble(upper), true, true);
+        NumericUtils.sortableLongToDouble(lower), NumericUtils.sortableLongToDouble(upper), true, true);
     TopDocs tTopDocs = searcher.search(tq, 1);
     assertEquals("Returned count of range query must be equal to inclusive range length", upper-lower+1, tTopDocs.totalHits );
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
index 19096c7..66a1593 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -393,8 +393,8 @@ public class TestPointQueries extends LuceneTestCase {
     for(int i=0;i<10000;i++) {
       long v = random().nextLong();
       byte[] tmp = new byte[8];
-      NumericUtils.longToBytes(v, tmp, 0);
-      long v2 = NumericUtils.bytesToLong(tmp, 0);
+      NumericUtils.longToSortableBytes(v, tmp, 0);
+      long v2 = NumericUtils.sortableBytesToLong(tmp, 0);
       assertEquals("got bytes=" + Arrays.toString(tmp), v, v2);
     }
   }
@@ -467,7 +467,7 @@ public class TestPointQueries extends LuceneTestCase {
       if (missing.get(id) == false) {
         doc.add(new LongPoint("sn_value", values[id]));
         byte[] bytes = new byte[8];
-        NumericUtils.longToBytes(values[id], bytes, 0);
+        NumericUtils.longToSortableBytes(values[id], bytes, 0);
         doc.add(new BinaryPoint("ss_value", bytes));
       }
     }
@@ -529,11 +529,11 @@ public class TestPointQueries extends LuceneTestCase {
                 System.out.println("\n" + Thread.currentThread().getName() + ": TEST: iter=" + iter + " value=" + lower + " TO " + upper);
                 byte[] tmp = new byte[8];
                 if (lower != null) {
-                  NumericUtils.longToBytes(lower, tmp, 0);
+                  NumericUtils.longToSortableBytes(lower, tmp, 0);
                   System.out.println("  lower bytes=" + Arrays.toString(tmp));
                 }
                 if (upper != null) {
-                  NumericUtils.longToBytes(upper, tmp, 0);
+                  NumericUtils.longToSortableBytes(upper, tmp, 0);
                   System.out.println("  upper bytes=" + Arrays.toString(tmp));
                 }
               }
@@ -542,9 +542,9 @@ public class TestPointQueries extends LuceneTestCase {
                 query = LongPoint.newRangeQuery("sn_value", lower, upper);
               } else {
                 byte[] lowerBytes = new byte[8];
-                NumericUtils.longToBytes(lower, lowerBytes, 0);
+                NumericUtils.longToSortableBytes(lower, lowerBytes, 0);
                 byte[] upperBytes = new byte[8];
-                NumericUtils.longToBytes(upper, upperBytes, 0);
+                NumericUtils.longToSortableBytes(upper, upperBytes, 0);
                 query = BinaryPoint.newRangeQuery("ss_value", lowerBytes, upperBytes);
               }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/test/org/apache/lucene/search/TestSortedNumericSortField.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSortedNumericSortField.java b/lucene/core/src/test/org/apache/lucene/search/TestSortedNumericSortField.java
index 1795461..210ff89 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSortedNumericSortField.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSortedNumericSortField.java
@@ -16,7 +16,6 @@
  */
 package org.apache.lucene.search;
 
-
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.SortedNumericDocValuesField;
@@ -25,8 +24,8 @@ import org.apache.lucene.index.MultiReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.LegacyNumericUtils;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NumericUtils;
 
 /** Simple tests for SortedNumericSortField */
 public class TestSortedNumericSortField extends LuceneTestCase {
@@ -223,12 +222,12 @@ public class TestSortedNumericSortField extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    doc.add(new SortedNumericDocValuesField("value", LegacyNumericUtils.floatToSortableInt(-3f)));
+    doc.add(new SortedNumericDocValuesField("value", NumericUtils.floatToSortableInt(-3f)));
     doc.add(newStringField("id", "2", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
-    doc.add(new SortedNumericDocValuesField("value", LegacyNumericUtils.floatToSortableInt(-5f)));
-    doc.add(new SortedNumericDocValuesField("value", LegacyNumericUtils.floatToSortableInt(7f)));
+    doc.add(new SortedNumericDocValuesField("value", NumericUtils.floatToSortableInt(-5f)));
+    doc.add(new SortedNumericDocValuesField("value", NumericUtils.floatToSortableInt(7f)));
     doc.add(newStringField("id", "1", Field.Store.YES));
     writer.addDocument(doc);
     IndexReader ir = writer.getReader();
@@ -251,12 +250,12 @@ public class TestSortedNumericSortField extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    doc.add(new SortedNumericDocValuesField("value", LegacyNumericUtils.doubleToSortableLong(-3d)));
+    doc.add(new SortedNumericDocValuesField("value", NumericUtils.doubleToSortableLong(-3d)));
     doc.add(newStringField("id", "2", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
-    doc.add(new SortedNumericDocValuesField("value", LegacyNumericUtils.doubleToSortableLong(-5d)));
-    doc.add(new SortedNumericDocValuesField("value", LegacyNumericUtils.doubleToSortableLong(7d)));
+    doc.add(new SortedNumericDocValuesField("value", NumericUtils.doubleToSortableLong(-5d)));
+    doc.add(new SortedNumericDocValuesField("value", NumericUtils.doubleToSortableLong(7d)));
     doc.add(newStringField("id", "1", Field.Store.YES));
     writer.addDocument(doc);
     IndexReader ir = writer.getReader();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/test/org/apache/lucene/util/TestLegacyNumericUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestLegacyNumericUtils.java b/lucene/core/src/test/org/apache/lucene/util/TestLegacyNumericUtils.java
index 621ab96..2fb20d1 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestLegacyNumericUtils.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestLegacyNumericUtils.java
@@ -145,8 +145,8 @@ public class TestLegacyNumericUtils extends LuceneTestCase {
     
     // check forward and back conversion
     for (int i=0; i<vals.length; i++) {
-      longVals[i]= LegacyNumericUtils.doubleToSortableLong(vals[i]);
-      assertTrue( "forward and back conversion should generate same double", Double.compare(vals[i], LegacyNumericUtils.sortableLongToDouble(longVals[i]))==0 );
+      longVals[i]= NumericUtils.doubleToSortableLong(vals[i]);
+      assertTrue( "forward and back conversion should generate same double", Double.compare(vals[i], NumericUtils.sortableLongToDouble(longVals[i]))==0 );
     }
     
     // check sort order (prefixVals should be ascending)
@@ -164,10 +164,10 @@ public class TestLegacyNumericUtils extends LuceneTestCase {
   };
 
   public void testSortableDoubleNaN() {
-    final long plusInf = LegacyNumericUtils.doubleToSortableLong(Double.POSITIVE_INFINITY);
+    final long plusInf = NumericUtils.doubleToSortableLong(Double.POSITIVE_INFINITY);
     for (double nan : DOUBLE_NANs) {
       assertTrue(Double.isNaN(nan));
-      final long sortable = LegacyNumericUtils.doubleToSortableLong(nan);
+      final long sortable = NumericUtils.doubleToSortableLong(nan);
       assertTrue("Double not sorted correctly: " + nan + ", long repr: " 
           + sortable + ", positive inf.: " + plusInf, sortable > plusInf);
     }
@@ -182,8 +182,8 @@ public class TestLegacyNumericUtils extends LuceneTestCase {
     
     // check forward and back conversion
     for (int i=0; i<vals.length; i++) {
-      intVals[i]= LegacyNumericUtils.floatToSortableInt(vals[i]);
-      assertTrue( "forward and back conversion should generate same double", Float.compare(vals[i], LegacyNumericUtils.sortableIntToFloat(intVals[i]))==0 );
+      intVals[i]= NumericUtils.floatToSortableInt(vals[i]);
+      assertTrue( "forward and back conversion should generate same double", Float.compare(vals[i], NumericUtils.sortableIntToFloat(intVals[i]))==0 );
     }
     
     // check sort order (prefixVals should be ascending)
@@ -201,10 +201,10 @@ public class TestLegacyNumericUtils extends LuceneTestCase {
   };
 
   public void testSortableFloatNaN() {
-    final int plusInf = LegacyNumericUtils.floatToSortableInt(Float.POSITIVE_INFINITY);
+    final int plusInf = NumericUtils.floatToSortableInt(Float.POSITIVE_INFINITY);
     for (float nan : FLOAT_NANs) {
       assertTrue(Float.isNaN(nan));
-      final int sortable = LegacyNumericUtils.floatToSortableInt(nan);
+      final int sortable = NumericUtils.floatToSortableInt(nan);
       assertTrue("Float not sorted correctly: " + nan + ", int repr: " 
           + sortable + ", positive inf.: " + plusInf, sortable > plusInf);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/test/org/apache/lucene/util/TestNumericUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestNumericUtils.java b/lucene/core/src/test/org/apache/lucene/util/TestNumericUtils.java
new file mode 100644
index 0000000..c56dd1f
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/util/TestNumericUtils.java
@@ -0,0 +1,491 @@
+/*
+ * 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.util;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+
+/**
+ * Tests for NumericUtils static methods.
+ */
+public class TestNumericUtils extends LuceneTestCase {
+
+  /**
+   * generate a series of encoded longs, each numerical one bigger than the one before.
+   * check for correct ordering of the encoded bytes and that values round-trip.
+   */
+  public void testLongConversionAndOrdering() throws Exception {
+    BytesRef previous = null;
+    BytesRef current = new BytesRef(new byte[Long.BYTES]);
+    for (long value = -100000L; value < 100000L; value++) {
+      NumericUtils.longToSortableBytes(value, current.bytes, current.offset);
+      if (previous == null) {
+        previous = new BytesRef(new byte[Long.BYTES]);
+      } else {
+        // test if smaller
+        assertTrue("current bigger than previous: ", previous.compareTo(current) < 0);
+      }
+      // test is back and forward conversion works
+      assertEquals("forward and back conversion should generate same long", value, NumericUtils.sortableBytesToLong(current.bytes, current.offset));
+      // next step
+      System.arraycopy(current.bytes, current.offset, previous.bytes, previous.offset, current.length);
+    }
+  }
+
+  /**
+   * generate a series of encoded ints, each numerical one bigger than the one before.
+   * check for correct ordering of the encoded bytes and that values round-trip.
+   */
+  public void testIntConversionAndOrdering() throws Exception {
+    BytesRef previous = null;
+    BytesRef current = new BytesRef(new byte[Integer.BYTES]);
+    for (int value = -100000; value < 100000; value++) {
+      NumericUtils.intToSortableBytes(value, current.bytes, current.offset);
+      if (previous == null) {
+        previous = new BytesRef(new byte[Integer.BYTES]);
+      } else {
+        // test if smaller
+        assertTrue("current bigger than previous: ", previous.compareTo(current) < 0);
+      }
+      // test is back and forward conversion works
+      assertEquals("forward and back conversion should generate same int", value, NumericUtils.sortableBytesToInt(current.bytes, current.offset));
+      // next step
+      System.arraycopy(current.bytes, current.offset, previous.bytes, previous.offset, current.length);
+    }
+  }
+  
+  /**
+   * generate a series of encoded BigIntegers, each numerical one bigger than the one before.
+   * check for correct ordering of the encoded bytes and that values round-trip.
+   */
+  public void testBigIntConversionAndOrdering() throws Exception {
+    // we need at least 3 bytes of storage.
+    int size = TestUtil.nextInt(random(), 3, 16);
+    BytesRef previous = null;
+    BytesRef current = new BytesRef(new byte[size]);
+    for (long value = -100000L; value < 100000L; value++) {
+      NumericUtils.bigIntToSortableBytes(BigInteger.valueOf(value), size, current.bytes, current.offset);
+      if (previous == null) {
+        previous = new BytesRef(new byte[size]);
+      } else {
+        // test if smaller
+        assertTrue("current bigger than previous: ", previous.compareTo(current) < 0);
+      }
+      // test is back and forward conversion works
+      assertEquals("forward and back conversion should generate same BigInteger", 
+                   BigInteger.valueOf(value), 
+                   NumericUtils.sortableBytesToBigInt(current.bytes, current.offset, current.length));
+      // next step
+      System.arraycopy(current.bytes, current.offset, previous.bytes, previous.offset, current.length);
+    }
+  }
+
+  /**
+   * check extreme values of longs 
+   * check for correct ordering of the encoded bytes and that values round-trip.
+   */
+  public void testLongSpecialValues() throws Exception {
+    long[] values = new long[] {
+      Long.MIN_VALUE, Long.MIN_VALUE+1, Long.MIN_VALUE+2, -5003400000000L,
+      -4000L, -3000L, -2000L, -1000L, -1L, 0L, 1L, 10L, 300L, 50006789999999999L, Long.MAX_VALUE-2, Long.MAX_VALUE-1, Long.MAX_VALUE
+    };
+    BytesRef[] encoded = new BytesRef[values.length];
+    
+    for (int i = 0; i < values.length; i++) {
+      encoded[i] = new BytesRef(new byte[Long.BYTES]);
+      NumericUtils.longToSortableBytes(values[i], encoded[i].bytes, encoded[i].offset);
+      
+      // check forward and back conversion
+      assertEquals("forward and back conversion should generate same long", 
+                   values[i], 
+                   NumericUtils.sortableBytesToLong(encoded[i].bytes, encoded[i].offset));
+    }
+    
+    // check sort order (encoded values should be ascending)
+    for (int i = 1; i < encoded.length; i++) {
+      assertTrue("check sort order", encoded[i-1].compareTo(encoded[i]) < 0);
+    }
+  }
+
+  /**
+   * check extreme values of ints
+   * check for correct ordering of the encoded bytes and that values round-trip.
+   */
+  public void testIntSpecialValues() throws Exception {
+    int[] values = new int[] {
+      Integer.MIN_VALUE, Integer.MIN_VALUE+1, Integer.MIN_VALUE+2, -64765767,
+      -4000, -3000, -2000, -1000, -1, 0, 1, 10, 300, 765878989, Integer.MAX_VALUE-2, Integer.MAX_VALUE-1, Integer.MAX_VALUE
+    };
+    BytesRef[] encoded = new BytesRef[values.length];
+    
+    for (int i = 0; i < values.length; i++) {
+      encoded[i] = new BytesRef(new byte[Integer.BYTES]);
+      NumericUtils.intToSortableBytes(values[i], encoded[i].bytes, encoded[i].offset);
+      
+      // check forward and back conversion
+      assertEquals("forward and back conversion should generate same int", 
+                   values[i], 
+                   NumericUtils.sortableBytesToInt(encoded[i].bytes, encoded[i].offset));
+    }
+    
+    // check sort order (encoded values should be ascending)
+    for (int i = 1; i < encoded.length; i++) {
+      assertTrue("check sort order", encoded[i-1].compareTo(encoded[i]) < 0);
+    }
+  }
+  
+  /**
+   * check extreme values of big integers (4 bytes)
+   * check for correct ordering of the encoded bytes and that values round-trip.
+   */
+  public void testBigIntSpecialValues() throws Exception {
+    BigInteger[] values = new BigInteger[] {
+      BigInteger.valueOf(Integer.MIN_VALUE), BigInteger.valueOf(Integer.MIN_VALUE+1), 
+      BigInteger.valueOf(Integer.MIN_VALUE+2), BigInteger.valueOf(-64765767),
+      BigInteger.valueOf(-4000), BigInteger.valueOf(-3000), BigInteger.valueOf(-2000), 
+      BigInteger.valueOf(-1000), BigInteger.valueOf(-1), BigInteger.valueOf(0), 
+      BigInteger.valueOf(1), BigInteger.valueOf(10), BigInteger.valueOf(300), 
+      BigInteger.valueOf(765878989), BigInteger.valueOf(Integer.MAX_VALUE-2), 
+      BigInteger.valueOf(Integer.MAX_VALUE-1), BigInteger.valueOf(Integer.MAX_VALUE)
+    };
+    BytesRef[] encoded = new BytesRef[values.length];
+    
+    for (int i = 0; i < values.length; i++) {
+      encoded[i] = new BytesRef(new byte[Integer.BYTES]);
+      NumericUtils.bigIntToSortableBytes(values[i], Integer.BYTES, encoded[i].bytes, encoded[i].offset);
+      
+      // check forward and back conversion
+      assertEquals("forward and back conversion should generate same big integer", 
+                   values[i], 
+                   NumericUtils.sortableBytesToBigInt(encoded[i].bytes, encoded[i].offset, Integer.BYTES));
+    }
+    
+    // check sort order (encoded values should be ascending)
+    for (int i = 1; i < encoded.length; i++) {
+      assertTrue("check sort order", encoded[i-1].compareTo(encoded[i]) < 0);
+    }
+  }
+
+  /**
+   * check various sorted values of doubles (including extreme values)
+   * check for correct ordering of the encoded bytes and that values round-trip.
+   */
+  public void testDoubles() throws Exception {
+    double[] values = new double[] {
+      Double.NEGATIVE_INFINITY, -2.3E25, -1.0E15, -1.0, -1.0E-1, -1.0E-2, -0.0, 
+      +0.0, 1.0E-2, 1.0E-1, 1.0, 1.0E15, 2.3E25, Double.POSITIVE_INFINITY, Double.NaN
+    };
+    long[] encoded = new long[values.length];
+    
+    // check forward and back conversion
+    for (int i = 0; i < values.length; i++) {
+      encoded[i] = NumericUtils.doubleToSortableLong(values[i]);
+      assertTrue("forward and back conversion should generate same double", 
+                 Double.compare(values[i], NumericUtils.sortableLongToDouble(encoded[i])) == 0);
+    }
+    
+    // check sort order (encoded values should be ascending)
+    for (int i = 1; i < encoded.length; i++) {
+      assertTrue("check sort order", encoded[i-1] < encoded[i]);
+    }
+  }
+
+  public static final double[] DOUBLE_NANs = {
+    Double.NaN,
+    Double.longBitsToDouble(0x7ff0000000000001L),
+    Double.longBitsToDouble(0x7fffffffffffffffL),
+    Double.longBitsToDouble(0xfff0000000000001L),
+    Double.longBitsToDouble(0xffffffffffffffffL)
+  };
+
+  public void testSortableDoubleNaN() {
+    final long plusInf = NumericUtils.doubleToSortableLong(Double.POSITIVE_INFINITY);
+    for (double nan : DOUBLE_NANs) {
+      assertTrue(Double.isNaN(nan));
+      final long sortable = NumericUtils.doubleToSortableLong(nan);
+      assertTrue("Double not sorted correctly: " + nan + ", long repr: " 
+          + sortable + ", positive inf.: " + plusInf, sortable > plusInf);
+    }
+  }
+  
+  /**
+   * check various sorted values of floats (including extreme values)
+   * check for correct ordering of the encoded bytes and that values round-trip.
+   */
+  public void testFloats() throws Exception {
+    float[] values = new float[] {
+      Float.NEGATIVE_INFINITY, -2.3E25f, -1.0E15f, -1.0f, -1.0E-1f, -1.0E-2f, -0.0f, 
+      +0.0f, 1.0E-2f, 1.0E-1f, 1.0f, 1.0E15f, 2.3E25f, Float.POSITIVE_INFINITY, Float.NaN
+    };
+    int[] encoded = new int[values.length];
+    
+    // check forward and back conversion
+    for (int i = 0; i < values.length; i++) {
+      encoded[i] = NumericUtils.floatToSortableInt(values[i]);
+      assertTrue("forward and back conversion should generate same float", 
+                 Float.compare(values[i], NumericUtils.sortableIntToFloat(encoded[i])) == 0);
+    }
+    
+    // check sort order (encoded values should be ascending)
+    for (int i = 1; i < encoded.length; i++) {
+      assertTrue( "check sort order", encoded[i-1] < encoded[i] );
+    }
+  }
+
+  public static final float[] FLOAT_NANs = {
+    Float.NaN,
+    Float.intBitsToFloat(0x7f800001),
+    Float.intBitsToFloat(0x7fffffff),
+    Float.intBitsToFloat(0xff800001),
+    Float.intBitsToFloat(0xffffffff)
+  };
+
+  public void testSortableFloatNaN() {
+    final int plusInf = NumericUtils.floatToSortableInt(Float.POSITIVE_INFINITY);
+    for (float nan : FLOAT_NANs) {
+      assertTrue(Float.isNaN(nan));
+      final int sortable = NumericUtils.floatToSortableInt(nan);
+      assertTrue("Float not sorted correctly: " + nan + ", int repr: " 
+          + sortable + ", positive inf.: " + plusInf, sortable > plusInf);
+    }
+  }
+  
+  public void testAdd() throws Exception {
+    int iters = atLeast(10000);
+    int numBytes = TestUtil.nextInt(random(), 1, 100);
+    for(int iter=0;iter<iters;iter++) {
+      BigInteger v1 = new BigInteger(8*numBytes-1, random());
+      BigInteger v2 = new BigInteger(8*numBytes-1, random());
+
+      byte[] v1Bytes = new byte[numBytes];
+      byte[] v1RawBytes = v1.toByteArray();
+      assert v1RawBytes.length <= numBytes;
+      System.arraycopy(v1RawBytes, 0, v1Bytes, v1Bytes.length-v1RawBytes.length, v1RawBytes.length);
+
+      byte[] v2Bytes = new byte[numBytes];
+      byte[] v2RawBytes = v2.toByteArray();
+      assert v1RawBytes.length <= numBytes;
+      System.arraycopy(v2RawBytes, 0, v2Bytes, v2Bytes.length-v2RawBytes.length, v2RawBytes.length);
+
+      byte[] result = new byte[numBytes];
+      NumericUtils.add(numBytes, 0, v1Bytes, v2Bytes, result);
+
+      BigInteger sum = v1.add(v2);
+      assertTrue("sum=" + sum + " v1=" + v1 + " v2=" + v2 + " but result=" + new BigInteger(1, result), sum.equals(new BigInteger(1, result)));
+    }
+  }
+
+  public void testIllegalAdd() throws Exception {
+    byte[] bytes = new byte[4];
+    Arrays.fill(bytes, (byte) 0xff);
+    byte[] one = new byte[4];
+    one[3] = 1;
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+      NumericUtils.add(4, 0, bytes, one, new byte[4]);
+    });
+    assertEquals("a + b overflows bytesPerDim=4", expected.getMessage());
+  }
+  
+  public void testSubtract() throws Exception {
+    int iters = atLeast(10000);
+    int numBytes = TestUtil.nextInt(random(), 1, 100);
+    for(int iter=0;iter<iters;iter++) {
+      BigInteger v1 = new BigInteger(8*numBytes-1, random());
+      BigInteger v2 = new BigInteger(8*numBytes-1, random());
+      if (v1.compareTo(v2) < 0) {
+        BigInteger tmp = v1;
+        v1 = v2;
+        v2 = tmp;
+      }
+
+      byte[] v1Bytes = new byte[numBytes];
+      byte[] v1RawBytes = v1.toByteArray();
+      assert v1RawBytes.length <= numBytes: "length=" + v1RawBytes.length + " vs numBytes=" + numBytes;
+      System.arraycopy(v1RawBytes, 0, v1Bytes, v1Bytes.length-v1RawBytes.length, v1RawBytes.length);
+
+      byte[] v2Bytes = new byte[numBytes];
+      byte[] v2RawBytes = v2.toByteArray();
+      assert v2RawBytes.length <= numBytes;
+      assert v2RawBytes.length <= numBytes: "length=" + v2RawBytes.length + " vs numBytes=" + numBytes;
+      System.arraycopy(v2RawBytes, 0, v2Bytes, v2Bytes.length-v2RawBytes.length, v2RawBytes.length);
+
+      byte[] result = new byte[numBytes];
+      NumericUtils.subtract(numBytes, 0, v1Bytes, v2Bytes, result);
+
+      BigInteger diff = v1.subtract(v2);
+
+      assertTrue("diff=" + diff + " vs result=" + new BigInteger(result) + " v1=" + v1 + " v2=" + v2, diff.equals(new BigInteger(result)));
+    }
+  }
+
+  public void testIllegalSubtract() throws Exception {
+    byte[] v1 = new byte[4];
+    v1[3] = (byte) 0xf0;
+    byte[] v2 = new byte[4];
+    v2[3] = (byte) 0xf1;
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+      NumericUtils.subtract(4, 0, v1, v2, new byte[4]);
+    });
+    assertEquals("a < b", expected.getMessage());
+  }
+  
+  /** test round-trip encoding of random integers */
+  public void testIntsRoundTrip() {
+    byte[] encoded = new byte[Integer.BYTES];
+
+    for (int i = 0; i < 10000; i++) {
+      int value = random().nextInt();
+      NumericUtils.intToSortableBytes(value, encoded, 0);
+      assertEquals(value, NumericUtils.sortableBytesToInt(encoded, 0));
+    }
+  }
+  
+  /** test round-trip encoding of random longs */
+  public void testLongsRoundTrip() {
+    byte[] encoded = new byte[Long.BYTES];
+
+    for (int i = 0; i < 10000; i++) {
+      long value = TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE);
+      NumericUtils.longToSortableBytes(value, encoded, 0);
+      assertEquals(value, NumericUtils.sortableBytesToLong(encoded, 0));
+    }
+  }
+  
+  /** test round-trip encoding of random floats */
+  public void testFloatsRoundTrip() {
+    byte[] encoded = new byte[Float.BYTES];
+
+    for (int i = 0; i < 10000; i++) {
+      float value = Float.intBitsToFloat(random().nextInt());
+      NumericUtils.intToSortableBytes(NumericUtils.floatToSortableInt(value), encoded, 0);
+      float actual = NumericUtils.sortableIntToFloat(NumericUtils.sortableBytesToInt(encoded, 0));
+      assertEquals(Float.floatToIntBits(value), Float.floatToIntBits(actual));
+    }
+  }
+  
+  /** test round-trip encoding of random doubles */
+  public void testDoublesRoundTrip() {
+    byte[] encoded = new byte[Double.BYTES];
+
+    for (int i = 0; i < 10000; i++) {
+      double value = Double.longBitsToDouble(TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE));
+      NumericUtils.longToSortableBytes(NumericUtils.doubleToSortableLong(value), encoded, 0);
+      double actual = NumericUtils.sortableLongToDouble(NumericUtils.sortableBytesToLong(encoded, 0));
+      assertEquals(Double.doubleToLongBits(value), Double.doubleToLongBits(actual));
+    }
+  }
+  
+  /** test round-trip encoding of random big integers */
+  public void testBigIntsRoundTrip() {
+    for (int i = 0; i < 10000; i++) {
+      BigInteger value = TestUtil.nextBigInteger(random(), 16);
+      int length = value.toByteArray().length;
+
+      // make sure sign extension is tested: sometimes pad to more bytes when encoding.
+      int maxLength = TestUtil.nextInt(random(), length, length + 3);
+      byte[] encoded = new byte[maxLength];
+      NumericUtils.bigIntToSortableBytes(value, maxLength, encoded, 0);
+      assertEquals(value, NumericUtils.sortableBytesToBigInt(encoded, 0, maxLength));
+    }
+  }
+  
+  /** check sort order of random integers consistent with Integer.compare */
+  public void testIntsCompare() {
+    BytesRef left = new BytesRef(new byte[Integer.BYTES]);
+    BytesRef right = new BytesRef(new byte[Integer.BYTES]);
+
+    for (int i = 0; i < 10000; i++) {
+      int leftValue = random().nextInt();
+      NumericUtils.intToSortableBytes(leftValue, left.bytes, left.offset);
+
+      int rightValue = random().nextInt();
+      NumericUtils.intToSortableBytes(rightValue, right.bytes, right.offset);
+      
+      assertEquals(Integer.signum(Integer.compare(leftValue, rightValue)),
+                   Integer.signum(left.compareTo(right)));
+    }
+  }
+  
+  /** check sort order of random longs consistent with Long.compare */
+  public void testLongsCompare() {
+    BytesRef left = new BytesRef(new byte[Long.BYTES]);
+    BytesRef right = new BytesRef(new byte[Long.BYTES]);
+
+    for (int i = 0; i < 10000; i++) {
+      long leftValue = TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE);
+      NumericUtils.longToSortableBytes(leftValue, left.bytes, left.offset);
+
+      long rightValue = TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE);
+      NumericUtils.longToSortableBytes(rightValue, right.bytes, right.offset);
+      
+      assertEquals(Integer.signum(Long.compare(leftValue, rightValue)),
+                   Integer.signum(left.compareTo(right)));
+    }
+  }
+  
+  /** check sort order of random floats consistent with Float.compare */
+  public void testFloatsCompare() {
+    BytesRef left = new BytesRef(new byte[Float.BYTES]);
+    BytesRef right = new BytesRef(new byte[Float.BYTES]);
+
+    for (int i = 0; i < 10000; i++) {
+      float leftValue = Float.intBitsToFloat(random().nextInt());
+      NumericUtils.intToSortableBytes(NumericUtils.floatToSortableInt(leftValue), left.bytes, left.offset);
+
+      float rightValue = Float.intBitsToFloat(random().nextInt());
+      NumericUtils.intToSortableBytes(NumericUtils.floatToSortableInt(rightValue), right.bytes, right.offset);
+      
+      assertEquals(Integer.signum(Float.compare(leftValue, rightValue)),
+                   Integer.signum(left.compareTo(right)));
+    }
+  }
+  
+  /** check sort order of random doubles consistent with Double.compare */
+  public void testDoublesCompare() {
+    BytesRef left = new BytesRef(new byte[Double.BYTES]);
+    BytesRef right = new BytesRef(new byte[Double.BYTES]);
+
+    for (int i = 0; i < 10000; i++) {
+      double leftValue = Double.longBitsToDouble(TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE));
+      NumericUtils.longToSortableBytes(NumericUtils.doubleToSortableLong(leftValue), left.bytes, left.offset);
+
+      double rightValue = Double.longBitsToDouble(TestUtil.nextLong(random(), Long.MIN_VALUE, Long.MAX_VALUE));
+      NumericUtils.longToSortableBytes(NumericUtils.doubleToSortableLong(rightValue), right.bytes, right.offset);
+      
+      assertEquals(Integer.signum(Double.compare(leftValue, rightValue)),
+                   Integer.signum(left.compareTo(right)));
+    }
+  }
+  
+  /** check sort order of random bigintegers consistent with BigInteger.compareTo */
+  public void testBigIntsCompare() {
+    for (int i = 0; i < 10000; i++) {
+      int maxLength = TestUtil.nextInt(random(), 1, 16);
+      
+      BigInteger leftValue = TestUtil.nextBigInteger(random(), maxLength);
+      BytesRef left = new BytesRef(new byte[maxLength]);
+      NumericUtils.bigIntToSortableBytes(leftValue, maxLength, left.bytes, left.offset);
+      
+      BigInteger rightValue = TestUtil.nextBigInteger(random(), maxLength);
+      BytesRef right = new BytesRef(new byte[maxLength]);
+      NumericUtils.bigIntToSortableBytes(rightValue, maxLength, right.bytes, right.offset);
+      
+      assertEquals(Integer.signum(leftValue.compareTo(rightValue)),
+                   Integer.signum(left.compareTo(right)));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java b/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
index 2be2b42..f1402fc 100644
--- a/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
+++ b/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
@@ -45,7 +45,7 @@ public class TestBKD extends LuceneTestCase {
       BKDWriter w = new BKDWriter(100, dir, "tmp", 1, 4, 2, 1.0f);
       byte[] scratch = new byte[4];
       for(int docID=0;docID<100;docID++) {
-        NumericUtils.intToBytes(docID, scratch, 0);
+        NumericUtils.intToSortableBytes(docID, scratch, 0);
         w.add(scratch, docID);
       }
 
@@ -74,7 +74,7 @@ public class TestBKD extends LuceneTestCase {
 
             @Override
             public void visit(int docID, byte[] packedValue) {
-              int x = NumericUtils.bytesToInt(packedValue, 0);
+              int x = NumericUtils.sortableBytesToInt(packedValue, 0);
               if (VERBOSE) {
                 System.out.println("visit docID=" + docID + " x=" + x);
               }
@@ -85,8 +85,8 @@ public class TestBKD extends LuceneTestCase {
 
             @Override
             public Relation compare(byte[] minPacked, byte[] maxPacked) {
-              int min = NumericUtils.bytesToInt(minPacked, 0);
-              int max = NumericUtils.bytesToInt(maxPacked, 0);
+              int min = NumericUtils.sortableBytesToInt(minPacked, 0);
+              int max = NumericUtils.sortableBytesToInt(maxPacked, 0);
               assert max >= min;
               if (VERBOSE) {
                 System.out.println("compare: min=" + min + " max=" + max + " vs queryMin=" + queryMin + " queryMax=" + queryMax);
@@ -141,7 +141,7 @@ public class TestBKD extends LuceneTestCase {
           if (values[dim] > maxValue[dim]) {
             maxValue[dim] = values[dim];
           }
-          NumericUtils.intToBytes(values[dim], scratch, dim * Integer.BYTES);
+          NumericUtils.intToSortableBytes(values[dim], scratch, dim * Integer.BYTES);
           if (VERBOSE) {
             System.out.println("    " + dim + " -> " + values[dim]);
           }
@@ -162,8 +162,8 @@ public class TestBKD extends LuceneTestCase {
         byte[] minPackedValue = r.getMinPackedValue();
         byte[] maxPackedValue = r.getMaxPackedValue();
         for(int dim=0;dim<numDims;dim++) {
-          assertEquals(minValue[dim], NumericUtils.bytesToInt(minPackedValue, dim * Integer.BYTES));
-          assertEquals(maxValue[dim], NumericUtils.bytesToInt(maxPackedValue, dim * Integer.BYTES));
+          assertEquals(minValue[dim], NumericUtils.sortableBytesToInt(minPackedValue, dim * Integer.BYTES));
+          assertEquals(maxValue[dim], NumericUtils.sortableBytesToInt(maxPackedValue, dim * Integer.BYTES));
         }
 
         int iters = atLeast(100);
@@ -197,7 +197,7 @@ public class TestBKD extends LuceneTestCase {
             public void visit(int docID, byte[] packedValue) {
               //System.out.println("visit check docID=" + docID);
               for(int dim=0;dim<numDims;dim++) {
-                int x = NumericUtils.bytesToInt(packedValue, dim * Integer.BYTES);
+                int x = NumericUtils.sortableBytesToInt(packedValue, dim * Integer.BYTES);
                 if (x < queryMin[dim] || x > queryMax[dim]) {
                   //System.out.println("  no");
                   return;
@@ -212,8 +212,8 @@ public class TestBKD extends LuceneTestCase {
             public Relation compare(byte[] minPacked, byte[] maxPacked) {
               boolean crosses = false;
               for(int dim=0;dim<numDims;dim++) {
-                int min = NumericUtils.bytesToInt(minPacked, dim * Integer.BYTES);
-                int max = NumericUtils.bytesToInt(maxPacked, dim * Integer.BYTES);
+                int min = NumericUtils.sortableBytesToInt(minPacked, dim * Integer.BYTES);
+                int max = NumericUtils.sortableBytesToInt(maxPacked, dim * Integer.BYTES);
                 assert max >= min;
 
                 if (max < queryMin[dim] || min > queryMax[dim]) {
@@ -269,7 +269,7 @@ public class TestBKD extends LuceneTestCase {
         }
         for(int dim=0;dim<numDims;dim++) {
           values[dim] = randomBigInt(numBytesPerDim);
-          NumericUtils.bigIntToBytes(values[dim], numBytesPerDim, scratch, dim * numBytesPerDim);
+          NumericUtils.bigIntToSortableBytes(values[dim], numBytesPerDim, scratch, dim * numBytesPerDim);
           if (VERBOSE) {
             System.out.println("    " + dim + " -> " + values[dim]);
           }
@@ -318,7 +318,7 @@ public class TestBKD extends LuceneTestCase {
             public void visit(int docID, byte[] packedValue) {
               //System.out.println("visit check docID=" + docID);
               for(int dim=0;dim<numDims;dim++) {
-                BigInteger x = NumericUtils.bytesToBigInt(packedValue, dim * numBytesPerDim, numBytesPerDim);
+                BigInteger x = NumericUtils.sortableBytesToBigInt(packedValue, dim * numBytesPerDim, numBytesPerDim);
                 if (x.compareTo(queryMin[dim]) < 0 || x.compareTo(queryMax[dim]) > 0) {
                   //System.out.println("  no");
                   return;
@@ -333,8 +333,8 @@ public class TestBKD extends LuceneTestCase {
             public Relation compare(byte[] minPacked, byte[] maxPacked) {
               boolean crosses = false;
               for(int dim=0;dim<numDims;dim++) {
-                BigInteger min = NumericUtils.bytesToBigInt(minPacked, dim * numBytesPerDim, numBytesPerDim);
-                BigInteger max = NumericUtils.bytesToBigInt(maxPacked, dim * numBytesPerDim, numBytesPerDim);
+                BigInteger min = NumericUtils.sortableBytesToBigInt(minPacked, dim * numBytesPerDim, numBytesPerDim);
+                BigInteger max = NumericUtils.sortableBytesToBigInt(maxPacked, dim * numBytesPerDim, numBytesPerDim);
                 assert max.compareTo(min) >= 0;
 
                 if (max.compareTo(queryMin[dim]) < 0 || min.compareTo(queryMax[dim]) > 0) {
@@ -533,84 +533,7 @@ public class TestBKD extends LuceneTestCase {
     verify(docValuesArray, docIDsArray, numDims, numBytesPerDim);
   }
 
-  public void testNumericUtilsAdd() throws Exception {
-    int iters = atLeast(10000);
-    int numBytes = TestUtil.nextInt(random(), 1, 100);
-    for(int iter=0;iter<iters;iter++) {
-      BigInteger v1 = new BigInteger(8*numBytes-1, random());
-      BigInteger v2 = new BigInteger(8*numBytes-1, random());
 
-      byte[] v1Bytes = new byte[numBytes];
-      byte[] v1RawBytes = v1.toByteArray();
-      assert v1RawBytes.length <= numBytes;
-      System.arraycopy(v1RawBytes, 0, v1Bytes, v1Bytes.length-v1RawBytes.length, v1RawBytes.length);
-
-      byte[] v2Bytes = new byte[numBytes];
-      byte[] v2RawBytes = v2.toByteArray();
-      assert v1RawBytes.length <= numBytes;
-      System.arraycopy(v2RawBytes, 0, v2Bytes, v2Bytes.length-v2RawBytes.length, v2RawBytes.length);
-
-      byte[] result = new byte[numBytes];
-      NumericUtils.add(numBytes, 0, v1Bytes, v2Bytes, result);
-
-      BigInteger sum = v1.add(v2);
-      assertTrue("sum=" + sum + " v1=" + v1 + " v2=" + v2 + " but result=" + new BigInteger(1, result), sum.equals(new BigInteger(1, result)));
-    }
-  }
-
-  public void testIllegalNumericUtilsAdd() throws Exception {
-    byte[] bytes = new byte[4];
-    Arrays.fill(bytes, (byte) 0xff);
-    byte[] one = new byte[4];
-    one[3] = 1;
-    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
-      NumericUtils.add(4, 0, bytes, one, new byte[4]);
-    });
-    assertEquals("a + b overflows bytesPerDim=4", expected.getMessage());
-  }
-  
-  public void testNumericUtilsSubtract() throws Exception {
-    int iters = atLeast(10000);
-    int numBytes = TestUtil.nextInt(random(), 1, 100);
-    for(int iter=0;iter<iters;iter++) {
-      BigInteger v1 = new BigInteger(8*numBytes-1, random());
-      BigInteger v2 = new BigInteger(8*numBytes-1, random());
-      if (v1.compareTo(v2) < 0) {
-        BigInteger tmp = v1;
-        v1 = v2;
-        v2 = tmp;
-      }
-
-      byte[] v1Bytes = new byte[numBytes];
-      byte[] v1RawBytes = v1.toByteArray();
-      assert v1RawBytes.length <= numBytes: "length=" + v1RawBytes.length + " vs numBytes=" + numBytes;
-      System.arraycopy(v1RawBytes, 0, v1Bytes, v1Bytes.length-v1RawBytes.length, v1RawBytes.length);
-
-      byte[] v2Bytes = new byte[numBytes];
-      byte[] v2RawBytes = v2.toByteArray();
-      assert v2RawBytes.length <= numBytes;
-      assert v2RawBytes.length <= numBytes: "length=" + v2RawBytes.length + " vs numBytes=" + numBytes;
-      System.arraycopy(v2RawBytes, 0, v2Bytes, v2Bytes.length-v2RawBytes.length, v2RawBytes.length);
-
-      byte[] result = new byte[numBytes];
-      NumericUtils.subtract(numBytes, 0, v1Bytes, v2Bytes, result);
-
-      BigInteger diff = v1.subtract(v2);
-
-      assertTrue("diff=" + diff + " vs result=" + new BigInteger(result) + " v1=" + v1 + " v2=" + v2, diff.equals(new BigInteger(result)));
-    }
-  }
-
-  public void testIllegalNumericUtilsSubtract() throws Exception {
-    byte[] v1 = new byte[4];
-    v1[3] = (byte) 0xf0;
-    byte[] v2 = new byte[4];
-    v2[3] = (byte) 0xf1;
-    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
-      NumericUtils.subtract(4, 0, v1, v2, new byte[4]);
-    });
-    assertEquals("a < b", expected.getMessage());
-  }
 
   /** docIDs can be null, for the single valued case, else it maps value to docID */
   private void verify(byte[][][] docValues, int[] docIDs, int numDims, int numBytesPerDim) throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
index c246d74..7585708 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
@@ -33,6 +33,7 @@ import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TwoPhaseIterator;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.LegacyNumericUtils;
+import org.apache.lucene.util.NumericUtils;
 
 /** Represents a range over double values.
  *
@@ -83,8 +84,8 @@ public final class DoubleRange extends Range {
 
   LongRange toLongRange() {
     return new LongRange(label,
-                         LegacyNumericUtils.doubleToSortableLong(min), true,
-                         LegacyNumericUtils.doubleToSortableLong(max), true);
+                         NumericUtils.doubleToSortableLong(min), true,
+                         NumericUtils.doubleToSortableLong(max), true);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
index a39ea7e..2d0ba5c 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
@@ -37,7 +37,7 @@ import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
-import org.apache.lucene.util.LegacyNumericUtils;
+import org.apache.lucene.util.NumericUtils;
 
 /** {@link Facets} implementation that computes counts for
  *  dynamic double ranges from a provided {@link
@@ -88,8 +88,8 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts {
     for(int i=0;i<ranges.length;i++) {
       DoubleRange range = ranges[i];
       longRanges[i] =  new LongRange(range.label,
-                                     LegacyNumericUtils.doubleToSortableLong(range.min), true,
-                                     LegacyNumericUtils.doubleToSortableLong(range.max), true);
+                                     NumericUtils.doubleToSortableLong(range.min), true,
+                                     NumericUtils.doubleToSortableLong(range.max), true);
     }
 
     LongRangeCounter counter = new LongRangeCounter(longRanges);
@@ -130,7 +130,7 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts {
         }
         // Skip missing docs:
         if (fv.exists(doc)) {
-          counter.add(LegacyNumericUtils.doubleToSortableLong(fv.doubleVal(doc)));
+          counter.add(NumericUtils.doubleToSortableLong(fv.doubleVal(doc)));
         } else {
           missingCount++;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java b/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java
index 183b7dd..d3ff410 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java
@@ -26,7 +26,7 @@ import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.LegacyNumericUtils;
+import org.apache.lucene.util.NumericUtils;
 
 import java.io.IOException;
 
@@ -198,7 +198,7 @@ public class ToParentBlockJoinSortField extends SortField {
         return new NumericDocValues() {
           @Override
           public long get(int docID) {
-            return LegacyNumericUtils.sortableFloatBits((int) view.get(docID));
+            return NumericUtils.sortableFloatBits((int) view.get(docID));
           }
         };
       }
@@ -223,7 +223,7 @@ public class ToParentBlockJoinSortField extends SortField {
         return new NumericDocValues() {
           @Override
           public long get(int docID) {
-            return LegacyNumericUtils.sortableDoubleBits(view.get(docID));
+            return NumericUtils.sortableDoubleBits(view.get(docID));
           }
         };
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java b/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
index 0015c6c..df1c80f 100644
--- a/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
+++ b/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
@@ -173,7 +173,7 @@ public abstract class SorterTestBase extends LuceneTestCase {
     doc.add(new SortedNumericDocValuesField(SORTED_NUMERIC_DV_FIELD, id + 1));
     doc.add(new Field(TERM_VECTORS_FIELD, Integer.toString(id), TERM_VECTORS_TYPE));
     byte[] bytes = new byte[4];
-    NumericUtils.intToBytes(id, bytes, 0);
+    NumericUtils.intToSortableBytes(id, bytes, 0);
     // TODO: index time sorting doesn't yet support points
     //doc.add(new BinaryPoint(DIMENSIONAL_FIELD, bytes));
     return doc;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
index 9f765ab..f175858 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
@@ -142,12 +142,12 @@ public class BigIntegerPoint extends Field {
   
   /** Encode single BigInteger dimension */
   public static void encodeDimension(BigInteger value, byte dest[], int offset) {
-    NumericUtils.bigIntToBytes(value, BYTES, dest, offset);
+    NumericUtils.bigIntToSortableBytes(value, BYTES, dest, offset);
   }
   
   /** Decode single BigInteger dimension */
   public static BigInteger decodeDimension(byte value[], int offset) {
-    return NumericUtils.bytesToBigInt(value, offset, BYTES);
+    return NumericUtils.sortableBytesToBigInt(value, offset, BYTES);
   }
 
   // static methods for generating queries

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
index aeb0a0f..5f45cb5 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
@@ -67,8 +67,8 @@ public class LatLonPoint extends Field {
    */
   public void setLocationValue(double latitude, double longitude) {
     byte[] bytes = new byte[8];
-    NumericUtils.intToBytes(encodeLatitude(latitude), bytes, 0);
-    NumericUtils.intToBytes(encodeLongitude(longitude), bytes, Integer.BYTES);
+    NumericUtils.intToSortableBytes(encodeLatitude(latitude), bytes, 0);
+    NumericUtils.intToSortableBytes(encodeLongitude(longitude), bytes, Integer.BYTES);
     fieldsData = new BytesRef(bytes);
   }
 
@@ -159,7 +159,7 @@ public class LatLonPoint extends Field {
    * @return decoded latitude value.
    */
   public static double decodeLatitude(byte[] src, int offset) {
-    return decodeLatitude(NumericUtils.bytesToInt(src, offset));
+    return decodeLatitude(NumericUtils.sortableBytesToInt(src, offset));
   }
 
   /** 
@@ -180,16 +180,16 @@ public class LatLonPoint extends Field {
    * @return decoded longitude value.
    */
   public static double decodeLongitude(byte[] src, int offset) {
-    return decodeLongitude(NumericUtils.bytesToInt(src, offset));
+    return decodeLongitude(NumericUtils.sortableBytesToInt(src, offset));
   }
   
   /** sugar encodes a single point as a 2D byte array */
   private static byte[][] encode(double latitude, double longitude) {
     byte[][] bytes = new byte[2][];
     bytes[0] = new byte[4];
-    NumericUtils.intToBytes(encodeLatitude(latitude), bytes[0], 0);
+    NumericUtils.intToSortableBytes(encodeLatitude(latitude), bytes[0], 0);
     bytes[1] = new byte[4];
-    NumericUtils.intToBytes(encodeLongitude(longitude), bytes[1], 0);
+    NumericUtils.intToSortableBytes(encodeLongitude(longitude), bytes[1], 0);
     return bytes;
   }
 
@@ -235,7 +235,7 @@ public class LatLonPoint extends Field {
       leftOpen[0] = lower[0];
       // leave longitude open
       leftOpen[1] = new byte[Integer.BYTES];
-      NumericUtils.intToBytes(Integer.MIN_VALUE, leftOpen[1], 0);
+      NumericUtils.intToSortableBytes(Integer.MIN_VALUE, leftOpen[1], 0);
       Query left = newBoxInternal(field, leftOpen, upper);
       q.add(new BooleanClause(left, BooleanClause.Occur.SHOULD));
 
@@ -243,7 +243,7 @@ public class LatLonPoint extends Field {
       rightOpen[0] = upper[0];
       // leave longitude open
       rightOpen[1] = new byte[Integer.BYTES];
-      NumericUtils.intToBytes(Integer.MAX_VALUE, rightOpen[1], 0);
+      NumericUtils.intToSortableBytes(Integer.MAX_VALUE, rightOpen[1], 0);
       Query right = newBoxInternal(field, lower, rightOpen);
       q.add(new BooleanClause(right, BooleanClause.Occur.SHOULD));
       return new ConstantScoreQuery(q.build());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ffeccab/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
index f2ec17e..2f79679 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
@@ -81,7 +81,7 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
       return null;
     } else {
       byte[] bytes = new byte[Long.BYTES];
-      NumericUtils.longToBytes(l, bytes, 0);
+      NumericUtils.longToSortableBytes(l, bytes, 0);
       return new BytesRef(bytes);
     }
   }