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 2015/04/02 17:05:49 UTC

svn commit: r1670918 [1/5] - in /lucene/dev/trunk/lucene: ./ codecs/src/java/org/apache/lucene/codecs/autoprefix/ codecs/src/resources/META-INF/services/ codecs/src/test/org/apache/lucene/codecs/autoprefix/ core/src/java/org/apache/lucene/codecs/ core/...

Author: mikemccand
Date: Thu Apr  2 15:05:48 2015
New Revision: 1670918

URL: http://svn.apache.org/r1670918
Log:
LUCENE-5879: add auto-prefix terms to block tree, and experimental AutoPrefixTermsPostingsFormat

Added:
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java   (with props)
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java   (with props)
    lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/
    lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java   (with props)
    lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/RandomPostingsTester.java   (with props)
Removed:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermRangeTermsEnum.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermContext.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PrefixQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1670918&r1=1670917&r2=1670918&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu Apr  2 15:05:48 2015
@@ -19,6 +19,10 @@ New Features
   for counting ranges that align with the underlying terms as defined by the
   NumberRangePrefixTree (e.g. familiar date units like days).  (David Smiley)
 
+* LUCENE-5879: Added experimental auto-prefix terms to BlockTree terms
+  dictionary, exposed as AutoPrefixPostingsFormat (Adrien Grand,
+  Uwe Schindler, Robert Muir, Mike McCandless)
+
 API Changes
 
 * LUCENE-3312: The API of oal.document was restructured to

Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java?rev=1670918&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java Thu Apr  2 15:05:48 2015
@@ -0,0 +1,125 @@
+package org.apache.lucene.codecs.autoprefix;
+
+/*
+ * 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.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.PostingsReaderBase;
+import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
+import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
+import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
+import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.IOUtils;
+
+/**
+ * Just like {@link Lucene50PostingsFormat} except this format
+ * exposes the experimental auto-prefix terms.
+ *
+ * @lucene.experimental
+ */
+
+public final class AutoPrefixPostingsFormat extends PostingsFormat {
+
+  private final int minItemsInBlock;
+  private final int maxItemsInBlock;
+  private final int minItemsInAutoPrefix;
+  private final int maxItemsInAutoPrefix;
+
+  /** Creates {@code AutoPrefixPostingsFormat} with default settings. */
+  public AutoPrefixPostingsFormat() {
+    this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
+         BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE,
+         25, 48);
+  }
+
+  /** Creates {@code Lucene50PostingsFormat} with custom
+   *  values for {@code minBlockSize} and {@code
+   *  maxBlockSize} passed to block terms dictionary.
+   *  @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
+  public AutoPrefixPostingsFormat(int minItemsInAutoPrefix, int maxItemsInAutoPrefix) {
+    this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
+         BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE,
+         minItemsInAutoPrefix,
+         maxItemsInAutoPrefix);
+  }
+
+  /** Creates {@code Lucene50PostingsFormat} with custom
+   *  values for {@code minBlockSize}, {@code
+   *  maxBlockSize}, {@code minItemsInAutoPrefix} and {@code maxItemsInAutoPrefix}, passed
+   *  to block tree terms dictionary.
+   *  @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int,int,int) */
+  public AutoPrefixPostingsFormat(int minItemsInBlock, int maxItemsInBlock, int minItemsInAutoPrefix, int maxItemsInAutoPrefix) {
+    super("AutoPrefix");
+    BlockTreeTermsWriter.validateSettings(minItemsInBlock,
+                                          maxItemsInBlock);
+    BlockTreeTermsWriter.validateAutoPrefixSettings(minItemsInAutoPrefix,
+                                                    maxItemsInAutoPrefix);
+    this.minItemsInBlock = minItemsInBlock;
+    this.maxItemsInBlock = maxItemsInBlock;
+    this.minItemsInAutoPrefix = minItemsInAutoPrefix;
+    this.maxItemsInAutoPrefix = maxItemsInAutoPrefix;
+  }
+
+  @Override
+  public String toString() {
+    return getName();
+  }
+
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    PostingsWriterBase postingsWriter = new Lucene50PostingsWriter(state);
+
+    boolean success = false;
+    try {
+      FieldsConsumer ret = new BlockTreeTermsWriter(state, 
+                                                    postingsWriter,
+                                                    minItemsInBlock, 
+                                                    maxItemsInBlock,
+                                                    minItemsInAutoPrefix,
+                                                    maxItemsInAutoPrefix);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(postingsWriter);
+      }
+    }
+  }
+
+  @Override
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+    PostingsReaderBase postingsReader = new Lucene50PostingsReader(state);
+    boolean success = false;
+    try {
+      FieldsProducer ret = new BlockTreeTermsReader(postingsReader, state);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(postingsReader);
+      }
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java?rev=1670918&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java Thu Apr  2 15:05:48 2015
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * An experimental postings format that automatically indexes appropriate
+ * prefix terms for fast range and prefix queries.
+ */
+package org.apache.lucene.codecs.autoprefix;

Modified: lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat?rev=1670918&r1=1670917&r2=1670918&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat (original)
+++ lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat Thu Apr  2 15:05:48 2015
@@ -20,3 +20,4 @@ org.apache.lucene.codecs.memory.FSTOrdPo
 org.apache.lucene.codecs.memory.FSTPostingsFormat
 org.apache.lucene.codecs.memory.MemoryPostingsFormat
 org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat
+org.apache.lucene.codecs.autoprefix.AutoPrefixPostingsFormat

Added: lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java?rev=1670918&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java Thu Apr  2 15:05:48 2015
@@ -0,0 +1,38 @@
+package org.apache.lucene.codecs.autoprefix;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.RandomPostingsTester;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+/**
+ * Tests AutoPrefix's postings
+ */
+
+// NOTE: we don't extend BasePostingsFormatTestCase becase we can only handle DOCS_ONLY fields:
+
+public class TestAutoPrefixPostingsFormat extends LuceneTestCase {
+  public void test() throws Exception {
+    new RandomPostingsTester(random()).testFull(TestUtil.alwaysPostingsFormat(new AutoPrefixPostingsFormat()),
+                                                createTempDir("autoprefix"),
+                                                IndexOptions.DOCS,
+                                                false);
+  }
+}

Added: lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java?rev=1670918&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java Thu Apr  2 15:05:48 2015
@@ -0,0 +1,738 @@
+package org.apache.lucene.codecs.autoprefix;
+
+/*
+ * 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.IndexOptions;
+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.PostingsEnum;
+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.Automata;
+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 AutoPrefixPostingsFormat(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(Automata.makeBinaryInterval(minTerm, minInclusive, maxTerm, maxInclusive),
+                                                   true, false, Integer.MAX_VALUE, true);
+
+      TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
+      NumericDocValues docValues = MultiDocValues.getNumericValues(r, "field");
+      PostingsEnum postingsEnum = 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());
+        postingsEnum = te.postings(null, postingsEnum);
+        int docID;
+        while ((docID = postingsEnum.nextDoc()) != PostingsEnum.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 {
+    if (VERBOSE) {
+      System.out.println("TEST: minItemsPerBlock=" + minItemsPerBlock);
+      System.out.println("TEST: maxItemsPerBlock=" + maxItemsPerBlock);
+      System.out.println("TEST: minTermsAutoPrefix=" + minTermsAutoPrefix);
+      System.out.println("TEST: maxTermsAutoPrefix=" + maxTermsAutoPrefix);
+    }
+    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()) {
+      if (VERBOSE) System.out.println("TEST: now force merge");
+      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(Automata.makeBinaryInterval(minTerm, minInclusive, maxTerm, maxInclusive),
+                                                   true, false, Integer.MAX_VALUE, true);
+
+      TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
+      NumericDocValues docValues = MultiDocValues.getNumericValues(r, "field");
+      PostingsEnum postingsEnum = 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());        
+        postingsEnum = te.postings(null, postingsEnum);
+        int docID;
+        while ((docID = postingsEnum.nextDoc()) != PostingsEnum.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()) {
+      if (VERBOSE) {
+        System.out.println("TEST: now force merge");
+      }
+      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(PrefixQuery.toAutomaton(prefixBR), true, false, Integer.MAX_VALUE, true);
+      TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
+      BinaryDocValues docValues = MultiDocValues.getBinaryValues(r, "field");
+      PostingsEnum postingsEnum = 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());        
+        postingsEnum = te.postings(null, postingsEnum);
+        int docID;
+        while ((docID = postingsEnum.nextDoc()) != PostingsEnum.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 {
+    if (VERBOSE) {
+      System.out.println("\nTEST: minTermsAutoPrefix=" + minTermsAutoPrefix + " maxTermsAutoPrefix=" + maxTermsAutoPrefix);
+      System.out.println("\nTEST: minItemsPerBlock=" + minItemsPerBlock + " maxItemsPerBlock=" + maxItemsPerBlock);
+    }
+    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(PrefixQuery.toAutomaton(new BytesRef("a")), false, false, Integer.MAX_VALUE, true);
+    TermsEnum te = ca.getTermsEnum(terms);
+    PostingsEnum postingsEnum = 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());
+      postingsEnum = te.postings(null, postingsEnum);
+      int docID;
+      while ((docID = postingsEnum.nextDoc()) != PostingsEnum.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_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 {
+      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.bytesRefToString(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);
+    }
+  }
+
+  /** Make sure you get clear exc. if you try to use this within anything but IndexOptions.DOCS fields. */
+  public void testWithFreqs() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    iwc.setCodec(codec);
+    IndexWriter w = new IndexWriter(dir, iwc);
+    FieldType ft = new FieldType(StringField.TYPE_NOT_STORED);
+    ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
+    Document doc = new Document();
+    doc.add(new Field("foo", "bar bar", ft));
+    w.addDocument(doc);
+    try {
+      w.commit();
+    } catch (IllegalStateException ise) {
+      assertEquals("ranges can only be indexed with IndexOptions.DOCS (field: foo)", ise.getMessage());
+    }
+    w.close();
+    dir.close();
+  }
+
+  /** Make sure you get clear exc. if you try to use this within anything but IndexOptions.DOCS fields. */
+  public void testWithPositions() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    iwc.setCodec(codec);
+    IndexWriter w = new IndexWriter(dir, iwc);
+    FieldType ft = new FieldType(StringField.TYPE_NOT_STORED);
+    ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+    Document doc = new Document();
+    doc.add(new Field("foo", "bar bar", ft));
+    w.addDocument(doc);
+    try {
+      w.commit();
+    } catch (IllegalStateException ise) {
+      assertEquals("ranges can only be indexed with IndexOptions.DOCS (field: foo)", ise.getMessage());
+    }
+    w.close();
+    dir.close();
+  }
+
+  /** Make sure you get clear exc. if you try to use this within anything but IndexOptions.DOCS fields. */
+  public void testWithOffsets() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    iwc.setCodec(codec);
+    IndexWriter w = new IndexWriter(dir, iwc);
+    FieldType ft = new FieldType(StringField.TYPE_NOT_STORED);
+    ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Document doc = new Document();
+    doc.add(new Field("foo", "bar bar", ft));
+    w.addDocument(doc);
+    try {
+      w.commit();
+    } catch (IllegalStateException ise) {
+      assertEquals("ranges can only be indexed with IndexOptions.DOCS (field: foo)", ise.getMessage());
+    }
+    w.close();
+    dir.close();
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java?rev=1670918&r1=1670917&r2=1670918&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java Thu Apr  2 15:05:48 2015
@@ -16,6 +16,7 @@ package org.apache.lucene.codecs;
  * limitations under the License.
  */
 
+import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader; // javadocs
 import org.apache.lucene.index.OrdTermState;
 import org.apache.lucene.index.TermState;
 
@@ -23,6 +24,8 @@ import org.apache.lucene.index.TermState
  * Holds all state required for {@link PostingsReaderBase}
  * to produce a {@link org.apache.lucene.index.PostingsEnum} without re-seeking the
  * terms dict.
