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/10/28 23:52:50 UTC

svn commit: r1635002 [4/4] - in /lucene/dev/branches/lucene6005/lucene: codecs/src/java/org/apache/lucene/codecs/blocktreeords/ codecs/src/java/org/apache/lucene/codecs/memory/ core/src/java/org/apache/lucene/codecs/ core/src/java/org/apache/lucene/cod...

Added: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestAutoPrefixTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestAutoPrefixTerms.java?rev=1635002&view=auto
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestAutoPrefixTerms.java (added)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestAutoPrefixTerms.java Tue Oct 28 22:52:49 2014
@@ -0,0 +1,663 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * 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.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.MultiDocValues;
+import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.MultiTermQuery;
+import org.apache.lucene.search.PrefixQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.AttributeImpl;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+
+public class TestAutoPrefixTerms extends LuceneTestCase {
+
+  private int minItemsPerBlock = TestUtil.nextInt(random(), 2, 100);
+  private int maxItemsPerBlock = 2*(Math.max(2, minItemsPerBlock-1)) + random().nextInt(100);
+  private int minTermsAutoPrefix = TestUtil.nextInt(random(), 2, 100);
+  private int maxTermsAutoPrefix = random().nextBoolean() ? Math.max(2, (minTermsAutoPrefix-1)*2 + random().nextInt(100)) : Integer.MAX_VALUE;
+
+  private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene50PostingsFormat(minItemsPerBlock, maxItemsPerBlock,
+                                                                                       minTermsAutoPrefix, maxTermsAutoPrefix));
+
+  // Numbers in a restricted range, encoded in decimal, left-0-padded:
+  public void testBasicNumericRanges() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    iwc.setCodec(codec);
+    IndexWriter w = new IndexWriter(dir, iwc);
+    int numTerms = TestUtil.nextInt(random(), 3000, 50000);
+    Set<String> terms = new HashSet<>();
+    int digits = TestUtil.nextInt(random(), 5, 10);
+    int maxValue = 1;
+    for(int i=0;i<digits;i++) {
+      maxValue *= 10;
+    }
+    String format = "%0" + digits + "d";
+    while (terms.size() < numTerms) {
+      terms.add(String.format(Locale.ROOT, format, random().nextInt(maxValue)));
+    }
+
+    for(String term : terms) {
+      Document doc = new Document();
+      doc.add(new StringField("field", term, Field.Store.NO));
+      doc.add(new NumericDocValuesField("field", Long.parseLong(term)));
+      w.addDocument(doc);
+    }
+
+    if (VERBOSE) System.out.println("\nTEST: now optimize");
+    if (random().nextBoolean()) {
+      w.forceMerge(1);
+    }
+
+    if (VERBOSE) System.out.println("\nTEST: now done");
+    IndexReader r = DirectoryReader.open(w, true);
+
+    List<String> sortedTerms = new ArrayList<>(terms);
+    Collections.sort(sortedTerms);
+
+    if (VERBOSE) {
+      System.out.println("TEST: sorted terms:");
+      int idx = 0;
+      for(String term : sortedTerms) {
+        System.out.println(idx + ": " + term);
+        idx++;
+      }
+    }
+
+    int iters = atLeast(100);
+    for(int iter=0;iter<iters;iter++) {
+      int min, max;
+      while (true) {
+        min = random().nextInt(maxValue);
+        max = random().nextInt(maxValue);
+        if (min == max) {
+          continue;
+        } else if (min > max) {
+          int x = min;
+          min = max;
+          max = x;
+        }
+        break;
+      }
+      
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter + " min=" + min + " max=" + max);
+      }
+
+      boolean minInclusive = random().nextBoolean();
+      boolean maxInclusive = random().nextBoolean();
+      BytesRef minTerm = new BytesRef(String.format(Locale.ROOT, format, min));
+      BytesRef maxTerm = new BytesRef(String.format(Locale.ROOT, format, max));
+      CompiledAutomaton ca = new CompiledAutomaton(minTerm, minInclusive,
+                                                   maxTerm, maxInclusive);
+
+      TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
+      NumericDocValues docValues = MultiDocValues.getNumericValues(r, "field");
+      DocsEnum docsEnum = null;
+
+      VerifyAutoPrefixTerms verifier = new VerifyAutoPrefixTerms(r.maxDoc(), minTerm, maxTerm);
+
+      while (te.next() != null) {
+        if (VERBOSE) {
+          System.out.println("  got term=" + te.term().utf8ToString());
+        }
+        verifier.sawTerm(te.term());
+        docsEnum = te.docs(null, docsEnum);
+        int docID;
+        while ((docID = docsEnum.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
+          long v = docValues.get(docID);
+          assert v >= min && v <= max: "docID=" + docID + " v=" + v;
+          // The auto-prefix terms should never "overlap" one another, so we should only ever see a given docID one time:
+          if (VERBOSE) {
+            System.out.println("    got docID=" + docID + " v=" + v);
+          }
+          verifier.sawDoc(docID);
+        }
+      }
+      
+      int startLoc = Collections.binarySearch(sortedTerms, String.format(Locale.ROOT, format, min));
+      if (startLoc < 0) {
+        startLoc = -startLoc-1;
+      } else if (minInclusive == false) {
+        startLoc++;
+      }
+      int endLoc = Collections.binarySearch(sortedTerms, String.format(Locale.ROOT, format, max));
+      if (endLoc < 0) {
+        endLoc = -endLoc-2;
+      } else if (maxInclusive == false) {
+        endLoc--;
+      }
+      verifier.finish(endLoc-startLoc+1, maxTermsAutoPrefix);
+    }
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  private 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;
+  }
+
+  // Numbers are encoded in full binary (4 byte ints):
+  public void testBinaryNumericRanges() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    iwc.setCodec(codec);
+    IndexWriter w = new IndexWriter(dir, iwc);
+    int numTerms = TestUtil.nextInt(random(), 3000, 50000);
+    Set<Integer> terms = new HashSet<>();
+    while (terms.size() < numTerms) {
+      terms.add(random().nextInt());
+    }
+
+    for(Integer term : terms) {
+      Document doc = new Document();
+      doc.add(new BinaryField("field", intToBytes(term)));
+      doc.add(new NumericDocValuesField("field", term));
+      w.addDocument(doc);
+    }
+
+    if (random().nextBoolean()) {
+      w.forceMerge(1);
+    }
+
+    IndexReader r = DirectoryReader.open(w, true);
+
+    List<Integer> sortedTerms = new ArrayList<>(terms);
+    Collections.sort(sortedTerms);
+
+    if (VERBOSE) {
+      System.out.println("TEST: sorted terms:");
+      int idx = 0;
+      for(Integer term : sortedTerms) {
+        System.out.println(idx + ": " + term);
+        idx++;
+      }
+    }
+
+    int iters = atLeast(100);
+    for(int iter=0;iter<iters;iter++) {
+
+      int min, max;
+      while (true) {
+        min = random().nextInt();
+        max = random().nextInt();
+        if (min == max) {
+          continue;
+        } else if (min > max) {
+          int x = min;
+          min = max;
+          max = x;
+        }
+        break;
+      }
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter + " min=" + min + " (" + intToBytes(min) + ") max=" + max + " (" + intToBytes(max) + ")");
+      }
+      
+      boolean minInclusive = random().nextBoolean();
+      BytesRef minTerm = intToBytes(min);
+      boolean maxInclusive = random().nextBoolean();
+      BytesRef maxTerm = intToBytes(max);
+      CompiledAutomaton ca = new CompiledAutomaton(minTerm, minInclusive,
+                                                   maxTerm, maxInclusive);
+
+      TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
+      NumericDocValues docValues = MultiDocValues.getNumericValues(r, "field");
+      DocsEnum docsEnum = null;
+      VerifyAutoPrefixTerms verifier = new VerifyAutoPrefixTerms(r.maxDoc(), minTerm, maxTerm);
+      while (te.next() != null) {
+        if (VERBOSE) {
+          System.out.println("  got term=" + te.term() + " docFreq=" + te.docFreq());
+        }
+        verifier.sawTerm(te.term());        
+        docsEnum = te.docs(null, docsEnum);
+        int docID;
+        while ((docID = docsEnum.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
+          long v = docValues.get(docID);
+          assert v >= min && v <= max: "docID=" + docID + " v=" + v;
+          verifier.sawDoc(docID);
+        }
+      }
+      
+      int startLoc = Collections.binarySearch(sortedTerms, min);
+      if (startLoc < 0) {
+        startLoc = -startLoc-1;
+      } else if (minInclusive == false) {
+        startLoc++;
+      }
+      int endLoc = Collections.binarySearch(sortedTerms, max);
+      if (endLoc < 0) {
+        endLoc = -endLoc-2;
+      } else if (maxInclusive == false) {
+        endLoc--;
+      }
+      int expectedHits = endLoc-startLoc+1;
+      try {
+        verifier.finish(expectedHits, maxTermsAutoPrefix);
+      } catch (AssertionError ae) {
+        for(int i=0;i<numTerms;i++) {
+          if (verifier.allHits.get(i) == false) {
+            int v = (int) docValues.get(i);
+            boolean accept = (v > min || (v == min && minInclusive)) &&
+              (v < max || (v == max && maxInclusive));
+            if (accept) {
+              System.out.println("MISSING: docID=" + i + " v=" + v + " term=" + intToBytes(v));
+            }
+          }
+        }
+
+        throw ae;
+      }
+    }
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  // Non-numeric, simple prefix query
+  public void testBasicPrefixTerms() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    iwc.setCodec(codec);
+    iwc.setMergeScheduler(new SerialMergeScheduler());
+    IndexWriter w = new IndexWriter(dir, iwc);
+    int numTerms = TestUtil.nextInt(random(), 3000, 50000);
+    Set<String> terms = new HashSet<>();
+    while (terms.size() < numTerms) {
+      terms.add(TestUtil.randomSimpleString(random()));
+    }
+
+    for(String term : terms) {
+      Document doc = new Document();
+      doc.add(new StringField("field", term, Field.Store.NO));
+      doc.add(new BinaryDocValuesField("field", new BytesRef(term)));
+      w.addDocument(doc);
+    }
+
+    if (random().nextBoolean()) {
+      w.forceMerge(1);
+    }
+
+    IndexReader r = DirectoryReader.open(w, true);
+
+    List<String> sortedTerms = new ArrayList<>(terms);
+    Collections.sort(sortedTerms);
+
+    if (VERBOSE) {
+      System.out.println("TEST: sorted terms:");
+      int idx = 0;
+      for(String term : sortedTerms) {
+        System.out.println(idx + ": " + term);
+        idx++;
+      }
+    }
+
+    if (VERBOSE) {
+      System.out.println("TEST: r=" + r);
+    }
+
+    int iters = atLeast(100);
+    for(int iter=0;iter<iters;iter++) {
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      String prefix;
+      if (random().nextInt(100) == 42) {
+        prefix = "";
+      } else {
+        prefix = TestUtil.randomSimpleString(random(), 1, 4);
+      }
+      BytesRef prefixBR = new BytesRef(prefix);
+      if (VERBOSE) {
+        System.out.println("  prefix=" + prefix);
+      }
+
+      CompiledAutomaton ca = new CompiledAutomaton(prefixBR);
+      TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
+      BinaryDocValues docValues = MultiDocValues.getBinaryValues(r, "field");
+      DocsEnum docsEnum = null;
+
+      VerifyAutoPrefixTerms verifier = new VerifyAutoPrefixTerms(r.maxDoc(), prefixBR);
+
+      while (te.next() != null) {
+        if (VERBOSE) {
+          System.out.println("TEST: got term=" + te.term().utf8ToString() + " docFreq=" + te.docFreq());
+        }
+        verifier.sawTerm(te.term());        
+        docsEnum = te.docs(null, docsEnum);
+        int docID;
+        while ((docID = docsEnum.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
+          assertTrue("prefixBR=" + prefixBR + " docBR=" + docValues.get(docID), StringHelper.startsWith(docValues.get(docID), prefixBR));
+          // The auto-prefix terms should never "overlap" one another, so we should only ever see a given docID one time:
+          verifier.sawDoc(docID);
+        }
+      }
+      
+      int startLoc = Collections.binarySearch(sortedTerms, prefix);
+      if (startLoc < 0) {
+        startLoc = -startLoc-1;
+      }
+      int endLoc = Collections.binarySearch(sortedTerms, prefix + (char) ('z'+1));
+      if (endLoc < 0) {
+        endLoc = -endLoc-2;
+      }
+      int expectedHits = endLoc-startLoc+1;
+      try {
+        verifier.finish(expectedHits, maxTermsAutoPrefix);
+      } catch (AssertionError ae) {
+        for(int i=0;i<numTerms;i++) {
+          if (verifier.allHits.get(i) == false) {
+            String s = docValues.get(i).utf8ToString();
+            if (s.startsWith(prefix)) {
+              System.out.println("MISSING: docID=" + i + " term=" + s);
+            }
+          }
+        }
+
+        throw ae;
+      }
+    }
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testDemoPrefixTerms() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    iwc.setCodec(codec);
+    IndexWriter w = new IndexWriter(dir, iwc);
+    int numDocs = 30;
+
+    for(int i=0;i<numDocs;i++) {
+      Document doc = new Document();
+      doc.add(new StringField("field", "" + (char) (97+i), Field.Store.NO));
+      w.addDocument(doc);
+      doc = new Document();
+      doc.add(new StringField("field", "a" + (char) (97+i), Field.Store.NO));
+      w.addDocument(doc);
+    }
+
+    if (random().nextBoolean()) {
+      w.forceMerge(1);
+    }
+
+    IndexReader r = DirectoryReader.open(w, true);
+    Terms terms = MultiFields.getTerms(r, "field");
+    if (VERBOSE) {
+      System.out.println("\nTEST: now intersect");
+    }
+    CompiledAutomaton ca = new CompiledAutomaton(new BytesRef("a"));
+    TermsEnum te = ca.getTermsEnum(terms);
+    DocsEnum docsEnum = null;
+
+    VerifyAutoPrefixTerms verifier = new VerifyAutoPrefixTerms(r.maxDoc(), new BytesRef("a"));
+    //TermsEnum te = terms.intersect(new CompiledAutomaton(a, true, false), null);
+    while (te.next() != null) {
+      verifier.sawTerm(te.term());
+      docsEnum = te.docs(null, docsEnum);
+      int docID;
+      while ((docID = docsEnum.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
+        // The auto-prefix terms should never "overlap" one another, so we should only ever see a given docID one time:
+        verifier.sawDoc(docID);
+      }
+    }
+    // 1 document has exactly "a", and 30 documents had "a?"
+    verifier.finish(31, maxTermsAutoPrefix);
+    PrefixQuery q = new PrefixQuery(new Term("field", "a"));
+    q.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_BOOLEAN_QUERY_REWRITE);
+    assertEquals(31, newSearcher(r).search(q, 1).totalHits);
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  static final class BinaryTokenStream extends TokenStream {
+    private final ByteTermAttribute bytesAtt = addAttribute(ByteTermAttribute.class);
+    private boolean available = true;
+  
+    public BinaryTokenStream(BytesRef bytes) {
+      bytesAtt.setBytesRef(bytes);
+    }
+  
+    @Override
+    public boolean incrementToken() {
+      if (available) {
+        clearAttributes();
+        available = false;
+        return true;
+      }
+      return false;
+    }
+  
+    @Override
+    public void reset() {
+      available = true;
+    }
+  
+    public interface ByteTermAttribute extends TermToBytesRefAttribute {
+      public void setBytesRef(BytesRef bytes);
+    }
+  
+    public static class ByteTermAttributeImpl extends AttributeImpl implements ByteTermAttribute,TermToBytesRefAttribute {
+      private BytesRef bytes;
+    
+      @Override
+      public void fillBytesRef() {
+        // no-op: the bytes was already filled by our owner's incrementToken
+      }
+    
+      @Override
+      public BytesRef getBytesRef() {
+        return bytes;
+      }
+
+      @Override
+      public void setBytesRef(BytesRef bytes) {
+        this.bytes = bytes;
+      }
+    
+      @Override
+      public void clear() {}
+    
+      @Override
+      public void copyTo(AttributeImpl target) {
+        ByteTermAttributeImpl other = (ByteTermAttributeImpl) target;
+        other.bytes = bytes;
+      }
+    }
+  }
+
+  /** Basically a StringField that accepts binary term. */
+  private static class BinaryField extends Field {
+
+    final static FieldType TYPE;
+    static {
+      TYPE = new FieldType(StringField.TYPE_NOT_STORED);
+      // Necessary so our custom tokenStream is used by Field.tokenStream:
+      TYPE.setTokenized(true);
+      TYPE.freeze();
+    }
+
+    public BinaryField(String name, BytesRef value) {
+      super(name, new BinaryTokenStream(value), TYPE);
+    }
+  }
+
+  /** Helper class to ensure auto-prefix terms 1) never overlap one another, and 2) are used when they should be. */
+  private static class VerifyAutoPrefixTerms {
+    final FixedBitSet allHits;
+    private final Map<BytesRef,Integer> prefixCounts = new HashMap<>();
+    private int totPrefixCount;
+    private final BytesRef[] bounds;
+    private int totTermCount;
+    private BytesRef lastTerm;
+
+    public VerifyAutoPrefixTerms(int maxDoc, BytesRef... bounds) {
+      allHits = new FixedBitSet(maxDoc);
+      assert bounds.length > 0;
+      this.bounds = bounds;
+    }
+
+    public void sawTerm(BytesRef term) {
+      //System.out.println("saw term=" + term);
+      if (lastTerm != null) {
+        assertTrue(lastTerm.compareTo(term) < 0);
+      }
+      lastTerm = BytesRef.deepCopyOf(term);
+      totTermCount++;
+      totPrefixCount += term.length;
+      for(int i=1;i<=term.length;i++) {
+        BytesRef prefix = BytesRef.deepCopyOf(term);
+        prefix.length = i;
+        Integer count = prefixCounts.get(prefix);
+        if (count == null) {
+          count = 1;
+        } else {
+          count += 1;
+        }
+        prefixCounts.put(prefix, count);
+      }
+    }
+
+    public void sawDoc(int docID) {
+      // The auto-prefix terms should never "overlap" one another, so we should only ever see a given docID one time:
+      assertFalse(allHits.getAndSet(docID));
+    }
+
+    public void finish(int expectedNumHits, int maxPrefixCount) {
+
+      if (maxPrefixCount != -1) {
+        // Auto-terms were used in this test
+        long allowedMaxTerms;
+
+        if (bounds.length == 1) {
+          // Simple prefix query: we should never see more than maxPrefixCount terms:
+          allowedMaxTerms = maxPrefixCount;
+        } else {
+          // Trickier: we need to allow for maxPrefixTerms for each different leading byte in the min and max:
+          assert bounds.length == 2;
+          BytesRef minTerm = bounds[0];
+          BytesRef maxTerm = bounds[1];
+
+          int commonPrefix = 0;
+          for(int i=0;i<minTerm.length && i<maxTerm.length;i++) {
+            if (minTerm.bytes[minTerm.offset+i] != maxTerm.bytes[maxTerm.offset+i]) {
+              commonPrefix = i;
+              break;
+            }
+          }
+
+          allowedMaxTerms = maxPrefixCount * (long) ((minTerm.length-commonPrefix) + (maxTerm.length-commonPrefix));
+        }
+
+        assertTrue("totTermCount=" + totTermCount + " is > allowedMaxTerms=" + allowedMaxTerms, totTermCount <= allowedMaxTerms);
+      }
+
+      assertEquals(expectedNumHits, allHits.cardinality());
+      int sum = 0;
+      for(Map.Entry<BytesRef,Integer> ent : prefixCounts.entrySet()) {
+
+        BytesRef prefix = ent.getKey();
+        if (VERBOSE) {
+          System.out.println("  verify prefix=" + TestUtil.brToString(prefix) + " count=" + ent.getValue());
+        }
+
+        if (maxPrefixCount != -1) {
+          // Auto-terms were used in this test
+
+          int sumLeftoverSuffix = 0;
+          for(BytesRef bound : bounds) {
+
+            int minSharedLength = Math.min(bound.length, prefix.length);
+            int commonPrefix = minSharedLength;
+            for(int i=0;i<minSharedLength;i++) {
+              if (bound.bytes[bound.offset+i] != prefix.bytes[prefix.offset+i]) {
+                commonPrefix = i;
+                break;
+              }
+            }
+            sumLeftoverSuffix += bound.length - commonPrefix;
+          }
+
+          long limit = (1+sumLeftoverSuffix) * (long) maxPrefixCount;
+
+          assertTrue("maxPrefixCount=" + maxPrefixCount + " prefix=" + prefix + " sumLeftoverSuffix=" + sumLeftoverSuffix + " limit=" + limit + " vs actual=" +ent.getValue(),
+                     ent.getValue() <= limit);
+        }
+
+        sum += ent.getValue();
+      }
+
+      // Make sure no test bug:
+      assertEquals(totPrefixCount, sum);
+    }
+  }
+
+}

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestDocument2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestDocument2.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestDocument2.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestDocument2.java Tue Oct 28 22:52:49 2014
@@ -273,6 +273,10 @@ public class TestDocument2 extends Lucen
     dir.close();
   }
 
