You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2013/08/14 17:55:43 UTC

svn commit: r1513940 - in /lucene/dev/trunk/lucene: ./ suggest/ suggest/src/java/org/apache/lucene/search/suggest/analyzing/ suggest/src/test/org/apache/lucene/search/suggest/analyzing/ test-framework/src/java/org/apache/lucene/analysis/

Author: mikemccand
Date: Wed Aug 14 15:55:43 2013
New Revision: 1513940

URL: http://svn.apache.org/r1513940
Log:
LUCENE-5165: add SuggestStopFilter

Added:
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/SuggestStopFilter.java   (with props)
    lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestSuggestStopFilter.java   (with props)
Removed:
    lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/ForkLastTokenFilter.java
    lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/StopKeywordFilter.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/suggest/build.xml
    lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1513940&r1=1513939&r2=1513940&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Wed Aug 14 15:55:43 2013
@@ -83,6 +83,13 @@ New features
   FacetsAggregator.createOrdinalValueResolver. This gives better options for
   resolving an ordinal's value by FacetAggregators. (Shai Erera)
 
+* LUCENE-5165: Add SuggestStopFilter, to be used with analyzing
+  suggesters, so that a stop word at the very end of the lookup query,
+  and without any trailing token characters, will be preserved.  This
+  enables query "a" to suggest apple; see 
+  http://blog.mikemccandless.com/2013/08/suggeststopfilter-carefully-removes.html
+  for details.
+
 Bug Fixes
 
 * LUCENE-5116: IndexWriter.addIndexes(IndexReader...) should drop empty (or all

Modified: lucene/dev/trunk/lucene/suggest/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/build.xml?rev=1513940&r1=1513939&r2=1513940&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/build.xml (original)
+++ lucene/dev/trunk/lucene/suggest/build.xml Wed Aug 14 15:55:43 2013
@@ -34,6 +34,15 @@
     <path refid="base.classpath"/>
   </path>
 
+
+  <target name="javadocs" depends="javadocs-queries,compile-core">
+    <invoke-module-javadoc>
+      <links>
+        <link href="../analyzers-common"/>
+      </links>
+    </invoke-module-javadoc>
+  </target>
+
   <target name="compile-core" depends="jar-misc, jar-analyzers-common, common.compile-core" />
 
 </project>

Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/SuggestStopFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/SuggestStopFilter.java?rev=1513940&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/SuggestStopFilter.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/SuggestStopFilter.java Wed Aug 14 15:55:43 2013
@@ -0,0 +1,129 @@
+package org.apache.lucene.search.suggest.analyzing;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.core.StopFilter;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.KeywordAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.util.CharArraySet;
+import org.apache.lucene.util.AttributeSource.State;
+
+/** Like {@link StopFilter} except it will not remove the
+ *  last token if that token was not followed by some token
+ *  separator.  For example, a query 'find the' would
+ *  preserve the 'the' since it was not followed by a space or
+ *  punctuation or something, and mark it KEYWORD so future
+ *  stemmers won't touch it either while a query like "find
+ *  the popsicle' would remove 'the' as a stopword.
+ *
+ *  <p>Normally you'd use the ordinary {@link StopFilter}
+ *  in your indexAnalyzer and then this class in your
+ *  queryAnalyzer, when using one of the analyzing suggesters. */
+
+public final class SuggestStopFilter extends TokenFilter {
+
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+  private final KeywordAttribute keywordAtt = addAttribute(KeywordAttribute.class);
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+  private final CharArraySet stopWords;
+
+  private State endState;
+  private boolean ended;
+
+  /** Sole constructor. */
+  public SuggestStopFilter(TokenStream input, CharArraySet stopWords) {
+    super(input);
+    this.stopWords = stopWords;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    ended = false;
+    endState = null;
+  }
+
+  @Override
+  public void end() throws IOException {
+    if (!ended) {
+      super.end();
+    } else {
+      // NOTE: we already called .end() from our .next() when
+      // the stream was complete, so we do not call
+      // super.end() here
+
+      if (endState != null) {
+        restoreState(endState);
+      }
+    }
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+    if (ended) {
+      return false;
+    }
+
+    if (!input.incrementToken()) {
+      return false;
+    }
+
+    int skippedPositions = 0;
+    while (true) {
+      if (stopWords.contains(termAtt.buffer(), 0, termAtt.length())) {
+        int posInc = posIncAtt.getPositionIncrement();
+        int endOffset = offsetAtt.endOffset();
+        // This token may be a stopword, if it's not end:
+        State sav = captureState();
+        if (input.incrementToken()) {
+          // It was a stopword; skip it
+          skippedPositions += posInc;
+        } else {
+          input.end();
+          ended = true;
+          int finalEndOffset = offsetAtt.endOffset();
+          assert finalEndOffset >= endOffset;
+          if (finalEndOffset > endOffset) {
+            // OK there was a token separator after the
+            // stopword, so it was a stopword
+            return false;
+          } else {
+            // No token separator after final token that
+            // looked like a stop-word; don't filter it:
+            endState = captureState();
+            restoreState(sav);
+            posIncAtt.setPositionIncrement(skippedPositions + posIncAtt.getPositionIncrement());
+            keywordAtt.setKeyword(true);
+            return true;
+          }
+        }
+      } else {
+        // Not a stopword; return the current token:
+        posIncAtt.setPositionIncrement(skippedPositions + posIncAtt.getPositionIncrement());
+        return true;
+      }
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java?rev=1513940&r1=1513939&r2=1513940&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java Wed Aug 14 15:55:43 2013
@@ -25,11 +25,8 @@ import java.util.Locale;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.PrefixQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.analysis.core.StopFilter;
+import org.apache.lucene.analysis.util.CharArraySet;
 import org.apache.lucene.search.suggest.Lookup.LookupResult;
 import org.apache.lucene.search.suggest.TermFreqPayload;
 import org.apache.lucene.search.suggest.TermFreqPayloadArrayIterator;
@@ -294,64 +291,39 @@ public class AnalyzingInfixSuggesterTest
     suggester.close();
   }
 
-  public void testForkLastToken() throws Exception {
-    Analyzer a = new Analyzer() {
+  public void testSuggestStopFilter() throws Exception {
+    final CharArraySet stopWords = StopFilter.makeStopSet(TEST_VERSION_CURRENT, "a");
+    Analyzer indexAnalyzer = new Analyzer() {
         @Override
         protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
           MockTokenizer tokens = new MockTokenizer(reader);
-          // ForkLastTokenFilter is a bit evil:
-          tokens.setEnableChecks(false);
           return new TokenStreamComponents(tokens,
-                                           new StopKeywordFilter(TEST_VERSION_CURRENT,
-                                                                 new ForkLastTokenFilter(tokens), StopKeywordFilter.makeStopSet(TEST_VERSION_CURRENT, "a")));
+                                           new StopFilter(TEST_VERSION_CURRENT, tokens, stopWords));
         }
       };
 
-    TermFreqPayload keys[] = new TermFreqPayload[] {
-      new TermFreqPayload("a bob for apples", 10, new BytesRef("foobaz")),
-    };
-
-    File tempDir = _TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
-
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
+    Analyzer queryAnalyzer = new Analyzer() {
         @Override
-        protected Query finishQuery(BooleanQuery in, boolean allTermsRequired) {
-          List<BooleanClause> clauses = in.clauses();
-          if (clauses.size() >= 2 && allTermsRequired) {
-            String t1 = getTerm(clauses.get(clauses.size()-2).getQuery());
-            String t2 = getTerm(clauses.get(clauses.size()-1).getQuery());
-            if (t1.equals(t2)) {
-              // The last 2 tokens came from
-              // ForkLastTokenFilter; we remove them and
-              // replace them with a MUST BooleanQuery that
-              // SHOULDs the two of them together:
-              BooleanQuery sub = new BooleanQuery();
-              BooleanClause other = clauses.get(clauses.size()-2);
-              sub.add(new BooleanClause(clauses.get(clauses.size()-2).getQuery(), BooleanClause.Occur.SHOULD));
-              sub.add(new BooleanClause(clauses.get(clauses.size()-1).getQuery(), BooleanClause.Occur.SHOULD));
-              clauses.subList(clauses.size()-2, clauses.size()).clear();
-              clauses.add(new BooleanClause(sub, BooleanClause.Occur.MUST));
-            }
-          }
-          return in;
+        protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+          MockTokenizer tokens = new MockTokenizer(reader);
+          return new TokenStreamComponents(tokens,
+                                           new SuggestStopFilter(tokens, stopWords));
         }
+      };
 
-        private String getTerm(Query query) {
-          if (query instanceof TermQuery) {
-            return ((TermQuery) query).getTerm().text();
-          } else if (query instanceof PrefixQuery) {
-            return ((PrefixQuery) query).getPrefix().text();
-          } else {
-            return null;
-          }
-        }
+    File tempDir = _TestUtil.getTempDir("AnalyzingInfixSuggesterTest");
 
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, indexAnalyzer, queryAnalyzer, 3) {
         @Override
         protected Directory getDirectory(File path) {
           return newDirectory();
         }
       };
 
+    TermFreqPayload keys[] = new TermFreqPayload[] {
+      new TermFreqPayload("a bob for apples", 10, new BytesRef("foobaz")),
+    };
+
     suggester.build(new TermFreqPayloadArrayIterator(keys));
     List<LookupResult> results = suggester.lookup(_TestUtil.stringToCharSequence("a", random()), 10, true, true);
     assertEquals(1, results.size());

Added: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestSuggestStopFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestSuggestStopFilter.java?rev=1513940&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestSuggestStopFilter.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestSuggestStopFilter.java Wed Aug 14 15:55:43 2013
@@ -0,0 +1,140 @@
+package org.apache.lucene.search.suggest.analyzing;
+
+/*
+ * 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.
+ */
+
+import java.io.StringReader;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.core.StopFilter;
+import org.apache.lucene.analysis.util.CharArraySet;
+
+public class TestSuggestStopFilter extends BaseTokenStreamTestCase {
+
+  public void testEndNotStopWord() throws Exception {
+    CharArraySet stopWords = StopFilter.makeStopSet(TEST_VERSION_CURRENT, "to");
+    TokenStream stream = new MockTokenizer(new StringReader("go to"));
+    TokenStream filter = new SuggestStopFilter(stream, stopWords);
+    assertTokenStreamContents(filter,
+                              new String[] {"go", "to"},
+                              new int[] {0, 3},
+                              new int[] {2, 5},
+                              null,
+                              new int[] {1, 1},
+                              null,
+                              5,
+                              new boolean[] {false, true},
+                              true);
+  }
+
+  public void testEndIsStopWord() throws Exception {
+                              
+    CharArraySet stopWords = StopFilter.makeStopSet(TEST_VERSION_CURRENT, "to");
+    TokenStream stream = new MockTokenizer(new StringReader("go to "));
+    TokenStream filter = new SuggestStopFilter(stream, stopWords);
+
+    filter = new SuggestStopFilter(stream, stopWords);
+    assertTokenStreamContents(filter,
+                              new String[] {"go"},
+                              new int[] {0},
+                              new int[] {2},
+                              null,
+                              new int[] {1},
+                              null,
+                              6,
+                              new boolean[] {false},
+                              true);
+  }
+
+  public void testMidStopWord() throws Exception {
+                              
+    CharArraySet stopWords = StopFilter.makeStopSet(TEST_VERSION_CURRENT, "to");
+    TokenStream stream = new MockTokenizer(new StringReader("go to school"));
+    TokenStream filter = new SuggestStopFilter(stream, stopWords);
+
+    filter = new SuggestStopFilter(stream, stopWords);
+    assertTokenStreamContents(filter,
+                              new String[] {"go", "school"},
+                              new int[] {0, 6},
+                              new int[] {2, 12},
+                              null,
+                              new int[] {1, 2},
+                              null,
+                              12,
+                              new boolean[] {false, false},
+                              true);
+  }
+
+  public void testMultipleStopWords() throws Exception {
+                              
+    CharArraySet stopWords = StopFilter.makeStopSet(TEST_VERSION_CURRENT, "to", "the", "a");
+    TokenStream stream = new MockTokenizer(new StringReader("go to a the school"));
+    TokenStream filter = new SuggestStopFilter(stream, stopWords);
+
+    filter = new SuggestStopFilter(stream, stopWords);
+    assertTokenStreamContents(filter,
+                              new String[] { "go", "school" },
+                              new int[] {0, 12},
+                              new int[] {2, 18},
+                              null,
+                              new int[] {1, 4},
+                              null,
+                              18,
+                              new boolean[] {false, false},
+                              true);
+  }
+
+  public void testMultipleStopWordsEnd() throws Exception {
+                              
+    CharArraySet stopWords = StopFilter.makeStopSet(TEST_VERSION_CURRENT, "to", "the", "a");
+    TokenStream stream = new MockTokenizer(new StringReader("go to a the"));
+    TokenStream filter = new SuggestStopFilter(stream, stopWords);
+
+    filter = new SuggestStopFilter(stream, stopWords);
+    assertTokenStreamContents(filter,
+                              new String[] { "go", "the"},
+                              new int[] {0, 8},
+                              new int[] {2, 11},
+                              null,
+                              new int[] {1, 3},
+                              null,
+                              11,
+                              new boolean[] {false, true},
+                              true);
+  }
+
+  public void testMultipleStopWordsEnd2() throws Exception {
+                              
+    CharArraySet stopWords = StopFilter.makeStopSet(TEST_VERSION_CURRENT, "to", "the", "a");
+    TokenStream stream = new MockTokenizer(new StringReader("go to a the "));
+    TokenStream filter = new SuggestStopFilter(stream, stopWords);
+
+    filter = new SuggestStopFilter(stream, stopWords);
+    assertTokenStreamContents(filter,
+                              new String[] { "go"},
+                              new int[] {0},
+                              new int[] {2},
+                              null,
+                              new int[] {1},
+                              null,
+                              12,
+                              new boolean[] {false},
+                              true);
+  }
+}

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1513940&r1=1513939&r2=1513940&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Wed Aug 14 15:55:43 2013
@@ -111,7 +111,8 @@ public abstract class BaseTokenStreamTes
   //     arriving to pos Y have the same endOffset)
   //   - offsets only move forwards (startOffset >=
   //     lastStartOffset)
-  public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[], Integer finalOffset,
+  public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[],
+                                               int posLengths[], Integer finalOffset, boolean[] keywordAtts,
                                                boolean offsetsAreCorrect) throws IOException {
     assertNotNull(output);
     CheckClearAttributesAttribute checkClearAtt = ts.addAttribute(CheckClearAttributesAttribute.class);
@@ -145,6 +146,12 @@ public abstract class BaseTokenStreamTes
       assertTrue("has no PositionLengthAttribute", ts.hasAttribute(PositionLengthAttribute.class));
       posLengthAtt = ts.getAttribute(PositionLengthAttribute.class);
     }
+
+    KeywordAttribute keywordAtt = null;
+    if (keywordAtts != null) {
+      assertTrue("has no KeywordAttribute", ts.hasAttribute(KeywordAttribute.class));
+      keywordAtt = ts.getAttribute(KeywordAttribute.class);
+    }
     
     // Maps position to the start/end offset:
     final Map<Integer,Integer> posToStartOffset = new HashMap<Integer,Integer>();
@@ -161,22 +168,31 @@ public abstract class BaseTokenStreamTes
       if (typeAtt != null) typeAtt.setType("bogusType");
       if (posIncrAtt != null) posIncrAtt.setPositionIncrement(45987657);
       if (posLengthAtt != null) posLengthAtt.setPositionLength(45987653);
+      if (keywordAtt != null) keywordAtt.setKeyword((i&1) == 0);
       
       checkClearAtt.getAndResetClearCalled(); // reset it, because we called clearAttribute() before
       assertTrue("token "+i+" does not exist", ts.incrementToken());
       assertTrue("clearAttributes() was not called correctly in TokenStream chain", checkClearAtt.getAndResetClearCalled());
       
       assertEquals("term "+i, output[i], termAtt.toString());
-      if (startOffsets != null)
+      if (startOffsets != null) {
         assertEquals("startOffset "+i, startOffsets[i], offsetAtt.startOffset());
-      if (endOffsets != null)
+      }
+      if (endOffsets != null) {
         assertEquals("endOffset "+i, endOffsets[i], offsetAtt.endOffset());
-      if (types != null)
+      }
+      if (types != null) {
         assertEquals("type "+i, types[i], typeAtt.type());
-      if (posIncrements != null)
+      }
+      if (posIncrements != null) {
         assertEquals("posIncrement "+i, posIncrements[i], posIncrAtt.getPositionIncrement());
-      if (posLengths != null)
+      }
+      if (posLengths != null) {
         assertEquals("posLength "+i, posLengths[i], posLengthAtt.getPositionLength());
+      }
+      if (keywordAtts != null) {
+        assertEquals("keywordAtt " + i, keywordAtts[i], keywordAtt.isKeyword());
+      }
       
       // we can enforce some basic things about a few attributes even if the caller doesn't check:
       if (offsetAtt != null) {
@@ -239,7 +255,9 @@ public abstract class BaseTokenStreamTes
         assertTrue("posLength must be >= 1", posLengthAtt.getPositionLength() >= 1);
       }
     }
-    assertFalse("TokenStream has more tokens than expected (expected count=" + output.length + ")", ts.incrementToken());
+    if (ts.incrementToken()) {
+      fail("TokenStream has more tokens than expected (expected count=" + output.length + "); extra token=" + termAtt.toString());
+    }
     ts.end();
     if (finalOffset != null) {
       assertEquals("finalOffset ", finalOffset.intValue(), offsetAtt.endOffset());
@@ -250,6 +268,10 @@ public abstract class BaseTokenStreamTes
     ts.close();
   }
   
+  public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[], Integer finalOffset, boolean offsetsAreCorrect) throws IOException {
+    assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, posLengths, finalOffset, null, offsetsAreCorrect);
+  }
+
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[], Integer finalOffset) throws IOException {
     assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, posLengths, finalOffset, true);
   }