You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2016/09/12 12:02:43 UTC

[1/2] lucene-solr:branch_6_2: LUCENE-7417: Highlighter WSTE didn't handle single-term MultiPhraseQuery. Also updated to Java 5 for-each in this method.

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6_2 8027eb980 -> c7b3e9ae3


LUCENE-7417: Highlighter WSTE didn't handle single-term MultiPhraseQuery.
Also updated to Java 5 for-each in this method.

(cherry picked from commit 3966f99)

(cherry picked from commit 514bb1b)


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

Branch: refs/heads/branch_6_2
Commit: cddeb9dc3c8322b4149b910f509a93be37f5c17b
Parents: 8027eb9
Author: David Smiley <ds...@apache.org>
Authored: Fri Sep 9 19:36:39 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Mon Sep 12 17:31:19 2016 +0530

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  6 ++++
 .../highlight/WeightedSpanTermExtractor.java    | 26 +++++++++--------
 .../search/highlight/HighlighterTest.java       | 30 +++++++++++++-------
 3 files changed, 39 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cddeb9dc/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index fdc6d03..e9ae13d 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -10,6 +10,12 @@ API Changes
 * LUCENE-7436: MinHashFilter's constructor, and some of its default
   settings, should be public.  (Doug Turnbull via Mike McCandless)
 