+  // nocommit testTermRangeQuery
+  // nocommit test range exc
+  
+
   public void testIntRangeQuery() throws Exception {
     Directory dir = newDirectory();
 
@@ -558,6 +562,7 @@ public class TestDocument2 extends Lucen
     FieldTypes fieldTypes = w.getFieldTypes();
     fieldTypes.setPostingsFormat("id", "Memory");
     fieldTypes.enableStored("id");
+    fieldTypes.disableFastRanges("id");
 
     Document2 doc = w.newDocument();
     doc.addAtom("id", "0");
@@ -618,24 +623,6 @@ public class TestDocument2 extends Lucen
     dir.close();
   }
 
-  public void testNumericPrecisionStep() throws Exception {
-    Directory dir = newDirectory();
-    IndexWriterConfig iwc = newIndexWriterConfig();
-    IndexWriter w = new IndexWriter(dir, iwc);
-    FieldTypes fieldTypes = w.getFieldTypes();
-    fieldTypes.setNumericPrecisionStep("long", 4);
-
-    Document2 doc = w.newDocument();
-    doc.addLong("long", 17);
-    w.addDocument(doc);
-
-    IndexReader r = DirectoryReader.open(w, true);
-    assertEquals(16, MultiFields.getTerms(r, "long").size());
-    r.close();
-    w.close();
-    dir.close();
-  }
-
   public void testBinaryTermQuery() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java Tue Oct 28 22:52:49 2014
