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 0ef21292c04..b0091fd6970 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
@@ -34,8 +34,6 @@ import org.apache.lucene.index.IndexFileNames;
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;
@@ -59,14 +57,6 @@ import org.apache.lucene.util.fst.Outputs;
* min/maxItemsPerBlock during indexing to control how
* much memory the terms index uses.
*
- * 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}.
- *
* 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),
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
index 4ee38262403..46aee6e48d7 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
@@ -127,7 +127,6 @@ public final class FieldReader extends Terms implements Accountable {
/** For debugging -- used by CheckIndex too*/
@Override
public Stats getStats() throws IOException {
- // TODO: add auto-prefix terms into stats
return new SegmentTermsEnum(this).computeBlockStats();
}
@@ -185,7 +184,7 @@ public final class FieldReader extends Terms implements Accountable {
if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
}
- return new IntersectTermsEnum(this, compiled.automaton, compiled.runAutomaton, compiled.commonSuffixRef, startTerm, compiled.sinkState);
+ return new IntersectTermsEnum(this, compiled.automaton, compiled.runAutomaton, compiled.commonSuffixRef, startTerm);
}
@Override
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
index bbd7e7b58fa..934b5f64a37 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
@@ -42,10 +42,7 @@ import org.apache.lucene.util.fst.Outputs;
* the terms. It does not use the terms index at all: on init, it
* loads the root block, and scans its way to the initial term.
* Likewise, in next it scans until it finds a term that matches the
- * current automaton transition. If the index has auto-prefix terms
- * (only for DOCS_ONLY fields currently) it will visit these terms
- * when possible and then skip the real terms that auto-prefix term
- * matched. */
+ * current automaton transition. */
final class IntersectTermsEnum extends TermsEnum {
@@ -69,29 +66,19 @@ final class IntersectTermsEnum extends TermsEnum {
private final FST.BytesReader fstReader;
- private final boolean allowAutoPrefixTerms;
-
final FieldReader fr;
- /** Which state in the automaton accepts all possible suffixes. */
- private final int sinkState;
-
private BytesRef savedStartTerm;
-
- /** True if we did return the current auto-prefix term */
- private boolean useAutoPrefixTerm;
// TODO: in some cases we can filter by length? eg
// regexp foo*bar must be at least length 6 bytes
- public IntersectTermsEnum(FieldReader fr, Automaton automaton, RunAutomaton runAutomaton, BytesRef commonSuffix, BytesRef startTerm, int sinkState) throws IOException {
+ public IntersectTermsEnum(FieldReader fr, Automaton automaton, RunAutomaton runAutomaton, BytesRef commonSuffix, BytesRef startTerm) throws IOException {
this.fr = fr;
- this.sinkState = sinkState;
assert automaton != null;
assert runAutomaton != null;
this.runAutomaton = runAutomaton;
- this.allowAutoPrefixTerms = sinkState != -1;
this.automaton = automaton;
this.commonSuffix = commonSuffix;
@@ -269,7 +256,6 @@ final class IntersectTermsEnum extends TermsEnum {
final int saveSuffix = currentFrame.suffix;
final long saveLastSubFP = currentFrame.lastSubFP;
final int saveTermBlockOrd = currentFrame.termState.termBlockOrd;
- final boolean saveIsAutoPrefixTerm = currentFrame.isAutoPrefixTerm;
final boolean isSubBlock = currentFrame.next();
@@ -297,11 +283,8 @@ final class IntersectTermsEnum extends TermsEnum {
}
continue;
} else if (cmp == 0) {
- if (allowAutoPrefixTerms == false && currentFrame.isAutoPrefixTerm) {
- continue;
- }
return;
- } else if (allowAutoPrefixTerms || currentFrame.isAutoPrefixTerm == false) {
+ } else {
// Fallback to prior entry: the semantics of
// this method is that the first call to
// next() will return the term after the
@@ -312,7 +295,6 @@ final class IntersectTermsEnum extends TermsEnum {
currentFrame.suffix = saveSuffix;
currentFrame.suffixesReader.setPosition(savePos);
currentFrame.termState.termBlockOrd = saveTermBlockOrd;
- currentFrame.isAutoPrefixTerm = saveIsAutoPrefixTerm;
System.arraycopy(currentFrame.suffixBytes, currentFrame.startBytePos, term.bytes, currentFrame.prefix, currentFrame.suffix);
term.length = currentFrame.prefix + currentFrame.suffix;
// If the last entry was a block we don't
@@ -349,139 +331,6 @@ final class IntersectTermsEnum extends TermsEnum {
return currentFrame.next();
}
- private boolean skipPastLastAutoPrefixTerm() throws IOException {
- assert currentFrame.isAutoPrefixTerm;
- useAutoPrefixTerm = false;
-
- // If we last returned an auto-prefix term, we must now skip all
- // actual terms sharing that prefix. At most, that skipping
- // requires popping one frame, but it can also require simply
- // scanning ahead within the current frame. This scanning will
- // skip sub-blocks that contain many terms, which is why the
- // optimization "works":
- int floorSuffixLeadEnd = currentFrame.floorSuffixLeadEnd;
-
- boolean isSubBlock;
-
- if (floorSuffixLeadEnd == -1) {
- // An ordinary prefix, e.g. foo*
- int prefix = currentFrame.prefix;
- int suffix = currentFrame.suffix;
- if (suffix == 0) {
-
- // Easy case: the prefix term's suffix is the empty string,
- // meaning the prefix corresponds to all terms in the
- // current block, so we just pop this entire block:
- if (currentFrame.ord == 0) {
- throw NoMoreTermsException.INSTANCE;
- }
- currentFrame = stack[currentFrame.ord-1];
- currentTransition = currentFrame.transition;
-
- return popPushNext();
-
- } else {
-
- // Just next() until we hit an entry that doesn't share this
- // prefix. The first next should be a sub-block sharing the
- // same prefix, because if there are enough terms matching a
- // given prefix to warrant an auto-prefix term, then there
- // must also be enough to make a sub-block (assuming
- // minItemsInPrefix > minItemsInBlock):
- scanPrefix:
- while (true) {
- if (currentFrame.nextEnt == currentFrame.entCount) {
- if (currentFrame.isLastInFloor == false) {
- currentFrame.loadNextFloorBlock();
- } else if (currentFrame.ord == 0) {
- throw NoMoreTermsException.INSTANCE;
- } else {
- // Pop frame, which also means we've moved beyond this
- // auto-prefix term:
- currentFrame = stack[currentFrame.ord-1];
- currentTransition = currentFrame.transition;
-
- return popPushNext();
- }
- }
- isSubBlock = currentFrame.next();
- for(int i=0;i floorSuffixLeadEnd
- //assert currentFrame.prefix == prefix-1;
- //prefix = currentFrame.prefix;
-
- // In case when we pop, and the parent block is not just prefix-1, e.g. in block 417* on
- // its first term = floor prefix term 41[7-9], popping to block 4*:
- prefix = currentFrame.prefix;
-
- suffix = term.length - currentFrame.prefix;
- } else {
- // No need to pop; just scan in currentFrame:
- }
-
- // Now we scan until the lead suffix byte is > floorSuffixLeadEnd
- scanFloor:
- while (true) {
- if (currentFrame.nextEnt == currentFrame.entCount) {
- if (currentFrame.isLastInFloor == false) {
- currentFrame.loadNextFloorBlock();
- } else if (currentFrame.ord == 0) {
- throw NoMoreTermsException.INSTANCE;
- } else {
- // Pop frame, which also means we've moved beyond this
- // auto-prefix term:
- currentFrame = stack[currentFrame.ord-1];
- currentTransition = currentFrame.transition;
-
- return popPushNext();
- }
- }
- isSubBlock = currentFrame.next();
- for(int i=0;i= suffix && (currentFrame.suffixBytes[currentFrame.startBytePos+suffix-1]&0xff) > floorSuffixLeadEnd) {
- // Done scanning: we are now on the first term after all
- // terms matched by this auto-prefix term
- break;
- }
- }
- }
-
- return isSubBlock;
- }
-
// Only used internally when there are no more terms in next():
private static final class NoMoreTermsException extends RuntimeException {
@@ -511,15 +360,7 @@ final class IntersectTermsEnum extends TermsEnum {
private BytesRef _next() throws IOException {
- boolean isSubBlock;
-
- if (useAutoPrefixTerm) {
- // If the current term was an auto-prefix term, we have to skip past it:
- isSubBlock = skipPastLastAutoPrefixTerm();
- assert useAutoPrefixTerm == false;
- } else {
- isSubBlock = popPushNext();
- }
+ boolean isSubBlock = popPushNext();
nextTerm:
@@ -669,41 +510,6 @@ final class IntersectTermsEnum extends TermsEnum {
currentFrame = pushFrame(state);
currentTransition = currentFrame.transition;
currentFrame.lastState = lastState;
- } else if (currentFrame.isAutoPrefixTerm) {
- // We are on an auto-prefix term, meaning this term was compiled
- // at indexing time, matching all terms sharing this prefix (or,
- // a floor'd subset of them if that count was too high). A
- // prefix term represents a range of terms, so we now need to
- // test whether, from the current state in the automaton, it
- // accepts all terms in that range. As long as it does, we can
- // use this term and then later skip ahead past all terms in
- // this range:
- if (allowAutoPrefixTerms) {
-
- if (currentFrame.floorSuffixLeadEnd == -1) {
- // Simple prefix case
- useAutoPrefixTerm = state == sinkState;
- } else {
- if (currentFrame.floorSuffixLeadStart == -1) {
- // Must also accept the empty string in this case
- if (automaton.isAccept(state)) {
- useAutoPrefixTerm = acceptsSuffixRange(state, 0, currentFrame.floorSuffixLeadEnd);
- }
- } else {
- useAutoPrefixTerm = acceptsSuffixRange(lastState, currentFrame.floorSuffixLeadStart, currentFrame.floorSuffixLeadEnd);
- }
- }
-
- if (useAutoPrefixTerm) {
- // All suffixes of this auto-prefix term are accepted by the automaton, so we can use it:
- copyTerm();
- return term;
- } else {
- // We move onto the next term
- }
- } else {
- // We are not allowed to use auto-prefix terms, so we just skip it
- }
} else if (runAutomaton.isAccept(state)) {
copyTerm();
assert savedStartTerm == null || term.compareTo(savedStartTerm) > 0: "saveStartTerm=" + savedStartTerm.utf8ToString() + " term=" + term.utf8ToString();
@@ -716,24 +522,6 @@ final class IntersectTermsEnum extends TermsEnum {
}
}
- private final Transition scratchTransition = new Transition();
-
- /** Returns true if, from this state, the automaton accepts any suffix
- * starting with a label between start and end, inclusive. We just
- * look for a transition, matching this range, to the sink state. */
- private boolean acceptsSuffixRange(int state, int start, int end) {
-
- int count = automaton.initTransition(state, scratchTransition);
- for(int i=0;i= scratchTransition.min && end <= scratchTransition.max && scratchTransition.dest == sinkState) {
- return true;
- }
- }
-
- return false;
- }
-
// for debugging
@SuppressWarnings("unused")
static String brToString(BytesRef b) {
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
index 236e77a7f67..b1cfa7c04d2 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
@@ -95,17 +95,6 @@ final class IntersectTermsEnumFrame {
int startBytePos;
int suffix;
- // When we are on an auto-prefix term this is the starting lead byte
- // of the suffix (e.g. 'a' for the foo[a-m]* case):
- int floorSuffixLeadStart;
-
- // When we are on an auto-prefix term this is the ending lead byte
- // of the suffix (e.g. 'm' for the foo[a-m]* case):
- int floorSuffixLeadEnd;
-
- // True if the term we are currently on is an auto-prefix term:
- boolean isAutoPrefixTerm;
-
private final IntersectTermsEnum ite;
public IntersectTermsEnumFrame(IntersectTermsEnum ite, int ord) throws IOException {
@@ -219,10 +208,6 @@ final class IntersectTermsEnumFrame {
// written one after another -- tail recurse:
fpEnd = ite.in.getFilePointer();
}
-
- // Necessary in case this ord previously was an auto-prefix
- // term but now we recurse to a new leaf block
- isAutoPrefixTerm = false;
}
// TODO: maybe add scanToLabel; should give perf boost
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
index 327c181229e..8e01275378c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
@@ -34,8 +34,7 @@ import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.Util;
-/** Iterates through terms in this field. This implementation skips
- * any auto-prefix terms it encounters. */
+/** Iterates through terms in this field. */
final class SegmentTermsEnum extends TermsEnum {
@@ -121,8 +120,6 @@ final class SegmentTermsEnum extends TermsEnum {
* computing aggregate statistics. */
public Stats computeBlockStats() throws IOException {
- // TODO: add total auto-prefix term count
-
Stats stats = new Stats(fr.parent.segment, fr.fieldInfo.name);
if (fr.index != null) {
stats.indexNumBytes = fr.index.ramBytesUsed();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java
index f7995a383f6..32f21424adf 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java
@@ -42,8 +42,6 @@ public class Stats {
/** Total number of bytes (sum of term lengths) across all terms in the field. */
public long totalTermBytes;
- // TODO: add total auto-prefix term count
-
/** The number of normal (non-floor) blocks in the terms file. */
public int nonFloorBlockCount;
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index 6ccb6ea1754..aa017237695 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -25,10 +25,8 @@ import java.nio.file.Paths;
import java.text.NumberFormat;
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.Deque;
import java.util.HashMap;
import java.util.Iterator;
-import java.util.LinkedList;
import java.util.List;
import java.util.Locale;
import java.util.Map;
@@ -1117,73 +1115,6 @@ public final class CheckIndex implements Closeable {
return intersectTermCount != normalTermCount;
}
- /** Make an effort to visit "fake" (e.g. auto-prefix) terms. We do this by running term range intersections across an initially wide
- * interval of terms, at different boundaries, and then gradually decrease the interval. This is not guaranteed to hit all non-real
- * terms (doing that in general is non-trivial), but it should hit many of them, and validate their postings against the postings for the
- * real terms. */
- private static void checkTermRanges(String field, int maxDoc, Terms terms, long numTerms) throws IOException {
-
- // We'll target this many terms in our interval for the current level:
- double currentInterval = numTerms;
-
- FixedBitSet normalDocs = new FixedBitSet(maxDoc);
- FixedBitSet intersectDocs = new FixedBitSet(maxDoc);
-
- //System.out.println("CI.checkTermRanges field=" + field + " numTerms=" + numTerms);
-
- while (currentInterval >= 10.0) {
- //System.out.println(" cycle interval=" + currentInterval);
-
- // We iterate this terms enum to locate min/max term for each sliding/overlapping interval we test at the current level:
- TermsEnum termsEnum = terms.iterator();
-
- long termCount = 0;
-
- Deque termBounds = new LinkedList<>();
-
- long lastTermAdded = Long.MIN_VALUE;
-
- BytesRefBuilder lastTerm = null;
-
- while (true) {
- BytesRef term = termsEnum.next();
- if (term == null) {
- break;
- }
- //System.out.println(" top: term=" + term.utf8ToString());
- if (termCount >= lastTermAdded + currentInterval/4) {
- termBounds.add(BytesRef.deepCopyOf(term));
- lastTermAdded = termCount;
- if (termBounds.size() == 5) {
- BytesRef minTerm = termBounds.removeFirst();
- BytesRef maxTerm = termBounds.getLast();
- checkSingleTermRange(field, maxDoc, terms, minTerm, maxTerm, normalDocs, intersectDocs);
- }
- }
- termCount++;
-
- if (lastTerm == null) {
- lastTerm = new BytesRefBuilder();
- lastTerm.copyBytes(term);
- } else {
- if (lastTerm.get().compareTo(term) >= 0) {
- throw new RuntimeException("terms out of order: lastTerm=" + lastTerm.get() + " term=" + term);
- }
- lastTerm.copyBytes(term);
- }
- }
- //System.out.println(" count=" + termCount);
-
- if (lastTerm != null && termBounds.isEmpty() == false) {
- BytesRef minTerm = termBounds.removeFirst();
- BytesRef maxTerm = lastTerm.get();
- checkSingleTermRange(field, maxDoc, terms, minTerm, maxTerm, normalDocs, intersectDocs);
- }
-
- currentInterval *= .75;
- }
- }
-
/**
* checks Fields api is consistent with itself.
* searcher is optional, to verify with queries. Can be null.
@@ -1703,12 +1634,6 @@ public final class CheckIndex implements Closeable {
long fieldTermCount = (status.delTermCount+status.termCount)-termCountStart;
- // LUCENE-5879: this is just too slow for now:
- if (false && hasFreqs == false) {
- // For DOCS_ONLY fields we recursively test term ranges:
- checkTermRanges(field, maxDoc, fieldTerms, fieldTermCount);
- }
-
final Object stats = fieldTerms.getStats();
assert stats != null;
if (status.blockTreeStats == null) {
diff --git a/lucene/core/src/java/org/apache/lucene/index/Terms.java b/lucene/core/src/java/org/apache/lucene/index/Terms.java
index dca8a276bbf..dabb8f7cdeb 100644
--- a/lucene/core/src/java/org/apache/lucene/index/Terms.java
+++ b/lucene/core/src/java/org/apache/lucene/index/Terms.java
@@ -19,7 +19,6 @@ package org.apache.lucene.index;
import java.io.IOException;
-import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.automaton.CompiledAutomaton;
@@ -55,12 +54,7 @@ public abstract class Terms {
* {@link CompiledAutomaton#getTermsEnum} instead.
*
* NOTE: the returned TermsEnum cannot seek
.
- *
- * NOTE: the terms dictionary is free to
- * return arbitrary terms as long as the resulted visited
- * docs is the same. E.g., {@link BlockTreeTermsWriter}
- * creates auto-prefix terms during indexing to reduce the
- * number of terms visited. */
+ */
public TermsEnum intersect(CompiledAutomaton compiled, final BytesRef startTerm) throws IOException {
// TODO: could we factor out a common interface b/w