You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2016/06/20 08:16:08 UTC

[2/3] lucene-solr:branch_6x: LUCENE-7317: Remove auto-prefix terms.

LUCENE-7317: Remove auto-prefix terms.


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

Branch: refs/heads/branch_6x
Commit: bac521d1aac6d5d7bbdfde195286c5a50e653364
Parents: d3fcac7
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Jun 9 18:44:25 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Mon Jun 20 09:58:42 2016 +0200

----------------------------------------------------------------------
 .../autoprefix/AutoPrefixPostingsFormat.java    | 125 ---
 .../lucene/codecs/autoprefix/package-info.java  |  22 -
 .../org.apache.lucene.codecs.PostingsFormat     |   1 -
 .../TestAutoPrefixPostingsFormat.java           |  38 -
 .../codecs/autoprefix/TestAutoPrefixTerms.java  | 941 -------------------
 .../apache/lucene/codecs/BlockTermState.java    |  14 +-
 .../codecs/blocktree/AutoPrefixTermsWriter.java | 438 ---------
 .../codecs/blocktree/BlockTreeTermsReader.java  |  11 +-
 .../codecs/blocktree/BlockTreeTermsWriter.java  | 222 +----
 .../codecs/blocktree/IntersectTermsEnum.java    |   2 -
 .../org/apache/lucene/index/TermContext.java    |  12 -
 .../java/org/apache/lucene/index/TermState.java |   6 -
 .../MultiTermQueryConstantScoreWrapper.java     |   4 -
 .../apache/lucene/search/ScoringRewrite.java    |   2 +-
 .../org/apache/lucene/search/TermQuery.java     |   8 -
 .../org/apache/solr/query/SolrRangeQuery.java   |   4 -
 16 files changed, 31 insertions(+), 1819 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bac521d1/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java
deleted file mode 100644
index b83866a..0000000
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/AutoPrefixPostingsFormat.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.codecs.autoprefix;
-
-
-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);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bac521d1/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java b/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java
deleted file mode 100644
index b15f2fa..0000000
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bac521d1/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
index 753b6d7..710c605 100644
--- a/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
+++ b/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
@@ -19,4 +19,3 @@ org.apache.lucene.codecs.memory.DirectPostingsFormat
 org.apache.lucene.codecs.memory.FSTOrdPostingsFormat
 org.apache.lucene.codecs.memory.FSTPostingsFormat
 org.apache.lucene.codecs.memory.MemoryPostingsFormat
