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 2018/08/30 14:45:27 UTC

[1/4] lucene-solr:master: LUCENE-8465: Remove more references to auto-prefix terms.

Repository: lucene-solr
Updated Branches:
  refs/heads/master 4096decd8 -> 81eeae6db


LUCENE-8465: Remove more references to 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/81eeae6d
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/81eeae6d
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/81eeae6d

Branch: refs/heads/master
Commit: 81eeae6db28dcc921d98b562fc75e6c5aa530799
Parents: ba83c5a
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Aug 30 12:07:26 2018 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Aug 30 16:44:56 2018 +0200

----------------------------------------------------------------------
 .../codecs/blocktree/BlockTreeTermsReader.java  |  10 -
 .../lucene/codecs/blocktree/FieldReader.java    |   3 +-
 .../codecs/blocktree/IntersectTermsEnum.java    | 220 +------------------
 .../blocktree/IntersectTermsEnumFrame.java      |  15 --
 .../codecs/blocktree/SegmentTermsEnum.java      |   5 +-
 .../apache/lucene/codecs/blocktree/Stats.java   |   2 -
 .../org/apache/lucene/index/CheckIndex.java     |  75 -------
 .../src/java/org/apache/lucene/index/Terms.java |   8 +-
 8 files changed, 7 insertions(+), 331 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81eeae6d/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 0ef2129..b0091fd 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.</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),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81eeae6d/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
----------------------------------------------------------------------
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 4ee3826..46aee6e 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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81eeae6d/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
----------------------------------------------------------------------
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 bbd7e7b..934b5f6 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<suffix;i++) {
-            if (term.bytes[prefix+i] != currentFrame.suffixBytes[currentFrame.startBytePos+i]) {
-              break scanPrefix;
-            }
-          }
-        }
-      }
-    } else {
-      // Floor'd auto-prefix term; in this case we must skip all
-      // terms e.g. matching foo[a-m]*.  We are currently "on" fooa,
-      // which the automaton accepted (fooa* through foom*), and
-      // floorSuffixLeadEnd is m, so we must now scan to foon:
-      int prefix = currentFrame.prefix;
-      int suffix = currentFrame.suffix;
-
-      if (currentFrame.floorSuffixLeadStart == -1) {
-        suffix++;
-      }
-
-      if (suffix == 0) {
-
-        // This means current frame is fooa*, so we have to first
-        // pop the current frame, then scan in parent frame:
-        if (currentFrame.ord == 0) {
-          throw NoMoreTermsException.INSTANCE;
-        }
-        currentFrame = stack[currentFrame.ord-1];
-        currentTransition = currentFrame.transition;
-
-        // Current (parent) frame is now foo*, so now we just scan
-        // until the lead suffix byte is > 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-1;i++) {
-          if (term.bytes[prefix+i] != currentFrame.suffixBytes[currentFrame.startBytePos+i]) {
-            break scanFloor;
-          }
-        }
-        if (currentFrame.suffix >= 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<count;i++) {
-      automaton.getNextTransition(scratchTransition);
-      if (start >= scratchTransition.min && end <= scratchTransition.max && scratchTransition.dest == sinkState) {
-        return true;
-      }
-    }
-
-    return false;
-  }
-
   // for debugging
   @SuppressWarnings("unused")
   static String brToString(BytesRef b) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81eeae6d/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
----------------------------------------------------------------------
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 236e77a..b1cfa7c 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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81eeae6d/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
----------------------------------------------------------------------
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 327c181..8e01275 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();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81eeae6d/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java
----------------------------------------------------------------------
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 f7995a3..32f2142 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;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81eeae6d/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
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 6ccb6ea..aa01723 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<BytesRef> 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) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81eeae6d/lucene/core/src/java/org/apache/lucene/index/Terms.java
----------------------------------------------------------------------
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 dca8a27..dabb8f7 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.
    *
    *  <p><b>NOTE</b>: the returned TermsEnum cannot seek</p>.
-   *
-   *  <p><b>NOTE</b>: 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


[3/4] lucene-solr:master: LUCENE-8432: TopFieldComparator stops calling the comparator when only counting hits.

Posted by jp...@apache.org.
LUCENE-8432: TopFieldComparator stops calling the comparator when only counting hits.


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

