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/04/24 16:40:02 UTC

svn commit: r1589749 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/codecs/ lucene/codecs/src/java/org/apache/lucene/codecs/bloom/ lucene/core/ lucene/core/src/java/org/apache/lucene/codecs/ lucene/core/src/java/org/apache/lucene/index/ lucene/...

Author: mikemccand
Date: Thu Apr 24 14:40:01 2014
New Revision: 1589749

URL: http://svn.apache.org/r1589749
Log:
LUCENE-5610: add Terms.getMin/Max

Added:
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestTerms.java
      - copied unchanged from r1589729, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTerms.java
Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/lucene/codecs/   (props changed)
    lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java
    lucene/dev/branches/branch_4x/lucene/core/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/Terms.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
    lucene/dev/branches/branch_4x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1589749&r1=1589748&r2=1589749&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Thu Apr 24 14:40:01 2014
@@ -5,6 +5,13 @@ http://s.apache.org/luceneversions
 
 ======================= Lucene 4.9.0 =======================
 
+New Features
+
+* LUCENE-5610: Add Terms.getMin and Terms.getMax to get the lowest and
+  highest terms, and NumericUtils.get{Min/Max}{Int/Long} to get the
+  minimum numeric values from the provided Terms.  (Robert Muir, Mike
+  McCandless)
+
 API Changes
 
 * LUCENE-5582: Deprecate IndexOutput.length (just use
@@ -21,7 +28,7 @@ Optimizations
   
 * LUCENE-5599: HttpReplicator did not properly delegate bulk read() to wrapped
   InputStream. (Christoph Kaser via Shai Erera)
-  
+
 Bug fixes
 
 * LUCENE-5600: HttpClientBase did not properly consume a connection if a server

Modified: lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java?rev=1589749&r1=1589748&r2=1589749&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java Thu Apr 24 14:40:01 2014
@@ -306,6 +306,16 @@ public final class BloomFilteringPosting
       public boolean hasPayloads() {
         return delegateTerms.hasPayloads();
       }
+
+      @Override
+      public BytesRef getMin() throws IOException {
+        return delegateTerms.getMin();
+      }
+
+      @Override
+      public BytesRef getMax() throws IOException {
+        return delegateTerms.getMax();
+      }
     }
     
     final class BloomFilteredTermsEnum extends TermsEnum {

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java?rev=1589749&r1=1589748&r2=1589749&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java Thu Apr 24 14:40:01 2014
@@ -168,6 +168,14 @@ public class BlockTreeTermsReader extend
         final long sumDocFreq = in.readVLong();
         final int docCount = in.readVInt();
         final int longsSize = version >= BlockTreeTermsWriter.VERSION_META_ARRAY ? in.readVInt() : 0;
+
+        BytesRef minTerm, maxTerm;
+        if (version >= BlockTreeTermsWriter.VERSION_MIN_MAX_TERMS) {
+          minTerm = readBytesRef(in);
+          maxTerm = readBytesRef(in);
+        } else {
+          minTerm = maxTerm = null;
+        }
         if (docCount < 0 || docCount > info.getDocCount()) { // #docs with field must be <= #docs
           throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + info.getDocCount() + " (resource=" + in + ")");
         }
@@ -178,7 +186,9 @@ public class BlockTreeTermsReader extend
           throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq + " (resource=" + in + ")");
         }
         final long indexStartFP = indexDivisor != -1 ? indexIn.readVLong() : 0;
-        FieldReader previous = fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount, indexStartFP, longsSize, indexIn));
+        FieldReader previous = fields.put(fieldInfo.name,       
+                                          new FieldReader(fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount,
+                                                          indexStartFP, longsSize, indexIn, minTerm, maxTerm));
         if (previous != null) {
           throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
         }
@@ -196,6 +206,14 @@ public class BlockTreeTermsReader extend
     }
   }
 
