You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/09/29 16:41:36 UTC

svn commit: r1628196 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/test/org/apache/lucene/index/

Author: rmuir
Date: Mon Sep 29 14:41:36 2014
New Revision: 1628196

URL: http://svn.apache.org/r1628196
Log:
LUCENE-5977: Fix tokenstream safety checks in IndexWriter to work across multi-valued fields

Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/core/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FieldInvertState.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java

Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1628196&r1=1628195&r2=1628196&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Mon Sep 29 14:41:36 2014
@@ -128,6 +128,10 @@ Bug Fixes
 * LUCENE-5948: RateLimiter now fully inits itself on init.  (Varun
   Thacker via Mike McCandless)
 
+* LUCENE-5977: Fix tokenstream safety checks in IndexWriter to properly
+  work across multi-valued fields. Previously some cases across multi-valued
+  fields would happily create a corrupt index. (Dawid Weiss, Robert Muir)
+
 Documentation
 
 * LUCENE-5392: Add/improve analysis package documentation to reflect

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java?rev=1628196&r1=1628195&r2=1628196&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java Mon Sep 29 14:41:36 2014
@@ -585,9 +585,6 @@ final class DefaultIndexingChain extends
       // TODO: after we fix analyzers, also check if termVectorOffsets will be indexed.
       final boolean checkOffsets = fieldType.indexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
 
-      int lastStartOffset = 0;
-      int lastPosition = 0;
-        
       /*
        * To assist people in tracking down problems in analysis components, we wish to write the field name to the infostream
        * when we fail. We expect some caller to eventually deal with the real exception, so we don't want any 'catch' clauses,
@@ -613,13 +610,13 @@ final class DefaultIndexingChain extends
 
           int posIncr = invertState.posIncrAttribute.getPositionIncrement();
           invertState.position += posIncr;
-          if (invertState.position < lastPosition) {
+          if (invertState.position < invertState.lastPosition) {
             if (posIncr == 0) {
               throw new IllegalArgumentException("first position increment must be > 0 (got 0) for field '" + field.name() + "'");
             }
             throw new IllegalArgumentException("position increments (and gaps) must be >= 0 (got " + posIncr + ") for field '" + field.name() + "'");
           }
-          lastPosition = invertState.position;
+          invertState.lastPosition = invertState.position;
           if (posIncr == 0) {
             invertState.numOverlap++;
           }
@@ -627,11 +624,11 @@ final class DefaultIndexingChain extends
           if (checkOffsets) {
             int startOffset = invertState.offset + invertState.offsetAttribute.startOffset();
             int endOffset = invertState.offset + invertState.offsetAttribute.endOffset();
-            if (startOffset < lastStartOffset || endOffset < startOffset) {
+            if (startOffset < invertState.lastStartOffset || endOffset < startOffset) {
               throw new IllegalArgumentException("startOffset must be non-negative, and endOffset must be >= startOffset, and offsets must not go backwards "
-                                                 + "startOffset=" + startOffset + ",endOffset=" + endOffset + ",lastStartOffset=" + lastStartOffset + " for field '" + field.name() + "'");
+                                                 + "startOffset=" + startOffset + ",endOffset=" + endOffset + ",lastStartOffset=" + invertState.lastStartOffset + " for field '" + field.name() + "'");
             }
-            lastStartOffset = startOffset;
+            invertState.lastStartOffset = startOffset;
           }
 
           //System.out.println("  term=" + invertState.termAttribute);

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FieldInvertState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FieldInvertState.java?rev=1628196&r1=1628195&r2=1628196&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FieldInvertState.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FieldInvertState.java Mon Sep 29 14:41:36 2014
@@ -39,6 +39,9 @@ public final class FieldInvertState {
   int maxTermFrequency;
   int uniqueTermCount;
   float boost;
+  // we must track these across field instances (multi-valued case)
+  int lastStartOffset = 0;
+  int lastPosition = 0;
   AttributeSource attributeSource;
 
   OffsetAttribute offsetAttribute;
@@ -74,6 +77,8 @@ public final class FieldInvertState {
     maxTermFrequency = 0;
     uniqueTermCount = 0;
     boost = 1.0f;
+    lastStartOffset = 0;
+    lastPosition = 0;
   }
   
   // TODO: better name?

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java?rev=1628196&r1=1628195&r2=1628196&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java Mon Sep 29 14:41:36 2014
@@ -426,6 +426,16 @@ public class TestPostingsOffsets extends
       //expected
     }
   }
+  
+  public void testIllegalOffsetsAcrossFieldInstances() throws Exception {
+    try {
+      checkTokens(new Token[] { makeToken("use", 1, 150, 160) }, 
+                  new Token[] { makeToken("use", 1, 50, 60) });
+      fail();
+    } catch (IllegalArgumentException expected) {
+      //expected
+    }
+  }
    
   public void testBackwardsOffsets() throws Exception {
     try {
@@ -508,6 +518,33 @@ public class TestPostingsOffsets extends
   }
   // TODO: more tests with other possibilities
   
+  private void checkTokens(Token[] field1, Token[] field2) throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter riw = new RandomIndexWriter(random(), dir, iwc);
+    boolean success = false;
+    try {
+      FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
+      ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+      // store some term vectors for the checkindex cross-check
+      ft.setStoreTermVectors(true);
+      ft.setStoreTermVectorPositions(true);
+      ft.setStoreTermVectorOffsets(true);
+     
+      Document doc = new Document();
+      doc.add(new Field("body", new CannedTokenStream(field1), ft));
+      doc.add(new Field("body", new CannedTokenStream(field2), ft));
+      riw.addDocument(doc);
+      riw.close();
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(dir);
+      } else {
+        IOUtils.closeWhileHandlingException(riw, dir);
+      }
+    }
+  }
+  
   private void checkTokens(Token[] tokens) throws IOException {
     Directory dir = newDirectory();
     RandomIndexWriter riw = new RandomIndexWriter(random(), dir, iwc);