Branch: refs/heads/master
Commit: ba83c5a26a9e789617bf8c4a0113fe62f9f56f66
Parents: a30eeae
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Aug 30 12:00:21 2018 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Aug 30 16:44:56 2018 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 ++
 .../apache/lucene/search/TopFieldCollector.java | 41 ++++++++++++--------
 2 files changed, 29 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ba83c5a2/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index a9f93b9..5120f28 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -274,6 +274,10 @@ Improvements
 
 * LUCENE-8460: Better argument validation in StoredField. (Namgyu Kim)
 
+* LUCENE-8432: TopFieldComparator stops comparing documents if the index is
+  sorted, even if hits still need to be visited to compute the hit count.
+  (Nikolay Khitrin)
+
 Other:
 
 * LUCENE-8366: Upgrade to ICU 62.1. Emoji handling now uses Unicode 11's

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ba83c5a2/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
index 8f0e059..90c4555 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
@@ -107,21 +107,25 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
 
       return new MultiComparatorLeafCollector(comparators, reverseMul) {
 
+        boolean collectedAllCompetitiveHits = false;
+
         @Override
         public void collect(int doc) throws IOException {
           ++totalHits;
           if (queueFull) {
-            if (reverseMul * comparator.compareBottom(doc) <= 0) {
+            if (collectedAllCompetitiveHits || reverseMul * comparator.compareBottom(doc) <= 0) {
               // since docs are visited in doc Id order, if compare is 0, it means
               // this document is largest than anything else in the queue, and
               // therefore not competitive.
-              if (canEarlyTerminate && totalHits >= totalHitsThreshold) {
-                totalHitsRelation = Relation.GREATER_THAN_OR_EQUAL_TO;
-                throw new CollectionTerminatedException();
-              } else {
-                // just move to the next doc
-                return;
+              if (canEarlyTerminate) {
+                if (totalHits >= totalHitsThreshold) {
+                  totalHitsRelation = Relation.GREATER_THAN_OR_EQUAL_TO;
+                  throw new CollectionTerminatedException();
+                } else {
+                  collectedAllCompetitiveHits = true;
+                }
               }
+              return;
             }
 
             // This hit is competitive - replace bottom element in queue & adjustTop
@@ -183,6 +187,8 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
           canEarlyTerminate(sort, indexSort);
       return new MultiComparatorLeafCollector(queue.getComparators(context), queue.getReverseMul()) {
 
+        boolean collectedAllCompetitiveHits = false;
+
         @Override
         public void collect(int doc) throws IOException {
           //System.out.println("  collect doc=" + doc);
@@ -192,16 +198,19 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
           if (queueFull) {
             // Fastmatch: return if this hit is no better than
             // the worst hit currently in the queue:
-            final int cmp = reverseMul * comparator.compareBottom(doc);
-            if (cmp <= 0) {
-              // not competitive since documents are visited in doc id order
-              if (canEarlyTerminate && totalHits >= totalHitsThreshold) {
-                totalHitsRelation = Relation.GREATER_THAN_OR_EQUAL_TO;
-                throw new CollectionTerminatedException();
-              } else {
-                // just move to the next doc
-                return;
+            if (collectedAllCompetitiveHits || reverseMul * comparator.compareBottom(doc) <= 0) {
+              // since docs are visited in doc Id order, if compare is 0, it means
+              // this document is largest than anything else in the queue, and
+              // therefore not competitive.
+              if (canEarlyTerminate) {
+                if (totalHits >= totalHitsThreshold) {
+                  totalHitsRelation = Relation.GREATER_THAN_OR_EQUAL_TO;
+                  throw new CollectionTerminatedException();
+                } else {
+                  collectedAllCompetitiveHits = true;
+                }
               }
+              return;
             }
           }
 


[4/4] lucene-solr:master: LUCENE-765: Improved oal.index javadocs.

Posted by jp...@apache.org.
LUCENE-765: Improved oal.index javadocs.


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

Branch: refs/heads/master
Commit: e2fc49cce21f4afb2e49ed4d3858ef8cc7dbd99d
Parents: 4096dec
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Aug 30 11:54:37 2018 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Aug 30 16:44:56 2018 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   2 +
 .../org/apache/lucene/index/package-info.java   | 276 +++++++++++++------
 2 files changed, 188 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2fc49cc/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 243984e..409419f 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -290,6 +290,8 @@ Other:
 
 * LUCENE-8456: Upgrade Apache Commons Compress to v1.18 (Steve Rowe)
 
+* LUCENE-765: Improved org.apache.lucene.index javadocs. (Mike Sokolov)
+
 ======================= Lucene 7.4.1 =======================
 
 Bug Fixes:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2fc49cc/lucene/core/src/java/org/apache/lucene/index/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/package-info.java b/lucene/core/src/java/org/apache/lucene/index/package-info.java
index d7d337c..55ee56c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/index/package-info.java
@@ -6,7 +6,7 @@
  * (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
+ *   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,
@@ -17,34 +17,130 @@
 
 /**
  * Code to maintain and access indices.
- * <!-- TODO: add IndexWriter, IndexWriterConfig, DocValues, etc etc -->
  * <h2>Table Of Contents</h2>
- *     <ol>
- *         <li><a href="#postings">Postings APIs</a>
- *             <ul>
- *                 <li><a href="#fields">Fields</a></li>
- *                 <li><a href="#terms">Terms</a></li>
- *                 <li><a href="#documents">Documents</a></li>
- *                 <li><a href="#positions">Positions</a></li>
- *             </ul>
- *         </li>
- *         <li><a href="#stats">Index Statistics</a>
- *             <ul>
- *                 <li><a href="#termstats">Term-level</a></li>
- *                 <li><a href="#fieldstats">Field-level</a></li>
- *                 <li><a href="#segmentstats">Segment-level</a></li>
- *                 <li><a href="#documentstats">Document-level</a></li>
- *             </ul>
- *         </li>
- *     </ol>
+ *   <ol>
+ *     <li><a href="#index">Index APIs</a>
+ *       <ul>
+ *         <li><a href="#writer">IndexWriter</a></li>
+ *         <li><a href="#reader">IndexReader</a></li>
+ *         <li><a href="#segments">Segments and docids</a></li>
+ *       </ul>
+ *     </li>
+ *     <li><a href="#field_types">Field types</a>
+ *       <ul>
+ *         <li><a href="#postings-desc">Postings</a></li>
+ *         <li><a href="#stored-fields">Stored Fields</a></li>
+ *         <li><a href="#docvalues">DocValues</a></li>
+ *         <li><a href="#points">Points</a></li>
+ *       </ul>
+ *     </li>
+ *     <li><a href="#postings">Postings APIs</a>
+ *       <ul>
+ *         <li><a href="#fields">Fields</a></li>
+ *         <li><a href="#terms">Terms</a></li>
+ *         <li><a href="#documents">Documents</a></li>
+ *         <li><a href="#positions">Positions</a></li>
+ *       </ul>
+ *     </li>
+ *     <li><a href="#stats">Index Statistics</a>
+ *       <ul>
+ *         <li><a href="#termstats">Term-level</a></li>
+ *         <li><a href="#fieldstats">Field-level</a></li>
+ *         <li><a href="#segmentstats">Segment-level</a></li>
+ *         <li><a href="#documentstats">Document-level</a></li>
+ *       </ul>
+ *     </li>
+ *   </ol>
+ * <a name="index"></a>
+ * <h2>Index APIs</h2>
+
+ * <a name="writer"></a>
+ * <h3>IndexWriter</h3>
+
+ * <p>{@link org.apache.lucene.index.IndexWriter} is used to create an index, and to add, update and
+ * delete documents. The IndexWriter class is thread safe, and enforces a single instance per
+ * index. Creating an IndexWriter creates a new index or opens an existing index for writing, in a
+ * {@link org.apache.lucene.store.Directory}, depending on the configuration in {@link
+ * org.apache.lucene.index.IndexWriterConfig}. A Directory is an abstraction that typically
+ * represents a local file-system directory (see various implementations of {@link
+ * org.apache.lucene.store.FSDirectory}), but it may also stand for some other storage, such as
+ * RAM.</p>
+
+ * <a name="reader"></a>
+ * <h3>IndexReader</h3>
+
+ * <p>{@link org.apache.lucene.index.IndexReader} is used to read data from the index, and supports
+ * searching. Many thread-safe readers may be {@link org.apache.lucene.index.DirectoryReader#open}
+ * concurrently with a single (or no) writer. Each reader maintains a consistent "point in time"
+ * view of an index and must be explicitly refreshed (see {@link
+ * org.apache.lucene.index.DirectoryReader#openIfChanged}) in order to incorporate writes that may
+ * occur after it is opened.</p>
+
+ * <a name="segments"></a>
+ * <h3>Segments and docids</h3>
+
+ * <p>Lucene's index is composed of segments, each of which contains a subset of all the documents
+ * in the index, and is a complete searchable index in itself, over that subset. As documents are
+ * written to the index, new segments are created and flushed to directory storage. Segments are
+ * immutable; updates and deletions may only create new segments and do not modify existing
+ * ones. Over time, the writer merges groups of smaller segments into single larger ones in order to
+ * maintain an index that is efficient to search, and to reclaim dead space left behind by deleted
+ * (and updated) documents.</p>
+
+ * <p>Each document is identified by a 32-bit number, its "docid," and is composed of a collection
+ * of Field values of diverse types (postings, stored fields, doc values, and points). Docids come
+ * in two flavors: global and per-segment. A document's global docid is just the sum of its
+ * per-segment docid and that segment's base docid offset. External, high-level APIs only handle
+ * global docids, but internal APIs that reference a {@link org.apache.lucene.index.LeafReader},
+ * which is a reader for a single segment, deal in per-segment docids.</p>
+ *
+ * <p>Docids are assigned sequentially within each segment (starting at 0). Thus the number of
+ * documents in a segment is the same as its maximum docid; some may be deleted, but their docids
+ * are retained until the segment is merged. When segments merge, their documents are assigned new
+ * sequential docids. Accordingly, docid values must always be treated as internal implementation,
+ * not exposed as part of an application, nor stored or referenced outside of Lucene's internal
+ * APIs.</p>
+
+ * <a name="field_types"></a>
+ * <h2>Field Types</h2>
+ *
+ * <a name="postings-desc"></a>
+ *
+ * <p>Lucene supports a variety of different document field data structures. Lucene's core, the
+ * inverted index, is comprised of "postings." The postings, with their term dictionary, can be
+ * thought of as a map that provides efficient lookup given a {@link org.apache.lucene.index.Term}
+ * (roughly, a word or token), to (the ordered list of) {@link org.apache.lucene.document.Document}s
+ * containing that Term.  Postings do not provide any way of retrieving terms given a document,
+ * short of scanning the entire index.</p>
+ *
+ * <a name="stored-fields"></a>
+ * <p>Stored fields are essentially the opposite of postings, providing efficient retrieval of field
+ * values given a docid.  All stored field values for a document are stored together in a
+ * block. Different types of stored field provide high-level datatypes such as strings and numbers
+ * on top of the underlying bytes. Stored field values are usually retrieved by the searcher using
+ * an implementation of {@link org.apache.lucene.index.StoredFieldVisitor}.</p>
+
+ * <a name="docvalues"></a>
+ * <p>{@link org.apache.lucene.index.DocValues} fields are what are sometimes referred to as
+ * columnar, or column-stride fields, by analogy to relational database terminology, in which
+ * documents are considered as rows, and fields, columns. DocValues fields store values per-field: a
+ * value for every document is held in a single data structure, providing for rapid, sequential
+ * lookup of a field-value given a docid. These fields are used for efficient value-based sorting,
+ * and for faceting, but they are not useful for filtering.</p>
+
+ * <a name="points"></a>
+ * <p>{@link org.apache.lucene.index.PointValues} represent numeric values using a kd-tree data
+ * structure. Efficient 1- and higher dimensional implementations make these the choice for numeric
+ * range and interval queries, and geo-spatial queries.</p>
+
  * <a name="postings"></a>
  * <h2>Postings APIs</h2>
  * <a name="fields"></a>
  * <h3>
- *     Fields
+ *   Fields
  * </h3>
  * <p>
- * {@link org.apache.lucene.index.Fields} is the initial entry point into the 
+ * {@link org.apache.lucene.index.Fields} is the initial entry point into the
  * postings APIs, this can be obtained in several ways:
  * <pre class="prettyprint">
  * // access indexed fields for an index segment
@@ -63,7 +159,7 @@
  * </pre>
  * <a name="terms"></a>
  * <h3>
- *     Terms
+ *   Terms
  * </h3>
  * <p>
  * {@link org.apache.lucene.index.Terms} represents the collection of terms
@@ -128,10 +224,10 @@
  *   System.out.println(docid);
  *   int freq = postings.freq();
  *   for (int i = 0; i &lt; freq; i++) {
- *      System.out.println(postings.nextPosition());
- *      System.out.println(postings.startOffset());
- *      System.out.println(postings.endOffset());
- *      System.out.println(postings.getPayload());
+ *    System.out.println(postings.nextPosition());
+ *    System.out.println(postings.startOffset());
+ *    System.out.println(postings.endOffset());
+ *    System.out.println(postings.getPayload());
  *   }
  * }
  * </pre>
@@ -139,7 +235,7 @@
  * <h2>Index Statistics</h2>
  * <a name="termstats"></a>
  * <h3>
- *     Term statistics
+ *   Term statistics
  * </h3>
  *     <ul>
  *        <li>{@link org.apache.lucene.index.TermsEnum#docFreq}: Returns the number of 
@@ -153,80 +249,80 @@
  *     </ul>
  * <a name="fieldstats"></a>
  * <h3>
- *     Field statistics
+ *   Field statistics
  * </h3>
- *     <ul>
- *        <li>{@link org.apache.lucene.index.Terms#size}: Returns the number of 
- *            unique terms in the field. This statistic may be unavailable 
- *            (returns <code>-1</code>) for some Terms implementations such as
- *            {@link org.apache.lucene.index.MultiTerms}, where it cannot be efficiently
- *            computed.  Note that this count also includes terms that appear only
- *            in deleted documents: when segments are merged such terms are also merged
- *            away and the statistic is then updated.
- *        <li>{@link org.apache.lucene.index.Terms#getDocCount}: Returns the number of
- *            documents that contain at least one occurrence of any term for this field.
- *            This can be thought of as a Field-level docFreq(). Like docFreq() it will
- *            also count deleted documents.
- *        <li>{@link org.apache.lucene.index.Terms#getSumDocFreq}: Returns the number of
- *            postings (term-document mappings in the inverted index) for the field. This
- *            can be thought of as the sum of {@link org.apache.lucene.index.TermsEnum#docFreq}
- *            across all terms in the field, and like docFreq() it will also count postings
- *            that appear in deleted documents.
- *        <li>{@link org.apache.lucene.index.Terms#getSumTotalTermFreq}: Returns the number
- *            of tokens for the field. This can be thought of as the sum of 
- *            {@link org.apache.lucene.index.TermsEnum#totalTermFreq} across all terms in the
- *            field, and like totalTermFreq() it will also count occurrences that appear in
- *            deleted documents.
- *     </ul>
+ *   <ul>
+ *    <li>{@link org.apache.lucene.index.Terms#size}: Returns the number of
+ *      unique terms in the field. This statistic may be unavailable
+ *      (returns <code>-1</code>) for some Terms implementations such as
+ *      {@link org.apache.lucene.index.MultiTerms}, where it cannot be efficiently
+ *      computed.  Note that this count also includes terms that appear only
+ *      in deleted documents: when segments are merged such terms are also merged
+ *      away and the statistic is then updated.
+ *    <li>{@link org.apache.lucene.index.Terms#getDocCount}: Returns the number of
+ *      documents that contain at least one occurrence of any term for this field.
+ *      This can be thought of as a Field-level docFreq(). Like docFreq() it will
+ *      also count deleted documents.
+ *    <li>{@link org.apache.lucene.index.Terms#getSumDocFreq}: Returns the number of
+ *      postings (term-document mappings in the inverted index) for the field. This
+ *      can be thought of as the sum of {@link org.apache.lucene.index.TermsEnum#docFreq}
+ *      across all terms in the field, and like docFreq() it will also count postings
+ *      that appear in deleted documents.
+ *    <li>{@link org.apache.lucene.index.Terms#getSumTotalTermFreq}: Returns the number
+ *      of tokens for the field. This can be thought of as the sum of
+ *      {@link org.apache.lucene.index.TermsEnum#totalTermFreq} across all terms in the
+ *      field, and like totalTermFreq() it will also count occurrences that appear in
+ *      deleted documents.
+ *   </ul>
  * <a name="segmentstats"></a>
  * <h3>
- *     Segment statistics
+ *   Segment statistics
  * </h3>
- *     <ul>
- *        <li>{@link org.apache.lucene.index.IndexReader#maxDoc}: Returns the number of 
- *            documents (including deleted documents) in the index. 
- *        <li>{@link org.apache.lucene.index.IndexReader#numDocs}: Returns the number 
- *            of live documents (excluding deleted documents) in the index.
- *        <li>{@link org.apache.lucene.index.IndexReader#numDeletedDocs}: Returns the
- *            number of deleted documents in the index.
- *        <li>{@link org.apache.lucene.index.Fields#size}: Returns the number of indexed
- *            fields.
- *     </ul>
+ *   <ul>
+ *    <li>{@link org.apache.lucene.index.IndexReader#maxDoc}: Returns the number of
+ *      documents (including deleted documents) in the index.
+ *    <li>{@link org.apache.lucene.index.IndexReader#numDocs}: Returns the number
+ *      of live documents (excluding deleted documents) in the index.
+ *    <li>{@link org.apache.lucene.index.IndexReader#numDeletedDocs}: Returns the
+ *      number of deleted documents in the index.
+ *    <li>{@link org.apache.lucene.index.Fields#size}: Returns the number of indexed
+ *      fields.
+ *   </ul>
  * <a name="documentstats"></a>
  * <h3>
- *     Document statistics
+ *   Document statistics
  * </h3>
  * <p>
  * Document statistics are available during the indexing process for an indexed field: typically
  * a {@link org.apache.lucene.search.similarities.Similarity} implementation will store some
  * of these values (possibly in a lossy way), into the normalization value for the document in
  * its {@link org.apache.lucene.search.similarities.Similarity#computeNorm} method.
- *     <ul>
- *        <li>{@link org.apache.lucene.index.FieldInvertState#getLength}: Returns the number of 
- *            tokens for this field in the document. Note that this is just the number
- *            of times that {@link org.apache.lucene.analysis.TokenStream#incrementToken} returned
- *            true, and is unrelated to the values in 
- *            {@link org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute}.
- *        <li>{@link org.apache.lucene.index.FieldInvertState#getNumOverlap}: Returns the number
- *            of tokens for this field in the document that had a position increment of zero. This
- *            can be used to compute a document length that discounts artificial tokens
- *            such as synonyms.
- *        <li>{@link org.apache.lucene.index.FieldInvertState#getPosition}: Returns the accumulated
- *            position value for this field in the document: computed from the values of
- *            {@link org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute} and including
- *            {@link org.apache.lucene.analysis.Analyzer#getPositionIncrementGap}s across multivalued
- *            fields.
- *        <li>{@link org.apache.lucene.index.FieldInvertState#getOffset}: Returns the total
- *            character offset value for this field in the document: computed from the values of
- *            {@link org.apache.lucene.analysis.tokenattributes.OffsetAttribute} returned by 
- *            {@link org.apache.lucene.analysis.TokenStream#end}, and including
- *            {@link org.apache.lucene.analysis.Analyzer#getOffsetGap}s across multivalued
- *            fields.
- *        <li>{@link org.apache.lucene.index.FieldInvertState#getUniqueTermCount}: Returns the number
- *            of unique terms encountered for this field in the document.
- *        <li>{@link org.apache.lucene.index.FieldInvertState#getMaxTermFrequency}: Returns the maximum
- *            frequency across all unique terms encountered for this field in the document. 
- *     </ul>
+ *   <ul>
+ *    <li>{@link org.apache.lucene.index.FieldInvertState#getLength}: Returns the number of
+ *      tokens for this field in the document. Note that this is just the number
+ *      of times that {@link org.apache.lucene.analysis.TokenStream#incrementToken} returned
+ *      true, and is unrelated to the values in
+ *      {@link org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute}.
+ *    <li>{@link org.apache.lucene.index.FieldInvertState#getNumOverlap}: Returns the number
+ *      of tokens for this field in the document that had a position increment of zero. This
+ *      can be used to compute a document length that discounts artificial tokens
+ *      such as synonyms.
+ *    <li>{@link org.apache.lucene.index.FieldInvertState#getPosition}: Returns the accumulated
+ *      position value for this field in the document: computed from the values of
+ *      {@link org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute} and including
+ *      {@link org.apache.lucene.analysis.Analyzer#getPositionIncrementGap}s across multivalued
+ *      fields.
+ *    <li>{@link org.apache.lucene.index.FieldInvertState#getOffset}: Returns the total
+ *      character offset value for this field in the document: computed from the values of
+ *      {@link org.apache.lucene.analysis.tokenattributes.OffsetAttribute} returned by
+ *      {@link org.apache.lucene.analysis.TokenStream#end}, and including
+ *      {@link org.apache.lucene.analysis.Analyzer#getOffsetGap}s across multivalued
+ *      fields.
+ *    <li>{@link org.apache.lucene.index.FieldInvertState#getUniqueTermCount}: Returns the number
+ *      of unique terms encountered for this field in the document.
+ *    <li>{@link org.apache.lucene.index.FieldInvertState#getMaxTermFrequency}: Returns the maximum
+ *      frequency across all unique terms encountered for this field in the document.
+ *   </ul>
  * <p>
  * Additional user-supplied statistics can be added to the document as DocValues fields and
  * accessed via {@link org.apache.lucene.index.LeafReader#getNumericDocValues}.


[2/4] lucene-solr:master: LUCENE-8460: Better argument validation in StoredField

Posted by jp...@apache.org.
LUCENE-8460: Better argument validation in StoredField

Signed-off-by: Namgyu Kim <kn...@gmail.com>
Signed-off-by: Adrien Grand <jp...@gmail.com>


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

Branch: refs/heads/master
Commit: a30eeae7956c8c59037ca1c08e7f69474da10e7a
Parents: e2fc49c
Author: Namgyu Kim <kn...@gmail.com>
Authored: Wed Aug 29 00:46:49 2018 +0900
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Aug 30 16:44:56 2018 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  2 ++
 .../java/org/apache/lucene/document/Field.java  | 30 +++++++++++---------
 .../org/apache/lucene/document/StoredField.java | 23 +++++++++------
 3 files changed, 33 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a30eeae7/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 409419f..a9f93b9 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -272,6 +272,8 @@ Improvements
 * LUCENE-8446: The UnifiedHighlighter's DefaultPassageFormatter now treats overlapping matches in
   the passage as merged (as if one larger match).  (David Smiley)
 
+* LUCENE-8460: Better argument validation in StoredField. (Namgyu Kim)
+
 Other:
 
 * LUCENE-8366: Upgrade to ICU 62.1. Emoji handling now uses Unicode 11's

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a30eeae7/lucene/core/src/java/org/apache/lucene/document/Field.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/Field.java b/lucene/core/src/java/org/apache/lucene/document/Field.java
index cbb559a..467fec7 100644
--- a/lucene/core/src/java/org/apache/lucene/document/Field.java
+++ b/lucene/core/src/java/org/apache/lucene/document/Field.java
@@ -169,9 +169,8 @@ public class Field implements IndexableField {
    * @param name field name
    * @param value byte array pointing to binary content (not copied)
    * @param type field type
-   * @throws IllegalArgumentException if the field name is null,
-   *         or the field's type is indexed()
-   * @throws NullPointerException if the type is null
+   * @throws IllegalArgumentException if the field name, value or type
+   *         is null, or the field's type is indexed().
    */
   public Field(String name, byte[] value, IndexableFieldType type) {
     this(name, value, 0, value.length, type);
@@ -187,12 +186,11 @@ public class Field implements IndexableField {
    * @param offset starting position of the byte array
    * @param length valid length of the byte array
    * @param type field type
-   * @throws IllegalArgumentException if the field name is null,
-   *         or the field's type is indexed()
-   * @throws NullPointerException if the type is null
+   * @throws IllegalArgumentException if the field name, value or type
+   *         is null, or the field's type is indexed().
    */
   public Field(String name, byte[] value, int offset, int length, IndexableFieldType type) {
-    this(name, new BytesRef(value, offset, length), type);
+    this(name, value != null ? new BytesRef(value, offset, length) : null, type);
   }
 
   /**
@@ -203,9 +201,8 @@ public class Field implements IndexableField {
    * @param name field name
    * @param bytes BytesRef pointing to binary content (not copied)
    * @param type field type
-   * @throws IllegalArgumentException if the field name is null,
-   *         or the field's type is indexed()
-   * @throws NullPointerException if the type is null
+   * @throws IllegalArgumentException if the field name, bytes or type
+   *         is null, or the field's type is indexed().
    */
   public Field(String name, BytesRef bytes, IndexableFieldType type) {
     if (name == null) {
@@ -214,9 +211,12 @@ public class Field implements IndexableField {
     if (bytes == null) {
       throw new IllegalArgumentException("bytes must not be null");
     }
+    if (type == null) {
+      throw new IllegalArgumentException("type must not be null");
+    }
+    this.name = name;
     this.fieldsData = bytes;
     this.type = type;
-    this.name = name;
   }
 
   // TODO: allow direct construction of int, long, float, double value too..?
@@ -226,10 +226,9 @@ public class Field implements IndexableField {
    * @param name field name
    * @param value string value
    * @param type field type
-   * @throws IllegalArgumentException if either the name or value
+   * @throws IllegalArgumentException if either the name, value or type
    *         is null, or if the field's type is neither indexed() nor stored(), 
    *         or if indexed() is false but storeTermVectors() is true.
-   * @throws NullPointerException if the type is null
    */
   public Field(String name, String value, IndexableFieldType type) {
     if (name == null) {
@@ -238,13 +237,16 @@ public class Field implements IndexableField {
     if (value == null) {
       throw new IllegalArgumentException("value must not be null");
     }
+    if (type == null) {
+      throw new IllegalArgumentException("type must not be null");
+    }
     if (!type.stored() && type.indexOptions() == IndexOptions.NONE) {
       throw new IllegalArgumentException("it doesn't make sense to have a field that "
         + "is neither indexed nor stored");
     }
-    this.type = type;
     this.name = name;
     this.fieldsData = value;
+    this.type = type;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a30eeae7/lucene/core/src/java/org/apache/lucene/document/StoredField.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/StoredField.java b/lucene/core/src/java/org/apache/lucene/document/StoredField.java
index 12b529c..7dc5a99 100644
--- a/lucene/core/src/java/org/apache/lucene/document/StoredField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/StoredField.java
@@ -40,12 +40,13 @@ public class StoredField extends Field {
    * FieldType}.
    * @param name field name
    * @param type custom {@link FieldType} for this field
-   * @throws IllegalArgumentException if the field name is null.
+   * @throws IllegalArgumentException if the field name or type
+   *         is null.
    */
   protected StoredField(String name, FieldType type) {
     super(name, type);
   }
-  
+
   /**
    * Expert: allows you to customize the {@link
    * FieldType}.
@@ -54,7 +55,8 @@ public class StoredField extends Field {
    * @param name field name
    * @param bytes byte array pointing to binary content (not copied)
    * @param type custom {@link FieldType} for this field
-   * @throws IllegalArgumentException if the field name is null.
+   * @throws IllegalArgumentException if the field name, value or type
+   *         is null.
    */
   public StoredField(String name, BytesRef bytes, FieldType type) {
     super(name, bytes, type);
@@ -66,7 +68,8 @@ public class StoredField extends Field {
    * not to change it until you're done with this field.
    * @param name field name
    * @param value byte array pointing to binary content (not copied)
-   * @throws IllegalArgumentException if the field name is null.
+   * @throws IllegalArgumentException if the field name or value
+   *         is null.
    */
   public StoredField(String name, byte[] value) {
     super(name, value, TYPE);
@@ -80,7 +83,8 @@ public class StoredField extends Field {
    * @param value byte array pointing to binary content (not copied)
    * @param offset starting position of the byte array
    * @param length valid length of the byte array
-   * @throws IllegalArgumentException if the field name is null.
+   * @throws IllegalArgumentException if the field name or value
+   *         is null.
    */
   public StoredField(String name, byte[] value, int offset, int length) {
     super(name, value, offset, length, TYPE);
@@ -92,7 +96,8 @@ public class StoredField extends Field {
    * not to change it until you're done with this field.
    * @param name field name
    * @param value BytesRef pointing to binary content (not copied)
-   * @throws IllegalArgumentException if the field name is null.
+   * @throws IllegalArgumentException if the field name or value
+   *         is null.
    */
   public StoredField(String name, BytesRef value) {
     super(name, value, TYPE);
@@ -102,7 +107,8 @@ public class StoredField extends Field {
    * Create a stored-only field with the given string value.
    * @param name field name
    * @param value string value
-   * @throws IllegalArgumentException if the field name or value is null.
+   * @throws IllegalArgumentException if the field name or value
+   *         is null.
    */
   public StoredField(String name, String value) {
     super(name, value, TYPE);
@@ -114,7 +120,8 @@ public class StoredField extends Field {
    * @param name field name
    * @param value string value
    * @param type custom {@link FieldType} for this field
-   * @throws IllegalArgumentException if the field name or value is null.
+   * @throws IllegalArgumentException if the field name, value or type
+   *         is null.
    */
   public StoredField(String name, String value, FieldType type) {
     super(name, value, type);