@@ -111,8 +111,8 @@ public class TestAutomatonQuery extends 
     assertAutomatonHits(2, Automata.makeString("doc"));
     assertAutomatonHits(1, Automata.makeChar('a'));
     assertAutomatonHits(2, Automata.makeCharRange('a', 'b'));
-    assertAutomatonHits(2, Automata.makeInterval(1233, 2346, 0));
-    assertAutomatonHits(1, Automata.makeInterval(0, 2000, 0));
+    assertAutomatonHits(2, Automata.makeDecimalInterval(1233, 2346, 0));
+    assertAutomatonHits(1, Automata.makeDecimalInterval(0, 2000, 0));
     assertAutomatonHits(2, Operations.union(Automata.makeChar('a'),
         Automata.makeChar('b')));
     assertAutomatonHits(0, Operations.intersection(Automata
@@ -188,8 +188,9 @@ public class TestAutomatonQuery extends 
     Automaton pfx = Automata.makeString("do");
     Automaton prefixAutomaton = Operations.concatenate(pfx, Automata.makeAnyString());
     AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), prefixAutomaton);
-    Terms terms = MultiFields.getTerms(searcher.getIndexReader(), FN);
-    assertTrue(aq.getTermsEnum(terms) instanceof PrefixTermsEnum);
+    // nocommit not true anymore
+    //Terms terms = MultiFields.getTerms(searcher.getIndexReader(), FN);
+    //assertTrue(aq.getTermsEnum(terms) instanceof PrefixTermsEnum);
     assertEquals(3, automatonQueryNrHits(aq));
   }
   

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java Tue Oct 28 22:52:49 2014
@@ -17,15 +17,29 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
+import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
-import org.apache.lucene.document.Document;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
 
 /**
  * Tests {@link PrefixQuery} class.
@@ -64,4 +78,90 @@ public class TestPrefixQuery extends Luc
     reader.close();
     directory.close();
   }
+
+  /** Make sure auto prefix terms are used with PrefixQuery. */
+  public void testAutoPrefixTermsKickIn() throws Exception {
+
+    List<String> prefixes = new ArrayList<>();
+    for(int i=1;i<5;i++) {
+      char[] chars = new char[i];
+      Arrays.fill(chars, 'a');
+      prefixes.add(new String(chars));
+    }
+
+    Set<String> randomTerms = new HashSet<>();
+    int numTerms = atLeast(10000);
+    while (randomTerms.size() < numTerms) {
+      for(String prefix : prefixes) {
+        randomTerms.add(prefix + TestUtil.randomRealisticUnicodeString(random()));
+      }
+    }
+
+    int actualCount = 0;
+    for(String term : randomTerms) {
+      if (term.startsWith("aa")) {
+        actualCount++;
+      }
+    }
+
+    //System.out.println("actual count " + actualCount);
+
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
+    int minTermsInBlock = TestUtil.nextInt(random(), 2, 100);
+    int maxTermsInBlock = Math.max(2, (minTermsInBlock-1)*2 + random().nextInt(100));
+
+    // As long as this is never > actualCount, aa should always see at least one auto-prefix term:
+    int minTermsAutoPrefix = TestUtil.nextInt(random(), 2, actualCount);
+    int maxTermsAutoPrefix = random().nextBoolean() ? Math.max(2, (minTermsAutoPrefix-1)*2 + random().nextInt(100)) : Integer.MAX_VALUE;
+
+    iwc.setCodec(TestUtil.alwaysPostingsFormat(new Lucene50PostingsFormat(minTermsInBlock, maxTermsInBlock,
+                                                                          minTermsAutoPrefix, maxTermsAutoPrefix)));
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+
+    for (String term : randomTerms) {
+      Document doc = new Document();
+      doc.add(new StringField("field", term, Field.Store.NO));
+      w.addDocument(doc);
+    }
+
+    w.forceMerge(1);
+    IndexReader r = w.getReader();
+    final Terms terms = MultiFields.getTerms(r, "field");
+    IndexSearcher s = new IndexSearcher(r);
+    final int finalActualCount = actualCount;
+    PrefixQuery q = new PrefixQuery(new Term("field", "aa")) {
+      public PrefixQuery checkTerms() throws IOException {
+        TermsEnum termsEnum = getTermsEnum(terms, new AttributeSource());
+        int count = 0;
+        while (termsEnum.next() != null) {
+          //System.out.println("got term: " + termsEnum.term().utf8ToString());
+          count++;
+        }
+
+        // Auto-prefix term(s) should have kicked in, so we should have visited fewer than the total number of aa* terms:
+        assertTrue(count < finalActualCount);
+
+        return this;
+      }
+    }.checkTerms();
+
+    int x = BooleanQuery.getMaxClauseCount();
+    try {
+      BooleanQuery.setMaxClauseCount(randomTerms.size());
+      if (random().nextBoolean()) {
+        q.setRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE);
+      } else if (random().nextBoolean()) {
+        q.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_BOOLEAN_QUERY_REWRITE);
+      }
+
+      assertEquals(actualCount, s.search(q, 1).totalHits);
+    } finally {
+      BooleanQuery.setMaxClauseCount(x);
+    }
+
+    r.close();
+    w.close();
+    dir.close();
+  }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java Tue Oct 28 22:52:49 2014
@@ -18,20 +18,32 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.analysis.*;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
 
 
 public class TestTermRangeQuery extends LuceneTestCase {
@@ -104,19 +116,20 @@ public class TestTermRangeQuery extends 
     initializeIndex(new String[]{"A", "B", "C", "D"});
     IndexReader reader = DirectoryReader.open(dir);
     IndexSearcher searcher = newSearcher(reader);
+
     TermRangeQuery query = new TermRangeQuery("content", null, null, true, true);
     Terms terms = MultiFields.getTerms(searcher.getIndexReader(), "content");
+    // Should return the unfiltered TermsEnum:
     assertFalse(query.getTermsEnum(terms) instanceof TermRangeTermsEnum);
     assertEquals(4, searcher.search(query, null, 1000).scoreDocs.length);
-    query = new TermRangeQuery("content", null, null, false, false);
-    assertFalse(query.getTermsEnum(terms) instanceof TermRangeTermsEnum);
-    assertEquals(4, searcher.search(query, null, 1000).scoreDocs.length);
-    query = TermRangeQuery.newStringRange("content", "", null, true, false);
+
+    query = TermRangeQuery.newStringRange("content", "", null, true, true);
+    // Should return the unfiltered TermsEnum:
     assertFalse(query.getTermsEnum(terms) instanceof TermRangeTermsEnum);
     assertEquals(4, searcher.search(query, null, 1000).scoreDocs.length);
-    // and now anothe one
-    query = TermRangeQuery.newStringRange("content", "B", null, true, false);
-    assertTrue(query.getTermsEnum(terms) instanceof TermRangeTermsEnum);
+
+    // and now another one
+    query = TermRangeQuery.newStringRange("content", "B", null, true, true);
     assertEquals(3, searcher.search(query, null, 1000).scoreDocs.length);
     reader.close();
   }