+ *
+ * @lucene.internal
  */
 public class BlockTermState extends OrdTermState {
   /** how many docs have this term */
@@ -36,6 +39,11 @@ public class BlockTermState extends OrdT
   // TODO: update BTR to nuke this
   public long blockFilePointer;
 
+  /** True if this term is "real" (e.g., not an auto-prefix term or
+   *  some other "secret" term; currently only {@link BlockTreeTermsReader}
+   *  sets this). */
+  public boolean isRealTerm;
+
   /** Sole constructor. (For invocation by subclass 
    *  constructors, typically implicit.) */
   protected BlockTermState() {
@@ -50,10 +58,11 @@ public class BlockTermState extends OrdT
     totalTermFreq = other.totalTermFreq;
     termBlockOrd = other.termBlockOrd;
     blockFilePointer = other.blockFilePointer;
+    isRealTerm = other.isRealTerm;
   }
 
   @Override
   public String toString() {
-    return "docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termBlockOrd=" + termBlockOrd + " blockFP=" + blockFilePointer;
+    return "docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termBlockOrd=" + termBlockOrd + " blockFP=" + blockFilePointer + " isRealTerm=" + isRealTerm;
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java?rev=1670918&r1=1670917&r2=1670918&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java Thu Apr  2 15:05:48 2015
@@ -62,6 +62,7 @@ public abstract class PostingsFormat imp
    * @param name must be all ascii alphanumeric, and less than 128 characters in length.
    */
   protected PostingsFormat(String name) {
+    // TODO: can we somehow detect name conflicts here?  Two different classes trying to claim the same name?  Otherwise you see confusing errors...
     NamedSPILoader.checkServiceName(name);
     this.name = name;
   }

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java?rev=1670918&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java Thu Apr  2 15:05:48 2015
@@ -0,0 +1,415 @@
+package org.apache.lucene.codecs.blocktree;
+
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.lucene.index.FilteredTermsEnum;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.StringHelper;
+
+// TODO: instead of inlining auto-prefix terms with normal terms,
+// we could write them into their own virtual/private field.  This
+// would make search time a bit more complex, since we'd need to
+// merge sort between two TermEnums, but it would also make stats
+// API (used by CheckIndex -verbose) easier to implement since we could
+// just walk this virtual field and gather its stats)
+
+/** Used in the first pass when writing a segment to locate
+ *  "appropriate" auto-prefix terms to pre-compile into the index.
+ *  This visits every term in the index to find prefixes that
+ *  match >= min and <= max number of terms. */
+
+class AutoPrefixTermsWriter {
+
+  //static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+  //static boolean DEBUG = false;
+  //static boolean DEBUG2 = BlockTreeTermsWriter.DEBUG2;
+  //static boolean DEBUG2 = true;
+
+  /** Describes a range of term-space to match, either a simple prefix
+   *  (foo*) or a floor-block range of a prefix (e.g. foo[a-m]*,
+   *  foo[n-z]*) when there are too many terms starting with foo*. */
+  public static final class PrefixTerm implements Comparable<PrefixTerm> {
+    /** Common prefix */
+    public final byte[] prefix;
+
+    /** If this is -2, this is a normal prefix (foo *), else it's the minimum lead byte of the suffix (e.g. 'd' in foo[d-m]*). */
+    public final int floorLeadStart;
+
+    /** The lead byte (inclusive) of the suffix for the term range we match (e.g. 'm' in foo[d-m*]); this is ignored when
+     *  floorLeadStart is -2. */
+    public final int floorLeadEnd;
+
+    public final BytesRef term;
+
+    /** Sole constructor. */
+    public PrefixTerm(byte[] prefix, int floorLeadStart, int floorLeadEnd) {
+      this.prefix = prefix;
+      this.floorLeadStart = floorLeadStart;
+      this.floorLeadEnd = floorLeadEnd;
+      this.term = toBytesRef(prefix, floorLeadStart);
+
+      assert floorLeadEnd >= floorLeadStart;
+      assert floorLeadEnd >= 0;
+      assert floorLeadStart == -2 || floorLeadStart >= 0;
+
+      // We should never create empty-string prefix term:
+      assert prefix.length > 0 || floorLeadStart != -2 || floorLeadEnd != 0xff;
+    }
+
+    @Override
+    public String toString() {
+      String s = brToString(new BytesRef(prefix));
+      if (floorLeadStart == -2) {
+        s += "[-" + Integer.toHexString(floorLeadEnd) + "]";
+      } else {
+        s += "[" + Integer.toHexString(floorLeadStart) + "-" + Integer.toHexString(floorLeadEnd) + "]";
+      }
+      return s;
+    }
+
+    @Override
+    public int compareTo(PrefixTerm other) {
+      int cmp = term.compareTo(other.term);
+      if (cmp == 0) {
+        if (prefix.length != other.prefix.length) {
+          return prefix.length - other.prefix.length;
+        }
+
+        // On tie, sort the bigger floorLeadEnd, earlier, since it
+        // spans more terms, so during intersect, we want to encounter this one
+        // first so we can use it if the automaton accepts the larger range:
+        cmp = other.floorLeadEnd - floorLeadEnd;
+      }
+
+      return cmp;
+    }
+
+    /** Returns the leading term for this prefix term, e.g. "foo" (for
+     *  the foo* prefix) or "foom" (for the foo[m-z]* case). */
+    private static BytesRef toBytesRef(byte[] prefix, int floorLeadStart) {
+      BytesRef br;
+      if (floorLeadStart != -2) {
+        assert floorLeadStart >= 0;
+        br = new BytesRef(prefix.length+1);
+      } else {
+        br = new BytesRef(prefix.length);
+      }
+      System.arraycopy(prefix, 0, br.bytes, 0, prefix.length);
+      br.length = prefix.length;
+      if (floorLeadStart != -2) {
+        assert floorLeadStart >= 0;
+        br.bytes[br.length++] = (byte) floorLeadStart;
+      }
+
+      return br;
+    }
+
+    public int compareTo(BytesRef term) {
+      return this.term.compareTo(term);
+    }
+
+    public TermsEnum getTermsEnum(TermsEnum in) {
+
+      final BytesRef prefixRef = new BytesRef(prefix);
+
+      return new FilteredTermsEnum(in) {
+          {
+            setInitialSeekTerm(term);
+          }
+
+          @Override
+          protected AcceptStatus accept(BytesRef term) {
+            if (StringHelper.startsWith(term, prefixRef) &&
+                (floorLeadEnd == -1 || term.length == prefixRef.length || (term.bytes[term.offset + prefixRef.length] & 0xff) <= floorLeadEnd)) {
+              return AcceptStatus.YES;
+            } else {
+              return AcceptStatus.END;
+            }
+          }
+        };
+    }
+  }
+
+  // for debugging
+  static String brToString(BytesRef b) {
+    try {
+      return b.utf8ToString() + " " + b;
+    } 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 b.toString();
+    }
+  }
+
+  final List<PrefixTerm> prefixes = new ArrayList<>();
+  private final int minItemsInPrefix;
+  private final int maxItemsInPrefix;
+
+  // Records index into pending where the current prefix at that
+  // length "started"; for example, if current term starts with 't',
+  // startsByPrefix[0] is the index into pending for the first
+  // term/sub-block starting with 't'.  We use this to figure out when
+  // to write a new block:
+  private final BytesRefBuilder lastTerm = new BytesRefBuilder();
+  private int[] prefixStarts = new int[8];
+  private List<Object> pending = new ArrayList<>();
+
+  //private final String segment;
+
+  public AutoPrefixTermsWriter(Terms terms, int minItemsInPrefix, int maxItemsInPrefix) throws IOException {
+    this.minItemsInPrefix = minItemsInPrefix;
+    this.maxItemsInPrefix = maxItemsInPrefix;
+    //this.segment = segment;
+
+    TermsEnum termsEnum = terms.iterator(null);
+    while (true) {
+      BytesRef term = termsEnum.next();
+      if (term == null) {
+        break;
+      }
+      //if (DEBUG) System.out.println("pushTerm: " + brToString(term));
+      pushTerm(term);
+    }
+
+    if (pending.size() > 1) {
+      pushTerm(BlockTreeTermsWriter.EMPTY_BYTES_REF);
+
+      // Also maybe save floor prefixes in root block; this can be a biggish perf gain for large ranges:
+      /*
+      System.out.println("root block pending.size=" + pending.size());
+      for(Object o : pending) {
+        System.out.println("  " + o);
+      }
+      */
+      while (pending.size() >= minItemsInPrefix) {
+        savePrefixes(0, pending.size());
+      }
+    }
+
+    Collections.sort(prefixes);
+  }
+
+  /** Pushes the new term to the top of the stack, and writes new blocks. */
+  private void pushTerm(BytesRef text) throws IOException {
+    int limit = Math.min(lastTerm.length(), text.length);
+
+    // Find common prefix between last term and current term:
+    int pos = 0;
+    while (pos < limit && lastTerm.byteAt(pos) == text.bytes[text.offset+pos]) {
+      pos++;
+    }
+
+    //if (DEBUG) System.out.println("  shared=" + pos + "  lastTerm.length=" + lastTerm.length());
+
+    // Close the "abandoned" suffix now:
+    for(int i=lastTerm.length()-1;i>=pos;i--) {
+
+      // How many items on top of the stack share the current suffix
+      // we are closing:
+      int prefixTopSize = pending.size() - prefixStarts[i];
+
+      while (prefixTopSize >= minItemsInPrefix) {       
+        //if (DEBUG) System.out.println("pushTerm i=" + i + " prefixTopSize=" + prefixTopSize + " minItemsInBlock=" + minItemsInPrefix);
+        savePrefixes(i+1, prefixTopSize);
+        //prefixStarts[i] -= prefixTopSize;
+        //System.out.println("    after savePrefixes: " + (pending.size() - prefixStarts[i]) + " pending.size()=" + pending.size() + " start=" + prefixStarts[i]);
+
+        // For large floor blocks, it's possible we should now re-run on the new prefix terms we just created:
+        prefixTopSize = pending.size() - prefixStarts[i];
+      }
+    }
+
+    if (prefixStarts.length < text.length) {
+      prefixStarts = ArrayUtil.grow(prefixStarts, text.length);
+    }
+
+    // Init new tail:
+    for(int i=pos;i<text.length;i++) {
+      prefixStarts[i] = pending.size();
+    }
+
+    lastTerm.copyBytes(text);
+
+    // Only append the first (optional) empty string, no the fake last one used to close all prefixes:
+    if (text.length > 0 || pending.isEmpty()) {
+      byte[] termBytes = new byte[text.length];
+      System.arraycopy(text.bytes, text.offset, termBytes, 0, text.length);
+      pending.add(termBytes);
+    }
+  }
+  
+  void savePrefixes(int prefixLength, int count) throws IOException {
+
+    assert count > 0;
+
+    //if (DEBUG2) {
+    //  BytesRef br = new BytesRef(lastTerm.bytes());
+    //  br.length = prefixLength;
+    //  System.out.println("  savePrefixes: seg=" + segment + " " + brToString(br) + " count=" + count + " pending.size()=" + pending.size());
+    //}
+
+    int lastSuffixLeadLabel = -2;
+
+    int start = pending.size()-count;
+    assert start >=0;
+
+    int end = pending.size();
+    int nextBlockStart = start;
+    int nextFloorLeadLabel = -1;
+    int prefixCount = 0;
+    int pendingCount = 0;
+    PrefixTerm lastPTEntry = null;
+    for (int i=start; i<end; i++) {
+
+      byte[] termBytes;
+      Object o = pending.get(i);
+      PrefixTerm ptEntry;
+      if (o instanceof byte[]) {
+        ptEntry = null;
+        termBytes = (byte[]) o;
+      } else {
+        ptEntry = (PrefixTerm) o;
+        termBytes = ptEntry.term.bytes;
+        if (ptEntry.prefix.length != prefixLength) {
+          assert ptEntry.prefix.length > prefixLength;
+          ptEntry = null;
+        }
+      }
+      pendingCount++;
+
+      //if (DEBUG) System.out.println("    check term=" + brToString(new BytesRef(termBytes)));
+
+      int suffixLeadLabel;
+
+      if (termBytes.length == prefixLength) {
+        // Suffix is 0, i.e. prefix 'foo' and term is
+        // 'foo' so the term has empty string suffix
+        // in this block
+        assert lastSuffixLeadLabel == -2;
+        suffixLeadLabel = -2;
+      } else {
+        suffixLeadLabel = termBytes[prefixLength] & 0xff;
+      }
+
+      // if (DEBUG) System.out.println("  i=" + i + " ent=" + ent + " suffixLeadLabel=" + suffixLeadLabel);
+
+      if (suffixLeadLabel != lastSuffixLeadLabel) {
+        // This is a boundary, a chance to make an auto-prefix term if we want:
+
+        // When we are "recursing" (generating auto-prefix terms on a block of
+        // floor'd auto-prefix terms), this assert is non-trivial because it
+        // ensures the floorLeadEnd of the previous terms is in fact less
+        // than the lead start of the current entry:
+        assert suffixLeadLabel > lastSuffixLeadLabel: "suffixLeadLabel=" + suffixLeadLabel + " vs lastSuffixLeadLabel=" + lastSuffixLeadLabel;
+
+        // NOTE: must check nextFloorLeadLabel in case minItemsInPrefix is 2 and prefix is 'a' and we've seen 'a' and then 'aa'
+        if (pendingCount >= minItemsInPrefix && end-nextBlockStart > maxItemsInPrefix && nextFloorLeadLabel != -1) {
+          // The count is too large for one block, so we must break it into "floor" blocks, where we record
+          // the leading label of the suffix of the first term in each floor block, so at search time we can
+          // jump to the right floor block.  We just use a naive greedy segmenter here: make a new floor
+          // block as soon as we have at least minItemsInBlock.  This is not always best: it often produces
+          // a too-small block as the final block:
+
+          // If the last entry was another prefix term of the same length, then it represents a range of terms, so we must use its ending
+          // prefix label as our ending label:
+          if (lastPTEntry != null) {
+            lastSuffixLeadLabel = lastPTEntry.floorLeadEnd;
+          }
+
+          savePrefix(prefixLength, nextFloorLeadLabel, lastSuffixLeadLabel);
+          pendingCount = 0;
+
+          prefixCount++;
+          nextFloorLeadLabel = suffixLeadLabel;
+          nextBlockStart = i;
+        }
+
+        if (nextFloorLeadLabel == -1) {
+          nextFloorLeadLabel = suffixLeadLabel;
+          //if (DEBUG) System.out.println("set first lead label=" + nextFloorLeadLabel);
+        }
+
+        lastSuffixLeadLabel = suffixLeadLabel;
+      }
+      lastPTEntry = ptEntry;
+    }
+
+    // Write last block, if any:
+    if (nextBlockStart < end) {
+      //System.out.println("  lastPTEntry=" + lastPTEntry + " lastSuffixLeadLabel=" + lastSuffixLeadLabel);
+      if (lastPTEntry != null) {
+        lastSuffixLeadLabel = lastPTEntry.floorLeadEnd;
+      }
+      assert lastSuffixLeadLabel >= nextFloorLeadLabel: "lastSuffixLeadLabel=" + lastSuffixLeadLabel + " nextFloorLeadLabel=" + nextFloorLeadLabel;
+      if (prefixCount == 0) {
+        if (prefixLength > 0) {
+          savePrefix(prefixLength, -2, 0xff);
+          prefixCount++;
+        } else {
+          // Don't add a prefix term for all terms in the index!
+        }
+      } else {
+        if (lastSuffixLeadLabel == -2) {
+          // Special case when closing the empty string root block:
+          lastSuffixLeadLabel = 0xff;
+        }
+        savePrefix(prefixLength, nextFloorLeadLabel, lastSuffixLeadLabel);
+        prefixCount++;
+      }
+    }
+
+    // Remove slice from the top of the pending stack, that we just wrote:
+    int sizeToClear = count;
+    if (prefixCount > 1) {
+      Object o = pending.get(pending.size()-count);
+      if (o instanceof byte[] && ((byte[]) o).length == prefixLength) {
+        // If we were just asked to write all f* terms, but there were too many and so we made floor blocks, the exact term 'f' will remain
+        // as its own item, followed by floor block terms like f[a-m]*, f[n-z]*, so in this case we leave 3 (not 2) items on the pending stack:
+        sizeToClear--;
+      }
+    }
+    pending.subList(pending.size()-sizeToClear, pending.size()).clear();
+
+    // Append prefix terms for each prefix, since these count like real terms that also need to be "rolled up":
+    for(int i=0;i<prefixCount;i++) {
+      PrefixTerm pt = prefixes.get(prefixes.size()-(prefixCount-i));
+      pending.add(pt);
+    }
+  }
+
+  private void savePrefix(int prefixLength, int floorLeadStart, int floorLeadEnd) {
+    byte[] prefix = new byte[prefixLength];
+    System.arraycopy(lastTerm.bytes(), 0, prefix, 0, prefixLength);
+    assert floorLeadStart != -1;
+    assert floorLeadEnd != -1;
+
+    PrefixTerm pt = new PrefixTerm(prefix, floorLeadStart, floorLeadEnd); 
+    //if (DEBUG2) System.out.println("    savePrefix: seg=" + segment + " " + pt + " count=" + count);
+    prefixes.add(pt);
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java?rev=1670918&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java Thu Apr  2 15:05:48 2015
@@ -0,0 +1,95 @@
+package org.apache.lucene.codecs.blocktree;
+
+/*
+ * 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.PostingsEnum;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.BitSetIterator;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FixedBitSet; // javadocs
+
+/** Takes a {@link FixedBitSet} and creates a DOCS {@link PostingsEnum} from it. */
+
+class BitSetPostingsEnum extends PostingsEnum {
+  private final BitSet bits;
+  private DocIdSetIterator in;
+  
+  BitSetPostingsEnum(BitSet bits) {
+    this.bits = bits;
+    reset();
+  }
+
+  @Override
+  public int freq() throws IOException {
+    return 1;
+  }
+
+  @Override
+  public int docID() {
+    if (in == null) {
+      return -1;
+    } else {
+      return in.docID();
+    }
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    if (in == null) {
+      in = new BitSetIterator(bits, 0);
+    }
+    return in.nextDoc();
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    return in.advance(target);
+  }
+
+  @Override
+  public long cost() {
+    return in.cost();
+  }
+  
+  void reset() {
+    in = null;
+  }
+
+  @Override
+  public BytesRef getPayload() {
+    return null;
+  }
+
+  @Override
+  public int nextPosition() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int startOffset() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int endOffset() {
+    throw new UnsupportedOperationException();
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java?rev=1670918&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java Thu Apr  2 15:05:48 2015
@@ -0,0 +1,87 @@
+package org.apache.lucene.codecs.blocktree;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+
+/** Silly stub class, used only when writing an auto-prefix
+ *  term in order to expose DocsEnum over a FixedBitSet.  We
+ *  pass this to {@link PostingsWriterBase#writeTerm} so 
+ *  that it can pull .docs() multiple times for the
+ *  current term. */
+
+class BitSetTermsEnum extends TermsEnum {
+  private final BitSetPostingsEnum postingsEnum;
+
+  public BitSetTermsEnum(BitSet docs) {
+    postingsEnum = new BitSetPostingsEnum(docs);
+  }
+
+  @Override
+  public SeekStatus seekCeil(BytesRef text) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void seekExact(long ord) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public BytesRef term() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public BytesRef next() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public long ord() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int docFreq() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public long totalTermFreq() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) {
+    if (flags != PostingsEnum.NONE) {
+      // We only work with DOCS_ONLY fields
+      return null;
+    }
+    if (liveDocs != null) {
+      throw new IllegalArgumentException("cannot handle live docs");
+    }
+    postingsEnum.reset();
+    return postingsEnum;
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java?rev=1670918&r1=1670917&r2=1670918&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java Thu Apr  2 15:05:48 2015
@@ -34,6 +34,8 @@ import org.apache.lucene.index.IndexFile
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.Terms;
+import org.apache.lucene.search.PrefixQuery;  // javadocs
+import org.apache.lucene.search.TermRangeQuery;  // javadocs
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
@@ -57,6 +59,14 @@ import org.apache.lucene.util.fst.Output
  *  min/maxItemsPerBlock during indexing to control how
  *  much memory the terms index uses.</p>
  *
+ *  <p>If auto-prefix terms were indexed (see
+ *  {@link BlockTreeTermsWriter}), then the {@link Terms#intersect}
+ *  implementation here will make use of these terms only if the
+ *  automaton has a binary sink state, i.e. an accept state
+ *  which has a transition to itself accepting all byte values.
+ *  For example, both {@link PrefixQuery} and {@link TermRangeQuery}
+ *  pass such automata to {@link Terms#intersect}.</p>
+ *
  *  <p>The data structure used by this implementation is very
  *  similar to a burst trie
  *  (http://citeseer.ist.psu.edu/viewdoc/summary?doi=10.1.1.18.3499),
@@ -90,8 +100,11 @@ public final class BlockTreeTermsReader
   /** Initial terms format. */
   public static final int VERSION_START = 0;
 
+  /** Auto-prefix terms. */
+  public static final int VERSION_AUTO_PREFIX_TERMS = 1;
+
   /** Current terms format. */
-  public static final int VERSION_CURRENT = VERSION_START;
+  public static final int VERSION_CURRENT = VERSION_AUTO_PREFIX_TERMS;
 
   /** Extension of terms index file */
   static final String TERMS_INDEX_EXTENSION = "tip";
@@ -116,7 +129,7 @@ public final class BlockTreeTermsReader
 
   final String segment;
   
-  private final int version;
+  final int version;
 
   /** Sole constructor. */
   public BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState state) throws IOException {



Re: svn commit: r1670918 [1/5] - in /lucene/dev/trunk/lucene: ./ codecs/src/java/org/apache/lucene/codecs/autoprefix/ codecs/src/resources/META-INF/services/ codecs/src/test/org/apache/lucene/codecs/autoprefix/ core/src/java/org/apache/lucene/codecs/ core/...

Posted by Michael McCandless <lu...@mikemccandless.com>.
Woops, thanks Rob!

Mike McCandless

http://blog.mikemccandless.com


On Thu, Apr 2, 2015 at 11:17 AM, Robert Muir <rc...@gmail.com> wrote:
> NOTE: i hit compile failure like this (TermRangeTermsEnum got removed).
>
> I am going to remove these asserts: to me they don't look very useful,
> and fix the build for now.
>
> compile-test:
>     [mkdir] Created dir:
> /home/rmuir/workspace/trunk/lucene/build/core/classes/test
>     [javac] Compiling 431 source files to
> /home/rmuir/workspace/trunk/lucene/build/core/classes/test
>     [javac] /home/rmuir/workspace/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java:123:
> error: cannot find symbol
>     [javac]     assertFalse(query.getTermsEnum(terms) instanceof
> TermRangeTermsEnum);
>     [javac]                                                      ^
>     [javac]   symbol:   class TermRangeTermsEnum
>     [javac]   location: class TestTermRangeQuery
>     [javac] /home/rmuir/workspace/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java:128:
> error: cannot find symbol
>     [javac]     assertFalse(query.getTermsEnum(terms) instanceof
> TermRangeTermsEnum);
>     [javac]                                                      ^
>     [javac]   symbol:   class TermRangeTermsEnum
>     [javac]   location: class TestTermRangeQuery
>     [javac] /home/rmuir/workspace/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java:132:
> error: cannot find symbol
>     [javac]     assertFalse(query.getTermsEnum(terms) instanceof
> TermRangeTermsEnum);
>     [javac]                                                      ^
>     [javac]   symbol:   class TermRangeTermsEnum
>     [javac]   location: class TestTermRangeQuery
>     [javac] Note: Some input files use or override a deprecated API.
>     [javac] Note: Recompile with -Xlint:deprecation for details.
>     [javac] 3 errors
>
> On Thu, Apr 2, 2015 at 11:05 AM,  <mi...@apache.org> wrote:
>> Author: mikemccand
>> Date: Thu Apr  2 15:05:48 2015
>> New Revision: 1670918
>>
>> URL: http://svn.apache.org/r1670918
>> Log:
>> LUCENE-5879: add auto-prefix terms to block tree, and experimental AutoPrefixTermsPostingsFormat
>>
>> Added:
>>     lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/
>>     lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java   (with props)
>>     lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java   (with props)
>>     lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/
>>     lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java   (with props)
>>     lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java   (with props)
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java   (with props)
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java   (with props)
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java   (with props)
>>     lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/RandomPostingsTester.java   (with props)
>> Removed:
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermRangeTermsEnum.java
>> Modified:
>>     lucene/dev/trunk/lucene/CHANGES.txt
>>     lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermContext.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PrefixQuery.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
>>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java
>>     lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java
>>     lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java
>>     lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java
>>     lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java
>>     lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java
>>     lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java
>>     lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
>>     lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
>>     lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
>>
>> Modified: lucene/dev/trunk/lucene/CHANGES.txt
>> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1670918&r1=1670917&r2=1670918&view=diff
>> ==============================================================================
>> --- lucene/dev/trunk/lucene/CHANGES.txt (original)
>> +++ lucene/dev/trunk/lucene/CHANGES.txt Thu Apr  2 15:05:48 2015
>> @@ -19,6 +19,10 @@ New Features
>>    for counting ranges that align with the underlying terms as defined by the
>>    NumberRangePrefixTree (e.g. familiar date units like days).  (David Smiley)
>>
>> +* LUCENE-5879: Added experimental auto-prefix terms to BlockTree terms
>> +  dictionary, exposed as AutoPrefixPostingsFormat (Adrien Grand,
>> +  Uwe Schindler, Robert Muir, Mike McCandless)
>> +
>>  API Changes
>>
>>  * LUCENE-3312: The API of oal.document was restructured to
>>
>> Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java
>> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java?rev=1670918&view=auto
>> ==============================================================================
>> --- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java (added)
>> +++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java Thu Apr  2 15:05:48 2015
>> @@ -0,0 +1,125 @@
>> +package org.apache.lucene.codecs.autoprefix;
>> +
>> +/*
>> + * 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.codecs.FieldsConsumer;
>> +import org.apache.lucene.codecs.FieldsProducer;
>> +import org.apache.lucene.codecs.PostingsFormat;
>> +import org.apache.lucene.codecs.PostingsReaderBase;
>> +import org.apache.lucene.codecs.PostingsWriterBase;
>> +import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
>> +import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
>> +import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
>> +import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
>> +import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
>> +import org.apache.lucene.index.SegmentReadState;
>> +import org.apache.lucene.index.SegmentWriteState;
>> +import org.apache.lucene.util.IOUtils;
>> +
>> +/**
>> + * Just like {@link Lucene50PostingsFormat} except this format
>> + * exposes the experimental auto-prefix terms.
>> + *
>> + * @lucene.experimental
>> + */
>> +
>> +public final class AutoPrefixPostingsFormat extends PostingsFormat {
>> +
>> +  private final int minItemsInBlock;
>> +  private final int maxItemsInBlock;
>> +  private final int minItemsInAutoPrefix;
>> +  private final int maxItemsInAutoPrefix;
>> +
>> +  /** Creates {@code AutoPrefixPostingsFormat} with default settings. */
>> +  public AutoPrefixPostingsFormat() {
>> +    this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
>> +         BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE,
>> +         25, 48);
>> +  }
>> +
>> +  /** Creates {@code Lucene50PostingsFormat} with custom
>> +   *  values for {@code minBlockSize} and {@code
>> +   *  maxBlockSize} passed to block terms dictionary.
>> +   *  @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
>> +  public AutoPrefixPostingsFormat(int minItemsInAutoPrefix, int maxItemsInAutoPrefix) {
>> +    this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
>> +         BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE,
>> +         minItemsInAutoPrefix,
>> +         maxItemsInAutoPrefix);
>> +  }
>> +
>> +  /** Creates {@code Lucene50PostingsFormat} with custom
>> +   *  values for {@code minBlockSize}, {@code
>> +   *  maxBlockSize}, {@code minItemsInAutoPrefix} and {@code maxItemsInAutoPrefix}, passed
>> +   *  to block tree terms dictionary.
>> +   *  @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int,int,int) */
>> +  public AutoPrefixPostingsFormat(int minItemsInBlock, int maxItemsInBlock, int minItemsInAutoPrefix, int maxItemsInAutoPrefix) {
>> +    super("AutoPrefix");
>> +    BlockTreeTermsWriter.validateSettings(minItemsInBlock,
>> +                                          maxItemsInBlock);
>> +    BlockTreeTermsWriter.validateAutoPrefixSettings(minItemsInAutoPrefix,
>> +                                                    maxItemsInAutoPrefix);
>> +    this.minItemsInBlock = minItemsInBlock;
>> +    this.maxItemsInBlock = maxItemsInBlock;
>> +    this.minItemsInAutoPrefix = minItemsInAutoPrefix;
>> +    this.maxItemsInAutoPrefix = maxItemsInAutoPrefix;
>> +  }
>> +
>> +  @Override
>> +  public String toString() {
>> +    return getName();
>> +  }
>> +
>> +  @Override
>> +  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
>> +    PostingsWriterBase postingsWriter = new Lucene50PostingsWriter(state);
>> +
>> +    boolean success = false;
>> +    try {
>> +      FieldsConsumer ret = new BlockTreeTermsWriter(state,
>> +                                                    postingsWriter,
>> +                                                    minItemsInBlock,
>> +                                                    maxItemsInBlock,
>> +                                                    minItemsInAutoPrefix,
>> +                                                    maxItemsInAutoPrefix);
>> +      success = true;
>> +      return ret;
>> +    } finally {
>> +      if (!success) {
>> +        IOUtils.closeWhileHandlingException(postingsWriter);
>> +      }
>> +    }
>> +  }
>> +
>> +  @Override
>> +  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
>> +    PostingsReaderBase postingsReader = new Lucene50PostingsReader(state);
>> +    boolean success = false;
>> +    try {
>> +      FieldsProducer ret = new BlockTreeTermsReader(postingsReader, state);
>> +      success = true;
>> +      return ret;
>> +    } finally {
>> +      if (!success) {
>> +        IOUtils.closeWhileHandlingException(postingsReader);
>> +      }
>> +    }
>> +  }
>> +}
>>
>> Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java
>> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java?rev=1670918&view=auto
>> ==============================================================================
>> --- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java (added)
>> +++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java Thu Apr  2 15:05:48 2015
>> @@ -0,0 +1,22 @@
>> +/*
>> + * 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.
>> + */
>> +
>> +/**
>> + * An experimental postings format that automatically indexes appropriate
>> + * prefix terms for fast range and prefix queries.
>> + */
>> +package org.apache.lucene.codecs.autoprefix;
>>
>> Modified: lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
>> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat?rev=1670918&r1=1670917&r2=1670918&view=diff
>> ==============================================================================
>> --- lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat (original)
>> +++ lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat Thu Apr  2 15:05:48 2015
>> @@ -20,3 +20,4 @@ org.apache.lucene.codecs.memory.FSTOrdPo
>>  org.apache.lucene.codecs.memory.FSTPostingsFormat
>>  org.apache.lucene.codecs.memory.MemoryPostingsFormat
>>  org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat
>> +org.apache.lucene.codecs.autoprefix.AutoPrefixPostingsFormat
>>
>> Added: lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java
>> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java?rev=1670918&view=auto
>> ==============================================================================
>> --- lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java (added)
>> +++ lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java Thu Apr  2 15:05:48 2015
>> @@ -0,0 +1,38 @@
>> +package org.apache.lucene.codecs.autoprefix;
>> +
>> +/*
>> + * Licensed to the Apache Software Foundation (ASF) under one or more
>> + * contributor license agreements.  See the NOTICE file distributed with
>> + * this work for additional information regarding copyright ownership.
>> + * The ASF licenses this file to You under the Apache License, Version 2.0
>> + * (the "License"); you may not use this file except in compliance with
>> + * the License.  You may obtain a copy of the License at
>> + *
>> + *     http://www.apache.org/licenses/LICENSE-2.0
>> + *
>> + * Unless required by applicable law or agreed to in writing, software
>> + * distributed under the License is distributed on an "AS IS" BASIS,
>> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
>> + * See the License for the specific language governing permissions and
>> + * limitations under the License.
>> + */
>> +
>> +import org.apache.lucene.index.IndexOptions;
>> +import org.apache.lucene.index.RandomPostingsTester;
>> +import org.apache.lucene.util.LuceneTestCase;
>> +import org.apache.lucene.util.TestUtil;
>> +
>> +/**
>> + * Tests AutoPrefix's postings
>> + */
>> +
>> +// NOTE: we don't extend BasePostingsFormatTestCase becase we can only handle DOCS_ONLY fields:
>> +
>> +public class TestAutoPrefixPostingsFormat extends LuceneTestCase {
>> +  public void test() throws Exception {
>> +    new RandomPostingsTester(random()).testFull(TestUtil.alwaysPostingsFormat(new AutoPrefixPostingsFormat()),
>> +                                                createTempDir("autoprefix"),
>> +                                                IndexOptions.DOCS,
>> +                                                false);
>> +  }
>> +}
>>
>> Added: lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java
>> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java?rev=1670918&view=auto
>> ==============================================================================
>> --- lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java (added)
>> +++ lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java Thu Apr  2 15:05:48 2015
>> @@ -0,0 +1,738 @@
>> +package org.apache.lucene.codecs.autoprefix;
>> +
>> +/*
>> + * 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.IndexOptions;
>> +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.PostingsEnum;
>> +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.Automata;
>> +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 AutoPrefixPostingsFormat(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(Automata.makeBinaryInterval(minTerm, minInclusive, maxTerm, maxInclusive),
>> +                                                   true, false, Integer.MAX_VALUE, true);
>> +
>> +      TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
>> +      NumericDocValues docValues = MultiDocValues.getNumericValues(r, "field");
>> +      PostingsEnum postingsEnum = 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());
>> +        postingsEnum = te.postings(null, postingsEnum);
>> +        int docID;
>> +        while ((docID = postingsEnum.nextDoc()) != PostingsEnum.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 {
>> +    if (VERBOSE) {
>> +      System.out.println("TEST: minItemsPerBlock=" + minItemsPerBlock);
>> +      System.out.println("TEST: maxItemsPerBlock=" + maxItemsPerBlock);
>> +      System.out.println("TEST: minTermsAutoPrefix=" + minTermsAutoPrefix);
>> +      System.out.println("TEST: maxTermsAutoPrefix=" + maxTermsAutoPrefix);
>> +    }
>> +    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()) {
>> +      if (VERBOSE) System.out.println("TEST: now force merge");
>> +      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(Automata.makeBinaryInterval(minTerm, minInclusive, maxTerm, maxInclusive),
>> +                                                   true, false, Integer.MAX_VALUE, true);
>> +
>> +      TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
>> +      NumericDocValues docValues = MultiDocValues.getNumericValues(r, "field");
>> +      PostingsEnum postingsEnum = 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());
>> +        postingsEnum = te.postings(null, postingsEnum);
>> +        int docID;
>> +        while ((docID = postingsEnum.nextDoc()) != PostingsEnum.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()) {
>> +      if (VERBOSE) {
>> +        System.out.println("TEST: now force merge");
>> +      }
>> +      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(PrefixQuery.toAutomaton(prefixBR), true, false, Integer.MAX_VALUE, true);
>> +      TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
>> +      BinaryDocValues docValues = MultiDocValues.getBinaryValues(r, "field");
>> +      PostingsEnum postingsEnum = 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());
>> +        postingsEnum = te.postings(null, postingsEnum);
>> +        int docID;
>> +        while ((docID = postingsEnum.nextDoc()) != PostingsEnum.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 {
>> +    if (VERBOSE) {
>> +      System.out.println("\nTEST: minTermsAutoPrefix=" + minTermsAutoPrefix + " maxTermsAutoPrefix=" + maxTermsAutoPrefix);
>> +      System.out.println("\nTEST: minItemsPerBlock=" + minItemsPerBlock + " maxItemsPerBlock=" + maxItemsPerBlock);
>> +    }
>> +    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(PrefixQuery.toAutomaton(new BytesRef("a")), false, false, Integer.MAX_VALUE, true);
>> +    TermsEnum te = ca.getTermsEnum(terms);
>> +    PostingsEnum postingsEnum = 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());
>> +      postingsEnum = te.postings(null, postingsEnum);
>> +      int docID;
>> +      while ((docID = postingsEnum.nextDoc()) != PostingsEnum.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_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 {
>> +      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.bytesRefToString(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);
>> +    }
>> +  }
>> +
>> +  /** Make sure you get clear exc. if you try to use this within anything but IndexOptions.DOCS fields. */
>> +  public void testWithFreqs() throws Exception {
>> +    Directory dir = newDirectory();
>> +    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
>> +    iwc.setCodec(codec);
>> +    IndexWriter w = new IndexWriter(dir, iwc);
>> +    FieldType ft = new FieldType(StringField.TYPE_NOT_STORED);
>> +    ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
>> +    Document doc = new Document();
>> +    doc.add(new Field("foo", "bar bar", ft));
>> +    w.addDocument(doc);
>> +    try {
>> +      w.commit();
>> +    } catch (IllegalStateException ise) {
>> +      assertEquals("ranges can only be indexed with IndexOptions.DOCS (field: foo)", ise.getMessage());
>> +    }
>> +    w.close();
>> +    dir.close();
>> +  }
>> +
>> +  /** Make sure you get clear exc. if you try to use this within anything but IndexOptions.DOCS fields. */
>> +  public void testWithPositions() throws Exception {
>> +    Directory dir = newDirectory();
>> +    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
>> +    iwc.setCodec(codec);
>> +    IndexWriter w = new IndexWriter(dir, iwc);
>> +    FieldType ft = new FieldType(StringField.TYPE_NOT_STORED);
>> +    ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
>> +    Document doc = new Document();
>> +    doc.add(new Field("foo", "bar bar", ft));
>> +    w.addDocument(doc);
>> +    try {
>> +      w.commit();
>> +    } catch (IllegalStateException ise) {
>> +      assertEquals("ranges can only be indexed with IndexOptions.DOCS (field: foo)", ise.getMessage());
>> +    }
>> +    w.close();
>> +    dir.close();
>> +  }
>> +
>> +  /** Make sure you get clear exc. if you try to use this within anything but IndexOptions.DOCS fields. */
>> +  public void testWithOffsets() throws Exception {
>> +    Directory dir = newDirectory();
>> +    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
>> +    iwc.setCodec(codec);
>> +    IndexWriter w = new IndexWriter(dir, iwc);
>> +    FieldType ft = new FieldType(StringField.TYPE_NOT_STORED);
>> +    ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
>> +    Document doc = new Document();
>> +    doc.add(new Field("foo", "bar bar", ft));
>> +    w.addDocument(doc);
>> +    try {
>> +      w.commit();
>> +    } catch (IllegalStateException ise) {
>> +      assertEquals("ranges can only be indexed with IndexOptions.DOCS (field: foo)", ise.getMessage());
>> +    }
>> +    w.close();
>> +    dir.close();
>> +  }
>> +}
>>
>> Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java
>> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java?rev=1670918&r1=1670917&r2=1670918&view=diff
>> ==============================================================================
>> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java (original)
>> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java Thu Apr  2 15:05:48 2015
>> @@ -16,6 +16,7 @@ package org.apache.lucene.codecs;
>>   * limitations under the License.
>>   */
>>
>> +import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader; // javadocs
>>  import org.apache.lucene.index.OrdTermState;
>>  import org.apache.lucene.index.TermState;
>>
>> @@ -23,6 +24,8 @@ import org.apache.lucene.index.TermState
>>   * Holds all state required for {@link PostingsReaderBase}
>>   * to produce a {@link org.apache.lucene.index.PostingsEnum} without re-seeking the
>>   * terms dict.
>> + *
>> + * @lucene.internal
>>   */
>>  public class BlockTermState extends OrdTermState {
>>    /** how many docs have this term */
>> @@ -36,6 +39,11 @@ public class BlockTermState extends OrdT
>>    // TODO: update BTR to nuke this
>>    public long blockFilePointer;
>>
>> +  /** True if this term is "real" (e.g., not an auto-prefix term or
>> +   *  some other "secret" term; currently only {@link BlockTreeTermsReader}
>> +   *  sets this). */
>> +  public boolean isRealTerm;
>> +
>>    /** Sole constructor. (For invocation by subclass
>>     *  constructors, typically implicit.) */
>>    protected BlockTermState() {
>> @@ -50,10 +58,11 @@ public class BlockTermState extends OrdT
>>      totalTermFreq = other.totalTermFreq;
>>      termBlockOrd = other.termBlockOrd;
>>      blockFilePointer = other.blockFilePointer;
>> +    isRealTerm = other.isRealTerm;
>>    }
>>
>>    @Override
>>    public String toString() {
>> -    return "docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termBlockOrd=" + termBlockOrd + " blockFP=" + blockFilePointer;
>> +    return "docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termBlockOrd=" + termBlockOrd + " blockFP=" + blockFilePointer + " isRealTerm=" + isRealTerm;
>>    }
>>  }
>>
>> Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java
>> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java?rev=1670918&r1=1670917&r2=1670918&view=diff
>> ==============================================================================
>> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java (original)
>> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java Thu Apr  2 15:05:48 2015
>> @@ -62,6 +62,7 @@ public abstract class PostingsFormat imp
>>     * @param name must be all ascii alphanumeric, and less than 128 characters in length.
>>     */
>>    protected PostingsFormat(String name) {
>> +    // TODO: can we somehow detect name conflicts here?  Two different classes trying to claim the same name?  Otherwise you see confusing errors...
>>      NamedSPILoader.checkServiceName(name);
>>      this.name = name;
>>    }
>>
>> Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java
>> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java?rev=1670918&view=auto
>> ==============================================================================
>> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java (added)
>> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java Thu Apr  2 15:05:48 2015
>> @@ -0,0 +1,415 @@
>> +package org.apache.lucene.codecs.blocktree;
>> +
>> +/*
>> + * 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 java.util.ArrayList;
>> +import java.util.Collections;
>> +import java.util.List;
>> +
>> +import org.apache.lucene.index.FilteredTermsEnum;
>> +import org.apache.lucene.index.Terms;
>> +import org.apache.lucene.index.TermsEnum;
>> +import org.apache.lucene.util.ArrayUtil;
>> +import org.apache.lucene.util.BytesRef;
>> +import org.apache.lucene.util.BytesRefBuilder;
>> +import org.apache.lucene.util.StringHelper;
>> +
>> +// TODO: instead of inlining auto-prefix terms with normal terms,
>> +// we could write them into their own virtual/private field.  This
>> +// would make search time a bit more complex, since we'd need to
>> +// merge sort between two TermEnums, but it would also make stats
>> +// API (used by CheckIndex -verbose) easier to implement since we could
>> +// just walk this virtual field and gather its stats)
>> +
>> +/** Used in the first pass when writing a segment to locate
>> + *  "appropriate" auto-prefix terms to pre-compile into the index.
>> + *  This visits every term in the index to find prefixes that
>> + *  match >= min and <= max number of terms. */
>> +
>> +class AutoPrefixTermsWriter {
>> +
>> +  //static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
>> +  //static boolean DEBUG = false;
>> +  //static boolean DEBUG2 = BlockTreeTermsWriter.DEBUG2;
>> +  //static boolean DEBUG2 = true;
>> +
>> +  /** Describes a range of term-space to match, either a simple prefix
>> +   *  (foo*) or a floor-block range of a prefix (e.g. foo[a-m]*,
>> +   *  foo[n-z]*) when there are too many terms starting with foo*. */
>> +  public static final class PrefixTerm implements Comparable<PrefixTerm> {
>> +    /** Common prefix */
>> +    public final byte[] prefix;
>> +
>> +    /** If this is -2, this is a normal prefix (foo *), else it's the minimum lead byte of the suffix (e.g. 'd' in foo[d-m]*). */
>> +    public final int floorLeadStart;
>> +
>> +    /** The lead byte (inclusive) of the suffix for the term range we match (e.g. 'm' in foo[d-m*]); this is ignored when
>> +     *  floorLeadStart is -2. */
>> +    public final int floorLeadEnd;
>> +
>> +    public final BytesRef term;
>> +
>> +    /** Sole constructor. */
>> +    public PrefixTerm(byte[] prefix, int floorLeadStart, int floorLeadEnd) {
>> +      this.prefix = prefix;
>> +      this.floorLeadStart = floorLeadStart;
>> +      this.floorLeadEnd = floorLeadEnd;
>> +      this.term = toBytesRef(prefix, floorLeadStart);
>> +
>> +      assert floorLeadEnd >= floorLeadStart;
>> +      assert floorLeadEnd >= 0;
>> +      assert floorLeadStart == -2 || floorLeadStart >= 0;
>> +
>> +      // We should never create empty-string prefix term:
>> +      assert prefix.length > 0 || floorLeadStart != -2 || floorLeadEnd != 0xff;
>> +    }
>> +
>> +    @Override
>> +    public String toString() {
>> +      String s = brToString(new BytesRef(prefix));
>> +      if (floorLeadStart == -2) {
>> +        s += "[-" + Integer.toHexString(floorLeadEnd) + "]";
>> +      } else {
>> +        s += "[" + Integer.toHexString(floorLeadStart) + "-" + Integer.toHexString(floorLeadEnd) + "]";
>> +      }
>> +      return s;
>> +    }
>> +
>> +    @Override
>> +    public int compareTo(PrefixTerm other) {
>> +      int cmp = term.compareTo(other.term);
>> +      if (cmp == 0) {
>> +        if (prefix.length != other.prefix.length) {
>> +          return prefix.length - other.prefix.length;
>> +        }
>> +
>> +        // On tie, sort the bigger floorLeadEnd, earlier, since it
>> +        // spans more terms, so during intersect, we want to encounter this one
>> +        // first so we can use it if the automaton accepts the larger range:
>> +        cmp = other.floorLeadEnd - floorLeadEnd;
>> +      }
>> +
>> +      return cmp;
>> +    }
>> +
>> +    /** Returns the leading term for this prefix term, e.g. "foo" (for
>> +     *  the foo* prefix) or "foom" (for the foo[m-z]* case). */
>> +    private static BytesRef toBytesRef(byte[] prefix, int floorLeadStart) {
>> +      BytesRef br;
>> +      if (floorLeadStart != -2) {
>> +        assert floorLeadStart >= 0;
>> +        br = new BytesRef(prefix.length+1);
>> +      } else {
>> +        br = new BytesRef(prefix.length);
>> +      }
>> +      System.arraycopy(prefix, 0, br.bytes, 0, prefix.length);
>> +      br.length = prefix.length;
>> +      if (floorLeadStart != -2) {
>> +        assert floorLeadStart >= 0;
>> +        br.bytes[br.length++] = (byte) floorLeadStart;
>> +      }
>> +
>> +      return br;
>> +    }
>> +
>> +    public int compareTo(BytesRef term) {
>> +      return this.term.compareTo(term);
>> +    }
>> +
>> +    public TermsEnum getTermsEnum(TermsEnum in) {
>> +
>> +      final BytesRef prefixRef = new BytesRef(prefix);
>> +
>> +      return new FilteredTermsEnum(in) {
>> +          {
>> +            setInitialSeekTerm(term);
>> +          }
>> +
>> +          @Override
>> +          protected AcceptStatus accept(BytesRef term) {
>> +            if (StringHelper.startsWith(term, prefixRef) &&
>> +                (floorLeadEnd == -1 || term.length == prefixRef.length || (term.bytes[term.offset + prefixRef.length] & 0xff) <= floorLeadEnd)) {
>> +              return AcceptStatus.YES;
>> +            } else {
>> +              return AcceptStatus.END;
>> +            }
>> +          }
>> +        };
>> +    }
>> +  }
>> +
>> +  // for debugging
>> +  static String brToString(BytesRef b) {
>> +    try {
>> +      return b.utf8ToString() + " " + b;
>> +    } 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 b.toString();
>> +    }
>> +  }
>> +
>> +  final List<PrefixTerm> prefixes = new ArrayList<>();
>> +  private final int minItemsInPrefix;
>> +  private final int maxItemsInPrefix;
>> +
>> +  // Records index into pending where the current prefix at that
>> +  // length "started"; for example, if current term starts with 't',
>> +  // startsByPrefix[0] is the index into pending for the first
>> +  // term/sub-block starting with 't'.  We use this to figure out when
>> +  // to write a new block:
>> +  private final BytesRefBuilder lastTerm = new BytesRefBuilder();
>> +  private int[] prefixStarts = new int[8];
>> +  private List<Object> pending = new ArrayList<>();
>> +
>> +  //private final String segment;
>> +
>> +  public AutoPrefixTermsWriter(Terms terms, int minItemsInPrefix, int maxItemsInPrefix) throws IOException {
>> +    this.minItemsInPrefix = minItemsInPrefix;
>> +    this.maxItemsInPrefix = maxItemsInPrefix;
>> +    //this.segment = segment;
>> +
>> +    TermsEnum termsEnum = terms.iterator(null);
>> +    while (true) {
>> +      BytesRef term = termsEnum.next();
>> +      if (term == null) {
>> +        break;
>> +      }
>> +      //if (DEBUG) System.out.println("pushTerm: " + brToString(term));
>> +      pushTerm(term);
>> +    }
>> +
>> +    if (pending.size() > 1) {
>> +      pushTerm(BlockTreeTermsWriter.EMPTY_BYTES_REF);
>> +
>> +      // Also maybe save floor prefixes in root block; this can be a biggish perf gain for large ranges:
>> +      /*
>> +      System.out.println("root block pending.size=" + pending.size());
>> +      for(Object o : pending) {
>> +        System.out.println("  " + o);
>> +      }
>> +      */
>> +      while (pending.size() >= minItemsInPrefix) {
>> +        savePrefixes(0, pending.size());
>> +      }
>> +    }
>> +
>> +    Collections.sort(prefixes);
>> +  }
>> +
>> +  /** Pushes the new term to the top of the stack, and writes new blocks. */
>> +  private void pushTerm(BytesRef text) throws IOException {
>> +    int limit = Math.min(lastTerm.length(), text.length);
>> +
>> +    // Find common prefix between last term and current term:
>> +    int pos = 0;
>> +    while (pos < limit && lastTerm.byteAt(pos) == text.bytes[text.offset+pos]) {
>> +      pos++;
>> +    }
>> +
>> +    //if (DEBUG) System.out.println("  shared=" + pos + "  lastTerm.length=" + lastTerm.length());
>> +
>> +    // Close the "abandoned" suffix now:
>> +    for(int i=lastTerm.length()-1;i>=pos;i--) {
>> +
>> +      // How many items on top of the stack share the current suffix
>> +      // we are closing:
>> +      int prefixTopSize = pending.size() - prefixStarts[i];
>> +
>> +      while (prefixTopSize >= minItemsInPrefix) {
>> +        //if (DEBUG) System.out.println("pushTerm i=" + i + " prefixTopSize=" + prefixTopSize + " minItemsInBlock=" + minItemsInPrefix);
>> +        savePrefixes(i+1, prefixTopSize);
>> +        //prefixStarts[i] -= prefixTopSize;
>> +        //System.out.println("    after savePrefixes: " + (pending.size() - prefixStarts[i]) + " pending.size()=" + pending.size() + " start=" + prefixStarts[i]);
>> +
>> +        // For large floor blocks, it's possible we should now re-run on the new prefix terms we just created:
>> +        prefixTopSize = pending.size() - prefixStarts[i];
>> +      }
>> +    }
>> +
>> +    if (prefixStarts.length < text.length) {
>> +      prefixStarts = ArrayUtil.grow(prefixStarts, text.length);
>> +    }
>> +
>> +    // Init new tail:
>> +    for(int i=pos;i<text.length;i++) {
>> +      prefixStarts[i] = pending.size();
>> +    }
>> +
>> +    lastTerm.copyBytes(text);
>> +
>> +    // Only append the first (optional) empty string, no the fake last one used to close all prefixes:
>> +    if (text.length > 0 || pending.isEmpty()) {
>> +      byte[] termBytes = new byte[text.length];
>> +      System.arraycopy(text.bytes, text.offset, termBytes, 0, text.length);
>> +      pending.add(termBytes);
>> +    }
>> +  }
>> +
>> +  void savePrefixes(int prefixLength, int count) throws IOException {
>> +
>> +    assert count > 0;
>> +
>> +    //if (DEBUG2) {
>> +    //  BytesRef br = new BytesRef(lastTerm.bytes());
>> +    //  br.length = prefixLength;
>> +    //  System.out.println("  savePrefixes: seg=" + segment + " " + brToString(br) + " count=" + count + " pending.size()=" + pending.size());
>> +    //}
>> +
>> +    int lastSuffixLeadLabel = -2;
>> +
>> +    int start = pending.size()-count;
>> +    assert start >=0;
>> +
>> +    int end = pending.size();
>> +    int nextBlockStart = start;
>> +    int nextFloorLeadLabel = -1;
>> +    int prefixCount = 0;
>> +    int pendingCount = 0;
>> +    PrefixTerm lastPTEntry = null;
>> +    for (int i=start; i<end; i++) {
>> +
>> +      byte[] termBytes;
>> +      Object o = pending.get(i);
>> +      PrefixTerm ptEntry;
>> +      if (o instanceof byte[]) {
>> +        ptEntry = null;
>> +        termBytes = (byte[]) o;
>> +      } else {
>> +        ptEntry = (PrefixTerm) o;
>> +        termBytes = ptEntry.term.bytes;
>> +        if (ptEntry.prefix.length != prefixLength) {
>> +          assert ptEntry.prefix.length > prefixLength;
>> +          ptEntry = null;
>> +        }
>> +      }
>> +      pendingCount++;
>> +
>> +      //if (DEBUG) System.out.println("    check term=" + brToString(new BytesRef(termBytes)));
>> +
>> +      int suffixLeadLabel;
>> +
>> +      if (termBytes.length == prefixLength) {
>> +        // Suffix is 0, i.e. prefix 'foo' and term is
>> +        // 'foo' so the term has empty string suffix
>> +        // in this block
>> +        assert lastSuffixLeadLabel == -2;
>> +        suffixLeadLabel = -2;
>> +      } else {
>> +        suffixLeadLabel = termBytes[prefixLength] & 0xff;
>> +      }
>> +
>> +      // if (DEBUG) System.out.println("  i=" + i + " ent=" + ent + " suffixLeadLabel=" + suffixLeadLabel);
>> +
>> +      if (suffixLeadLabel != lastSuffixLeadLabel) {
>> +        // This is a boundary, a chance to make an auto-prefix term if we want:
>> +
>> +        // When we are "recursing" (generating auto-prefix terms on a block of
>> +        // floor'd auto-prefix terms), this assert is non-trivial because it
>> +        // ensures the floorLeadEnd of the previous terms is in fact less
>> +        // than the lead start of the current entry:
>> +        assert suffixLeadLabel > lastSuffixLeadLabel: "suffixLeadLabel=" + suffixLeadLabel + " vs lastSuffixLeadLabel=" + lastSuffixLeadLabel;
>> +
>> +        // NOTE: must check nextFloorLeadLabel in case minItemsInPrefix is 2 and prefix is 'a' and we've seen 'a' and then 'aa'
>> +        if (pendingCount >= minItemsInPrefix && end-nextBlockStart > maxItemsInPrefix && nextFloorLeadLabel != -1) {
>> +          // The count is too large for one block, so we must break it into "floor" blocks, where we record
>> +          // the leading label of the suffix of the first term in each floor block, so at search time we can
>> +          // jump to the right floor block.  We just use a naive greedy segmenter here: make a new floor
>> +          // block as soon as we have at least minItemsInBlock.  This is not always best: it often produces
>> +          // a too-small block as the final block:
>> +
>> +          // If the last entry was another prefix term of the same length, then it represents a range of terms, so we must use its ending
>> +          // prefix label as our ending label:
>> +          if (lastPTEntry != null) {
>> +            lastSuffixLeadLabel = lastPTEntry.floorLeadEnd;
>> +          }
>> +
>> +          savePrefix(prefixLength, nextFloorLeadLabel, lastSuffixLeadLabel);
>> +          pendingCount = 0;
>> +
>> +          prefixCount++;
>> +          nextFloorLeadLabel = suffixLeadLabel;
>> +          nextBlockStart = i;
>> +        }
>> +
>> +        if (nextFloorLeadLabel == -1) {
>> +          nextFloorLeadLabel = suffixLeadLabel;
>> +          //if (DEBUG) System.out.println("set first lead label=" + nextFloorLeadLabel);
>> +        }
>> +
>> +        lastSuffixLeadLabel = suffixLeadLabel;
>> +      }
>> +      lastPTEntry = ptEntry;
>> +    }
>> +
>> +    // Write last block, if any:
>> +    if (nextBlockStart < end) {
>> +      //System.out.println("  lastPTEntry=" + lastPTEntry + " lastSuffixLeadLabel=" + lastSuffixLeadLabel);
>> +      if (lastPTEntry != null) {
>> +        lastSuffixLeadLabel = lastPTEntry.floorLeadEnd;
>> +      }
>> +      assert lastSuffixLeadLabel >= nextFloorLeadLabel: "lastSuffixLeadLabel=" + lastSuffixLeadLabel + " nextFloorLeadLabel=" + nextFloorLeadLabel;
>> +      if (prefixCount == 0) {
>> +        if (prefixLength > 0) {
>> +          savePrefix(prefixLength, -2, 0xff);
>> +          prefixCount++;
>> +        } else {
>> +          // Don't add a prefix term for all terms in the index!
>> +        }
>> +      } else {
>> +        if (lastSuffixLeadLabel == -2) {
>> +          // Special case when closing the empty string root block:
>> +          lastSuffixLeadLabel = 0xff;
>> +        }
>> +        savePrefix(prefixLength, nextFloorLeadLabel, lastSuffixLeadLabel);
>> +        prefixCount++;
>> +      }
>> +    }
>> +
>> +    // Remove slice from the top of the pending stack, that we just wrote:
>> +    int sizeToClear = count;
>> +    if (prefixCount > 1) {
>> +      Object o = pending.get(pending.size()-count);
>> +      if (o instanceof byte[] && ((byte[]) o).length == prefixLength) {
>> +        // If we were just asked to write all f* terms, but there were too many and so we made floor blocks, the exact term 'f' will remain
>> +        // as its own item, followed by floor block terms like f[a-m]*, f[n-z]*, so in this case we leave 3 (not 2) items on the pending stack:
>> +        sizeToClear--;
>> +      }
>> +    }
>> +    pending.subList(pending.size()-sizeToClear, pending.size()).clear();
>> +
>> +    // Append prefix terms for each prefix, since these count like real terms that also need to be "rolled up":
>> +    for(int i=0;i<prefixCount;i++) {
>> +      PrefixTerm pt = prefixes.get(prefixes.size()-(prefixCount-i));
>> +      pending.add(pt);
>> +    }
>> +  }
>> +
>> +  private void savePrefix(int prefixLength, int floorLeadStart, int floorLeadEnd) {
>> +    byte[] prefix = new byte[prefixLength];
>> +    System.arraycopy(lastTerm.bytes(), 0, prefix, 0, prefixLength);
>> +    assert floorLeadStart != -1;
>> +    assert floorLeadEnd != -1;
>> +
>> +    PrefixTerm pt = new PrefixTerm(prefix, floorLeadStart, floorLeadEnd);
>> +    //if (DEBUG2) System.out.println("    savePrefix: seg=" + segment + " " + pt + " count=" + count);
>> +    prefixes.add(pt);
>> +  }
>> +}
>>
>> Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java
>> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java?rev=1670918&view=auto
>> ==============================================================================
>> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java (added)
>> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java Thu Apr  2 15:05:48 2015
>> @@ -0,0 +1,95 @@
>> +package org.apache.lucene.codecs.blocktree;
>> +
>> +/*
>> + * 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.PostingsEnum;
>> +import org.apache.lucene.search.DocIdSetIterator;
>> +import org.apache.lucene.util.BitSet;
>> +import org.apache.lucene.util.BitSetIterator;
>> +import org.apache.lucene.util.BytesRef;
>> +import org.apache.lucene.util.FixedBitSet; // javadocs
>> +
>> +/** Takes a {@link FixedBitSet} and creates a DOCS {@link PostingsEnum} from it. */
>> +
>> +class BitSetPostingsEnum extends PostingsEnum {
>> +  private final BitSet bits;
>> +  private DocIdSetIterator in;
>> +
>> +  BitSetPostingsEnum(BitSet bits) {
>> +    this.bits = bits;
>> +    reset();
>> +  }
>> +
>> +  @Override
>> +  public int freq() throws IOException {
>> +    return 1;
>> +  }
>> +
>> +  @Override
>> +  public int docID() {
>> +    if (in == null) {
>> +      return -1;
>> +    } else {
>> +      return in.docID();
>> +    }
>> +  }
>> +
>> +  @Override
>> +  public int nextDoc() throws IOException {
>> +    if (in == null) {
>> +      in = new BitSetIterator(bits, 0);
>> +    }
>> +    return in.nextDoc();
>> +  }
>> +
>> +  @Override
>> +  public int advance(int target) throws IOException {
>> +    return in.advance(target);
>> +  }
>> +
>> +  @Override
>> +  public long cost() {
>> +    return in.cost();
>> +  }
>> +
>> +  void reset() {
>> +    in = null;
>> +  }
>> +
>> +  @Override
>> +  public BytesRef getPayload() {
>> +    return null;
>> +  }
>> +
>> +  @Override
>> +  public int nextPosition() {
>> +    throw new UnsupportedOperationException();
>> +  }
>> +
>> +  @Override
>> +  public int startOffset() {
>> +    throw new UnsupportedOperationException();
>> +  }
>> +
>> +  @Override
>> +  public int endOffset() {
>> +    throw new UnsupportedOperationException();
>> +  }
>> +}
>>
>> Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java
>> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java?rev=1670918&view=auto
>> ==============================================================================
>> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java (added)
>> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java Thu Apr  2 15:05:48 2015
>> @@ -0,0 +1,87 @@
>> +package org.apache.lucene.codecs.blocktree;
>> +
>> +/*
>> + * Licensed to the Apache Software Foundation (ASF) under one or more
>> + * contributor license agreements.  See the NOTICE file distributed with
>> + * this work for additional information regarding copyright ownership.
>> + * The ASF licenses this file to You under the Apache License, Version 2.0
>> + * (the "License"); you may not use this file except in compliance with
>> + * the License.  You may obtain a copy of the License at
>> + *
>> + *     http://www.apache.org/licenses/LICENSE-2.0
>> + *
>> + * Unless required by applicable law or agreed to in writing, software
>> + * distributed under the License is distributed on an "AS IS" BASIS,
>> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
>> + * See the License for the specific language governing permissions and
>> + * limitations under the License.
>> + */
>> +
>> +import org.apache.lucene.codecs.PostingsWriterBase;
>> +import org.apache.lucene.index.PostingsEnum;
>> +import org.apache.lucene.index.TermsEnum;
>> +import org.apache.lucene.util.BitSet;
>> +import org.apache.lucene.util.Bits;
>> +import org.apache.lucene.util.BytesRef;
>> +
>> +/** Silly stub class, used only when writing an auto-prefix
>> + *  term in order to expose DocsEnum over a FixedBitSet.  We
>> + *  pass this to {@link PostingsWriterBase#writeTerm} so
>> + *  that it can pull .docs() multiple times for the
>> + *  current term. */
>> +
>> +class BitSetTermsEnum extends TermsEnum {
>> +  private final BitSetPostingsEnum postingsEnum;
>> +
>> +  public BitSetTermsEnum(BitSet docs) {
>> +    postingsEnum = new BitSetPostingsEnum(docs);
>> +  }
>> +
>> +  @Override
>> +  public SeekStatus seekCeil(BytesRef text) {
>> +    throw new UnsupportedOperationException();
>> +  }
>> +
>> +  @Override
>> +  public void seekExact(long ord) {
>> +    throw new UnsupportedOperationException();
>> +  }
>> +
>> +  @Override
>> +  public BytesRef term() {
>> +    throw new UnsupportedOperationException();
>> +  }
>> +
>> +  @Override
>> +  public BytesRef next() {
>> +    throw new UnsupportedOperationException();
>> +  }
>> +
>> +  @Override
>> +  public long ord() {
>> +    throw new UnsupportedOperationException();
>> +  }
>> +
>> +  @Override
>> +  public int docFreq() {
>> +    throw new UnsupportedOperationException();
>> +  }
>> +
>> +  @Override
>> +  public long totalTermFreq() {
>> +    throw new UnsupportedOperationException();
>> +  }
>> +
>> +  @Override
>> +  public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) {
>> +    if (flags != PostingsEnum.NONE) {
>> +      // We only work with DOCS_ONLY fields
>> +      return null;
>> +    }
>> +    if (liveDocs != null) {
>> +      throw new IllegalArgumentException("cannot handle live docs");
>> +    }
>> +    postingsEnum.reset();
>> +    return postingsEnum;
>> +  }
>> +}
>>
>> Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
>> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java?rev=1670918&r1=1670917&r2=1670918&view=diff
>> ==============================================================================
>> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java (original)
>> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java Thu Apr  2 15:05:48 2015
>> @@ -34,6 +34,8 @@ import org.apache.lucene.index.IndexFile
>>  import org.apache.lucene.index.IndexOptions;
>>  import org.apache.lucene.index.SegmentReadState;
>>  import org.apache.lucene.index.Terms;
>> +import org.apache.lucene.search.PrefixQuery;  // javadocs
>> +import org.apache.lucene.search.TermRangeQuery;  // javadocs
>>  import org.apache.lucene.store.IndexInput;
>>  import org.apache.lucene.util.Accountable;
>>  import org.apache.lucene.util.Accountables;
>> @@ -57,6 +59,14 @@ import org.apache.lucene.util.fst.Output
>>   *  min/maxItemsPerBlock during indexing to control how
>>   *  much memory the terms index uses.</p>
>>   *
>> + *  <p>If auto-prefix terms were indexed (see
>> + *  {@link BlockTreeTermsWriter}), then the {@link Terms#intersect}
>> + *  implementation here will make use of these terms only if the
>> + *  automaton has a binary sink state, i.e. an accept state
>> + *  which has a transition to itself accepting all byte values.
>> + *  For example, both {@link PrefixQuery} and {@link TermRangeQuery}
>> + *  pass such automata to {@link Terms#intersect}.</p>
>> + *
>>   *  <p>The data structure used by this implementation is very
>>   *  similar to a burst trie
>>   *  (http://citeseer.ist.psu.edu/viewdoc/summary?doi=10.1.1.18.3499),
>> @@ -90,8 +100,11 @@ public final class BlockTreeTermsReader
>>    /** Initial terms format. */
>>    public static final int VERSION_START = 0;
>>
>> +  /** Auto-prefix terms. */
>> +  public static final int VERSION_AUTO_PREFIX_TERMS = 1;
>> +
>>    /** Current terms format. */
>> -  public static final int VERSION_CURRENT = VERSION_START;
>> +  public static final int VERSION_CURRENT = VERSION_AUTO_PREFIX_TERMS;
>>
>>    /** Extension of terms index file */
>>    static final String TERMS_INDEX_EXTENSION = "tip";
>> @@ -116,7 +129,7 @@ public final class BlockTreeTermsReader
>>
>>    final String segment;
>>
>> -  private final int version;
>> +  final int version;
>>
>>    /** Sole constructor. */
>>    public BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState state) throws IOException {
>>
>>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: dev-help@lucene.apache.org
>

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


Re: svn commit: r1670918 [1/5] - in /lucene/dev/trunk/lucene: ./ codecs/src/java/org/apache/lucene/codecs/autoprefix/ codecs/src/resources/META-INF/services/ codecs/src/test/org/apache/lucene/codecs/autoprefix/ core/src/java/org/apache/lucene/codecs/ core/...

Posted by Robert Muir <rc...@gmail.com>.
NOTE: i hit compile failure like this (TermRangeTermsEnum got removed).

I am going to remove these asserts: to me they don't look very useful,
and fix the build for now.

compile-test:
    [mkdir] Created dir:
/home/rmuir/workspace/trunk/lucene/build/core/classes/test
    [javac] Compiling 431 source files to
/home/rmuir/workspace/trunk/lucene/build/core/classes/test
    [javac] /home/rmuir/workspace/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java:123:
error: cannot find symbol
    [javac]     assertFalse(query.getTermsEnum(terms) instanceof
TermRangeTermsEnum);
    [javac]                                                      ^
    [javac]   symbol:   class TermRangeTermsEnum
    [javac]   location: class TestTermRangeQuery
    [javac] /home/rmuir/workspace/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java:128:
error: cannot find symbol
    [javac]     assertFalse(query.getTermsEnum(terms) instanceof
TermRangeTermsEnum);
    [javac]                                                      ^
    [javac]   symbol:   class TermRangeTermsEnum
    [javac]   location: class TestTermRangeQuery
    [javac] /home/rmuir/workspace/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java:132:
error: cannot find symbol
    [javac]     assertFalse(query.getTermsEnum(terms) instanceof
TermRangeTermsEnum);
    [javac]                                                      ^
    [javac]   symbol:   class TermRangeTermsEnum
    [javac]   location: class TestTermRangeQuery
    [javac] Note: Some input files use or override a deprecated API.
    [javac] Note: Recompile with -Xlint:deprecation for details.
    [javac] 3 errors

On Thu, Apr 2, 2015 at 11:05 AM,  <mi...@apache.org> wrote:
> Author: mikemccand
> Date: Thu Apr  2 15:05:48 2015
> New Revision: 1670918
>
> URL: http://svn.apache.org/r1670918
> Log:
> LUCENE-5879: add auto-prefix terms to block tree, and experimental AutoPrefixTermsPostingsFormat
>
> Added:
>     lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/
>     lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java   (with props)
>     lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java   (with props)
>     lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/
>     lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java   (with props)
>     lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java   (with props)
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java   (with props)
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java   (with props)
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java   (with props)
>     lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/RandomPostingsTester.java   (with props)
> Removed:
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermRangeTermsEnum.java
> Modified:
>     lucene/dev/trunk/lucene/CHANGES.txt
>     lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermContext.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PrefixQuery.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java
>     lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java
>     lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java
>     lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestPrefixQuery.java
>     lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java
>     lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestWildcard.java
>     lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java
>     lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
>     lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
>     lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
>
> Modified: lucene/dev/trunk/lucene/CHANGES.txt
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1670918&r1=1670917&r2=1670918&view=diff
> ==============================================================================
> --- lucene/dev/trunk/lucene/CHANGES.txt (original)
> +++ lucene/dev/trunk/lucene/CHANGES.txt Thu Apr  2 15:05:48 2015
> @@ -19,6 +19,10 @@ New Features
>    for counting ranges that align with the underlying terms as defined by the
>    NumberRangePrefixTree (e.g. familiar date units like days).  (David Smiley)
>
> +* LUCENE-5879: Added experimental auto-prefix terms to BlockTree terms
> +  dictionary, exposed as AutoPrefixPostingsFormat (Adrien Grand,
> +  Uwe Schindler, Robert Muir, Mike McCandless)
> +
>  API Changes
>
>  * LUCENE-3312: The API of oal.document was restructured to
>
> Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java?rev=1670918&view=auto
> ==============================================================================
> --- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java (added)
> +++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java Thu Apr  2 15:05:48 2015
> @@ -0,0 +1,125 @@
> +package org.apache.lucene.codecs.autoprefix;
> +
> +/*
> + * 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.codecs.FieldsConsumer;
> +import org.apache.lucene.codecs.FieldsProducer;
> +import org.apache.lucene.codecs.PostingsFormat;
> +import org.apache.lucene.codecs.PostingsReaderBase;
> +import org.apache.lucene.codecs.PostingsWriterBase;
> +import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
> +import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
> +import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
> +import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
> +import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
> +import org.apache.lucene.index.SegmentReadState;
> +import org.apache.lucene.index.SegmentWriteState;
> +import org.apache.lucene.util.IOUtils;
> +
> +/**
> + * Just like {@link Lucene50PostingsFormat} except this format
> + * exposes the experimental auto-prefix terms.
> + *
> + * @lucene.experimental
> + */
> +
> +public final class AutoPrefixPostingsFormat extends PostingsFormat {
> +
> +  private final int minItemsInBlock;
> +  private final int maxItemsInBlock;
> +  private final int minItemsInAutoPrefix;
> +  private final int maxItemsInAutoPrefix;
> +
> +  /** Creates {@code AutoPrefixPostingsFormat} with default settings. */
> +  public AutoPrefixPostingsFormat() {
> +    this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
> +         BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE,
> +         25, 48);
> +  }
> +
> +  /** Creates {@code Lucene50PostingsFormat} with custom
> +   *  values for {@code minBlockSize} and {@code
> +   *  maxBlockSize} passed to block terms dictionary.
> +   *  @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
> +  public AutoPrefixPostingsFormat(int minItemsInAutoPrefix, int maxItemsInAutoPrefix) {
> +    this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
> +         BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE,
> +         minItemsInAutoPrefix,
> +         maxItemsInAutoPrefix);
> +  }
> +
> +  /** Creates {@code Lucene50PostingsFormat} with custom
> +   *  values for {@code minBlockSize}, {@code
> +   *  maxBlockSize}, {@code minItemsInAutoPrefix} and {@code maxItemsInAutoPrefix}, passed
> +   *  to block tree terms dictionary.
> +   *  @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int,int,int) */
> +  public AutoPrefixPostingsFormat(int minItemsInBlock, int maxItemsInBlock, int minItemsInAutoPrefix, int maxItemsInAutoPrefix) {
> +    super("AutoPrefix");
> +    BlockTreeTermsWriter.validateSettings(minItemsInBlock,
> +                                          maxItemsInBlock);
> +    BlockTreeTermsWriter.validateAutoPrefixSettings(minItemsInAutoPrefix,
> +                                                    maxItemsInAutoPrefix);
> +    this.minItemsInBlock = minItemsInBlock;
> +    this.maxItemsInBlock = maxItemsInBlock;
> +    this.minItemsInAutoPrefix = minItemsInAutoPrefix;
> +    this.maxItemsInAutoPrefix = maxItemsInAutoPrefix;
> +  }
> +
> +  @Override
> +  public String toString() {
> +    return getName();
> +  }
> +
> +  @Override
> +  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
> +    PostingsWriterBase postingsWriter = new Lucene50PostingsWriter(state);
> +
> +    boolean success = false;
> +    try {
> +      FieldsConsumer ret = new BlockTreeTermsWriter(state,
> +                                                    postingsWriter,
> +                                                    minItemsInBlock,
> +                                                    maxItemsInBlock,
> +                                                    minItemsInAutoPrefix,
> +                                                    maxItemsInAutoPrefix);
> +      success = true;
> +      return ret;
> +    } finally {
> +      if (!success) {
> +        IOUtils.closeWhileHandlingException(postingsWriter);
> +      }
> +    }
> +  }
> +
> +  @Override
> +  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
> +    PostingsReaderBase postingsReader = new Lucene50PostingsReader(state);
> +    boolean success = false;
> +    try {
> +      FieldsProducer ret = new BlockTreeTermsReader(postingsReader, state);
> +      success = true;
> +      return ret;
> +    } finally {
> +      if (!success) {
> +        IOUtils.closeWhileHandlingException(postingsReader);
> +      }
> +    }
> +  }
> +}
>
> Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java?rev=1670918&view=auto
> ==============================================================================
> --- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java (added)
> +++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java Thu Apr  2 15:05:48 2015
> @@ -0,0 +1,22 @@
> +/*
> + * 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.
> + */
> +
> +/**
> + * An experimental postings format that automatically indexes appropriate
> + * prefix terms for fast range and prefix queries.
> + */
> +package org.apache.lucene.codecs.autoprefix;
>
> Modified: lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat?rev=1670918&r1=1670917&r2=1670918&view=diff
> ==============================================================================
> --- lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat (original)
> +++ lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat Thu Apr  2 15:05:48 2015
> @@ -20,3 +20,4 @@ org.apache.lucene.codecs.memory.FSTOrdPo
>  org.apache.lucene.codecs.memory.FSTPostingsFormat
>  org.apache.lucene.codecs.memory.MemoryPostingsFormat
>  org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat
> +org.apache.lucene.codecs.autoprefix.AutoPrefixPostingsFormat
>
> Added: lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java?rev=1670918&view=auto
> ==============================================================================
> --- lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java (added)
> +++ lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java Thu Apr  2 15:05:48 2015
> @@ -0,0 +1,38 @@
> +package org.apache.lucene.codecs.autoprefix;
> +
> +/*
> + * Licensed to the Apache Software Foundation (ASF) under one or more
> + * contributor license agreements.  See the NOTICE file distributed with
> + * this work for additional information regarding copyright ownership.
> + * The ASF licenses this file to You under the Apache License, Version 2.0
> + * (the "License"); you may not use this file except in compliance with
> + * the License.  You may obtain a copy of the License at
> + *
> + *     http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + */
> +
> +import org.apache.lucene.index.IndexOptions;
> +import org.apache.lucene.index.RandomPostingsTester;
> +import org.apache.lucene.util.LuceneTestCase;
> +import org.apache.lucene.util.TestUtil;
> +
> +/**
> + * Tests AutoPrefix's postings
> + */
> +
> +// NOTE: we don't extend BasePostingsFormatTestCase becase we can only handle DOCS_ONLY fields:
> +
> +public class TestAutoPrefixPostingsFormat extends LuceneTestCase {
> +  public void test() throws Exception {
> +    new RandomPostingsTester(random()).testFull(TestUtil.alwaysPostingsFormat(new AutoPrefixPostingsFormat()),
> +                                                createTempDir("autoprefix"),
> +                                                IndexOptions.DOCS,
> +                                                false);
> +  }
> +}
>
> Added: lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java?rev=1670918&view=auto
> ==============================================================================
> --- lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java (added)
> +++ lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java Thu Apr  2 15:05:48 2015
> @@ -0,0 +1,738 @@
> +package org.apache.lucene.codecs.autoprefix;
> +
> +/*
> + * 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.IndexOptions;
> +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.PostingsEnum;
> +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.Automata;
> +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 AutoPrefixPostingsFormat(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(Automata.makeBinaryInterval(minTerm, minInclusive, maxTerm, maxInclusive),
> +                                                   true, false, Integer.MAX_VALUE, true);
> +
> +      TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
> +      NumericDocValues docValues = MultiDocValues.getNumericValues(r, "field");
> +      PostingsEnum postingsEnum = 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());
> +        postingsEnum = te.postings(null, postingsEnum);
> +        int docID;
> +        while ((docID = postingsEnum.nextDoc()) != PostingsEnum.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 {
> +    if (VERBOSE) {
> +      System.out.println("TEST: minItemsPerBlock=" + minItemsPerBlock);
> +      System.out.println("TEST: maxItemsPerBlock=" + maxItemsPerBlock);
> +      System.out.println("TEST: minTermsAutoPrefix=" + minTermsAutoPrefix);
> +      System.out.println("TEST: maxTermsAutoPrefix=" + maxTermsAutoPrefix);
> +    }
> +    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()) {
> +      if (VERBOSE) System.out.println("TEST: now force merge");
> +      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(Automata.makeBinaryInterval(minTerm, minInclusive, maxTerm, maxInclusive),
> +                                                   true, false, Integer.MAX_VALUE, true);
> +
> +      TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
> +      NumericDocValues docValues = MultiDocValues.getNumericValues(r, "field");
> +      PostingsEnum postingsEnum = 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());
> +        postingsEnum = te.postings(null, postingsEnum);
> +        int docID;
> +        while ((docID = postingsEnum.nextDoc()) != PostingsEnum.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()) {
> +      if (VERBOSE) {
> +        System.out.println("TEST: now force merge");
> +      }
> +      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(PrefixQuery.toAutomaton(prefixBR), true, false, Integer.MAX_VALUE, true);
> +      TermsEnum te = ca.getTermsEnum(MultiFields.getTerms(r, "field"));
> +      BinaryDocValues docValues = MultiDocValues.getBinaryValues(r, "field");
> +      PostingsEnum postingsEnum = 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());
> +        postingsEnum = te.postings(null, postingsEnum);
> +        int docID;
> +        while ((docID = postingsEnum.nextDoc()) != PostingsEnum.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 {
> +    if (VERBOSE) {
> +      System.out.println("\nTEST: minTermsAutoPrefix=" + minTermsAutoPrefix + " maxTermsAutoPrefix=" + maxTermsAutoPrefix);
> +      System.out.println("\nTEST: minItemsPerBlock=" + minItemsPerBlock + " maxItemsPerBlock=" + maxItemsPerBlock);
> +    }
> +    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(PrefixQuery.toAutomaton(new BytesRef("a")), false, false, Integer.MAX_VALUE, true);
> +    TermsEnum te = ca.getTermsEnum(terms);
> +    PostingsEnum postingsEnum = 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());
> +      postingsEnum = te.postings(null, postingsEnum);
> +      int docID;
> +      while ((docID = postingsEnum.nextDoc()) != PostingsEnum.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_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 {
> +      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.bytesRefToString(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);
> +    }
> +  }
> +
> +  /** Make sure you get clear exc. if you try to use this within anything but IndexOptions.DOCS fields. */
> +  public void testWithFreqs() throws Exception {
> +    Directory dir = newDirectory();
> +    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
> +    iwc.setCodec(codec);
> +    IndexWriter w = new IndexWriter(dir, iwc);
> +    FieldType ft = new FieldType(StringField.TYPE_NOT_STORED);
> +    ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
> +    Document doc = new Document();
> +    doc.add(new Field("foo", "bar bar", ft));
> +    w.addDocument(doc);
> +    try {
> +      w.commit();
> +    } catch (IllegalStateException ise) {
> +      assertEquals("ranges can only be indexed with IndexOptions.DOCS (field: foo)", ise.getMessage());
> +    }
> +    w.close();
> +    dir.close();
> +  }
> +
> +  /** Make sure you get clear exc. if you try to use this within anything but IndexOptions.DOCS fields. */
> +  public void testWithPositions() throws Exception {
> +    Directory dir = newDirectory();
> +    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
> +    iwc.setCodec(codec);
> +    IndexWriter w = new IndexWriter(dir, iwc);
> +    FieldType ft = new FieldType(StringField.TYPE_NOT_STORED);
> +    ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
> +    Document doc = new Document();
> +    doc.add(new Field("foo", "bar bar", ft));
> +    w.addDocument(doc);
> +    try {
> +      w.commit();
> +    } catch (IllegalStateException ise) {
> +      assertEquals("ranges can only be indexed with IndexOptions.DOCS (field: foo)", ise.getMessage());
> +    }
> +    w.close();
> +    dir.close();
> +  }
> +
> +  /** Make sure you get clear exc. if you try to use this within anything but IndexOptions.DOCS fields. */
> +  public void testWithOffsets() throws Exception {
> +    Directory dir = newDirectory();
> +    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
> +    iwc.setCodec(codec);
> +    IndexWriter w = new IndexWriter(dir, iwc);
> +    FieldType ft = new FieldType(StringField.TYPE_NOT_STORED);
> +    ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
> +    Document doc = new Document();
> +    doc.add(new Field("foo", "bar bar", ft));
> +    w.addDocument(doc);
> +    try {
> +      w.commit();
> +    } catch (IllegalStateException ise) {
> +      assertEquals("ranges can only be indexed with IndexOptions.DOCS (field: foo)", ise.getMessage());
> +    }
> +    w.close();
> +    dir.close();
> +  }
> +}
>
> Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java?rev=1670918&r1=1670917&r2=1670918&view=diff
> ==============================================================================
> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java (original)
> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java Thu Apr  2 15:05:48 2015
> @@ -16,6 +16,7 @@ package org.apache.lucene.codecs;
>   * limitations under the License.
>   */
>
> +import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader; // javadocs
>  import org.apache.lucene.index.OrdTermState;
>  import org.apache.lucene.index.TermState;
>
> @@ -23,6 +24,8 @@ import org.apache.lucene.index.TermState
>   * Holds all state required for {@link PostingsReaderBase}
>   * to produce a {@link org.apache.lucene.index.PostingsEnum} without re-seeking the
>   * terms dict.
> + *
> + * @lucene.internal
>   */
>  public class BlockTermState extends OrdTermState {
>    /** how many docs have this term */
> @@ -36,6 +39,11 @@ public class BlockTermState extends OrdT
>    // TODO: update BTR to nuke this
>    public long blockFilePointer;
>
> +  /** True if this term is "real" (e.g., not an auto-prefix term or
> +   *  some other "secret" term; currently only {@link BlockTreeTermsReader}
> +   *  sets this). */
> +  public boolean isRealTerm;
> +
>    /** Sole constructor. (For invocation by subclass
>     *  constructors, typically implicit.) */
>    protected BlockTermState() {
> @@ -50,10 +58,11 @@ public class BlockTermState extends OrdT
>      totalTermFreq = other.totalTermFreq;
>      termBlockOrd = other.termBlockOrd;
>      blockFilePointer = other.blockFilePointer;
> +    isRealTerm = other.isRealTerm;
>    }
>
>    @Override
>    public String toString() {
> -    return "docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termBlockOrd=" + termBlockOrd + " blockFP=" + blockFilePointer;
> +    return "docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termBlockOrd=" + termBlockOrd + " blockFP=" + blockFilePointer + " isRealTerm=" + isRealTerm;
>    }
>  }
>
> Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java?rev=1670918&r1=1670917&r2=1670918&view=diff
> ==============================================================================
> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java (original)
> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java Thu Apr  2 15:05:48 2015
> @@ -62,6 +62,7 @@ public abstract class PostingsFormat imp
>     * @param name must be all ascii alphanumeric, and less than 128 characters in length.
>     */
>    protected PostingsFormat(String name) {
> +    // TODO: can we somehow detect name conflicts here?  Two different classes trying to claim the same name?  Otherwise you see confusing errors...
>      NamedSPILoader.checkServiceName(name);
>      this.name = name;
>    }
>
> Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java?rev=1670918&view=auto
> ==============================================================================
> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java (added)
> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java Thu Apr  2 15:05:48 2015
> @@ -0,0 +1,415 @@
> +package org.apache.lucene.codecs.blocktree;
> +
> +/*
> + * 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 java.util.ArrayList;
> +import java.util.Collections;
> +import java.util.List;
> +
> +import org.apache.lucene.index.FilteredTermsEnum;
> +import org.apache.lucene.index.Terms;
> +import org.apache.lucene.index.TermsEnum;
> +import org.apache.lucene.util.ArrayUtil;
> +import org.apache.lucene.util.BytesRef;
> +import org.apache.lucene.util.BytesRefBuilder;
> +import org.apache.lucene.util.StringHelper;
> +
> +// TODO: instead of inlining auto-prefix terms with normal terms,
> +// we could write them into their own virtual/private field.  This
> +// would make search time a bit more complex, since we'd need to
> +// merge sort between two TermEnums, but it would also make stats
> +// API (used by CheckIndex -verbose) easier to implement since we could
> +// just walk this virtual field and gather its stats)
> +
> +/** Used in the first pass when writing a segment to locate
> + *  "appropriate" auto-prefix terms to pre-compile into the index.
> + *  This visits every term in the index to find prefixes that
> + *  match >= min and <= max number of terms. */
> +
> +class AutoPrefixTermsWriter {
> +
> +  //static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
> +  //static boolean DEBUG = false;
> +  //static boolean DEBUG2 = BlockTreeTermsWriter.DEBUG2;
> +  //static boolean DEBUG2 = true;
> +
> +  /** Describes a range of term-space to match, either a simple prefix
> +   *  (foo*) or a floor-block range of a prefix (e.g. foo[a-m]*,
> +   *  foo[n-z]*) when there are too many terms starting with foo*. */
> +  public static final class PrefixTerm implements Comparable<PrefixTerm> {
> +    /** Common prefix */
> +    public final byte[] prefix;
> +
> +    /** If this is -2, this is a normal prefix (foo *), else it's the minimum lead byte of the suffix (e.g. 'd' in foo[d-m]*). */
> +    public final int floorLeadStart;
> +
> +    /** The lead byte (inclusive) of the suffix for the term range we match (e.g. 'm' in foo[d-m*]); this is ignored when
> +     *  floorLeadStart is -2. */
> +    public final int floorLeadEnd;
> +
> +    public final BytesRef term;
> +
> +    /** Sole constructor. */
> +    public PrefixTerm(byte[] prefix, int floorLeadStart, int floorLeadEnd) {
> +      this.prefix = prefix;
> +      this.floorLeadStart = floorLeadStart;
> +      this.floorLeadEnd = floorLeadEnd;
> +      this.term = toBytesRef(prefix, floorLeadStart);
> +
> +      assert floorLeadEnd >= floorLeadStart;
> +      assert floorLeadEnd >= 0;
> +      assert floorLeadStart == -2 || floorLeadStart >= 0;
> +
> +      // We should never create empty-string prefix term:
> +      assert prefix.length > 0 || floorLeadStart != -2 || floorLeadEnd != 0xff;
> +    }
> +
> +    @Override
> +    public String toString() {
> +      String s = brToString(new BytesRef(prefix));
> +      if (floorLeadStart == -2) {
> +        s += "[-" + Integer.toHexString(floorLeadEnd) + "]";
> +      } else {
> +        s += "[" + Integer.toHexString(floorLeadStart) + "-" + Integer.toHexString(floorLeadEnd) + "]";
> +      }
> +      return s;
> +    }
> +
> +    @Override
> +    public int compareTo(PrefixTerm other) {
> +      int cmp = term.compareTo(other.term);
> +      if (cmp == 0) {
> +        if (prefix.length != other.prefix.length) {
> +          return prefix.length - other.prefix.length;
> +        }
> +
> +        // On tie, sort the bigger floorLeadEnd, earlier, since it
> +        // spans more terms, so during intersect, we want to encounter this one
> +        // first so we can use it if the automaton accepts the larger range:
> +        cmp = other.floorLeadEnd - floorLeadEnd;
> +      }
> +
> +      return cmp;
> +    }
> +
> +    /** Returns the leading term for this prefix term, e.g. "foo" (for
> +     *  the foo* prefix) or "foom" (for the foo[m-z]* case). */
> +    private static BytesRef toBytesRef(byte[] prefix, int floorLeadStart) {
> +      BytesRef br;
> +      if (floorLeadStart != -2) {
> +        assert floorLeadStart >= 0;
> +        br = new BytesRef(prefix.length+1);
> +      } else {
> +        br = new BytesRef(prefix.length);
> +      }
> +      System.arraycopy(prefix, 0, br.bytes, 0, prefix.length);
> +      br.length = prefix.length;
> +      if (floorLeadStart != -2) {
> +        assert floorLeadStart >= 0;
> +        br.bytes[br.length++] = (byte) floorLeadStart;
> +      }
> +
> +      return br;
> +    }
> +
> +    public int compareTo(BytesRef term) {
> +      return this.term.compareTo(term);
> +    }
> +
> +    public TermsEnum getTermsEnum(TermsEnum in) {
> +
> +      final BytesRef prefixRef = new BytesRef(prefix);
> +
> +      return new FilteredTermsEnum(in) {
> +          {
> +            setInitialSeekTerm(term);
> +          }
> +
> +          @Override
> +          protected AcceptStatus accept(BytesRef term) {
> +            if (StringHelper.startsWith(term, prefixRef) &&
> +                (floorLeadEnd == -1 || term.length == prefixRef.length || (term.bytes[term.offset + prefixRef.length] & 0xff) <= floorLeadEnd)) {
> +              return AcceptStatus.YES;
> +            } else {
> +              return AcceptStatus.END;
> +            }
> +          }
> +        };
> +    }
> +  }
> +
> +  // for debugging
> +  static String brToString(BytesRef b) {
> +    try {
> +      return b.utf8ToString() + " " + b;
> +    } 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 b.toString();
> +    }
> +  }
> +
> +  final List<PrefixTerm> prefixes = new ArrayList<>();
> +  private final int minItemsInPrefix;
> +  private final int maxItemsInPrefix;
> +
> +  // Records index into pending where the current prefix at that
> +  // length "started"; for example, if current term starts with 't',
> +  // startsByPrefix[0] is the index into pending for the first
> +  // term/sub-block starting with 't'.  We use this to figure out when
> +  // to write a new block:
> +  private final BytesRefBuilder lastTerm = new BytesRefBuilder();
> +  private int[] prefixStarts = new int[8];
> +  private List<Object> pending = new ArrayList<>();
> +
> +  //private final String segment;
> +
> +  public AutoPrefixTermsWriter(Terms terms, int minItemsInPrefix, int maxItemsInPrefix) throws IOException {
> +    this.minItemsInPrefix = minItemsInPrefix;
> +    this.maxItemsInPrefix = maxItemsInPrefix;
> +    //this.segment = segment;
> +
> +    TermsEnum termsEnum = terms.iterator(null);
> +    while (true) {
> +      BytesRef term = termsEnum.next();
> +      if (term == null) {
> +        break;
> +      }
> +      //if (DEBUG) System.out.println("pushTerm: " + brToString(term));
> +      pushTerm(term);
> +    }
> +
> +    if (pending.size() > 1) {
> +      pushTerm(BlockTreeTermsWriter.EMPTY_BYTES_REF);
> +
> +      // Also maybe save floor prefixes in root block; this can be a biggish perf gain for large ranges:
> +      /*
> +      System.out.println("root block pending.size=" + pending.size());
> +      for(Object o : pending) {
> +        System.out.println("  " + o);
> +      }
> +      */
> +      while (pending.size() >= minItemsInPrefix) {
> +        savePrefixes(0, pending.size());
> +      }
> +    }
> +
> +    Collections.sort(prefixes);
> +  }
> +
> +  /** Pushes the new term to the top of the stack, and writes new blocks. */
> +  private void pushTerm(BytesRef text) throws IOException {
> +    int limit = Math.min(lastTerm.length(), text.length);
> +
> +    // Find common prefix between last term and current term:
> +    int pos = 0;
> +    while (pos < limit && lastTerm.byteAt(pos) == text.bytes[text.offset+pos]) {
> +      pos++;
> +    }
> +
> +    //if (DEBUG) System.out.println("  shared=" + pos + "  lastTerm.length=" + lastTerm.length());
> +
> +    // Close the "abandoned" suffix now:
> +    for(int i=lastTerm.length()-1;i>=pos;i--) {
> +
> +      // How many items on top of the stack share the current suffix
> +      // we are closing:
> +      int prefixTopSize = pending.size() - prefixStarts[i];
> +
> +      while (prefixTopSize >= minItemsInPrefix) {
> +        //if (DEBUG) System.out.println("pushTerm i=" + i + " prefixTopSize=" + prefixTopSize + " minItemsInBlock=" + minItemsInPrefix);
> +        savePrefixes(i+1, prefixTopSize);
> +        //prefixStarts[i] -= prefixTopSize;
> +        //System.out.println("    after savePrefixes: " + (pending.size() - prefixStarts[i]) + " pending.size()=" + pending.size() + " start=" + prefixStarts[i]);
> +
> +        // For large floor blocks, it's possible we should now re-run on the new prefix terms we just created:
> +        prefixTopSize = pending.size() - prefixStarts[i];
> +      }
> +    }
> +
> +    if (prefixStarts.length < text.length) {
> +      prefixStarts = ArrayUtil.grow(prefixStarts, text.length);
> +    }
> +
> +    // Init new tail:
> +    for(int i=pos;i<text.length;i++) {
> +      prefixStarts[i] = pending.size();
> +    }
> +
> +    lastTerm.copyBytes(text);
> +
> +    // Only append the first (optional) empty string, no the fake last one used to close all prefixes:
> +    if (text.length > 0 || pending.isEmpty()) {
> +      byte[] termBytes = new byte[text.length];
> +      System.arraycopy(text.bytes, text.offset, termBytes, 0, text.length);
> +      pending.add(termBytes);
> +    }
> +  }
> +
> +  void savePrefixes(int prefixLength, int count) throws IOException {
> +
> +    assert count > 0;
> +
> +    //if (DEBUG2) {
> +    //  BytesRef br = new BytesRef(lastTerm.bytes());
> +    //  br.length = prefixLength;
> +    //  System.out.println("  savePrefixes: seg=" + segment + " " + brToString(br) + " count=" + count + " pending.size()=" + pending.size());
> +    //}
> +
> +    int lastSuffixLeadLabel = -2;
> +
> +    int start = pending.size()-count;
> +    assert start >=0;
> +
> +    int end = pending.size();
> +    int nextBlockStart = start;
> +    int nextFloorLeadLabel = -1;
> +    int prefixCount = 0;
> +    int pendingCount = 0;
> +    PrefixTerm lastPTEntry = null;
> +    for (int i=start; i<end; i++) {
> +
> +      byte[] termBytes;
> +      Object o = pending.get(i);
> +      PrefixTerm ptEntry;
> +      if (o instanceof byte[]) {
> +        ptEntry = null;
> +        termBytes = (byte[]) o;
> +      } else {
> +        ptEntry = (PrefixTerm) o;
> +        termBytes = ptEntry.term.bytes;
> +        if (ptEntry.prefix.length != prefixLength) {
> +          assert ptEntry.prefix.length > prefixLength;
> +          ptEntry = null;
> +        }
> +      }
> +      pendingCount++;
> +
> +      //if (DEBUG) System.out.println("    check term=" + brToString(new BytesRef(termBytes)));
> +
> +      int suffixLeadLabel;
> +
> +      if (termBytes.length == prefixLength) {
> +        // Suffix is 0, i.e. prefix 'foo' and term is
> +        // 'foo' so the term has empty string suffix
> +        // in this block
> +        assert lastSuffixLeadLabel == -2;
> +        suffixLeadLabel = -2;
> +      } else {
> +        suffixLeadLabel = termBytes[prefixLength] & 0xff;
> +      }
> +
> +      // if (DEBUG) System.out.println("  i=" + i + " ent=" + ent + " suffixLeadLabel=" + suffixLeadLabel);
> +
> +      if (suffixLeadLabel != lastSuffixLeadLabel) {
> +        // This is a boundary, a chance to make an auto-prefix term if we want:
> +
> +        // When we are "recursing" (generating auto-prefix terms on a block of
> +        // floor'd auto-prefix terms), this assert is non-trivial because it
> +        // ensures the floorLeadEnd of the previous terms is in fact less
> +        // than the lead start of the current entry:
> +        assert suffixLeadLabel > lastSuffixLeadLabel: "suffixLeadLabel=" + suffixLeadLabel + " vs lastSuffixLeadLabel=" + lastSuffixLeadLabel;
> +
> +        // NOTE: must check nextFloorLeadLabel in case minItemsInPrefix is 2 and prefix is 'a' and we've seen 'a' and then 'aa'
> +        if (pendingCount >= minItemsInPrefix && end-nextBlockStart > maxItemsInPrefix && nextFloorLeadLabel != -1) {
> +          // The count is too large for one block, so we must break it into "floor" blocks, where we record
> +          // the leading label of the suffix of the first term in each floor block, so at search time we can
> +          // jump to the right floor block.  We just use a naive greedy segmenter here: make a new floor
> +          // block as soon as we have at least minItemsInBlock.  This is not always best: it often produces
> +          // a too-small block as the final block:
> +
> +          // If the last entry was another prefix term of the same length, then it represents a range of terms, so we must use its ending
> +          // prefix label as our ending label:
> +          if (lastPTEntry != null) {
> +            lastSuffixLeadLabel = lastPTEntry.floorLeadEnd;
> +          }
> +
> +          savePrefix(prefixLength, nextFloorLeadLabel, lastSuffixLeadLabel);
> +          pendingCount = 0;
> +
> +          prefixCount++;
> +          nextFloorLeadLabel = suffixLeadLabel;
> +          nextBlockStart = i;
> +        }
> +
> +        if (nextFloorLeadLabel == -1) {
> +          nextFloorLeadLabel = suffixLeadLabel;
> +          //if (DEBUG) System.out.println("set first lead label=" + nextFloorLeadLabel);
> +        }
> +
> +        lastSuffixLeadLabel = suffixLeadLabel;
> +      }
> +      lastPTEntry = ptEntry;
> +    }
> +
> +    // Write last block, if any:
> +    if (nextBlockStart < end) {
> +      //System.out.println("  lastPTEntry=" + lastPTEntry + " lastSuffixLeadLabel=" + lastSuffixLeadLabel);
> +      if (lastPTEntry != null) {
> +        lastSuffixLeadLabel = lastPTEntry.floorLeadEnd;
> +      }
> +      assert lastSuffixLeadLabel >= nextFloorLeadLabel: "lastSuffixLeadLabel=" + lastSuffixLeadLabel + " nextFloorLeadLabel=" + nextFloorLeadLabel;
> +      if (prefixCount == 0) {
> +        if (prefixLength > 0) {
> +          savePrefix(prefixLength, -2, 0xff);
> +          prefixCount++;
> +        } else {
> +          // Don't add a prefix term for all terms in the index!
> +        }
> +      } else {
> +        if (lastSuffixLeadLabel == -2) {
> +          // Special case when closing the empty string root block:
> +          lastSuffixLeadLabel = 0xff;
> +        }
> +        savePrefix(prefixLength, nextFloorLeadLabel, lastSuffixLeadLabel);
> +        prefixCount++;
> +      }
> +    }
> +
> +    // Remove slice from the top of the pending stack, that we just wrote:
> +    int sizeToClear = count;
> +    if (prefixCount > 1) {
> +      Object o = pending.get(pending.size()-count);
> +      if (o instanceof byte[] && ((byte[]) o).length == prefixLength) {
> +        // If we were just asked to write all f* terms, but there were too many and so we made floor blocks, the exact term 'f' will remain
> +        // as its own item, followed by floor block terms like f[a-m]*, f[n-z]*, so in this case we leave 3 (not 2) items on the pending stack:
> +        sizeToClear--;
> +      }
> +    }
> +    pending.subList(pending.size()-sizeToClear, pending.size()).clear();
> +
> +    // Append prefix terms for each prefix, since these count like real terms that also need to be "rolled up":
> +    for(int i=0;i<prefixCount;i++) {
> +      PrefixTerm pt = prefixes.get(prefixes.size()-(prefixCount-i));
> +      pending.add(pt);
> +    }
> +  }
> +
> +  private void savePrefix(int prefixLength, int floorLeadStart, int floorLeadEnd) {
> +    byte[] prefix = new byte[prefixLength];
> +    System.arraycopy(lastTerm.bytes(), 0, prefix, 0, prefixLength);
> +    assert floorLeadStart != -1;
> +    assert floorLeadEnd != -1;
> +
> +    PrefixTerm pt = new PrefixTerm(prefix, floorLeadStart, floorLeadEnd);
> +    //if (DEBUG2) System.out.println("    savePrefix: seg=" + segment + " " + pt + " count=" + count);
> +    prefixes.add(pt);
> +  }
> +}
>
> Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java?rev=1670918&view=auto
> ==============================================================================
> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java (added)
> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetPostingsEnum.java Thu Apr  2 15:05:48 2015
> @@ -0,0 +1,95 @@
> +package org.apache.lucene.codecs.blocktree;
> +
> +/*
> + * 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.PostingsEnum;
> +import org.apache.lucene.search.DocIdSetIterator;
> +import org.apache.lucene.util.BitSet;
> +import org.apache.lucene.util.BitSetIterator;
> +import org.apache.lucene.util.BytesRef;
> +import org.apache.lucene.util.FixedBitSet; // javadocs
> +
> +/** Takes a {@link FixedBitSet} and creates a DOCS {@link PostingsEnum} from it. */
> +
> +class BitSetPostingsEnum extends PostingsEnum {
> +  private final BitSet bits;
> +  private DocIdSetIterator in;
> +
> +  BitSetPostingsEnum(BitSet bits) {
> +    this.bits = bits;
> +    reset();
> +  }
> +
> +  @Override
> +  public int freq() throws IOException {
> +    return 1;
> +  }
> +
> +  @Override
> +  public int docID() {
> +    if (in == null) {
> +      return -1;
> +    } else {
> +      return in.docID();
> +    }
> +  }
> +
> +  @Override
> +  public int nextDoc() throws IOException {
> +    if (in == null) {
> +      in = new BitSetIterator(bits, 0);
> +    }
> +    return in.nextDoc();
> +  }
> +
> +  @Override
> +  public int advance(int target) throws IOException {
> +    return in.advance(target);
> +  }
> +
> +  @Override
> +  public long cost() {
> +    return in.cost();
> +  }
> +
> +  void reset() {
> +    in = null;
> +  }
> +
> +  @Override
> +  public BytesRef getPayload() {
> +    return null;
> +  }
> +
> +  @Override
> +  public int nextPosition() {
> +    throw new UnsupportedOperationException();
> +  }
> +
> +  @Override
> +  public int startOffset() {
> +    throw new UnsupportedOperationException();
> +  }
> +
> +  @Override
> +  public int endOffset() {
> +    throw new UnsupportedOperationException();
> +  }
> +}
>
> Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java?rev=1670918&view=auto
> ==============================================================================
> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java (added)
> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BitSetTermsEnum.java Thu Apr  2 15:05:48 2015
> @@ -0,0 +1,87 @@
> +package org.apache.lucene.codecs.blocktree;
> +
> +/*
> + * Licensed to the Apache Software Foundation (ASF) under one or more
> + * contributor license agreements.  See the NOTICE file distributed with
> + * this work for additional information regarding copyright ownership.
> + * The ASF licenses this file to You under the Apache License, Version 2.0
> + * (the "License"); you may not use this file except in compliance with
> + * the License.  You may obtain a copy of the License at
> + *
> + *     http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + */
> +
> +import org.apache.lucene.codecs.PostingsWriterBase;
> +import org.apache.lucene.index.PostingsEnum;
> +import org.apache.lucene.index.TermsEnum;
> +import org.apache.lucene.util.BitSet;
> +import org.apache.lucene.util.Bits;
> +import org.apache.lucene.util.BytesRef;
> +
> +/** Silly stub class, used only when writing an auto-prefix
> + *  term in order to expose DocsEnum over a FixedBitSet.  We
> + *  pass this to {@link PostingsWriterBase#writeTerm} so
> + *  that it can pull .docs() multiple times for the
> + *  current term. */
> +
> +class BitSetTermsEnum extends TermsEnum {
> +  private final BitSetPostingsEnum postingsEnum;
> +
> +  public BitSetTermsEnum(BitSet docs) {
> +    postingsEnum = new BitSetPostingsEnum(docs);
> +  }
> +
> +  @Override
> +  public SeekStatus seekCeil(BytesRef text) {
> +    throw new UnsupportedOperationException();
> +  }
> +
> +  @Override
> +  public void seekExact(long ord) {
> +    throw new UnsupportedOperationException();
> +  }
> +
> +  @Override
> +  public BytesRef term() {
> +    throw new UnsupportedOperationException();
> +  }
> +
> +  @Override
> +  public BytesRef next() {
> +    throw new UnsupportedOperationException();
> +  }
> +
> +  @Override
> +  public long ord() {
> +    throw new UnsupportedOperationException();
> +  }
> +
> +  @Override
> +  public int docFreq() {
> +    throw new UnsupportedOperationException();
> +  }
> +
> +  @Override
> +  public long totalTermFreq() {
> +    throw new UnsupportedOperationException();
> +  }
> +
> +  @Override
> +  public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) {
> +    if (flags != PostingsEnum.NONE) {
> +      // We only work with DOCS_ONLY fields
> +      return null;
> +    }
> +    if (liveDocs != null) {
> +      throw new IllegalArgumentException("cannot handle live docs");
> +    }
> +    postingsEnum.reset();
> +    return postingsEnum;
> +  }
> +}
>
> Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java?rev=1670918&r1=1670917&r2=1670918&view=diff
> ==============================================================================
> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java (original)
> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java Thu Apr  2 15:05:48 2015
> @@ -34,6 +34,8 @@ import org.apache.lucene.index.IndexFile
>  import org.apache.lucene.index.IndexOptions;
>  import org.apache.lucene.index.SegmentReadState;
>  import org.apache.lucene.index.Terms;
> +import org.apache.lucene.search.PrefixQuery;  // javadocs
> +import org.apache.lucene.search.TermRangeQuery;  // javadocs
>  import org.apache.lucene.store.IndexInput;
>  import org.apache.lucene.util.Accountable;
>  import org.apache.lucene.util.Accountables;
> @@ -57,6 +59,14 @@ import org.apache.lucene.util.fst.Output
>   *  min/maxItemsPerBlock during indexing to control how
>   *  much memory the terms index uses.</p>
>   *
> + *  <p>If auto-prefix terms were indexed (see
> + *  {@link BlockTreeTermsWriter}), then the {@link Terms#intersect}
> + *  implementation here will make use of these terms only if the
> + *  automaton has a binary sink state, i.e. an accept state
> + *  which has a transition to itself accepting all byte values.
> + *  For example, both {@link PrefixQuery} and {@link TermRangeQuery}
> + *  pass such automata to {@link Terms#intersect}.</p>
> + *
>   *  <p>The data structure used by this implementation is very
>   *  similar to a burst trie
>   *  (http://citeseer.ist.psu.edu/viewdoc/summary?doi=10.1.1.18.3499),
> @@ -90,8 +100,11 @@ public final class BlockTreeTermsReader
>    /** Initial terms format. */
>    public static final int VERSION_START = 0;
>
> +  /** Auto-prefix terms. */
> +  public static final int VERSION_AUTO_PREFIX_TERMS = 1;
> +
>    /** Current terms format. */
> -  public static final int VERSION_CURRENT = VERSION_START;
> +  public static final int VERSION_CURRENT = VERSION_AUTO_PREFIX_TERMS;
>
>    /** Extension of terms index file */
>    static final String TERMS_INDEX_EXTENSION = "tip";
> @@ -116,7 +129,7 @@ public final class BlockTreeTermsReader
>
>    final String segment;
>
> -  private final int version;
> +  final int version;
>
>    /** Sole constructor. */
>    public BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState state) throws IOException {
>
>

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org