+  private static BytesRef readBytesRef(IndexInput in) throws IOException {
+    BytesRef bytes = new BytesRef();
+    bytes.length = in.readVInt();
+    bytes.bytes = new byte[bytes.length];
+    in.readBytes(bytes.bytes, 0, bytes.length);
+    return bytes;
+  }
+
   /** Reads terms file header. */
   protected int readHeader(IndexInput input) throws IOException {
     int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_CODEC_NAME,
@@ -463,12 +481,15 @@ public class BlockTreeTermsReader extend
     final long indexStartFP;
     final long rootBlockFP;
     final BytesRef rootCode;
+    final BytesRef minTerm;
+    final BytesRef maxTerm;
     final int longsSize;
 
     private final FST<BytesRef> index;
     //private boolean DEBUG;
 
-    FieldReader(FieldInfo fieldInfo, long numTerms, BytesRef rootCode, long sumTotalTermFreq, long sumDocFreq, int docCount, long indexStartFP, int longsSize, IndexInput indexIn) throws IOException {
+    FieldReader(FieldInfo fieldInfo, long numTerms, BytesRef rootCode, long sumTotalTermFreq, long sumDocFreq, int docCount,
+                long indexStartFP, int longsSize, IndexInput indexIn, BytesRef minTerm, BytesRef maxTerm) throws IOException {
       assert numTerms > 0;
       this.fieldInfo = fieldInfo;
       //DEBUG = BlockTreeTermsReader.DEBUG && fieldInfo.name.equals("id");
@@ -479,6 +500,8 @@ public class BlockTreeTermsReader extend
       this.indexStartFP = indexStartFP;
       this.rootCode = rootCode;
       this.longsSize = longsSize;
+      this.minTerm = minTerm;
+      this.maxTerm = maxTerm;
       // if (DEBUG) {
       //   System.out.println("BTTR: seg=" + segment + " field=" + fieldInfo.name + " rootBlockCode=" + rootCode + " divisor=" + indexDivisor);
       // }
@@ -505,6 +528,26 @@ public class BlockTreeTermsReader extend
       }
     }
 
+    @Override
+    public BytesRef getMin() throws IOException {
+      if (minTerm == null) {
+        // Older index that didn't store min/maxTerm
+        return super.getMin();
+      } else {
+        return minTerm;
+      }
+    }
+
+    @Override
+    public BytesRef getMax() throws IOException {
+      if (maxTerm == null) {
+        // Older index that didn't store min/maxTerm
+        return super.getMax();
+      } else {
+        return maxTerm;
+      }
+    }
+
     /** For debugging -- used by CheckIndex too*/
     // TODO: maybe push this into Terms?
     public Stats computeStats() throws IOException {

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java?rev=1589749&r1=1589748&r2=1589749&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java Thu Apr 24 14:40:01 2014
@@ -111,11 +111,12 @@ import org.apache.lucene.util.packed.Pac
  *    <li>InnerNode --&gt; EntryCount, SuffixLength[,Sub?], Byte<sup>SuffixLength</sup>, StatsLength, &lt; TermStats ? &gt;<sup>EntryCount</sup>, MetaLength, &lt;<i>TermMetadata ? </i>&gt;<sup>EntryCount</sup></li>
  *    <li>TermStats --&gt; DocFreq, TotalTermFreq </li>
  *    <li>FieldSummary --&gt; NumFields, &lt;FieldNumber, NumTerms, RootCodeLength, Byte<sup>RootCodeLength</sup>,
- *                            SumTotalTermFreq?, SumDocFreq, DocCount&gt;<sup>NumFields</sup></li>
+ *                            SumTotalTermFreq?, SumDocFreq, DocCount, LongsSize, MinTerm, MaxTerm&gt;<sup>NumFields</sup></li>
  *    <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
  *    <li>DirOffset --&gt; {@link DataOutput#writeLong Uint64}</li>
+ *    <li>MinTerm,MaxTerm --&gt; {@link DataOutput#writeVInt VInt} length followed by the byte[]</li>
  *    <li>EntryCount,SuffixLength,StatsLength,DocFreq,MetaLength,NumFields,
- *        FieldNumber,RootCodeLength,DocCount --&gt; {@link DataOutput#writeVInt VInt}</li>
+ *        FieldNumber,RootCodeLength,DocCount,LongsSize --&gt; {@link DataOutput#writeVInt VInt}</li>
  *    <li>TotalTermFreq,NumTerms,SumTotalTermFreq,SumDocFreq --&gt; 
  *        {@link DataOutput#writeVLong VLong}</li>
  *    <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
@@ -134,6 +135,9 @@ import org.apache.lucene.util.packed.Pac
  *    <li>SumDocFreq is the total number of postings, the number of term-document pairs across
  *        the entire field.</li>
  *    <li>DocCount is the number of documents that have at least one posting for this field.</li>
+ *    <li>LongsSize records how many long values the postings writer/reader record per term
+ *        (e.g., to hold freq/prox/doc file offsets).
+ *    <li>MinTerm, MaxTerm are the lowest and highest term in this field.</li>
  *    <li>PostingsHeader and TermMetadata are plugged into by the specific postings implementation:
  *        these contain arbitrary per-file data (such as parameters or versioning information) 
  *        and per-term data (such as pointers to inverted files).</li>
@@ -212,8 +216,11 @@ public class BlockTreeTermsWriter extend
   /** checksums */
   public static final int VERSION_CHECKSUM = 3;
 
+  /** min/max term */
+  public static final int VERSION_MIN_MAX_TERMS = 4;
+
   /** Current terms format. */
-  public static final int VERSION_CURRENT = VERSION_CHECKSUM;
+  public static final int VERSION_CURRENT = VERSION_MIN_MAX_TERMS;
 
   /** Extension of terms index file */
   static final String TERMS_INDEX_EXTENSION = "tip";
@@ -237,8 +244,11 @@ public class BlockTreeTermsWriter extend
     public final long sumDocFreq;
     public final int docCount;
     private final int longsSize;
+    public final BytesRef minTerm;
+    public final BytesRef maxTerm;
 
-    public FieldMetaData(FieldInfo fieldInfo, BytesRef rootCode, long numTerms, long indexStartFP, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize) {
+    public FieldMetaData(FieldInfo fieldInfo, BytesRef rootCode, long numTerms, long indexStartFP, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize,
+                         BytesRef minTerm, BytesRef maxTerm) {
       assert numTerms > 0;
       this.fieldInfo = fieldInfo;
       assert rootCode != null: "field=" + fieldInfo.name + " numTerms=" + numTerms;
@@ -249,6 +259,8 @@ public class BlockTreeTermsWriter extend
       this.sumDocFreq = sumDocFreq;
       this.docCount = docCount;
       this.longsSize = longsSize;
+      this.minTerm = minTerm;
+      this.maxTerm = maxTerm;
     }
   }
 
@@ -507,6 +519,9 @@ public class BlockTreeTermsWriter extend
     private int[] subTermCountSums = new int[10];
     private int[] subSubCounts = new int[10];
 
+    private BytesRef minTerm;
+    private BytesRef maxTerm = new BytesRef();
+
     // This class assigns terms to blocks "naturally", ie,
     // according to the number of terms under a given prefix
     // that we encounter:
@@ -1045,6 +1060,11 @@ public class BlockTreeTermsWriter extend
       PendingTerm term = new PendingTerm(BytesRef.deepCopyOf(text), state);
       pending.add(term);
       numTerms++;
+
+      if (minTerm == null) {
+        minTerm = BytesRef.deepCopyOf(text);
+      }
+      maxTerm.copyBytes(text);
     }
 
     // Finishes all terms in this field
@@ -1083,7 +1103,8 @@ public class BlockTreeTermsWriter extend
                                      sumTotalTermFreq,
                                      sumDocFreq,
                                      docCount,
-                                     longsSize));
+                                     longsSize,
+                                     minTerm, maxTerm));
       } else {
         assert sumTotalTermFreq == 0 || fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY && sumTotalTermFreq == -1;
         assert sumDocFreq == 0;
@@ -1111,6 +1132,7 @@ public class BlockTreeTermsWriter extend
       for(FieldMetaData field : fields) {
         //System.out.println("  field " + field.fieldInfo.name + " " + field.numTerms + " terms");
         out.writeVInt(field.fieldInfo.number);
+        assert field.numTerms > 0;
         out.writeVLong(field.numTerms);
         out.writeVInt(field.rootCode.length);
         out.writeBytes(field.rootCode.bytes, field.rootCode.offset, field.rootCode.length);
@@ -1121,6 +1143,8 @@ public class BlockTreeTermsWriter extend
         out.writeVInt(field.docCount);
         out.writeVInt(field.longsSize);
         indexOut.writeVLong(field.indexStartFP);
+        writeBytesRef(out, field.minTerm);
+        writeBytesRef(out, field.maxTerm);
       }
       writeTrailer(out, dirStart);
       CodecUtil.writeFooter(out);
@@ -1132,4 +1156,9 @@ public class BlockTreeTermsWriter extend
       IOUtils.closeWhileHandlingException(ioe, out, indexOut, postingsWriter);
     }
   }
