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 2017/02/08 10:22:14 UTC

[3/3] lucene-solr:branch_5_5: LUCENE-7440: fix MultiLevelSkipListReader overflow

LUCENE-7440: fix MultiLevelSkipListReader overflow


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

Branch: refs/heads/branch_5_5
Commit: 1c219d2da692d8b41c02d8a52ffacd813cd44a7b
Parents: 91147a8
Author: yonik <yo...@apache.org>
Authored: Sun Sep 11 01:28:24 2016 +0530
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed Feb 8 11:01:23 2017 +0100

----------------------------------------------------------------------
 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/1c219d2d/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 41eea64..5229247 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -27,6 +27,10 @@ Bug Fixes
 * LUCENE-6974: Fixed DecimalDigitFilter in case of supplementary code points.
   (Hossman)
 
+* LUCENE-7440: Document id skipping (PostingsEnum.advance) could throw an
+  ArrayIndexOutOfBoundsException exception on large index segments (>1.8B docs)
+  with large skips. (yonik)
+
 Other
 
 * LUCENE-6989: Backport MMapDirectory's unmapping code from Lucene 6.4 to use

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1c219d2d/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/1c219d2d/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;
+  }
+  
+}