You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/12/07 11:52:05 UTC

svn commit: r1643659 [3/7] - in /lucene/dev/branches/lucene6005/lucene: ./ analysis/common/src/test/org/apache/lucene/analysis/core/ analysis/icu/src/test/org/apache/lucene/collation/ backward-codecs/src/test/org/apache/lucene/index/ benchmark/src/java...

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1643659&r1=1643658&r2=1643659&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java Sun Dec  7 10:52:03 2014
@@ -18,14 +18,15 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.Comparator;
 
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefHash.BytesStartArray;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IntBlockPool;
-import org.apache.lucene.util.BytesRefHash.BytesStartArray;
 
 abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
   private static final int HASH_INIT_SIZE = 4;
@@ -50,13 +51,16 @@ abstract class TermsHashPerField impleme
 
   final BytesRefHash bytesHash;
 
+  int maxTermLength;
+
   ParallelPostingsArray postingsArray;
   private final Counter bytesUsed;
+  final boolean rightJustifyTerms;
 
   /** streamCount: how many streams this field stores per term.
    * E.g. doc(+freq) is 1 stream, prox+offset is a second. */
 
-  public TermsHashPerField(int streamCount, FieldInvertState fieldState, TermsHash termsHash, TermsHashPerField nextPerField, FieldInfo fieldInfo) {
+  public TermsHashPerField(int streamCount, FieldInvertState fieldState, TermsHash termsHash, TermsHashPerField nextPerField, FieldInfo fieldInfo, boolean rightJustifyTerms) {
     intPool = termsHash.intPool;
     bytePool = termsHash.bytePool;
     termBytePool = termsHash.termBytePool;
@@ -68,6 +72,7 @@ abstract class TermsHashPerField impleme
     numPostingInt = 2*streamCount;
     this.fieldInfo = fieldInfo;
     this.nextPerField = nextPerField;
+    this.rightJustifyTerms = rightJustifyTerms;
     PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
     bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts);
   }
@@ -94,7 +99,14 @@ abstract class TermsHashPerField impleme
   /** Collapse the hash table and sort in-place; also sets
    * this.sortedTermIDs to the results */
   public int[] sortPostings() {
-    sortedTermIDs = bytesHash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
+    Comparator<BytesRef> cmp;
+    if (rightJustifyTerms) {
+      // Sort by a comparator as if terms were right justified:
+      cmp = BytesRef.getRightJustifiedComparator();
+    } else {
+      cmp = BytesRef.getUTF8SortedAsUnicodeComparator();
+    }
+    sortedTermIDs = bytesHash.sort(cmp);
     return sortedTermIDs;
   }
 