+Bug Fixes
+
+* LUCENE-7417: The standard Highlighter could throw an IllegalArgumentException when
+  trying to highlight a query containing a degenerate case of a MultiPhraseQuery with one
+  term.  (Thomas Kappler via David Smiley)
+
 ======================= Lucene 6.2.0 =======================
 
 API Changes

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cddeb9dc/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
index 7507bdd..644dad3 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
@@ -118,8 +118,7 @@ public class WeightedSpanTermExtractor {
       Term[] phraseQueryTerms = phraseQuery.getTerms();
       if (phraseQueryTerms.length == 1) {
         extractWeightedSpanTerms(terms, new SpanTermQuery(phraseQueryTerms[0]), boost);
-      }
-      else {
+      } else {
         SpanQuery[] clauses = new SpanQuery[phraseQueryTerms.length];
         for (int i = 0; i < phraseQueryTerms.length; i++) {
           clauses[i] = new SpanTermQuery(phraseQueryTerms[i]);
@@ -153,8 +152,8 @@ public class WeightedSpanTermExtractor {
       // this query is TermContext sensitive.
       extractWeightedTerms(terms, query, boost);
     } else if (query instanceof DisjunctionMaxQuery) {
-      for (Iterator<Query> iterator = ((DisjunctionMaxQuery) query).iterator(); iterator.hasNext();) {
-        extract(iterator.next(), boost, terms);
+      for (Query clause : ((DisjunctionMaxQuery) query)) {
+        extract(clause, boost, terms);
       }
     } else if (query instanceof ToParentBlockJoinQuery) {
       extract(((ToParentBlockJoinQuery) query).getChildQuery(), boost, terms);
@@ -184,16 +183,15 @@ public class WeightedSpanTermExtractor {
             disjuncts = (disjunctLists[positions[i]] = new ArrayList<>(termArray.length));
             ++distinctPositions;
           }
-          for (int j = 0; j < termArray.length; ++j) {
-            disjuncts.add(new SpanTermQuery(termArray[j]));
+          for (Term aTermArray : termArray) {
+            disjuncts.add(new SpanTermQuery(aTermArray));
           }
         }
 
         int positionGaps = 0;
         int position = 0;
         final SpanQuery[] clauses = new SpanQuery[distinctPositions];
-        for (int i = 0; i < disjunctLists.length; ++i) {
-          List<SpanQuery> disjuncts = disjunctLists[i];
+        for (List<SpanQuery> disjuncts : disjunctLists) {
           if (disjuncts != null) {
             clauses[position++] = new SpanOrQuery(disjuncts
                 .toArray(new SpanQuery[disjuncts.size()]));
@@ -202,11 +200,15 @@ public class WeightedSpanTermExtractor {
           }
         }
 
-        final int slop = mpq.getSlop();
-        final boolean inorder = (slop == 0);
+        if (clauses.length == 1) {
+          extractWeightedSpanTerms(terms, clauses[0], boost);
+        } else {
+          final int slop = mpq.getSlop();
+          final boolean inorder = (slop == 0);
 
-        SpanNearQuery sp = new SpanNearQuery(clauses, slop + positionGaps, inorder);
-        extractWeightedSpanTerms(terms, sp, boost);
+          SpanNearQuery sp = new SpanNearQuery(clauses, slop + positionGaps, inorder);
+          extractWeightedSpanTerms(terms, sp, boost);
+        }
       }
     } else if (query instanceof MatchAllDocsQuery) {
       //nothing

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cddeb9dc/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
index cf727d7..fc402ba 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
@@ -94,7 +94,6 @@ import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
-import org.junit.Test;
 import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
 
@@ -1580,30 +1579,39 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
     helper.start();
   }
 
-  @Test
   public void testHighlighterWithPhraseQuery() throws IOException, InvalidTokenOffsetsException {
+    final String fieldName = "substring";
+
+    final PhraseQuery query = new PhraseQuery(fieldName, new BytesRef[] { new BytesRef("uchu") });
+
+    assertHighlighting(query, new SimpleHTMLFormatter("<b>", "</b>"), "Buchung", "B<b>uchu</b>ng", fieldName);
+  }
+
+  public void testHighlighterWithMultiPhraseQuery() throws IOException, InvalidTokenOffsetsException {
+    final String fieldName = "substring";
+
+    final MultiPhraseQuery mpq = new MultiPhraseQuery.Builder()
+        .add(new Term(fieldName, "uchu")).build();
 
+    assertHighlighting(mpq, new SimpleHTMLFormatter("<b>", "</b>"), "Buchung", "B<b>uchu</b>ng", fieldName);
+  }
+
+  private void assertHighlighting(Query query, Formatter formatter, String text, String expected, String fieldName)
+      throws IOException, InvalidTokenOffsetsException {
     final Analyzer analyzer = new Analyzer() {
       @Override
       protected TokenStreamComponents createComponents(String fieldName) {
         return new TokenStreamComponents(new NGramTokenizer(4, 4));
       }
     };
-    final String fieldName = "substring";
-
-    final List<BytesRef> list = new ArrayList<>();
-    list.add(new BytesRef("uchu"));
-    final PhraseQuery query = new PhraseQuery(fieldName, list.toArray(new BytesRef[list.size()]));
 
     final QueryScorer fragmentScorer = new QueryScorer(query, fieldName);
-    final SimpleHTMLFormatter formatter = new SimpleHTMLFormatter("<b>", "</b>");
 
     final Highlighter highlighter = new Highlighter(formatter, fragmentScorer);
     highlighter.setTextFragmenter(new SimpleFragmenter(100));
-    final String fragment = highlighter.getBestFragment(analyzer, fieldName, "Buchung");
-
-    assertEquals("B<b>uchu</b>ng",fragment);
+    final String fragment = highlighter.getBestFragment(analyzer, fieldName, text);
 
+    assertEquals(expected, fragment);
   }
 
   public void testUnRewrittenQuery() throws Exception {


[2/2] lucene-solr:branch_6_2: LUCENE-7440: fix MultiLevelSkipListReader overflow

Posted by sh...@apache.org.
LUCENE-7440: fix MultiLevelSkipListReader overflow

(cherry picked from commit cf72eeb)


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

Branch: refs/heads/branch_6_2
Commit: c7b3e9ae3695a13dacb81312db0d470ada273808
Parents: cddeb9d
Author: yonik <yo...@apache.org>
Authored: Sun Sep 11 01:28:24 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Mon Sep 12 17:32:22 2016 +0530

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +
 .../lucene/codecs/MultiLevelSkipListReader.java |   9 +-
 .../org/apache/lucene/index/Test2BDocs.java     | 135 +++++++++++++++++++
 3 files changed, 145 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c7b3e9ae/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index e9ae13d..f62e470 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -16,6 +16,10 @@ Bug Fixes
   trying to highlight a query containing a degenerate case of a MultiPhraseQuery with one
   term.  (Thomas Kappler via David Smiley)
 
+* LUCENE-7440: Document id skipping (PostingsEnum.advance) could throw an
+  ArrayIndexOutOfBoundsException exception on large index segments (>1.8B docs)
+  with large skips. (yonik)
+
 ======================= Lucene 6.2.0 =======================
 
 API Changes

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c7b3e9ae/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListReader.java b/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListReader.java
index 72ffe9f..c937886 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListReader.java
@@ -63,7 +63,9 @@ public abstract class MultiLevelSkipListReader implements Closeable {
   /**  skipInterval of each level. */
   private int skipInterval[];
 
-  /** Number of docs skipped per level. */
+  /** Number of docs skipped per level.
+   * It's possible for some values to overflow a signed int, but this has been accounted for.
+   */
   private int[] numSkipped;
 
   /** Doc id of current skip entry per level. */
@@ -150,8 +152,9 @@ public abstract class MultiLevelSkipListReader implements Closeable {
     setLastSkipData(level);
       
     numSkipped[level] += skipInterval[level];
-      
-    if (numSkipped[level] > docCount) {
+
+    // numSkipped may overflow a signed int, so compare as unsigned.
+    if (Integer.compareUnsigned(numSkipped[level], docCount) > 0) {
       // this skip list is exhausted
       skipDoc[level] = Integer.MAX_VALUE;
       if (numberOfSkipLevels > level) numberOfSkipLevels = level; 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c7b3e9ae/lucene/core/src/test/org/apache/lucene/index/Test2BDocs.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BDocs.java b/lucene/core/src/test/org/apache/lucene/index/Test2BDocs.java
new file mode 100644
index 0000000..4fab45a
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/index/Test2BDocs.java
@@ -0,0 +1,135 @@
+/*
+ * 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.index;
+
+
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.store.BaseDirectoryWrapper;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.Monster;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.TimeUnits;
+
+@SuppressCodecs({"SimpleText", "Memory", "Direct"})
+@TimeoutSuite(millis = 80 * TimeUnits.HOUR) // effectively no limit
+@Monster("Takes ~30min")
+@SuppressSysoutChecks(bugUrl = "Stuff gets printed")
+public class Test2BDocs extends LuceneTestCase {
+  
+  // indexes Integer.MAX_VALUE docs with indexed field(s)
+  public void test2BDocs() throws Exception {
+    BaseDirectoryWrapper dir = newFSDirectory(createTempDir("2BDocs"));
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper)dir).setThrottling(MockDirectoryWrapper.Throttling.NEVER);
+    }
+    
+    IndexWriter w = new IndexWriter(dir,
+        new IndexWriterConfig(new MockAnalyzer(random()))
+        .setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH)
+        .setRAMBufferSizeMB(256.0)
+        .setMergeScheduler(new ConcurrentMergeScheduler())
+        .setMergePolicy(newLogMergePolicy(false, 10))
+        .setOpenMode(IndexWriterConfig.OpenMode.CREATE)
+        .setCodec(TestUtil.getDefaultCodec()));
+
+    Document doc = new Document();
+    Field field = new Field("f1", "a", StringField.TYPE_NOT_STORED);
+    doc.add(field);
+    
+    for (int i = 0; i < IndexWriter.MAX_DOCS; i++) {
+      w.addDocument(doc);
+      if (i % (10*1000*1000) == 0) {
+        System.out.println("indexed: " + i);
+        System.out.flush();
+      }
+    }
+    
+    w.forceMerge(1);
+    w.close();
+    
+    System.out.println("verifying...");
+    System.out.flush();
+    
+    DirectoryReader r = DirectoryReader.open(dir);
+
+    BytesRef term = new BytesRef(1);
+    term.bytes[0] = (byte)'a';
+    term.length = 1;
+
+    long skips = 0;
+
+    Random rnd = random();
+
+    long start = System.nanoTime();
+
+    for (LeafReaderContext context : r.leaves()) {
+      LeafReader reader = context.reader();
+      int lim = context.reader().maxDoc();
+
+      Terms terms = reader.fields().terms("f1");
+      for (int i=0; i<10000; i++) {
+        TermsEnum te = terms.iterator();
+        assertTrue( te.seekExact(term) );
+        PostingsEnum docs = te.postings(null);
+
+        // skip randomly through the term
+        for (int target = -1;;)
+        {
+          int maxSkipSize = lim - target + 1;
+          // do a smaller skip half of the time
+          if (rnd.nextBoolean()) {
+            maxSkipSize = Math.min(256, maxSkipSize);
+          }
+          int newTarget = target + rnd.nextInt(maxSkipSize) + 1;
+          if (newTarget >= lim) {
+            if (target+1 >= lim) break; // we already skipped to end, so break.
+            newTarget = lim-1;  // skip to end
+          }
+          target = newTarget;
+
+          int res = docs.advance(target);
+          if (res == PostingsEnum.NO_MORE_DOCS) break;
+
+          assertTrue( res >= target );
+
+          skips++;
+          target = res;
+        }
+      }
+    }
+    
+    r.close();
+    dir.close();
+
+    long end = System.nanoTime();
+
+    System.out.println("Skip count=" + skips + " seconds=" + TimeUnit.NANOSECONDS.toSeconds(end-start));
+    assert skips > 0;
+  }
+  
+}