@@ -336,4 +349,123 @@ public class TestTermRangeQuery extends 
     //assertEquals("C added => A,B,<empty string>,C in range", 3, hits.length());
      reader.close();
   }
+
+  /** Make sure auto prefix terms are used with TermRangeQuery */
+  public void testAutoPrefixTermsKickIn() throws Exception {
+
+    List<String> prefixes = new ArrayList<>();
+    for(int i=1;i<5;i++) {
+      char[] chars = new char[i];
+      Arrays.fill(chars, 'a');
+      prefixes.add(new String(chars));
+    }
+
+    Set<String> randomTerms = new HashSet<>();
+    int numTerms = atLeast(10000);
+    while (randomTerms.size() < numTerms) {
+      for(String prefix : prefixes) {
+        randomTerms.add(prefix + TestUtil.randomSimpleString(random()));
+      }
+    }
+
+    // We make term range aa<start> - aa<end>
+    char start;
+    char end;
+
+    int actualCount;
+    boolean startInclusive = random().nextBoolean();
+    boolean endInclusive = random().nextBoolean();
+    String startTerm;
+    String endTerm;
+
+    while (true) {
+      start = (char) TestUtil.nextInt(random(), 'a', 'm');
+      end = (char) TestUtil.nextInt(random(), start+1, 'z');
+
+      actualCount = 0;
+
+      startTerm = "aa" + start;
+      endTerm = "aa" + end;
+
+      for(String term : randomTerms) {
+        int cmpStart = startTerm.compareTo(term);
+        int cmpEnd = endTerm.compareTo(term);
+        if ((cmpStart < 0 || (startInclusive && cmpStart == 0)) &&
+            (cmpEnd > 0 || (endInclusive && cmpEnd == 0))) {
+          actualCount++;
+        }
+      }
+
+      if (actualCount > 2000) {
+        break;
+      }
+    }
+
+    //System.out.println("start " + startTerm + " inclusive? " + startInclusive);
+    //System.out.println("end " + endTerm + " inclusive? " + endInclusive);
+    //System.out.println("actual count " + actualCount);
+
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
+    int minTermsInBlock = TestUtil.nextInt(random(), 2, 100);
+    int maxTermsInBlock = Math.max(2, (minTermsInBlock-1)*2 + random().nextInt(100));
+
+    int minTermsAutoPrefix = TestUtil.nextInt(random(), 2, 100);
+    int maxTermsAutoPrefix = random().nextBoolean() ? Math.max(2, (minTermsAutoPrefix-1)*2 + random().nextInt(100)) : Integer.MAX_VALUE;
+
+    //System.out.println("minTermsAutoPrefix " + minTermsAutoPrefix);
+    //System.out.println("maxTermsAutoPrefix " + maxTermsAutoPrefix);
+
+    iwc.setCodec(TestUtil.alwaysPostingsFormat(new Lucene50PostingsFormat(minTermsInBlock, maxTermsInBlock,
+                                                                          minTermsAutoPrefix, maxTermsAutoPrefix)));
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+
+    for (String term : randomTerms) {
+      Document doc = new Document();
+      doc.add(new StringField("field", term, Field.Store.NO));
+      w.addDocument(doc);
+    }
+
+    w.forceMerge(1);
+    IndexReader r = w.getReader();
+    final Terms terms = MultiFields.getTerms(r, "field");
+    IndexSearcher s = new IndexSearcher(r);
+    final int finalActualCount = actualCount;
+    TermRangeQuery q = new TermRangeQuery("field", new BytesRef(startTerm), new BytesRef(endTerm), startInclusive, endInclusive) {
+        public TermRangeQuery checkTerms() throws IOException {
+        TermsEnum termsEnum = getTermsEnum(terms, new AttributeSource());
+        int count = 0;
+        while (termsEnum.next() != null) {
+          //System.out.println("got term: " + termsEnum.term().utf8ToString());
+          count++;
+        }
+        //System.out.println("count " + count);
+
+        // Auto-prefix term(s) should have kicked in, so we should have visited fewer than the total number of aa* terms:
+        assertTrue(count < finalActualCount);
+
+        return this;
+      }
+    }.checkTerms();
+
+    if (random().nextBoolean()) {
+      q.setRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE);
+    } else if (random().nextBoolean()) {
+      q.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_BOOLEAN_QUERY_REWRITE);
+    }
+
+    assertEquals(actualCount, s.search(q, 1).totalHits);
+
+    // Test when min == max:
+    List<String> randomTermsList = new ArrayList<>(randomTerms);
+    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
+      String term = randomTermsList.get(random().nextInt(randomTermsList.size()));
+      q = new TermRangeQuery("field", new BytesRef(term), new BytesRef(term), true, true);
+      assertEquals(1, s.search(q, 1).totalHits);
+    }
+
+    r.close();
+    w.close();
+    dir.close();
+  }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java Tue Oct 28 22:52:49 2014
@@ -126,13 +126,14 @@ public class TestWildcard
 
     MultiTermQuery wq = new WildcardQuery(new Term("field", "prefix*"));
     assertMatches(searcher, wq, 2);