@@ -156,6 +168,13 @@ abstract class TermsHashPerField impleme
     //System.out.println("add term=" + termBytesRef.utf8ToString() + " doc=" + docState.docID + " termID=" + termID);
 
     if (termID >= 0) {// New posting
+      maxTermLength = Math.max(maxTermLength, termBytesRef.length);
+      /*
+      if (rightJustifyTerms && termBytesRef.length > 0 && termBytesRef.bytes[termBytesRef.offset] == 0) {
+        // nocommit catch this earlier so it's not an aborting exception?
+        throw new IllegalArgumentException("left-zero-pad fields cannot lead with 0 bytes");
+      }
+      */
       bytesHash.byteStart(termID);
       // Init stream slices
       if (numPostingInt + intPool.intUpto > IntBlockPool.INT_BLOCK_SIZE) {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeFilter.java?rev=1643659&r1=1643658&r2=1643659&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeFilter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeFilter.java Sun Dec  7 10:52:03 2014
@@ -25,6 +25,8 @@ import org.apache.lucene.index.NumericDo
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.HalfFloat;
+import org.apache.lucene.util.NumericUtils;
 
 /**
  * A range filter built on top of numeric doc values field 
@@ -65,13 +67,15 @@ public abstract class DocValuesRangeFilt
   final T upperVal;
   final boolean includeLower;
   final boolean includeUpper;
-  
-  private DocValuesRangeFilter(String field, T lowerVal, T upperVal, boolean includeLower, boolean includeUpper) {
+  final String desc;
+
+  private DocValuesRangeFilter(String field, T lowerVal, T upperVal, boolean includeLower, boolean includeUpper, String desc) {
     this.field = field;
     this.lowerVal = lowerVal;
     this.upperVal = upperVal;
     this.includeLower = includeLower;
     this.includeUpper = includeUpper;
+    this.desc = desc;
   }
   
   /** This method is implemented for each data type */
@@ -83,8 +87,8 @@ public abstract class DocValuesRangeFilt
    * fields containing zero or one term in the field. The range can be half-open by setting one
    * of the values to <code>null</code>.
    */
-  public static DocValuesRangeFilter<String> newStringRange(String field, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) {
-    return new DocValuesRangeFilter<String>(field, lowerVal, upperVal, includeLower, includeUpper) {
+  public static DocValuesRangeFilter<String> newStringRange(String field, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper, String desc) {
+    return new DocValuesRangeFilter<String>(field, lowerVal, upperVal, includeLower, includeUpper, desc) {
       @Override
       public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
         final SortedDocValues fcsi = DocValues.getSorted(context.reader(), field);
@@ -140,8 +144,8 @@ public abstract class DocValuesRangeFilt
    * of the values to <code>null</code>.
    */
   // TODO: bogus that newStringRange doesnt share this code... generics hell
-  public static DocValuesRangeFilter<BytesRef> newBytesRefRange(String field, BytesRef lowerVal, BytesRef upperVal, boolean includeLower, boolean includeUpper) {
-    return new DocValuesRangeFilter<BytesRef>(field, lowerVal, upperVal, includeLower, includeUpper) {
+  public static DocValuesRangeFilter<BytesRef> newBytesRefRange(String field, BytesRef lowerVal, BytesRef upperVal, boolean includeLower, boolean includeUpper, String desc) {
+    return new DocValuesRangeFilter<BytesRef>(field, lowerVal, upperVal, includeLower, includeUpper, desc) {
       @Override
       public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
         final SortedDocValues fcsi = DocValues.getSorted(context.reader(), field);
@@ -196,8 +200,8 @@ public abstract class DocValuesRangeFilt
    * int fields containing exactly one numeric term in the field. The range can be half-open by setting one
    * of the values to <code>null</code>.
    */
-  public static DocValuesRangeFilter<Integer> newIntRange(String field, Integer lowerVal, Integer upperVal, boolean includeLower, boolean includeUpper) {
-    return new DocValuesRangeFilter<Integer>(field, lowerVal, upperVal, includeLower, includeUpper) {
+  public static DocValuesRangeFilter<Integer> newIntRange(String field, Integer lowerVal, Integer upperVal, boolean includeLower, boolean includeUpper, String desc) {
+    return new DocValuesRangeFilter<Integer>(field, lowerVal, upperVal, includeLower, includeUpper, desc) {
       @Override
       public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
         final int inclusiveLowerPoint, inclusiveUpperPoint;
@@ -238,8 +242,8 @@ public abstract class DocValuesRangeFilt
    * long fields containing exactly one numeric term in the field. The range can be half-open by setting one
    * of the values to <code>null</code>.
    */
-  public static DocValuesRangeFilter<Long> newLongRange(String field, Long lowerVal, Long upperVal, boolean includeLower, boolean includeUpper) {
-    return new DocValuesRangeFilter<Long>(field, lowerVal, upperVal, includeLower, includeUpper) {
+  public static DocValuesRangeFilter<Long> newLongRange(String field, Long lowerVal, Long upperVal, boolean includeLower, boolean includeUpper, String desc) {
+    return new DocValuesRangeFilter<Long>(field, lowerVal, upperVal, includeLower, includeUpper, desc) {
       @Override
       public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
         final long inclusiveLowerPoint, inclusiveUpperPoint;
@@ -280,8 +284,8 @@ public abstract class DocValuesRangeFilt
    * float fields containing exactly one numeric term in the field. The range can be half-open by setting one
    * of the values to <code>null</code>.
    */
-  public static DocValuesRangeFilter<Float> newFloatRange(String field, Float lowerVal, Float upperVal, boolean includeLower, boolean includeUpper) {
-    return new DocValuesRangeFilter<Float>(field, lowerVal, upperVal, includeLower, includeUpper) {
+  public static DocValuesRangeFilter<Float> newFloatRange(String field, Float lowerVal, Float upperVal, boolean includeLower, boolean includeUpper, String desc) {
+    return new DocValuesRangeFilter<Float>(field, lowerVal, upperVal, includeLower, includeUpper, desc) {
       @Override
       public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
         // we transform the floating point numbers to sortable integers
@@ -291,8 +295,8 @@ public abstract class DocValuesRangeFilt
           float f = lowerVal.floatValue();
           if (!includeUpper && f > 0.0f && Float.isInfinite(f))
             return null;
-          int i = Document.floatToSortableInt(f);
-          inclusiveLowerPoint = Document.sortableIntToFloat(includeLower ? i : (i + 1));
+          int i = NumericUtils.floatToInt(f);
+          inclusiveLowerPoint = NumericUtils.intToFloat(includeLower ? i : (i + 1));
         } else {
           inclusiveLowerPoint = Float.NEGATIVE_INFINITY;
         }
@@ -300,8 +304,8 @@ public abstract class DocValuesRangeFilt
           float f = upperVal.floatValue();
           if (!includeUpper && f < 0.0f && Float.isInfinite(f))
             return null;
-          int i = Document.floatToSortableInt(f);
-          inclusiveUpperPoint = Document.sortableIntToFloat(includeUpper ? i : (i - 1));
+          int i = NumericUtils.floatToInt(f);
+          inclusiveUpperPoint = NumericUtils.intToFloat(includeUpper ? i : (i - 1));
         } else {
           inclusiveUpperPoint = Float.POSITIVE_INFINITY;
         }
@@ -326,8 +330,8 @@ public abstract class DocValuesRangeFilt
    * double fields containing exactly one numeric term in the field. The range can be half-open by setting one
    * of the values to <code>null</code>.
    */
-  public static DocValuesRangeFilter<Double> newDoubleRange(String field, Double lowerVal, Double upperVal, boolean includeLower, boolean includeUpper) {
-    return new DocValuesRangeFilter<Double>(field, lowerVal, upperVal, includeLower, includeUpper) {
+  public static DocValuesRangeFilter<Double> newDoubleRange(String field, Double lowerVal, Double upperVal, boolean includeLower, boolean includeUpper, String desc) {
+    return new DocValuesRangeFilter<Double>(field, lowerVal, upperVal, includeLower, includeUpper, desc) {
       @Override
       public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
         // we transform the floating point numbers to sortable integers
@@ -337,8 +341,8 @@ public abstract class DocValuesRangeFilt
           double f = lowerVal.doubleValue();
           if (!includeUpper && f > 0.0 && Double.isInfinite(f))
             return null;
-          long i = Document.doubleToSortableLong(f);
-          inclusiveLowerPoint = Document.sortableLongToDouble(includeLower ? i : (i + 1L));
+          long i = NumericUtils.doubleToLong(f);
+          inclusiveLowerPoint = NumericUtils.longToDouble(includeLower ? i : (i + 1L));
         } else {
           inclusiveLowerPoint = Double.NEGATIVE_INFINITY;
         }
@@ -346,8 +350,8 @@ public abstract class DocValuesRangeFilt
           double f = upperVal.doubleValue();
           if (!includeUpper && f < 0.0 && Double.isInfinite(f))
             return null;
-          long i = Document.doubleToSortableLong(f);
-          inclusiveUpperPoint = Document.sortableLongToDouble(includeUpper ? i : (i - 1L));
+          long i = NumericUtils.doubleToLong(f);
+          inclusiveUpperPoint = NumericUtils.longToDouble(includeUpper ? i : (i - 1L));
         } else {
           inclusiveUpperPoint = Double.POSITIVE_INFINITY;
         }
@@ -368,15 +372,77 @@ public abstract class DocValuesRangeFilt
     };
   }
   
+  /**
+   * Creates a numeric range filter using {@link org.apache.lucene.index.LeafReader#getNumericDocValues(String)}. This works with all
+   * half-float fields containing exactly one numeric term in the field. The range can be half-open by setting one
+   * of the values to <code>null</code>.
+   */
+  public static DocValuesRangeFilter<Float> newHalfFloatRange(String field, Float lowerVal, Float upperVal, boolean includeLower, boolean includeUpper, String desc) {
+    return new DocValuesRangeFilter<Float>(field, lowerVal, upperVal, includeLower, includeUpper, desc) {
+      @Override
+      public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
+        // we transform the floating point numbers to sortable shorts
+        // using NumericUtils to easier find the next bigger/lower value
+
+        final short inclusiveLowerShort;
+        if (lowerVal != null) {
+          float f = lowerVal.floatValue();
+          short s = NumericUtils.halfFloatToShort(f);
+          if (includeLower == false) {
+            if (s == Short.MAX_VALUE) {
+              return null;
+            }
+            s++;
+          }
+          inclusiveLowerShort = s;
+        } else {
+          inclusiveLowerShort = NumericUtils.halfFloatToShort(Float.NEGATIVE_INFINITY);
+        }
+
+        final short inclusiveUpperShort;
+        if (upperVal != null) {
+          float f = upperVal.floatValue();
+          short s = NumericUtils.halfFloatToShort(f);
+          if (includeUpper == false) {
+            if (s == Short.MIN_VALUE) {
+              return null;
+            }
+            s--;
+          }
+          inclusiveUpperShort = s;
+        } else {
+          inclusiveUpperShort = NumericUtils.halfFloatToShort(Float.POSITIVE_INFINITY);
+        }
+        
+        if (inclusiveLowerShort > inclusiveUpperShort) {
+          return null;
+        }
+
+        final NumericDocValues values = DocValues.getNumeric(context.reader(), field);
+        return new DocValuesDocIdSet(context.reader().maxDoc(), acceptDocs) {
+          @Override
+          protected boolean matchDoc(int doc) {
+            final short value = (short) values.get(doc);
+            return value >= inclusiveLowerShort && value <= inclusiveUpperShort;
+          }
+        };
+      }
+    };
+  }
+  
   @Override
   public final String toString() {
-    final StringBuilder sb = new StringBuilder(field).append(":");
-    return sb.append(includeLower ? '[' : '{')
-      .append((lowerVal == null) ? "*" : lowerVal.toString())
-      .append(" TO ")
-      .append((upperVal == null) ? "*" : upperVal.toString())
-      .append(includeUpper ? ']' : '}')
-      .toString();
+    if (desc == null) {
+      final StringBuilder sb = new StringBuilder(field).append(":");
+      return sb.append(includeLower ? '[' : '{')
+        .append((lowerVal == null) ? "*" : lowerVal.toString())
+        .append(" TO ")
+        .append((upperVal == null) ? "*" : upperVal.toString())
+        .append(includeUpper ? ']' : '}')
+        .toString();
+    } else {
+      return desc;
+    }
   }
 
   @Override

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java?rev=1643659&r1=1643658&r2=1643659&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java Sun Dec  7 10:52:03 2014
@@ -19,14 +19,15 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.NumericUtils;
 
 /**
  * Expert: a FieldComparator compares hits so as to determine their
@@ -277,7 +278,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public int compareBottom(int doc) {
-      double v2 = Double.longBitsToDouble(currentReaderValues.get(doc));
+      double v2 = NumericUtils.longToDouble(currentReaderValues.get(doc));
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -289,7 +290,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public void copy(int slot, int doc) {
-      double v2 = Double.longBitsToDouble(currentReaderValues.get(doc));
+      double v2 = NumericUtils.longToDouble(currentReaderValues.get(doc));
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -316,7 +317,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public int compareTop(int doc) {
-      double docValue = Double.longBitsToDouble(currentReaderValues.get(doc));
+      double docValue = NumericUtils.longToDouble(currentReaderValues.get(doc));
       // Test for docValue == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
@@ -350,7 +351,7 @@ public abstract class FieldComparator<T>
     @Override
     public int compareBottom(int doc) {
       // TODO: are there sneaky non-branch ways to compute sign of float?
-      float v2 = Float.intBitsToFloat((int)currentReaderValues.get(doc));
+      float v2 = NumericUtils.intToFloat((int)currentReaderValues.get(doc));
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -362,7 +363,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public void copy(int slot, int doc) {
-      float v2 =  Float.intBitsToFloat((int)currentReaderValues.get(doc));
+      float v2 =  NumericUtils.intToFloat((int)currentReaderValues.get(doc));
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -389,7 +390,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public int compareTop(int doc) {
-      float docValue = Float.intBitsToFloat((int)currentReaderValues.get(doc));
+      float docValue = NumericUtils.intToFloat((int)currentReaderValues.get(doc));
       // Test for docValue == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
@@ -1047,7 +1048,6 @@ public abstract class FieldComparator<T>
 
     @Override
     public int compareValues(BytesRef val1, BytesRef val2) {
-      // missing always sorts first:
       if (val1 == null) {
         if (val2 == null) {
           return 0;

Added: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LeftZeroPadTermRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LeftZeroPadTermRangeFilter.java?rev=1643659&view=auto
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LeftZeroPadTermRangeFilter.java (added)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LeftZeroPadTermRangeFilter.java Sun Dec  7 10:52:03 2014
@@ -0,0 +1,74 @@
+package org.apache.lucene.search;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.util.BytesRef;
+
+/*
+ * 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.
+ */
+
+/**
+ * A Filter that restricts search results to a range of term
+ * values in a given field.
+ *
+ * <p>This filter matches the documents looking for terms that fall into the
+ * supplied range according to {@link
+ * Byte#compareTo(Byte)}.
+ *
+ * <p>If you construct a large number of range filters with different ranges but on the 
+ * same field, {@link DocValuesRangeFilter} may have significantly better performance. 
+ * @since 2.9
+ */
+public class LeftZeroPadTermRangeFilter extends MultiTermQueryWrapperFilter<LeftZeroPadTermRangeQuery> {
+  private final String desc;
+
+  /**
+   * @param fieldName The field this range applies to
+   * @param lowerTerm The lower bound on this range
+   * @param upperTerm The upper bound on this range
+   * @param includeLower Does this range include the lower bound?
+   * @param includeUpper Does this range include the upper bound?
+   * @throws IllegalArgumentException if both terms are null or if
+   *  lowerTerm is null and includeLower is true (similar for upperTerm
+   *  and includeUpper)
+   */
+  public LeftZeroPadTermRangeFilter(String fieldName, BytesRef lowerTerm, BytesRef upperTerm,
+                         boolean includeLower, boolean includeUpper, String desc) {
+    super(new LeftZeroPadTermRangeQuery(fieldName, lowerTerm, upperTerm, includeLower, includeUpper));
+    this.desc = desc;
+  }
+
+  /** Returns the lower value of this range filter */
+  public BytesRef getLowerTerm() { return query.getLowerTerm(); }
+
+  /** Returns the upper value of this range filter */
+  public BytesRef getUpperTerm() { return query.getUpperTerm(); }
+  
+  /** Returns <code>true</code> if the lower endpoint is inclusive */
+  public boolean includesLower() { return query.includesLower(); }
+  
+  /** Returns <code>true</code> if the upper endpoint is inclusive */
+  public boolean includesUpper() { return query.includesUpper(); }
+
+  @Override
+  public String toString() {
+    if (desc == null) {
+      return super.toString();
+    } else {
+      return desc;
+    }
+  }
+}

Added: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LeftZeroPadTermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LeftZeroPadTermRangeQuery.java?rev=1643659&view=auto
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LeftZeroPadTermRangeQuery.java (added)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LeftZeroPadTermRangeQuery.java Sun Dec  7 10:52:03 2014
@@ -0,0 +1,203 @@
+package org.apache.lucene.search;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.ToStringUtils;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+
+// nocommit javadocs
+public class LeftZeroPadTermRangeQuery extends MultiTermQuery {
+  private final BytesRef lowerTerm;
+  private final BytesRef upperTerm;
+  private final boolean includeLower;
+  private final boolean includeUpper;
+
+  /**
+   * Constructs a query selecting all terms greater/equal than <code>lowerTerm</code>
+   * but less/equal than <code>upperTerm</code>. 
+   * 
+   * <p>
+   * If an endpoint is null, it is said 
+   * to be "open". Either or both endpoints may be open.  Open endpoints may not 
+   * be exclusive (you can't select all but the first or last term without 
+   * explicitly specifying the term to exclude.)
+   * 
+   * @param field The field that holds both lower and upper terms.
+   * @param lowerTerm
+   *          The term text at the lower end of the range
+   * @param upperTerm
+   *          The term text at the upper end of the range
+   * @param includeLower
+   *          If true, the <code>lowerTerm</code> is
+   *          included in the range.
+   * @param includeUpper
+   *          If true, the <code>upperTerm</code> is
+   *          included in the range.
+   */
+  public LeftZeroPadTermRangeQuery(String field, BytesRef lowerTerm, BytesRef upperTerm, boolean includeLower, boolean includeUpper) {
+    super(field);
+    this.lowerTerm = lowerTerm;
+    this.upperTerm = upperTerm;
+    this.includeLower = includeLower;
+    this.includeUpper = includeUpper;
+  }
+
+  /** Returns the lower value of this range query */
+  public BytesRef getLowerTerm() { return lowerTerm; }
+
+  /** Returns the upper value of this range query */
+  public BytesRef getUpperTerm() { return upperTerm; }
+  
+  /** Returns <code>true</code> if the lower endpoint is inclusive */
+  public boolean includesLower() { return includeLower; }
+  
+  /** Returns <code>true</code> if the upper endpoint is inclusive */
+  public boolean includesUpper() { return includeUpper; }
+  
+  private BytesRef leftZeroPad(int newLength, BytesRef term) {
+    if (term == null) {
+      return null;
+    }
+    byte[] bytes = new byte[newLength];
+    if (newLength < term.length) {
+      // This is actually OK: it means the query range is larger than what's in the index
+      return term;
+    }
+
+    int prefix = newLength - term.length;
+    for(int i=0;i<prefix;i++) {
+      bytes[i] = 0;
+    }
+    System.arraycopy(term.bytes, term.offset, bytes, prefix, term.length);
+    return new BytesRef(bytes);
+  }
+
+  @Override
+  protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
+
+    int fixedLength = terms.getMin().length;
+    assert fixedLength == terms.getMax().length;
+
+    boolean segIncludeUpper = includeUpper;
+
+    if (lowerTerm.length > fixedLength) {
+      return TermsEnum.EMPTY;
+    }
+
+    BytesRef segUpperTerm;
+    if (upperTerm.length > fixedLength) {
+      segUpperTerm = null;
+    } else {
+      segUpperTerm = leftZeroPad(fixedLength, upperTerm);
+    }
+
+    // Zero-pad by segment:
+    BytesRef segLowerTerm = leftZeroPad(fixedLength, lowerTerm);
+    
+    if (segLowerTerm != null && segUpperTerm != null && segLowerTerm.compareTo(segUpperTerm) > 0) {
+      // Matches no terms:
+      return TermsEnum.EMPTY;
+    }
+
+    if (terms.size() == 0) {
+      // No terms
+      return TermsEnum.EMPTY;
+    }
+
+    // Optimization: if our range is outside of the range indexed in this segment, skip it:
+    if (segUpperTerm != null && terms.getMin().compareTo(segUpperTerm) > 0) {
+      return TermsEnum.EMPTY;
+    }
+
+    if (segLowerTerm != null && terms.getMax().compareTo(segLowerTerm) < 0) {
+      return TermsEnum.EMPTY;
+    }      
+     
+    TermsEnum tenum = terms.iterator(null);
+    
+    if ((segLowerTerm == null || (includeLower && segLowerTerm.length == 0)) && segUpperTerm == null) {
+      // Matches all terms:
+      return terms.iterator(null);
+    }
+
+    return new CompiledAutomaton(segLowerTerm, segLowerTerm == null || includeLower, segUpperTerm, segUpperTerm == null || includeUpper).getTermsEnum(terms);
+  }
+
+  /** Prints a user-readable version of this query. */
+  @Override
+  public String toString(String field) {
+      StringBuilder buffer = new StringBuilder();
+      if (!getField().equals(field)) {
+          buffer.append(getField());
+          buffer.append(":");
+      }
+      buffer.append(includeLower ? '[' : '{');
+      // TODO: all these toStrings for queries should just output the bytes, it might not be UTF-8!
+      buffer.append(lowerTerm != null ? ("*".equals(Term.toString(lowerTerm)) ? "\\*" : Term.toString(lowerTerm))  : "*");
+      buffer.append(" TO ");
+      buffer.append(upperTerm != null ? ("*".equals(Term.toString(upperTerm)) ? "\\*" : Term.toString(upperTerm)) : "*");
+      buffer.append(includeUpper ? ']' : '}');
+      buffer.append(ToStringUtils.boost(getBoost()));
+      return buffer.toString();
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = super.hashCode();
+    result = prime * result + (includeLower ? 1231 : 1237);
+    result = prime * result + (includeUpper ? 1231 : 1237);
+    result = prime * result + ((lowerTerm == null) ? 0 : lowerTerm.hashCode());
+    result = prime * result + ((upperTerm == null) ? 0 : upperTerm.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (!super.equals(obj))
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    LeftZeroPadTermRangeQuery other = (LeftZeroPadTermRangeQuery) obj;
+    if (includeLower != other.includeLower)
+      return false;
+    if (includeUpper != other.includeUpper)
+      return false;
+    if (lowerTerm == null) {
+      if (other.lowerTerm != null)
+        return false;
+    } else if (!lowerTerm.equals(other.lowerTerm))
+      return false;
+    if (upperTerm == null) {
+      if (other.upperTerm != null)
+        return false;
+    } else if (!upperTerm.equals(other.upperTerm))
+      return false;
+    return true;
+  }
+
+}

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/Query.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/Query.java?rev=1643659&r1=1643658&r2=1643659&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/Query.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/Query.java Sun Dec  7 10:52:03 2014
@@ -36,7 +36,6 @@ import org.apache.lucene.index.Term;
     <li> {@link FuzzyQuery}
     <li> {@link RegexpQuery}
     <li> {@link TermRangeQuery}
-    <li> {@link NumericRangeQuery}
     <li> {@link ConstantScoreQuery}
     <li> {@link DisjunctionMaxQuery}
     <li> {@link MatchAllDocsQuery}

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java?rev=1643659&r1=1643658&r2=1643659&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java Sun Dec  7 10:52:03 2014
@@ -21,6 +21,7 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.util.NumericUtils;
 
 /** 
  * Selects a value from the document's list to use as the representative value 
@@ -81,14 +82,14 @@ public class SortedNumericSelector {
         return new NumericDocValues() {
           @Override
           public long get(int docID) {
-            return Document.sortableFloatBits((int) view.get(docID));
+            return (int) view.get(docID);
           }
         };
       case DOUBLE:
         return new NumericDocValues() {
           @Override
           public long get(int docID) {
-            return Document.sortableDoubleBits(view.get(docID));
+            return view.get(docID);
           }
         };
       default:

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermRangeFilter.java?rev=1643659&r1=1643658&r2=1643659&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermRangeFilter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermRangeFilter.java Sun Dec  7 10:52:03 2014
@@ -1,5 +1,6 @@
 package org.apache.lucene.search;
 
+import org.apache.lucene.document.Document;
 import org.apache.lucene.util.BytesRef;
 
 /*
@@ -98,7 +99,7 @@ public class TermRangeFilter extends Mul
     if (desc == null) {
       return super.toString();
     } else {
-      return "TermRangeFilter(field=" + getField() + " " + desc + ")";
+      return desc;
     }
   }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/BytesRef.java?rev=1643659&r1=1643658&r2=1643659&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/BytesRef.java Sun Dec  7 10:52:03 2014
@@ -178,6 +178,25 @@ public final class BytesRef implements C
     return utf8SortedAsUnicodeSortOrder;
   }
 
+  private final static Comparator<BytesRef> rightJustifiedComparator = new Comparator<BytesRef>() {
+    @Override
+    public int compare(BytesRef a, BytesRef b) {
+      //assert a.length == 0 || a.bytes[a.offset] != 0;
+      //assert b.length == 0 || b.bytes[b.offset] != 0;
+      if (a.length < b.length) {
+        return -1;
+      } else if (a.length > b.length) {
+        return 1;
+      } else {
+        return utf8SortedAsUnicodeSortOrder.compare(a, b);
+      }
+    }
+  };
+
+  public static Comparator<BytesRef> getRightJustifiedComparator() {
+    return rightJustifiedComparator;
+  }
+
   private static class UTF8SortedAsUnicodeComparator implements Comparator<BytesRef> {
     // Only singleton
     private UTF8SortedAsUnicodeComparator() {};

Copied: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/HalfFloat.java (from r1642843, lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/HalfFloat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/HalfFloat.java?p2=lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/HalfFloat.java&p1=lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/HalfFloat.java&r1=1642843&r2=1643659&rev=1643659&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/HalfFloat.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/HalfFloat.java Sun Dec  7 10:52:03 2014
@@ -1,4 +1,4 @@
-package org.apache.lucene.document;
+package org.apache.lucene.util;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -30,8 +30,7 @@ public class HalfFloat {
   private HalfFloat() {
   }
 
-  // Only uses bottom 16 bits:
-  public static float intBitsToFloat(int hbits) {
+  public static float shortBitsToFloat(short hbits) {
     int mant = hbits & 0x03ff;          // 10 bits mantissa
     int exp = hbits & 0x7c00;           // 5 bits exponent
     if (exp == 0x7c00) {                // NaN/Inf
@@ -56,7 +55,7 @@ public class HalfFloat {
   }
 
   // Only uses bottom 16 bits:
-  public static int floatToIntBits(float fval) {
+  public static short floatToShortBits(float fval) {
     int fbits = Float.floatToIntBits(fval);
     int sign = fbits >>> 16 & 0x8000;           // sign only
     int val = (fbits & 0x7fffffff) + 0x1000;    // rounded value
@@ -64,23 +63,23 @@ public class HalfFloat {
       // avoid Inf due to rounding
       if ((fbits & 0x7fffffff) >= 0x47800000) { // is or must become NaN/Inf
         if (val < 0x7f800000) {             // was value but too large
-          return sign | 0x7c00;           // make it +/-Inf
+          return (short) (sign | 0x7c00);           // make it +/-Inf
         }
-        return sign | 0x7c00 |              // remains +/-Inf or NaN
-          (fbits & 0x007fffff) >>> 13; // keep NaN (and Inf) bits
+        return (short) (sign | 0x7c00 |              // remains +/-Inf or NaN
+                        (fbits & 0x007fffff) >>> 13); // keep NaN (and Inf) bits
       }
-      return sign | 0x7bff;                   // unrounded not quite Inf
+      return (short) (sign | 0x7bff);                   // unrounded not quite Inf
     }
     if (val >= 0x38800000) {                    // remains normalized value
-      return sign | val - 0x38000000 >>> 13;  // exp - 127 + 15
+      return (short) (sign | val - 0x38000000 >>> 13);  // exp - 127 + 15
     }
     if (val < 0x33000000) {                     // too small for subnormal
-      return sign;                            // becomes +/-0
+      return (short) sign;                            // becomes +/-0
     }
     val = (fbits & 0x7fffffff) >>> 23;          // tmp exp for subnormal calc
-    return sign | ((fbits & 0x7fffff | 0x800000) // add subnormal bit
-                   + (0x800000 >>> val - 102)          // round depending on cut off
-                   >>> 126 - val); // div by 2^(1-(exp-127+15)) and >> 13 | exp=0
+    return (short) (sign | ((fbits & 0x7fffff | 0x800000) // add subnormal bit
+                            + (0x800000 >>> val - 102)          // round depending on cut off
+                            >>> 126 - val)); // div by 2^(1-(exp-127+15)) and >> 13 | exp=0
   }
 }
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java?rev=1643659&r1=1643658&r2=1643659&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java Sun Dec  7 10:52:03 2014
@@ -18,244 +18,46 @@ package org.apache.lucene.util;
  */
 
 import java.io.IOException;
+import java.math.BigInteger;
 
-import org.apache.lucene.analysis.NumericTokenStream;
 import org.apache.lucene.index.FilterLeafReader;
 import org.apache.lucene.index.FilteredTermsEnum;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.NumericRangeFilter;
-import org.apache.lucene.search.NumericRangeQuery; // for javadocs
 
 /**
  * This is a helper class to generate prefix-encoded representations for numerical values
  * and supplies converters to represent float/double values as sortable integers/longs.
  *
- * <p>To quickly execute range queries in Apache Lucene, a range is divided recursively
- * into multiple intervals for searching: The center of the range is searched only with
- * the lowest possible precision in the trie, while the boundaries are matched
- * more exactly. This reduces the number of terms dramatically.
- *
- * <p>This class generates terms to achieve this: First the numerical integer values need to
- * be converted to bytes. For that integer values (32 bit or 64 bit) are made unsigned
- * and the bits are converted to ASCII chars with each 7 bit. The resulting byte[] is
- * sortable like the original integer value (even using UTF-8 sort order). Each value is also
- * 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
+ * to integer values by changing their bit layout: {@link #doubleToLong},
+ * {@link #floatToInt}. 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 NumericTokenStream} that can index <code>int</code>, <code>long</code>,
- * <code>float</code>, and <code>double</code>. For querying,
- * {@link NumericRangeQuery} and {@link NumericRangeFilter} implement the query part
- * for the same data types.
- *
- * <p>This class can also be used, to generate lexicographically sortable (according to
- * {@link BytesRef#getUTF8SortedAsUTF16Comparator()}) representations of numeric data
- * types for other usages (e.g. sorting).
- *
  * @lucene.internal
  * @since 2.9, API changed non backwards-compliant in 4.0
  */
 public final class NumericUtils {
 
   private NumericUtils() {} // no instance!
-  
-  /**
-   * The default precision step used by {@link LongField},
-   * {@link DoubleField}, {@link NumericTokenStream}, {@link
-   * NumericRangeQuery}, and {@link NumericRangeFilter}.
-   */
-  public static final int PRECISION_STEP_DEFAULT = 16;
-  
-  /**
-   * The default precision step used by {@link IntField} and
-   * {@link FloatField}.
-   */
-  public static final int PRECISION_STEP_DEFAULT_32 = 8;
-  
-  /**
-   * Longs are stored at lower precision by shifting off lower bits. The shift count is
-   * stored as <code>SHIFT_START_LONG+shift</code> in the first byte
-   */
-  public static final byte SHIFT_START_LONG = 0x20;
 
-  /**
-   * The maximum term length (used for <code>byte[]</code> buffer size)
-   * for encoding <code>long</code> values.
-   * @see #longToPrefixCodedBytes
-   */
-  public static final int BUF_SIZE_LONG = 63/7 + 2;
-
-  /**
-   * Integers are stored at lower precision by shifting off lower bits. The shift count is
-   * stored as <code>SHIFT_START_INT+shift</code> in the first byte
-   */
-  public static final byte SHIFT_START_INT  = 0x60;
-
-  /**
-   * The maximum term length (used for <code>byte[]</code> buffer size)
-   * for encoding <code>int</code> values.
-   * @see #intToPrefixCodedBytes
-   */
-  public static final int BUF_SIZE_INT = 31/7 + 2;
-
-  /**
-   * Returns prefix coded bits after reducing the precision by <code>shift</code> bits.
-   * This is method is used by {@link NumericTokenStream}.
-   * After encoding, {@code bytes.offset} will always be 0. 
-   * @param val the numeric value
-   * @param shift how many bits to strip from the right
-   * @param bytes will contain the encoded value
-   */
-  public static void longToPrefixCoded(final long val, final int shift, final BytesRefBuilder bytes) {
-    longToPrefixCodedBytes(val, shift, bytes);
+  public static short halfFloatToShort(float value) {
+    return sortableHalfFloatBits((short) HalfFloat.floatToShortBits(value));
   }
 
   /**
-   * Returns prefix coded bits after reducing the precision by <code>shift</code> bits.
-   * This is method is used by {@link NumericTokenStream}.
-   * After encoding, {@code bytes.offset} will always be 0.
-   * @param val the numeric value
-   * @param shift how many bits to strip from the right
-   * @param bytes will contain the encoded value
-   */
-  public static void intToPrefixCoded(final int val, final int shift, final BytesRefBuilder bytes) {
-    intToPrefixCodedBytes(val, shift, bytes);
-  }
-
-  /**
-   * Returns prefix coded bits after reducing the precision by <code>shift</code> bits.
-   * This is method is used by {@link NumericTokenStream}.
-   * After encoding, {@code bytes.offset} will always be 0.
-   * @param val the numeric value
-   * @param shift how many bits to strip from the right
-   * @param bytes will contain the encoded value
-   */
-  public static void longToPrefixCodedBytes(final long val, final int shift, final BytesRefBuilder bytes) {
-    // ensure shift is 0..63
-    if ((shift & ~0x3f) != 0) {
-      throw new IllegalArgumentException("Illegal shift value, must be 0..63; got shift=" + shift);
-    }
-    int nChars = (((63-shift)*37)>>8) + 1;    // i/7 is the same as (i*37)>>8 for i in 0..63
-    bytes.setLength(nChars+1);   // one extra for the byte that contains the shift info
-    bytes.grow(BUF_SIZE_LONG);
-    bytes.setByteAt(0, (byte)(SHIFT_START_LONG + shift));
-    long sortableBits = val ^ 0x8000000000000000L;
-    sortableBits >>>= shift;
-    while (nChars > 0) {
-      // Store 7 bits per byte for compatibility
-      // with UTF-8 encoding of terms
-      bytes.setByteAt(nChars--, (byte)(sortableBits & 0x7f));
-      sortableBits >>>= 7;
-    }
-  }
-
-
-  /**
-   * Returns prefix coded bits after reducing the precision by <code>shift</code> bits.
-   * This is method is used by {@link NumericTokenStream}.
-   * After encoding, {@code bytes.offset} will always be 0. 
-   * @param val the numeric value
-   * @param shift how many bits to strip from the right
-   * @param bytes will contain the encoded value
-   */
-  public static void intToPrefixCodedBytes(final int val, final int shift, final BytesRefBuilder bytes) {
-    // ensure shift is 0..31
-    if ((shift & ~0x1f) != 0) {
-      throw new IllegalArgumentException("Illegal shift value, must be 0..31; got shift=" + shift);
-    }
-    int nChars = (((31-shift)*37)>>8) + 1;    // i/7 is the same as (i*37)>>8 for i in 0..63
-    bytes.setLength(nChars+1);   // one extra for the byte that contains the shift info
-    bytes.grow(NumericUtils.BUF_SIZE_LONG);  // use the max
-    bytes.setByteAt(0, (byte)(SHIFT_START_INT + shift));
-    int sortableBits = val ^ 0x80000000;
-    sortableBits >>>= shift;
-    while (nChars > 0) {
-      // Store 7 bits per byte for compatibility
-      // with UTF-8 encoding of terms
-      bytes.setByteAt(nChars--, (byte)(sortableBits & 0x7f));
-      sortableBits >>>= 7;
-    }
-  }
-
-
-  /**
-   * Returns the shift value from a prefix encoded {@code long}.
-   * @throws NumberFormatException if the supplied {@link BytesRef} is
-   * not correctly prefix encoded.
-   */
-  public static int getPrefixCodedLongShift(final BytesRef val) {
-    final int shift = val.bytes[val.offset] - SHIFT_START_LONG;
-    if (shift > 63 || shift < 0)
-      throw new NumberFormatException("Invalid shift value (" + shift + ") in prefixCoded bytes (is encoded value really an INT?)");
-    return shift;
-  }
-
-  /**
-   * Returns the shift value from a prefix encoded {@code int}.
-   * @throws NumberFormatException if the supplied {@link BytesRef} is
-   * not correctly prefix encoded.
-   */
-  public static int getPrefixCodedIntShift(final BytesRef val) {
-    final int shift = val.bytes[val.offset] - SHIFT_START_INT;
-    if (shift > 31 || shift < 0)
-      throw new NumberFormatException("Invalid shift value in prefixCoded bytes (is encoded value really an INT?)");
-    return shift;
-  }
-
-  /**
-   * Returns a long from prefixCoded bytes.
-   * Rightmost bits will be zero for lower precision codes.
-   * This method can be used to decode a term's value.
-   * @throws NumberFormatException if the supplied {@link BytesRef} is
-   * not correctly prefix encoded.
-   * @see #longToPrefixCodedBytes
-   */
-  public static long prefixCodedToLong(final BytesRef val) {
-    long sortableBits = 0L;
-    for (int i=val.offset+1, limit=val.offset+val.length; i<limit; i++) {
-      sortableBits <<= 7;
-      final byte b = val.bytes[i];
-      if (b < 0) {
-        throw new NumberFormatException(
-          "Invalid prefixCoded numerical value representation (byte "+
-          Integer.toHexString(b&0xff)+" at position "+(i-val.offset)+" is invalid)"
-        );
-      }
-      sortableBits |= b;
-    }
-    return (sortableBits << getPrefixCodedLongShift(val)) ^ 0x8000000000000000L;
-  }
-
-  /**
-   * Returns an int from prefixCoded bytes.
-   * Rightmost bits will be zero for lower precision codes.
-   * This method can be used to decode a term's value.
-   * @throws NumberFormatException if the supplied {@link BytesRef} is
-   * not correctly prefix encoded.
-   * @see #intToPrefixCodedBytes
+   * 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 #intToFloat
    */
-  public static int prefixCodedToInt(final BytesRef val) {
-    int sortableBits = 0;
-    for (int i=val.offset+1, limit=val.offset+val.length; i<limit; i++) {
-      sortableBits <<= 7;
-      final byte b = val.bytes[i];
-      if (b < 0) {
-        throw new NumberFormatException(
-          "Invalid prefixCoded numerical value representation (byte "+
-          Integer.toHexString(b&0xff)+" at position "+(i-val.offset)+" is invalid)"
-        );
-      }
-      sortableBits |= b;
-    }
-    return (sortableBits << getPrefixCodedIntShift(val)) ^ 0x80000000;
+  public static int floatToInt(float val) {
+    return sortableFloatBits(Float.floatToIntBits(val));
   }
 
   /**
@@ -265,318 +67,162 @@ public final class NumericUtils {
    * 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
+   * @see #longToDouble
    */
-  public static long doubleToSortableLong(double val) {
+  public static long doubleToLong(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));
+  public static float shortToHalfFloat(short v) {
+    return HalfFloat.shortBitsToFloat(sortableHalfFloatBits(v));
   }
 
   /**
    * Converts a sortable <code>int</code> back to a <code>float</code>.
-   * @see #floatToSortableInt
+   * @see #floatToInt
    */
-  public static float sortableIntToFloat(int val) {
+  public static float intToFloat(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 a sortable <code>long</code> back to a <code>double</code>.
+   * @see #doubleToLong
+   */
+  public static double longToDouble(long val) {
+    return Double.longBitsToDouble(sortableDoubleBits(val));
   }
-  
+
+  /** Converts IEEE 754 representation of a half float to sortable order (or back to the original) */
+  public static short sortableHalfFloatBits(short bits) {
+    return (short) (bits ^ (bits >> 15) & 0x7fff);
+  }
+
   /** 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
-   * {@link LongRangeBuilder#addRange(BytesRef,BytesRef)}
-   * method.
-   * <p>This method is used by {@link NumericRangeQuery}.
-   */
-  public static void splitLongRange(final LongRangeBuilder builder,
-    final int precisionStep,  final long minBound, final long maxBound
-  ) {
-    splitRange(builder, 64, precisionStep, minBound, maxBound);
-  }
-  
-  /**
-   * Splits an int range recursively.
-   * You may implement a builder that adds clauses to a
-   * {@link org.apache.lucene.search.BooleanQuery} for each call to its
-   * {@link IntRangeBuilder#addRange(BytesRef,BytesRef)}
-   * method.
-   * <p>This method is used by {@link NumericRangeQuery}.
-   */
-  public static void splitIntRange(final IntRangeBuilder builder,
-    final int precisionStep,  final int minBound, final int maxBound
-  ) {
-    splitRange(builder, 32, precisionStep, minBound, maxBound);
+  /** 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;
   }
   
-  /** This helper does the splitting for both 32 and 64 bit. */
-  private static void splitRange(
-    final Object builder, final int valSize,
-    final int precisionStep, long minBound, long maxBound
-  ) {
-    if (precisionStep < 1)
-      throw new IllegalArgumentException("precisionStep must be >=1");
-    if (minBound > maxBound) return;
-    for (int shift=0; ; shift += precisionStep) {
-      // calculate new bounds for inner precision
-      final long diff = 1L << (shift+precisionStep),
-        mask = ((1L<<precisionStep) - 1L) << shift;
-      final boolean
-        hasLower = (minBound & mask) != 0L,
-        hasUpper = (maxBound & mask) != mask;
-      final long
-        nextMinBound = (hasLower ? (minBound + diff) : minBound) & ~mask,
-        nextMaxBound = (hasUpper ? (maxBound - diff) : maxBound) & ~mask;
-      final boolean
-        lowerWrapped = nextMinBound < minBound,
-        upperWrapped = nextMaxBound > maxBound;
-      
-      if (shift+precisionStep>=valSize || nextMinBound>nextMaxBound || lowerWrapped || upperWrapped) {
-        // We are in the lowest precision or the next precision is not available.
-        addRange(builder, valSize, minBound, maxBound, shift);
-        // exit the split recursion loop
-        break;
-      }
-      
-      if (hasLower)
-        addRange(builder, valSize, minBound, minBound | mask, shift);
-      if (hasUpper)
-        addRange(builder, valSize, maxBound & ~mask, maxBound, shift);
-      
-      // recurse to next precision
-      minBound = nextMinBound;
-      maxBound = nextMaxBound;
+  public static short bytesToShort(BytesRef bytes) {
+    if (bytes.length != 2) {
+      throw new IllegalArgumentException("incoming bytes should be length=2; got length=" + bytes.length);
     }
-  }
-  
-  /** Helper that delegates to correct range builder */
-  private static void addRange(
-    final Object builder, final int valSize,
-    long minBound, long maxBound,
-    final int shift
-  ) {
-    // for the max bound set all lower bits (that were shifted away):
-    // this is important for testing or other usages of the splitted range
-    // (e.g. to reconstruct the full range). The prefixEncoding will remove
-    // the bits anyway, so they do not hurt!
-    maxBound |= (1L << shift) - 1L;
-    // delegate to correct range builder
-    switch(valSize) {
-      case 64:
-        ((LongRangeBuilder)builder).addRange(minBound, maxBound, shift);
-        break;
-      case 32:
-        ((IntRangeBuilder)builder).addRange((int)minBound, (int)maxBound, shift);
-        break;
-      default:
-        // Should not happen!
-        throw new IllegalArgumentException("valSize must be 32 or 64.");
+    short sortableBits = 0;
+    for(int i=0;i<2;i++) {
+      sortableBits = (short) ((sortableBits << 8) | bytes.bytes[bytes.offset + i] & 0xff);
     }
+
+    return (short) (sortableBits ^ 0x8000);
   }
 
-  /**
-   * Callback for {@link #splitLongRange}.
-   * You need to overwrite only one of the methods.
-   * @lucene.internal
-   * @since 2.9, API changed non backwards-compliant in 4.0
-   */
-  public static abstract class LongRangeBuilder {
-    
-    /**
-     * Overwrite this method, if you like to receive the already prefix encoded range bounds.
-     * You can directly build classical (inclusive) range queries from them.
-     */
-    public void addRange(BytesRef minPrefixCoded, BytesRef maxPrefixCoded) {
-      throw new UnsupportedOperationException();
+  public static int bytesToInt(BytesRef bytes) {
+    if (bytes.length != 4) {
+      throw new IllegalArgumentException("incoming bytes should be length=4; got length=" + bytes.length);
     }
-    
-    /**
-     * Overwrite this method, if you like to receive the raw long range bounds.
-     * You can use this for e.g. debugging purposes (print out range bounds).
-     */
-    public void addRange(final long min, final long max, final int shift) {
-      final BytesRefBuilder minBytes = new BytesRefBuilder(), maxBytes = new BytesRefBuilder();
-      longToPrefixCodedBytes(min, shift, minBytes);
-      longToPrefixCodedBytes(max, shift, maxBytes);
-      addRange(minBytes.get(), maxBytes.get());
+    int sortableBits = 0;
+    for(int i=0;i<4;i++) {
+      sortableBits = (sortableBits << 8) | bytes.bytes[bytes.offset + i] & 0xff;
     }
-  
+
+    return sortableBits ^ 0x80000000;
   }
-  
-  /**
-   * Callback for {@link #splitIntRange}.
-   * You need to overwrite only one of the methods.
-   * @lucene.internal
-   * @since 2.9, API changed non backwards-compliant in 4.0
-   */
-  public static abstract class IntRangeBuilder {
-    
-    /**
-     * Overwrite this method, if you like to receive the already prefix encoded range bounds.
-     * You can directly build classical range (inclusive) queries from them.
-     */
-    public void addRange(BytesRef minPrefixCoded, BytesRef maxPrefixCoded) {
-      throw new UnsupportedOperationException();
+
+  public static long bytesToLong(BytesRef bytes) {
+    if (bytes.length != 8) {
+      throw new IllegalArgumentException("incoming bytes should be length=8; got length=" + bytes.length);
     }
-    
-    /**
-     * Overwrite this method, if you like to receive the raw int range bounds.
-     * You can use this for e.g. debugging purposes (print out range bounds).
-     */
-    public void addRange(final int min, final int max, final int shift) {
-      final BytesRefBuilder minBytes = new BytesRefBuilder(), maxBytes = new BytesRefBuilder();
-      intToPrefixCodedBytes(min, shift, minBytes);
-      intToPrefixCodedBytes(max, shift, maxBytes);
-      addRange(minBytes.get(), maxBytes.get());
+    long sortableBits = 0;
+    for(int i=0;i<8;i++) {
+      sortableBits = (sortableBits << 8) | bytes.bytes[bytes.offset + i] & 0xff;
     }
-  
+
+    return sortableBits ^ 0x8000000000000000L;
   }
-  
-  /**
-   * Filters the given {@link TermsEnum} by accepting only prefix coded 64 bit
-   * terms with a shift value of <tt>0</tt>.
-   * 
-   * @param termsEnum
-   *          the terms enum to filter
-   * @return a filtered {@link TermsEnum} that only returns prefix coded 64 bit
-   *         terms with a shift value of <tt>0</tt>.
-   */
-  public static TermsEnum filterPrefixCodedLongs(TermsEnum termsEnum) {
-    return new SeekingNumericFilteredTermsEnum(termsEnum) {
 
-      @Override
-      protected AcceptStatus accept(BytesRef term) {
-        return NumericUtils.getPrefixCodedLongShift(term) == 0 ? AcceptStatus.YES : AcceptStatus.END;
-      }
-    };
+  public static BytesRef shortToBytes(short v) {
+    int sortableBits = v ^ 0x8000;
+    BytesRef token = new BytesRef(2);
+    token.length = 2;
+    int index = 1;
+    while (index >= 0) {
+      token.bytes[index] = (byte) (sortableBits & 0xff);
+      index--;
+      sortableBits >>>= 8;
+    }
+    return token;
   }
 
-  /**
-   * Filters the given {@link TermsEnum} by accepting only prefix coded 32 bit
-   * terms with a shift value of <tt>0</tt>.
-   * 
-   * @param termsEnum
-   *          the terms enum to filter
-   * @return a filtered {@link TermsEnum} that only returns prefix coded 32 bit
-   *         terms with a shift value of <tt>0</tt>.
-   */
-  public static TermsEnum filterPrefixCodedInts(TermsEnum termsEnum) {
-    return new SeekingNumericFilteredTermsEnum(termsEnum) {
-      
-      @Override
-      protected AcceptStatus accept(BytesRef term) {
-        return NumericUtils.getPrefixCodedIntShift(term) == 0 ? AcceptStatus.YES : AcceptStatus.END;
-      }
-    };
-  }
-
-  /** Just like FilteredTermsEnum, except it adds a limited
-   *  seekCeil implementation that only works with {@link
-   *  #filterPrefixCodedInts} and {@link
-   *  #filterPrefixCodedLongs}. */
-  private static abstract class SeekingNumericFilteredTermsEnum extends FilteredTermsEnum {
-    public SeekingNumericFilteredTermsEnum(final TermsEnum tenum) {
-      super(tenum, false);
+  public static BytesRef intToBytes(int v) {
+    int sortableBits = v ^ 0x80000000;
+    BytesRef token = new BytesRef(4);
+    token.length = 4;
+    int index = 3;
+    while (index >= 0) {
+      token.bytes[index] = (byte) (sortableBits & 0xff);
+      index--;
+      sortableBits >>>= 8;
     }
+    return token;
+  }
 
-    @Override
-    @SuppressWarnings("fallthrough")
-    public SeekStatus seekCeil(BytesRef term) throws IOException {
-
-      // NOTE: This is not general!!  It only handles YES
-      // and END, because that's all we need for the numeric
-      // case here
-
-      SeekStatus status = tenum.seekCeil(term);
-      if (status == SeekStatus.END) {
-        return SeekStatus.END;
-      }
-
-      actualTerm = tenum.term();
-
-      if (accept(actualTerm) == AcceptStatus.YES) {
-        return status;
-      } else {
-        return SeekStatus.END;
-      }
+  public static BytesRef longToBytes(long v) {
+    long sortableBits = v ^ 0x8000000000000000L;
+    BytesRef token = new BytesRef(8);
+    token.length = 8;
+    int index = 7;
+    while (index >= 0) {
+      token.bytes[index] = (byte) (sortableBits & 0xff);
+      index--;
+      sortableBits >>>= 8;
     }
+    return token;
   }
 
-  private static Terms intTerms(Terms terms) {
-    return new FilterLeafReader.FilterTerms(terms) {
-        @Override
-        public TermsEnum iterator(TermsEnum reuse) throws IOException {
-          return filterPrefixCodedInts(in.iterator(reuse));
-        }
-      };
-  }
-
-  private static Terms longTerms(Terms terms) {
-    return new FilterLeafReader.FilterTerms(terms) {
-        @Override
-        public TermsEnum iterator(TermsEnum reuse) throws IOException {
-          return filterPrefixCodedLongs(in.iterator(reuse));
-        }
-      };
-  }
-    
-  /** Returns the minimum int value indexed into this
-   *  numeric field. */
-  public static int getMinInt(Terms terms) throws IOException {
-    // All shift=0 terms are sorted first, so we don't need
-    // to filter the incoming terms; we can just get the
-    // min: 
-    return NumericUtils.prefixCodedToInt(terms.getMin());
-  }
-
-  /** Returns the maximum int value indexed into this
-   *  numeric field. */
-  public static int getMaxInt(Terms terms) throws IOException {
-    return NumericUtils.prefixCodedToInt(intTerms(terms).getMax());
-  }
-
-  /** Returns the minimum long value indexed into this
-   *  numeric field. */
-  public static long getMinLong(Terms terms) throws IOException {
-    // All shift=0 terms are sorted first, so we don't need
-    // to filter the incoming terms; we can just get the
-    // min: 
-    return NumericUtils.prefixCodedToLong(terms.getMin());
-  }
-
-  /** Returns the maximum long value indexed into this
-   *  numeric field. */
-  public static long getMaxLong(Terms terms) throws IOException {
-    return NumericUtils.prefixCodedToLong(longTerms(terms).getMax());
+  public static BytesRef halfFloatToBytes(float value) {
+    return shortToBytes(halfFloatToShort(value));
+  }
+
+  public static BytesRef floatToBytes(float value) {
+    return intToBytes(floatToInt(value));
+  }
+
+  public static BytesRef doubleToBytes(double value) {
+    return longToBytes(doubleToLong(value));
+  }
+
+  public static float bytesToHalfFloat(BytesRef bytes) {
+    return HalfFloat.shortBitsToFloat(sortableHalfFloatBits(bytesToShort(bytes)));
+  }
+
+  public static float bytesToFloat(BytesRef bytes) {
+    return intToFloat(bytesToInt(bytes));
+  }
+
+  public static double bytesToDouble(BytesRef bytes) {
+    return longToDouble(bytesToLong(bytes));
+  }
+
+  public static BytesRef bigIntToBytes(BigInteger value) {
+    byte[] bytes = value.toByteArray();
+    sortableBigIntBytes(bytes);
+    //System.out.println(value + " -> " + new BytesRef(bytes));
+    return new BytesRef(bytes);
+  }
+
+  public static BigInteger bytesToBigInt(BytesRef bytes) {
+    byte[] copy = new byte[bytes.length];
+    System.arraycopy(bytes.bytes, bytes.offset, copy, 0, bytes.length);
+    sortableBigIntBytes(copy);
+    return new BigInteger(copy);
+  }
+
+  private static void sortableBigIntBytes(byte[] bytes) {
+    // nocommit does NOT work
+    //    bytes[0] ^= 0x80;
   }
-  
 }

Added: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestBigInteger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestBigInteger.java?rev=1643659&view=auto
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestBigInteger.java (added)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestBigInteger.java Sun Dec  7 10:52:03 2014
@@ -0,0 +1,221 @@
+package org.apache.lucene.document;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Set;
+
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestBigInteger extends LuceneTestCase {
+  // nocommit test multi-valued too
+  public void testRange1() throws Exception {
+    Directory dir = newDirectory();
+
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
+    FieldTypes fieldTypes = w.getFieldTypes();
+    //System.out.println("id type: " + fieldTypes.getFieldType("id"));
+
+    // nocommit need random test, across segments, with merging
+    Document doc = w.newDocument();
+    doc.addBigInteger("big", new BigInteger("3000000000000000000"));
+    doc.addAtom("id", "one");
+    w.addDocument(doc);
+
+    doc = w.newDocument();
+    doc.addBigInteger("big", new BigInteger("2000000000000000000"));
+    doc.addAtom("id", "two");
+    w.addDocument(doc);
+
+    doc = w.newDocument();
+    doc.addBigInteger("big", new BigInteger("7000000000000000000"));
+    doc.addAtom("id", "three");
+    w.addDocument(doc);
+
+    IndexReader r = DirectoryReader.open(w, true);
+    IndexSearcher s = newSearcher(r);
+
+    // Make sure range query hits the right number of hits
+    assertEquals(2, s.search(new MatchAllDocsQuery(),
+                             fieldTypes.newBigIntRangeFilter("big",
+                                                             new BigInteger("0"), true,
+                                                             new BigInteger("3000000000000000000"), true), 1).totalHits);
+    assertEquals(3, s.search(new MatchAllDocsQuery(),
+                             fieldTypes.newBigIntRangeFilter("big",
+                                                             new BigInteger("0"), true,
+                                                             new BigInteger("10000000000000000000"), true), 1).totalHits);
+    TopDocs hits = s.search(new MatchAllDocsQuery(),
+                            fieldTypes.newBigIntRangeFilter("big",
+                                                            new BigInteger("1000000000000000000"), true,
+                                                            new BigInteger("2500000000000000000"), true), 1);
+    assertEquals(1, hits.totalHits);
+    assertEquals(new BigInteger("2000000000000000000"), s.doc(hits.scoreDocs[0].doc).getBigInteger("big"));
+
+    doc = w.newDocument();
+    doc.addBigInteger("big", new BigInteger("17"));
+    doc.addAtom("id", "four");
+    w.addDocument(doc);
+    w.forceMerge(1);
+    r.close();
+    r = DirectoryReader.open(w, true);
+    s = newSearcher(r);
+
+    assertEquals(4, s.search(new MatchAllDocsQuery(),
+                             fieldTypes.newBigIntRangeFilter("big",
+                                                             new BigInteger("0"), true,
+                                                             new BigInteger("10000000000000000000"), true), 1).totalHits);
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  /*
+  public void testFoo() throws Exception {
+    for(int i=-257;i<=257;i++) {
+      byte[] bytes = BigInteger.valueOf(i).toByteArray();
+      if (bytes.length == 2) {
+        System.out.println(String.format(Locale.ROOT, "%d -> %x %x", i, bytes[0] & 0xff, bytes[1] & 0xff));
+      } else {
+        System.out.println(String.format(Locale.ROOT, "%d -> %x", i, bytes[0] & 0xff));
+      }
+    }
+  }
+  */
+
+  /*
+  public void testBigIntRange2() throws Exception {
+    Directory dir = newDirectory();
+
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
+    FieldTypes fieldTypes = w.getFieldTypes();
+    //System.out.println("id type: " + fieldTypes.getFieldType("id"));
+
+    // nocommit need random test, across segments, with merging
+    Document doc = w.newDocument();
+    doc.addBigInteger("big", new BigInteger("-100"));
+    doc.addAtom("id", "one");
+    w.addDocument(doc);
+
+    doc = w.newDocument();
+    doc.addBigInteger("big", new BigInteger("200"));
+    doc.addAtom("id", "two");
+    w.addDocument(doc);
+
+    IndexReader r = DirectoryReader.open(w, true);
+    IndexSearcher s = newSearcher(r);
+
+    // nocommit add & test TermQuery
+
+    // Make sure range query hits the right number of hits
+    assertEquals(1, s.search(new MatchAllDocsQuery(),
+                             fieldTypes.newRangeFilter("big",
+                                                       new BigInteger("-200"), true,
+                                                       new BigInteger("17"), true), 1).totalHits);
+    r.close();
+    w.close();
+    dir.close();
+  }
+  */
+
+  // nocommit sorting
+
+  private BigInteger randomBigInt() {
+    BigInteger big = new BigInteger(TestUtil.nextInt(random(), 4, 100), random()); 
+    // nocommit only positive for now
+    if (false && random().nextBoolean()) {
+      big = big.negate();
+    }
+    return big;
+  }
+
+  public void testRandom() throws Exception {
+    Directory dir = newDirectory();
+    int numDocs = atLeast(100);
+    RandomIndexWriter w = newRandomIndexWriter(dir);
+    List<BigInteger> values = new ArrayList<>();
+    for(int i=0;i<numDocs;i++) {
+      Document doc = w.newDocument();
+      doc.addUniqueInt("id", i);
+      BigInteger big = randomBigInt();
+      values.add(big);
+      doc.addBigInteger("big", big);
+      w.addDocument(doc);
+      if (VERBOSE) {
+        System.out.println("id=" + i + " big=" + big);
+      }
+    }
+
+    IndexReader r = w.getReader();
+    FieldTypes fieldTypes = r.getFieldTypes();
+
+    IndexSearcher s = newSearcher(r);
+    int iters = atLeast(1000);
+    for(int iter=0;iter<iters;iter++) {
+      BigInteger x = randomBigInt();
+      BigInteger y = randomBigInt();
+
+      BigInteger min, max;
+      if (x.compareTo(y) < 0) {
+        min = x;
+        max = y;
+      } else {
+        min = y;
+        max = x;
+      }
+      Set<Integer> expected = new HashSet<>();
+      for(int i=0;i<values.size();i++) {
+        BigInteger value = values.get(i);
+        if (value.compareTo(min) >= 0 && value.compareTo(max) <= 0) {
+          expected.add(i);
+        }
+      }
+      if (VERBOSE) {
+        System.out.println("iter " + iter + " count=" + expected.size() + " min=" + min + " max=" + max);
+        for(int value : expected) {
+          System.out.println("  " + value);
+        }
+      }
+      
+      Set<Integer> actual = new HashSet<>();
+      TopDocs hits = s.search(new MatchAllDocsQuery(), fieldTypes.newBigIntRangeFilter("big", min, true, max, true), numDocs);
+      for(ScoreDoc hit : hits.scoreDocs) {
+        actual.add(s.doc(hit.doc).getInt("id"));
+      }
+      assertEquals(expected, actual);
+    }
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+}