-org.apache.lucene.codecs.autoprefix.AutoPrefixPostingsFormat

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bac521d1/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java
deleted file mode 100644
index 3c7059e..0000000
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixPostingsFormat.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.codecs.autoprefix;
-
-
-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);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bac521d1/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java b/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java
deleted file mode 100644
index 316fc9a..0000000
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/autoprefix/TestAutoPrefixTerms.java
+++ /dev/null
@@ -1,941 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.codecs.autoprefix;
-
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-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.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.RandomIndexWriter;
-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.BooleanQuery;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.PrefixQuery;
-import org.apache.lucene.search.TermRangeQuery;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.MathUtil;
-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;
-
-  public TestAutoPrefixTerms() {
-    if (LuceneTestCase.VERBOSE) {
-      System.out.println("TEST: using minItemsPerBlock=" + minItemsPerBlock + " maxItemsPerBlock=" + maxItemsPerBlock);
-      System.out.println("TEST: using minTermsAutoPrefix=" + minTermsAutoPrefix + " maxTermsAutoPrefix=" + maxTermsAutoPrefix);
-    }
-  }
-
-  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);
-
-    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(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(newStringField("field", intToBytes(term), Field.Store.NO));
-      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);
-
-    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(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);
-
-    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(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);
-    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(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"));
-    assertEquals(31, newSearcher(r).search(q, 1).totalHits);
-    r.close();
-    w.close();
-    dir.close();
-  }
-
-  /** 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-prefix terms were used in this test
-        long allowedMaxTerms;
-
-        if (bounds.length == 1) {
-          // Simple prefix query: we should never see more than maxPrefixCount terms, except for the empty string:
-          if (bounds[0].length == 0) {
-            allowedMaxTerms = Integer.MAX_VALUE;
-          } else {
-            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));
-          if (commonPrefix == 0) {
-            int min;
-            if (minTerm.length == 0) {
-              min = 0;
-            } else {
-              min = minTerm.bytes[minTerm.offset] & 0xff;
-            }
-            int max;
-            if (maxTerm.length == 0) {
-              max = 0;
-            } else {
-              max = maxTerm.bytes[maxTerm.offset] & 0xff;
-            }
-            if (max > min) {
-              // When maxPrefixCount is small (< 16), each byte of the term can require more than one "level" of auto-prefixing:
-              // NOTE: this is still only approximate ... it's tricky to get a closed form max bound that's "tight"
-              allowedMaxTerms += MathUtil.log(max-min, maxPrefixCount);
-            }
-          }
-        }
-
-        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-prefix 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);
-    IllegalStateException expected = expectThrows(IllegalStateException.class, () -> {
-      w.commit();
-    });
-    assertEquals("ranges can only be indexed with IndexOptions.DOCS (field: foo)", expected.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);
-    IllegalStateException expected = expectThrows(IllegalStateException.class, () -> {
-      w.commit();
-    });
-    assertEquals("ranges can only be indexed with IndexOptions.DOCS (field: foo)", expected.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);
-    IllegalStateException expected = expectThrows(IllegalStateException.class, () -> {
-      w.commit();
-    });
-    assertEquals("ranges can only be indexed with IndexOptions.DOCS (field: foo)", expected.getMessage());
-
-    w.close();
-    dir.close();
-  }
-
-  /** Make sure auto prefix terms are used with TermRangeQuery */
-  public void testTermRange() throws Exception {
-
-    List<String> prefixes = new ArrayList<>();
-    for(int i=1;i<5;i++) {
-      char[] chars = new char[i];
-      Arrays.fill(chars, 'a');
-      prefixes.add(new String(chars));
-    }
-
-    Set<String> randomTerms = new HashSet<>();
-    int numTerms = atLeast(10000);
-    while (randomTerms.size() < numTerms) {
-      for(String prefix : prefixes) {
-        randomTerms.add(prefix + TestUtil.randomSimpleString(random()));
-      }
-    }
-
-    // We make term range aa<start> - aa<end>
-    char start;
-    char end;
-
-    int actualCount;
-    boolean startInclusive = random().nextBoolean();
-    boolean endInclusive = random().nextBoolean();
-    String startTerm;
-    String endTerm;
-
-    while (true) {
-      start = (char) TestUtil.nextInt(random(), 'a', 'm');
-      end = (char) TestUtil.nextInt(random(), start+1, 'z');
-
-      actualCount = 0;
-
-      startTerm = "aa" + start;
-      endTerm = "aa" + end;
-
-      for(String term : randomTerms) {
-        int cmpStart = startTerm.compareTo(term);
-        int cmpEnd = endTerm.compareTo(term);
-        if ((cmpStart < 0 || (startInclusive && cmpStart == 0)) &&
-            (cmpEnd > 0 || (endInclusive && cmpEnd == 0))) {
-          actualCount++;
-        }
-      }
-
-      if (actualCount > 2000) {
-        break;
-      }
-    }
-
-    if (VERBOSE) {
-      System.out.println("start " + startTerm + " inclusive? " + startInclusive);
-      System.out.println("end " + endTerm + " inclusive? " + endInclusive);
-      System.out.println("actual count " + actualCount);
-    }
-
-    Directory dir = newDirectory();
-    IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
-    int minTermsInBlock = TestUtil.nextInt(random(), 2, 100);
-    int maxTermsInBlock = Math.max(2, (minTermsInBlock-1)*2 + random().nextInt(100));
-
-    int minTermsAutoPrefix = TestUtil.nextInt(random(), 2, 100);
-    int maxTermsAutoPrefix = random().nextBoolean() ? Math.max(2, (minTermsAutoPrefix-1)*2 + random().nextInt(100)) : Integer.MAX_VALUE;
-
-    if (VERBOSE) {
-      System.out.println("minTermsAutoPrefix " + minTermsAutoPrefix);
-      System.out.println("maxTermsAutoPrefix " + maxTermsAutoPrefix);
-    }
-
-    iwc.setCodec(TestUtil.alwaysPostingsFormat(new AutoPrefixPostingsFormat(minTermsInBlock, maxTermsInBlock,
-                                                                            minTermsAutoPrefix, maxTermsAutoPrefix)));
-    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
-
-    if (VERBOSE) {
-      System.out.println("TEST: index terms");
-    }
-    for (String term : randomTerms) {
-      Document doc = new Document();
-      doc.add(new StringField("field", term, Field.Store.NO));
-      w.addDocument(doc);
-      if (VERBOSE) {
-        System.out.println("  " + term);
-      }
-    }
-
-    if (VERBOSE) {
-      System.out.println("TEST: now force merge");
-    }
-
-    w.forceMerge(1);
-    IndexReader r = w.getReader();
-    final Terms terms = MultiFields.getTerms(r, "field");
-    IndexSearcher s = new IndexSearcher(r);
-    final int finalActualCount = actualCount;
-    if (VERBOSE) {
-      System.out.println("start=" + startTerm + " end=" + endTerm + " startIncl=" + startInclusive + " endIncl=" + endInclusive);
-    }
-    TermRangeQuery q = new TermRangeQuery("field", new BytesRef(startTerm), new BytesRef(endTerm), startInclusive, endInclusive) {
-      public TermRangeQuery checkTerms() throws IOException {
-        TermsEnum termsEnum = getTermsEnum(terms, new AttributeSource());
-        int count = 0;
-        while (termsEnum.next() != null) {
-          if (VERBOSE) {
-            System.out.println("got term: " + termsEnum.term().utf8ToString());
-          }
-          count++;
-        }
-        if (VERBOSE) {
-          System.out.println("count " + count + " vs finalActualCount=" + finalActualCount);
-        }
-
-        // Auto-prefix term(s) should have kicked in, so we should have visited fewer than the total number of aa* terms:
-        assertTrue(count < finalActualCount);
-
-        return this;
-      }
-    }.checkTerms();
-
-    int maxClauseCount = BooleanQuery.getMaxClauseCount();
-
-    try {
-
-      // TODO test with boolean rewrite as well once we can create term
-      // queries on fake terms
-      /*if (random().nextBoolean()) {
-        q.setRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_REWRITE);
-        BooleanQuery.setMaxClauseCount(actualCount);
-      } else if (random().nextBoolean()) {
-        q.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_BOOLEAN_REWRITE);
-        BooleanQuery.setMaxClauseCount(actualCount);
-      }*/
-
-      if (VERBOSE) {
-        System.out.println("TEST: use rewrite method " + q.getRewriteMethod());
-      }
-      assertEquals(actualCount, s.search(q, 1).totalHits);
-    } finally {
-      BooleanQuery.setMaxClauseCount(maxClauseCount);
-    }
-
-    // Test when min == max:
-    List<String> randomTermsList = new ArrayList<>(randomTerms);
-    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
-      String term = randomTermsList.get(random().nextInt(randomTermsList.size()));
-      q = new TermRangeQuery("field", new BytesRef(term), new BytesRef(term), true, true);
-      assertEquals(1, s.search(q, 1).totalHits);
-    }
-
-    r.close();
-    w.close();
-    dir.close();
-  }
-
-
-  /** Make sure auto prefix terms are used with PrefixQuery. */
-  public void testPrefixQuery() throws Exception {
-
-    List<String> prefixes = new ArrayList<>();
-    for(int i=1;i<5;i++) {
-      char[] chars = new char[i];
-      Arrays.fill(chars, 'a');
-      prefixes.add(new String(chars));
-    }
-
-    Set<String> randomTerms = new HashSet<>();
-    int numTerms = atLeast(10000);
-    while (randomTerms.size() < numTerms) {
-      for(String prefix : prefixes) {
-        randomTerms.add(prefix + TestUtil.randomRealisticUnicodeString(random()));
-      }
-    }
-
-    int actualCount = 0;
-    for(String term : randomTerms) {
-      if (term.startsWith("aa")) {
-        actualCount++;
-      }
-    }
-
-    //System.out.println("actual count " + actualCount);
-
-    Directory dir = newDirectory();
-    IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
-    int minTermsInBlock = TestUtil.nextInt(random(), 2, 100);
-    int maxTermsInBlock = Math.max(2, (minTermsInBlock-1)*2 + random().nextInt(100));
-
-    // As long as this is never > actualCount, aa should always see at least one auto-prefix term:
-    int minTermsAutoPrefix = TestUtil.nextInt(random(), 2, actualCount);
-    int maxTermsAutoPrefix = random().nextBoolean() ? Math.max(2, (minTermsAutoPrefix-1)*2 + random().nextInt(100)) : Integer.MAX_VALUE;
-
-    iwc.setCodec(TestUtil.alwaysPostingsFormat(new AutoPrefixPostingsFormat(minTermsInBlock, maxTermsInBlock,
-                                                                            minTermsAutoPrefix, maxTermsAutoPrefix)));
-    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
-
-    for (String term : randomTerms) {
-      Document doc = new Document();
-      doc.add(new StringField("field", term, Field.Store.NO));
-      w.addDocument(doc);
-    }
-
-    w.forceMerge(1);
-    IndexReader r = w.getReader();
-    final Terms terms = MultiFields.getTerms(r, "field");
-    IndexSearcher s = new IndexSearcher(r);
-    final int finalActualCount = actualCount;
-    PrefixQuery q = new PrefixQuery(new Term("field", "aa")) {
-      public PrefixQuery checkTerms() throws IOException {
-        TermsEnum termsEnum = getTermsEnum(terms, new AttributeSource());
-        int count = 0;
-        while (termsEnum.next() != null) {
-          //System.out.println("got term: " + termsEnum.term().utf8ToString());
-          count++;
-        }
-
-        // Auto-prefix term(s) should have kicked in, so we should have visited fewer than the total number of aa* terms:
-        assertTrue(count < finalActualCount);
-
-        return this;
-      }
-    }.checkTerms();
-
-    int x = BooleanQuery.getMaxClauseCount();
-    try {
-      // TODO test with boolean rewrite as well once we can create term
-      // queries on fake terms
-      /*BooleanQuery.setMaxClauseCount(randomTerms.size());
-      if (random().nextBoolean()) {
-        q.setRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_REWRITE);
-      } else if (random().nextBoolean()) {
-        q.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_BOOLEAN_REWRITE);
-      }*/
-
-      assertEquals(actualCount, s.search(q, 1).totalHits);
-    } finally {
-      BooleanQuery.setMaxClauseCount(x);
-    }
-
-    r.close();
-    w.close();
-    dir.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bac521d1/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java b/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java
index 595bfbb..c317668 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java
@@ -16,7 +16,6 @@
  */
 package org.apache.lucene.codecs;
 
-import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader; // javadocs
 import org.apache.lucene.index.OrdTermState;
 import org.apache.lucene.index.TermState;
 
@@ -39,11 +38,6 @@ public class BlockTermState extends OrdTermState {
   // 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 = true;
-
   /** Sole constructor. (For invocation by subclass 
    *  constructors, typically implicit.) */
   protected BlockTermState() {
@@ -58,16 +52,10 @@ public class BlockTermState extends OrdTermState {
     totalTermFreq = other.totalTermFreq;
     termBlockOrd = other.termBlockOrd;
     blockFilePointer = other.blockFilePointer;
-    isRealTerm = other.isRealTerm;
-  }
-
-  @Override
-  public boolean isRealTerm() {
-    return isRealTerm;
   }
 
   @Override
   public String toString() {
-    return "docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termBlockOrd=" + termBlockOrd + " blockFP=" + blockFilePointer + " isRealTerm=" + isRealTerm;
+    return "docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termBlockOrd=" + termBlockOrd + " blockFP=" + blockFilePointer;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bac521d1/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java
deleted file mode 100644
index 501068d..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java
+++ /dev/null
@@ -1,438 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.codecs.blocktree;
-
-
-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 {@code >= min} and {@code <= 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();
-    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());
-      }
-    }
-
-    // Even though we visited terms in already-sorted order, the prefixes
-    // can be slightly unsorted, e.g. aaaaa will be before aaa, so we
-    // must sort here so our caller can do merge sort into actual terms
-    // when writing.  Probably we should use CollectionUtil.timSort here?
-    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);
-    //if (DEBUG) System.out.println("\nterm: " + text.utf8ToString());
-
-    // 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("  pop: i=" + i + " prefixTopSize=" + prefixTopSize + " minItemsInBlock=" + minItemsInPrefix);
-        savePrefixes(i+1, prefixTopSize);
-        //prefixStarts[i] -= prefixTopSize;
-        //if (DEBUG) 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());
-      System.out.println("  savePrefixes: " + brToString(br) + " count=" + count + " pending.size()=" + pending.size());
-    }
-    */
-
-    int lastSuffixLeadLabel = -2;
-
-    int start = pending.size()-count;
-    assert start >=0;
-
-    // Special case empty-string suffix case: we are being asked to build prefix terms for all aaa* terms, but 
-    // the exact term aaa is here, and we must skip it (it is handled "higher", under the aa* terms):
-    Object o = pending.get(start);
-    boolean skippedEmptyStringSuffix = false;
-    if (o instanceof byte[]) {
-      if (((byte[]) o).length == prefixLength) {
-        start++;
-        count--;
-        //if (DEBUG) System.out.println("  skip empty-string term suffix");
-        skippedEmptyStringSuffix = true;
-      }
-    } else {
-      PrefixTerm prefix = (PrefixTerm) o;
-      if (prefix.term.bytes.length == prefixLength) {
-        start++;
-        count--;
-        //if (DEBUG) System.out.println("  skip empty-string PT suffix");
-        skippedEmptyStringSuffix = true;
-      }
-    }
-
-    int end = pending.size();
-    int nextBlockStart = start;
-    int nextFloorLeadLabel = -1;
-    int prefixCount = 0;
-
-    PrefixTerm lastPTEntry = null;
-
-    for (int i=start; i<end; i++) {
-
-      byte[] termBytes;
-      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;
-        }
-      }
-
-      //if (DEBUG) System.out.println("    check term=" + brToString(new BytesRef(termBytes)) + " o=" + o);
-
-      // We handled the empty-string suffix case up front:
-      assert termBytes.length > prefixLength;
-
-      int suffixLeadLabel = termBytes[prefixLength] & 0xff;
-
-      //if (DEBUG) System.out.println("  i=" + i + " o=" + o + " suffixLeadLabel=" + Integer.toHexString(suffixLeadLabel) + " pendingCount=" + (i - nextBlockStart) + " min=" + minItemsInPrefix);
-
-      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;
-
-        int itemsInBlock = i - nextBlockStart;
-
-        if (itemsInBlock >= minItemsInPrefix && end-nextBlockStart > maxItemsInPrefix) {
-          // 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) {
-            //if (DEBUG) System.out.println("  use last");
-            lastSuffixLeadLabel = lastPTEntry.floorLeadEnd;
-          }
-          savePrefix(prefixLength, nextFloorLeadLabel, lastSuffixLeadLabel);
-
-          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++;
-          
-          // If we skipped empty string suffix, e.g. term aaa for prefix aaa*, since we
-          // are now writing the full aaa* prefix term, we include it here:
-          if (skippedEmptyStringSuffix) {
-            count++;
-          }
-        } 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:
-
-    pending.subList(pending.size()-count, 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);
-    //if (DEBUG) System.out.println("    savePrefix: " + pt);
-
-    prefixes.add(pt);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bac521d1/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
index 03ec517..6fc9a24 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
@@ -107,8 +107,11 @@ public final class BlockTreeTermsReader extends FieldsProducer {
    *  this field did write any auto-prefix terms. */
   public static final int VERSION_AUTO_PREFIX_TERMS_COND = 2;
 
+  /** Auto-prefix terms have been superseded by points. */
+  public static final int VERSION_AUTO_PREFIX_TERMS_REMOVED = 3;
+
   /** Current terms format. */
-  public static final int VERSION_CURRENT = VERSION_AUTO_PREFIX_TERMS_COND;
+  public static final int VERSION_CURRENT = VERSION_AUTO_PREFIX_TERMS_REMOVED;
 
   /** Extension of terms index file */
   static final String TERMS_INDEX_EXTENSION = "tip";
@@ -150,15 +153,15 @@ public final class BlockTreeTermsReader extends FieldsProducer {
       termsIn = state.directory.openInput(termsName, state.context);
       version = CodecUtil.checkIndexHeader(termsIn, TERMS_CODEC_NAME, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
 
-      if (version < VERSION_AUTO_PREFIX_TERMS) {
-        // Old (pre-5.2.0) index, no auto-prefix terms:
+      if (version < VERSION_AUTO_PREFIX_TERMS || version >= VERSION_AUTO_PREFIX_TERMS_REMOVED) {
+        // Old (pre-5.2.0) or recent (6.2.0+) index, no auto-prefix terms:
         this.anyAutoPrefixTerms = false;
       } else if (version == VERSION_AUTO_PREFIX_TERMS) {
         // 5.2.x index, might have auto-prefix terms:
         this.anyAutoPrefixTerms = true;
       } else {
         // 5.3.x index, we record up front if we may have written any auto-prefix terms:
-        assert version >= VERSION_AUTO_PREFIX_TERMS_COND;
+        assert version == VERSION_AUTO_PREFIX_TERMS_COND;
         byte b = termsIn.readByte();
         if (b == 0) {
           this.anyAutoPrefixTerms = false;