-    Terms terms = MultiFields.getTerms(searcher.getIndexReader(), "field");
-    assertTrue(wq.getTermsEnum(terms) instanceof PrefixTermsEnum);
+    // nocommit not true anymore ... how can we re-assert?
+    //Terms terms = MultiFields.getTerms(searcher.getIndexReader(), "field");
+    // assertTrue(wq.getTermsEnum(terms) instanceof PrefixTermsEnum);
     
     wq = new WildcardQuery(new Term("field", "*"));
     assertMatches(searcher, wq, 2);
-    assertFalse(wq.getTermsEnum(terms) instanceof PrefixTermsEnum);
-    assertFalse(wq.getTermsEnum(terms).getClass().getSimpleName().contains("AutomatonTermsEnum"));
+    //assertFalse(wq.getTermsEnum(terms) instanceof PrefixTermsEnum);
+    //assertFalse(wq.getTermsEnum(terms).getClass().getSimpleName().contains("AutomatonTermsEnum"));
     reader.close();
     indexStore.close();
   }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java Tue Oct 28 22:52:49 2014
@@ -223,7 +223,7 @@ public class TestAutomaton extends Lucen
   }
 
   public void testInterval() throws Exception {
-    Automaton a = Operations.determinize(Automata.makeInterval(17, 100, 3));
+    Automaton a = Operations.determinize(Automata.makeDecimalInterval(17, 100, 3));
     assertFalse(Operations.run(a, ""));
     assertTrue(Operations.run(a, "017"));
     assertTrue(Operations.run(a, "100"));
@@ -414,7 +414,7 @@ public class TestAutomaton extends Lucen
   }
 
   public void testOneInterval() throws Exception {
-    Automaton a = Automata.makeInterval(999, 1032, 0);
+    Automaton a = Automata.makeDecimalInterval(999, 1032, 0);
     a = Operations.determinize(a);
     assertTrue(Operations.run(a, "0999"));
     assertTrue(Operations.run(a, "00999"));
@@ -422,7 +422,7 @@ public class TestAutomaton extends Lucen
   }
 
   public void testAnotherInterval() throws Exception {
-    Automaton a = Automata.makeInterval(1, 2, 0);
+    Automaton a = Automata.makeDecimalInterval(1, 2, 0);
     a = Operations.determinize(a);
     assertTrue(Operations.run(a, "01"));
   }