+
+  private static void writeBytesRef(IndexOutput out, BytesRef bytes) throws IOException {
+    out.writeVInt(bytes.length);
+    out.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1589749&r1=1589748&r2=1589749&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Thu Apr 24 14:40:01 2014
@@ -776,6 +776,30 @@ public class CheckIndex {
       final boolean hasPositions = terms.hasPositions();
       final boolean hasPayloads = terms.hasPayloads();
       final boolean hasOffsets = terms.hasOffsets();
+      
+      BytesRef bb = terms.getMin();
+      BytesRef minTerm;
+      if (bb != null) {
+        assert bb.isValid();
+        minTerm = BytesRef.deepCopyOf(bb);
+      } else {
+        minTerm = null;
+
+      }
+      BytesRef maxTerm;
+      bb = terms.getMax();
+      if (bb != null) {
+        assert bb.isValid();
+        maxTerm = BytesRef.deepCopyOf(bb);
+        if (minTerm == null) {
+          throw new RuntimeException("field \"" + field + "\" has null minTerm but non-null maxTerm");
+        }
+      } else {
+        maxTerm = null;
+        if (minTerm != null) {
+          throw new RuntimeException("field \"" + field + "\" has non-null minTerm but null maxTerm");
+        }
+      }
 
       // term vectors cannot omit TF:
       final boolean expectedHasFreqs = (isVectors || fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0);
@@ -838,6 +862,20 @@ public class CheckIndex {
           }
           lastTerm.copyBytes(term);
         }
+
+        if (minTerm == null) {
+          // We checked this above:
+          assert maxTerm == null;
+          throw new RuntimeException("field=\"" + field + "\": invalid term: term=" + term + ", minTerm=" + minTerm);
+        }
+        
+        if (term.compareTo(minTerm) < 0) {
+          throw new RuntimeException("field=\"" + field + "\": invalid term: term=" + term + ", minTerm=" + minTerm);
+        }
+        
+        if (term.compareTo(maxTerm) > 0) {
+          throw new RuntimeException("field=\"" + field + "\": invalid term: term=" + term + ", maxTerm=" + maxTerm);
+        }
         
         final int docFreq = termsEnum.docFreq();
         if (docFreq <= 0) {
@@ -1085,6 +1123,10 @@ public class CheckIndex {
           }
         }
       }