@@ -445,7 +445,7 @@ public class TestAutomaton extends Lucen
       }
       String prefix = b.toString();
 
-      Automaton a = Operations.determinize(Automata.makeInterval(min, max, digits));
+      Automaton a = Operations.determinize(Automata.makeDecimalInterval(min, max, digits));
       if (random().nextBoolean()) {
         a = MinimizationOperations.minimize(a);
       }
@@ -923,7 +923,7 @@ public class TestAutomaton extends Lucen
           if (VERBOSE) {
             System.out.println("  op=union interval min=" + min + " max=" + max + " digits=" + digits);
           }
-          a = Operations.union(a, Automata.makeInterval(min, max, digits));
+          a = Operations.union(a, Automata.makeDecimalInterval(min, max, digits));
           StringBuilder b = new StringBuilder();
           for(int i=0;i<digits;i++) {
             b.append('0');
@@ -1083,4 +1083,131 @@ public class TestAutomaton extends Lucen
       throw ae;
     }
   }
+
+  public void testMakeBinaryIntervalRandom() throws Exception {
+    int iters = atLeast(100);
+    for(int iter=0;iter<iters;iter++) {
+      BytesRef minTerm = TestUtil.randomBinaryTerm(random());
+      boolean minInclusive = random().nextBoolean();
+      BytesRef maxTerm = TestUtil.randomBinaryTerm(random());
+      boolean maxInclusive = random().nextBoolean();
+
+      if (VERBOSE) {
+        System.out.println("TEST: iter=" + iter + " minTerm=" + minTerm + " minInclusive=" + minInclusive + " maxTerm=" + maxTerm + " maxInclusive=" + maxInclusive);
+      }
+
+      Automaton a = Automata.makeBinaryInterval(minTerm, minInclusive, maxTerm, maxInclusive);
+
+      Automaton minA = MinimizationOperations.minimize(a);
+      if (minA.getNumStates() != a.getNumStates()) {
+        assertTrue(minA.getNumStates() < a.getNumStates());
+        System.out.println("Original was not minimal:");
+        System.out.println("Original:\n" + a.toDot());
+        System.out.println("Minimized:\n" + minA.toDot());
+        fail("auotmaton was not minimal");
+      }
+
+      if (VERBOSE) {
+        System.out.println(a.toDot());
+      }
+
+      for(int iter2=0;iter2<500;iter2++) {
+        BytesRef term = TestUtil.randomBinaryTerm(random());
+        int minCmp = minTerm.compareTo(term);
+        int maxCmp = maxTerm.compareTo(term);
+
+        boolean expected;
+        if (minCmp > 0 || maxCmp < 0) {
+          expected = false;
+        } else if (minCmp == 0 && maxCmp == 0) {
+          expected = minInclusive && maxInclusive;
+        } else if (minCmp == 0) {
+          expected = minInclusive;
+        } else if (maxCmp == 0) {
+          expected = maxInclusive;
+        } else {
+          expected = true;
+        }
+
+        if (VERBOSE) {
+          System.out.println("  check term=" + term + " expected=" + expected);
+        }
+        IntsRefBuilder intsBuilder = new IntsRefBuilder();
+        Util.toIntsRef(term, intsBuilder);
+        assertEquals(expected, Operations.run(a, intsBuilder.toIntsRef()));
+      }
+    }
+  }
+
+  private static IntsRef intsRef(String s) {
+    IntsRefBuilder intsBuilder = new IntsRefBuilder();
+    Util.toIntsRef(new BytesRef(s), intsBuilder);
+    return intsBuilder.toIntsRef();
+  }
+
+  public void testMakeBinaryIntervalBasic() throws Exception {
+    Automaton a = Automata.makeBinaryInterval(new BytesRef("bar"), true, new BytesRef("foo"), true);
+    assertTrue(Operations.run(a, intsRef("bar")));
+    assertTrue(Operations.run(a, intsRef("foo")));
+    assertTrue(Operations.run(a, intsRef("beep")));
+    assertFalse(Operations.run(a, intsRef("baq")));
+    assertTrue(Operations.run(a, intsRef("bara")));
+  }
+
+  public void testMakeBinaryIntervalEqual() throws Exception {
+    Automaton a = Automata.makeBinaryInterval(new BytesRef("bar"), true, new BytesRef("bar"), true);
+    assertTrue(Operations.run(a, intsRef("bar")));
+    assertTrue(Operations.isFinite(a));
+    assertEquals(1, Operations.getFiniteStrings(a, 10).size());
+  }
+
+  public void testMakeBinaryIntervalCommonPrefix() throws Exception {
+    Automaton a = Automata.makeBinaryInterval(new BytesRef("bar"), true, new BytesRef("barfoo"), true);
+    assertFalse(Operations.run(a, intsRef("bam")));
+    assertTrue(Operations.run(a, intsRef("bar")));
+    assertTrue(Operations.run(a, intsRef("bara")));
+    assertTrue(Operations.run(a, intsRef("barf")));
+    assertTrue(Operations.run(a, intsRef("barfo")));
+    assertTrue(Operations.run(a, intsRef("barfoo")));
+    assertTrue(Operations.run(a, intsRef("barfonz")));
+    assertFalse(Operations.run(a, intsRef("barfop")));
+    assertFalse(Operations.run(a, intsRef("barfoop")));
+  }
+
+  public void testMakeBinaryIntervalOpenMax() throws Exception {
+    Automaton a = Automata.makeBinaryInterval(new BytesRef("bar"), true, null, true);
+    assertFalse(Operations.run(a, intsRef("bam")));
+    assertTrue(Operations.run(a, intsRef("bar")));
+    assertTrue(Operations.run(a, intsRef("bara")));
+    assertTrue(Operations.run(a, intsRef("barf")));
+    assertTrue(Operations.run(a, intsRef("barfo")));
+    assertTrue(Operations.run(a, intsRef("barfoo")));
+    assertTrue(Operations.run(a, intsRef("barfonz")));
+    assertTrue(Operations.run(a, intsRef("barfop")));
+    assertTrue(Operations.run(a, intsRef("barfoop")));
+    assertTrue(Operations.run(a, intsRef("zzz")));
+  }
+
+  public void testMakeBinaryIntervalOpenMin() throws Exception {
+    Automaton a = Automata.makeBinaryInterval(null, true, new BytesRef("foo"), true);
+    assertFalse(Operations.run(a, intsRef("foz")));
+    assertFalse(Operations.run(a, intsRef("zzz")));
+    assertTrue(Operations.run(a, intsRef("foo")));
+    assertTrue(Operations.run(a, intsRef("")));
+    assertTrue(Operations.run(a, intsRef("a")));
+    assertTrue(Operations.run(a, intsRef("aaa")));
+    assertTrue(Operations.run(a, intsRef("bz")));
+  }
+
+  public void testMakeBinaryIntervalOpenBoth() throws Exception {
+    Automaton a = Automata.makeBinaryInterval(null, true, null, true);
+    System.out.println(a.toDot());
+    assertTrue(Operations.run(a, intsRef("foz")));
+    assertTrue(Operations.run(a, intsRef("zzz")));
+    assertTrue(Operations.run(a, intsRef("foo")));
+    assertTrue(Operations.run(a, intsRef("")));
+    assertTrue(Operations.run(a, intsRef("a")));
+    assertTrue(Operations.run(a, intsRef("aaa")));
+    assertTrue(Operations.run(a, intsRef("bz")));
+  }
 }