+
+      if (minTerm != null && status.termCount + status.delTermCount == 0) {
+        throw new RuntimeException("field=\"" + field + "\": minTerm is non-null yet we saw no terms: " + minTerm);
+      }
       
       final Terms fieldTerms = fields.terms(field);
       if (fieldTerms == null) {

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java?rev=1589749&r1=1589748&r2=1589749&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java Thu Apr 24 14:40:01 2014
@@ -37,11 +37,14 @@ import org.apache.lucene.util.Bits;
  */
 public abstract class FilteredTermsEnum extends TermsEnum {
 
-  private BytesRef initialSeekTerm = null;
+  private BytesRef initialSeekTerm;
   private boolean doSeek;
-  private BytesRef actualTerm = null;
 
-  private final TermsEnum tenum;
+  /** Which term the enum is currently positioned to. */
+  protected BytesRef actualTerm;
+
+  /** The delegate {@link TermsEnum}. */
+  protected final TermsEnum tenum;
 
   /** Return value, if term should be accepted or the iteration should
    * {@code END}. The {@code *_SEEK} values denote, that after handling the current term
@@ -252,6 +255,7 @@ public abstract class FilteredTermsEnum 
         case END:
           // we are supposed to end the enum
           return null;
+        // NO: we just fall through and iterate again
       }
     }
   }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java?rev=1589749&r1=1589748&r2=1589749&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java Thu Apr 24 14:40:01 2014
@@ -98,6 +98,32 @@ public final class MultiTerms extends Te
       return TermsEnum.EMPTY;
     }
   }
+  
+  @Override
+  public BytesRef getMin() throws IOException {
+    BytesRef minTerm = null;
+    for(Terms terms : subs) {
+      BytesRef term = terms.getMin();
+      if (minTerm == null || term.compareTo(minTerm) < 0) {
+        minTerm = term;
+      }
+    }
+
+    return minTerm;
+  }
+
+  @Override
+  public BytesRef getMax() throws IOException {
+    BytesRef maxTerm = null;
+    for(Terms terms : subs) {
+      BytesRef term = terms.getMax();
+      if (maxTerm == null || term.compareTo(maxTerm) > 0) {
+        maxTerm = term;
+      }
+    }
+
+    return maxTerm;
+  }
 
   @Override
   public TermsEnum iterator(TermsEnum reuse) throws IOException {

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/Terms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/Terms.java?rev=1589749&r1=1589748&r2=1589749&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/Terms.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/Terms.java Thu Apr 24 14:40:01 2014
@@ -125,4 +125,80 @@ public abstract class Terms {
 
   /** Zero-length array of {@link Terms}. */
   public final static Terms[] EMPTY_ARRAY = new Terms[0];
+  
+  /** Returns the smallest term (in lexicographic order) in the field. 
+   *  Note that, just like other term measures, this measure does not 
+   *  take deleted documents into account.  This returns
+   *  null when there are no terms. */
+  public BytesRef getMin() throws IOException {
+    return iterator(null).next();
+  }
+
+  /** Returns the largest term (in lexicographic order) in the field. 
+   *  Note that, just like other term measures, this measure does not 
+   *  take deleted documents into account.  This returns
+   *  null when there are no terms. */
+  @SuppressWarnings("fallthrough")
+  public BytesRef getMax() throws IOException {
+    long size = size();
+    
+    if (size == 0) {
+      // empty: only possible from a FilteredTermsEnum...
+      return null;
+    } else if (size >= 0) {
+      // try to seek-by-ord
+      try {
+        TermsEnum iterator = iterator(null);
+        iterator.seekExact(size - 1);
+        return iterator.term();
+      } catch (UnsupportedOperationException e) {
+        // ok
+      }
+    }
+    
+    // otherwise: binary search
+    TermsEnum iterator = iterator(null);
+    BytesRef v = iterator.next();
+    if (v == null) {
+      // empty: only possible from a FilteredTermsEnum...
+      return v;
+    }
+
+    BytesRef scratch = new BytesRef(1);
+
+    scratch.length = 1;
+
+    // Iterates over digits:
+    while (true) {
+
+      int low = 0;
+      int high = 256;
+
+      // Binary search current digit to find the highest
+      // digit before END:
+      while (low != high) {
+        int mid = (low+high) >>> 1;
+        scratch.bytes[scratch.length-1] = (byte) mid;
+        if (iterator.seekCeil(scratch) == TermsEnum.SeekStatus.END) {
+          // Scratch was too high
+          if (mid == 0) {
+            scratch.length--;
+            return scratch;
+          }
+          high = mid;
+        } else {
+          // Scratch was too low; there is at least one term
+          // still after it:
+          if (low == mid) {
+            break;
+          }
+          low = mid;
+        }
+      }
+
+      // Recurse to next digit:
+      scratch.length++;
+      scratch.grow(scratch.length);
+    }
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java?rev=1589749&r1=1589748&r2=1589749&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java Thu Apr 24 14:40:01 2014
@@ -17,12 +17,16 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.analysis.NumericTokenStream;
 import org.apache.lucene.document.DoubleField; // javadocs
 import org.apache.lucene.document.FloatField; // javadocs
 import org.apache.lucene.document.IntField; // javadocs
 import org.apache.lucene.document.LongField; // javadocs
+import org.apache.lucene.index.FilterAtomicReader;
 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
@@ -464,14 +468,15 @@ public final class NumericUtils {
    *         terms with a shift value of <tt>0</tt>.
    */
   public static TermsEnum filterPrefixCodedLongs(TermsEnum termsEnum) {
-    return new FilteredTermsEnum(termsEnum, false) {
+    return new SeekingNumericFilteredTermsEnum(termsEnum) {
+
       @Override
       protected AcceptStatus accept(BytesRef term) {
         return NumericUtils.getPrefixCodedLongShift(term) == 0 ? AcceptStatus.YES : AcceptStatus.END;
       }
     };
   }
-  
+
   /**
    * Filters the given {@link TermsEnum} by accepting only prefix coded 32 bit
    * terms with a shift value of <tt>0</tt>.
@@ -482,7 +487,7 @@ public final class NumericUtils {
    *         terms with a shift value of <tt>0</tt>.
    */
   public static TermsEnum filterPrefixCodedInts(TermsEnum termsEnum) {
-    return new FilteredTermsEnum(termsEnum, false) {
+    return new SeekingNumericFilteredTermsEnum(termsEnum) {
       
       @Override
       protected AcceptStatus accept(BytesRef term) {
@@ -490,5 +495,85 @@ public final class NumericUtils {
       }
     };
   }
+
+  /** 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);
+    }
+
+    @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;
+      }
+    }
+  }
+
+  private static Terms intTerms(Terms terms) {
+    return new FilterAtomicReader.FilterTerms(terms) {
+        @Override
+        public TermsEnum iterator(TermsEnum reuse) throws IOException {
+          return filterPrefixCodedInts(in.iterator(reuse));
+        }
+      };
+  }
+
+  private static Terms longTerms(Terms terms) {
+    return new FilterAtomicReader.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());
+  }
   
 }

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java?rev=1589749&r1=1589748&r2=1589749&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java Thu Apr 24 14:40:01 2014
@@ -91,6 +91,20 @@ public class AssertingAtomicReader exten
     }
 
     @Override
+    public BytesRef getMin() throws IOException {
+      BytesRef v = in.getMin();
+      assert v == null || v.isValid();
+      return v;
+    }
+
+    @Override
+    public BytesRef getMax() throws IOException {
+      BytesRef v = in.getMax();
+      assert v == null || v.isValid();
+      return v;
+    }
+
+    @Override
     public TermsEnum iterator(TermsEnum reuse) throws IOException {
       // TODO: should we give this thing a random to be super-evil,
       // and randomly *not* unwrap?