Modified: lucene/dev/branches/lucene6005/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java (original)
+++ lucene/dev/branches/lucene6005/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java Tue Oct 28 22:52:49 2014
@@ -164,7 +164,7 @@ public class BBoxStrategy extends Spatia
 
     //Is this a hack?  We assume that numericValue() is only called for DocValues purposes.
     @Override
-    public Number numericValue() {
+    public Number numericDocValue() {
       //Numeric DocValues only supports Long,
       final Number number = super.numericValue();
       if (number == null)

Modified: lucene/dev/branches/lucene6005/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java Tue Oct 28 22:52:49 2014
@@ -82,8 +82,9 @@ public abstract class SpatialTestCase ex
   protected IndexWriterConfig newIndexWriterConfig(Random random) {
     final IndexWriterConfig indexWriterConfig = LuceneTestCase.newIndexWriterConfig(random, new MockAnalyzer(random));
     //TODO can we randomly choose a doc-values supported format?
-    if (needsDocValues())
-      indexWriterConfig.setCodec( TestUtil.getDefaultCodec());
+    if (needsDocValues()) {
+      indexWriterConfig.setCodec(TestUtil.getDefaultCodec());
+    }
     return indexWriterConfig;
   }
 

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java Tue Oct 28 22:52:49 2014
@@ -235,13 +235,13 @@ public class AssertingLeafReader extends
     @Override
     public TermState termState() throws IOException {
       assert state == State.POSITIONED : "termState() called on unpositioned TermsEnum";
-      return super.termState();
+      return in.termState();
     }
 
     @Override
     public void seekExact(BytesRef term, TermState state) throws IOException {
       assert term.isValid();
-      super.seekExact(term, state);
+      in.seekExact(term, state);
       this.state = State.POSITIONED;
     }
 

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Tue Oct 28 22:52:49 2014
@@ -82,14 +82,13 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.search.FieldDoc;
-import org.apache.lucene.search.FilteredQuery;
 import org.apache.lucene.search.FilteredQuery.FilterStrategy;
+import org.apache.lucene.search.FilteredQuery;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.NoLockFactory;
 import org.junit.Assert;
-
 import com.carrotsearch.randomizedtesting.generators.RandomInts;
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 
@@ -699,6 +698,14 @@ public final class TestUtil {
     return new String(buffer, 0, i);
   }
 
+  /** Returns a random binary term. */
+  public static BytesRef randomBinaryTerm(Random r) {
+    int length = r.nextInt(15);
+    BytesRef b = new BytesRef(length);
+    r.nextBytes(b.bytes);
+    b.length = length;
+    return b;
+  }
   
   /** Return a Codec that can read any of the
    *  default codecs and formats, but always writes in the specified
@@ -1143,6 +1150,24 @@ public final class TestUtil {
       return sb.toString();
     }
   }
+
+  /** For debugging: tries to include br.utf8ToString(), but if that
+   *  fails (because it's not valid utf8, which is fine!), just
+   *  use ordinary toString. */
+  public static String brToString(BytesRef br) {
+    if (br == null) {
+      return "(null)";
+    } else {
+      try {
+        return br.utf8ToString() + " " + br.toString();
+      } catch (Throwable t) {
+        // If BytesRef isn't actually UTF8, or it's eg a
+        // prefix of UTF8 that ends mid-unicode-char, we
+        // fallback to hex:
+        return br.toString();
+      }
+    }
+  }
   
   /** List of characters that match {@link Character#isWhitespace} */
   public static final char[] WHITESPACE_CHARACTERS = new char[] {