You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by gu...@apache.org on 2021/04/29 00:47:11 UTC

[lucene-solr] branch branch_8x updated: LUCENE-9572 Backport from 9.0 (#2485)

This is an automated email from the ASF dual-hosted git repository.

gus pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new d7e4be6  LUCENE-9572 Backport from 9.0 (#2485)
d7e4be6 is described below

commit d7e4be6fc621a95165eb32d1d6f02bea4f5010ef
Author: Gus Heck <46...@users.noreply.github.com>
AuthorDate: Wed Apr 28 20:46:54 2021 -0400

    LUCENE-9572 Backport from 9.0 (#2485)
---
 .../miscellaneous/TypeAsSynonymFilter.java         |  37 ++++++-
 .../miscellaneous/TypeAsSynonymFilterFactory.java  |  17 ++-
 .../miscellaneous/TestTypeAsSynonymFilter.java     |  99 +++++++++++++++++
 .../TestTypeAsSynonymFilterFactory.java            |   7 ++
 .../lucene/analysis/BaseTokenStreamTestCase.java   | 122 ++++++++++++---------
 5 files changed, 220 insertions(+), 62 deletions(-)

diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilter.java
index 8269d5d..1ba0951 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilter.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilter.java
@@ -18,39 +18,59 @@
 package org.apache.lucene.analysis.miscellaneous;
 
 import java.io.IOException;
+import java.util.Set;
 
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 import org.apache.lucene.util.AttributeSource;
 
 /**
  * Adds the {@link TypeAttribute#type()} as a synonym,
- * i.e. another token at the same position, optionally with a specified prefix prepended.
+ * i.e. another token at the same position, optionally with a specified prefix prepended, optionally
+ * transfering flags, and optionally ignoring some types. See {@link TypeAsSynonymFilterFactory} for
+ * full details.
  */
 public final class TypeAsSynonymFilter extends TokenFilter {
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
   private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
+  private final FlagsAttribute flagsAtt = addAttribute(FlagsAttribute.class);
   private final String prefix;
+  private final Set<String> ignore;
+  private final int synFlagsMask;
 
-  AttributeSource.State savedToken = null;
+  private AttributeSource.State savedToken = null;
 
 
   public TypeAsSynonymFilter(TokenStream input) {
-    this(input, null);
+    this(input, null, null, ~0);
   }
-
   /**
    * @param input input tokenstream
    * @param prefix Prepend this string to every token type emitted as token text.
    *               If null, nothing will be prepended.
    */
+
   public TypeAsSynonymFilter(TokenStream input, String prefix) {
+    this(input, prefix, null, ~0);
+  }
+
+  /**
+   * @param input  input tokenstream
+   * @param prefix Prepend this string to every token type emitted as token text.
+   *               If null, nothing will be prepended.
+   * @param ignore types to ignore (and not convert to a synonym)
+   * @param synFlagsMask a mask to control what flags are propagated to the synonym.
+   */
+  public TypeAsSynonymFilter(TokenStream input, String prefix, Set<String> ignore, int synFlagsMask) {
     super(input);
     this.prefix = prefix;
+    this.ignore = ignore;
+    this.synFlagsMask = synFlagsMask;
   }
 
   @Override
@@ -64,9 +84,14 @@ public final class TypeAsSynonymFilter extends TokenFilter {
       }
       termAtt.append(typeAtt.type());
       posIncrAtt.setPositionIncrement(0);
+      // control what flags transfer to synonym
+      flagsAtt.setFlags(flagsAtt.getFlags() & synFlagsMask);
       return true;
-    } else if (input.incrementToken()) { // Ho pending token type to emit
-      savedToken = captureState();
+    } else if (input.incrementToken()) { // No pending token type to emit
+      String type = typeAtt.type();
+      if (ignore == null || !ignore.contains(type)) {
+        savedToken = captureState();
+      }
       return true;
     }
     return false;
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilterFactory.java
index 8dc0c46..cf39ea8 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilterFactory.java
@@ -18,24 +18,31 @@
 package org.apache.lucene.analysis.miscellaneous;
 
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.util.TokenFilterFactory;
 
 /**
  * Factory for {@link TypeAsSynonymFilter}.
+ *
+ * <p>In Solr this might be used as such
  * <pre class="prettyprint">
  * &lt;fieldType name="text_type_as_synonym" class="solr.TextField" positionIncrementGap="100"&gt;
  *   &lt;analyzer&gt;
  *     &lt;tokenizer class="solr.UAX29URLEmailTokenizerFactory"/&gt;
- *     &lt;filter class="solr.TypeAsSynonymFilterFactory" prefix="_type_" /&gt;
+ *     &lt;filter class="solr.TypeAsSynonymFilterFactory" prefix="_type_" synFlagsMask="5" ignore="foo,bar"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
  *
  * <p>
  * If the optional {@code prefix} parameter is used, the specified value will be prepended
  * to the type, e.g. with prefix="_type_", for a token "example.com" with type "&lt;URL&gt;",
- * the emitted synonym will have text "_type_&lt;URL&gt;".
+ * the emitted synonym will have text "_type_&lt;URL&gt;". If the optional synFlagsMask is used
+ * then the flags on the synonym will be set to <code>synFlagsMask &amp; tokenFlags</code>. The
+ * example above transfers only the lowest and third lowest bits. If no mask is set then
+ * all flags are transferred. The ignore parameter can be used to avoid creating synonyms for
+ * some types.
  *
  * @since 7.3.0
  * @lucene.spi {@value #NAME}
@@ -46,10 +53,14 @@ public class TypeAsSynonymFilterFactory extends TokenFilterFactory {
   public static final String NAME = "typeAsSynonym";
 
   private final String prefix;
+  private final Set<String> ignore;
+  private final int synFlagMask;
 
   public TypeAsSynonymFilterFactory(Map<String,String> args) {
     super(args);
     prefix = get(args, "prefix");  // default value is null
+    ignore = getSet(args, "ignore");
+    synFlagMask = getInt(args,"synFlagsMask", ~0);
     if (!args.isEmpty()) {
       throw new IllegalArgumentException("Unknown parameters: " + args);
     }
@@ -57,6 +68,6 @@ public class TypeAsSynonymFilterFactory extends TokenFilterFactory {
 
   @Override
   public TokenStream create(TokenStream input) {
-    return new TypeAsSynonymFilter(input, prefix);
+    return new TypeAsSynonymFilter(input, prefix, ignore, synFlagMask);
   }
 }
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTypeAsSynonymFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTypeAsSynonymFilter.java
new file mode 100644
index 0000000..8e7a5b8
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTypeAsSynonymFilter.java
@@ -0,0 +1,99 @@
+/*
+ * 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.analysis.miscellaneous;
+
+import java.util.Collections;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.CannedTokenStream;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+
+/**
+ * Test that this filter moves the value in type to a synonym token with the same offsets. This is rarely
+ * useful by itself, but in combination with another filter that updates the type value with an appropriate
+ * synonym can be used to identify synonyms before tokens are modified by further analysis, and then
+ * add them at the end, ensuring that the synonym value has not ben subjected to the intervening analysis.
+ * This typically applies when the analysis would remove characters that should remain in the synonym.
+ */
+public class TestTypeAsSynonymFilter extends BaseTokenStreamTestCase {
+
+  /**
+   * Test the straight forward case with the simplest constructor. Simply converts every
+   * type to a synonym. Typically one wants to also set an ignore list containing "word" unless
+   * that default value is removed by prior analysis.
+   */
+  public void testSimple() throws Exception {
+
+    Token token = new Token("foo", 0, 2);
+    token.setType("bar");
+    Token token2 = new Token("foo", 4, 6);
+    TokenStream ts = new CannedTokenStream(token, token2);
+    ts = new TypeAsSynonymFilter(ts);
+
+    // "word" is the default type!
+    assertTokenStreamContents(ts, new String[] {
+        "foo", "bar","foo","word"},new int[] {0,0,4,4}, new int[]{2,2,6,6}, new int[] {1,0,1,0});
+  }
+
+  /**
+   * Tests that we can add a prefix to the synonym (for example, to keep it from ever matching user input directly),
+   * and test that we can ignore a list of type values we don't wish to turn into synonyms.
+   */
+  public void testWithPrefixAndIgnore() throws Exception {
+    Token[] tokens = new Token[] {
+        new Token("foo", 1, 3),
+        new Token("foo", 5, 7),
+        new Token("foo", 9, 11),
+    } ;
+    tokens[0].setType("bar");
+    tokens[2].setType("ignoreme");
+    TokenStream ts = new CannedTokenStream(tokens);
+    ts = new TypeAsSynonymFilter(ts,"pfx_", Stream.of("word","ignoreme").collect(Collectors.toSet()), 0);
+
+    assertTokenStreamContents(ts, new String[] {
+        "foo", "pfx_bar","foo","foo"},new int[] {1,1,5,9}, new int[]{3,3,7,11}, new int[] {1,0,1,1});
+
+  }
+
+  /**
+   * Analysis chains that make use of flags may or may not want flags transferred to the synonym to be
+   * created. This tests the mask that can be used to control which flag bits are transferred.
+   */
+  public void testFlagMask() throws Exception {
+
+    Token token = new Token("foo", 0, 2);
+    token.setType("bar");
+    token.setFlags(7);
+    Token token2 = new Token("foo", 4, 6);
+    TokenStream ts = new CannedTokenStream(token, token2);
+
+    ts = new TypeAsSynonymFilter(ts,"", Collections.emptySet(), 5) ;
+
+    // "word" is the default type!
+    assertTokenStreamContents(ts, new String[] {
+        "foo", "bar","foo","word"},new int[] {0,0,4,4}, new int[]{2,2,6,6},
+        null, // not testing types
+        null,null, //positions tested above
+        // final values, keywords, graph, payloads not tested here
+        null,null, null, false,null,
+        new int[] {7,5,0,0}
+        );
+  }
+}
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTypeAsSynonymFilterFactory.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTypeAsSynonymFilterFactory.java
index 6beb139..6bdf194 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTypeAsSynonymFilterFactory.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTypeAsSynonymFilterFactory.java
@@ -40,6 +40,13 @@ public class TestTypeAsSynonymFilterFactory extends BaseTokenStreamFactoryTestCa
         null, null, new String[] { "<ALPHANUM>", "<ALPHANUM>", "<URL>", "<URL>" }, new int[] { 1, 0, 1, 0 });
   }
 
+  public void testIgnore() throws Exception {
+    TokenStream stream = new CannedTokenStream(TOKENS);
+    stream = tokenFilterFactory("typeAsSynonym", "prefix", "_type_","ignore", "<ALPHANUM>").create(stream);
+    assertTokenStreamContents(stream, new String[] { "Visit", "example.com", "_type_<URL>" },
+        null, null, new String[] { "<ALPHANUM>", "<URL>", "<URL>" }, new int[] { 1,  1, 0 });
+  }
+
   private static Token token(String term, String type) {
     Token token = new Token();
     token.setEmpty();
diff --git a/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
index 914ae37..fb9004a 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
@@ -52,22 +52,22 @@ import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.AutomatonTestUtil;
 import org.apache.lucene.util.fst.Util;
 
-/** 
- * Base class for all Lucene unit tests that use TokenStreams. 
+/**
+ * Base class for all Lucene unit tests that use TokenStreams.
  * <p>
  * When writing unit tests for analysis components, it's highly recommended
  * to use the helper methods here (especially in conjunction with {@link MockAnalyzer} or
- * {@link MockTokenizer}), as they contain many assertions and checks to 
+ * {@link MockTokenizer}), as they contain many assertions and checks to
  * catch bugs.
- * 
+ *
  * @see MockAnalyzer
  * @see MockTokenizer
  */
 public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
   // some helpers to test Analyzers and TokenStreams:
-  
+
   /**
-   * Attribute that records if it was cleared or not.  This is used 
+   * Attribute that records if it was cleared or not.  This is used
    * for testing that clearAttributes() was called correctly.
    */
   public static interface CheckClearAttributesAttribute extends Attribute {
@@ -75,12 +75,12 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
   }
 
   /**
-   * Attribute that records if it was cleared or not.  This is used 
+   * Attribute that records if it was cleared or not.  This is used
    * for testing that clearAttributes() was called correctly.
    */
   public static final class CheckClearAttributesAttributeImpl extends AttributeImpl implements CheckClearAttributesAttribute {
     private boolean clearCalled = false;
-    
+
     @Override
     public boolean getAndResetClearCalled() {
       try {
@@ -107,7 +107,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
     public int hashCode() {
       return 76137213 ^ Boolean.valueOf(clearCalled).hashCode();
     }
-    
+
     @Override
     public void copyTo(AttributeImpl target) {
       ((CheckClearAttributesAttributeImpl) target).clear();
@@ -125,28 +125,28 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
   //     arriving to pos Y have the same endOffset)
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[],
                                                int posLengths[], Integer finalOffset, Integer finalPosInc, boolean[] keywordAtts,
-                                               boolean graphOffsetsAreCorrect, byte[][] payloads) throws IOException {
+                                               boolean graphOffsetsAreCorrect, byte[][] payloads, int[] flags) throws IOException {
     assertNotNull(output);
     CheckClearAttributesAttribute checkClearAtt = ts.addAttribute(CheckClearAttributesAttribute.class);
-    
+
     CharTermAttribute termAtt = null;
     if (output.length > 0) {
       assertTrue("has no CharTermAttribute", ts.hasAttribute(CharTermAttribute.class));
       termAtt = ts.getAttribute(CharTermAttribute.class);
     }
-    
+
     OffsetAttribute offsetAtt = null;
     if (startOffsets != null || endOffsets != null || finalOffset != null) {
       assertTrue("has no OffsetAttribute", ts.hasAttribute(OffsetAttribute.class));
       offsetAtt = ts.getAttribute(OffsetAttribute.class);
     }
-    
+
     TypeAttribute typeAtt = null;
     if (types != null) {
       assertTrue("has no TypeAttribute", ts.hasAttribute(TypeAttribute.class));
       typeAtt = ts.getAttribute(TypeAttribute.class);
     }
-    
+
     PositionIncrementAttribute posIncrAtt = null;
     if (posIncrements != null || finalPosInc != null) {
       assertTrue("has no PositionIncrementAttribute", ts.hasAttribute(PositionIncrementAttribute.class));
@@ -170,7 +170,13 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
       assertTrue("has no PayloadAttribute", ts.hasAttribute(PayloadAttribute.class));
       payloadAtt = ts.getAttribute(PayloadAttribute.class);
     }
-    
+
+    FlagsAttribute flagsAtt = null;
+    if (flags != null) {
+      assertTrue("has no FlagsAttribute", ts.hasAttribute(FlagsAttribute.class));
+      flagsAtt = ts.getAttribute(FlagsAttribute.class);
+    }
+
     // Maps position to the start/end offset:
     final Map<Integer,Integer> posToStartOffset = new HashMap<>();
     final Map<Integer,Integer> posToEndOffset = new HashMap<>();
@@ -190,7 +196,8 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
       if (posLengthAtt != null) posLengthAtt.setPositionLength(45987653);
       if (keywordAtt != null) keywordAtt.setKeyword((i&1) == 0);
       if (payloadAtt != null) payloadAtt.setPayload(new BytesRef(new byte[] { 0x00, -0x21, 0x12, -0x43, 0x24 }));
-      
+      if (flagsAtt != null) flagsAtt.setFlags(~0); // all 1's
+
       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 at token " + i, checkClearAtt.getAndResetClearCalled());
@@ -214,6 +221,9 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
       if (keywordAtts != null) {
         assertEquals("keywordAtt " + i + " term=" + termAtt, keywordAtts[i], keywordAtt.isKeyword());
       }
+      if (flagsAtt != null) {
+        assertEquals("flagsAtt " + i + " term=" + termAtt, flags[i], flagsAtt.getFlags());
+      }
       if (payloads != null) {
         if (payloads[i] != null) {
           assertEquals("payloads " + i, new BytesRef(payloads[i]), payloadAtt.getPayload());
@@ -294,12 +304,13 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
     if (posLengthAtt != null) posLengthAtt.setPositionLength(45987653);
     if (keywordAtt != null) keywordAtt.setKeyword(true);
     if (payloadAtt != null) payloadAtt.setPayload(new BytesRef(new byte[] { 0x00, -0x21, 0x12, -0x43, 0x24 }));
+    if (flagsAtt != null) flagsAtt.setFlags(~0); // all 1's
 
     checkClearAtt.getAndResetClearCalled(); // reset it, because we called clearAttribute() before
 
     ts.end();
     assertTrue("super.end()/clearAttributes() was not called correctly in end()", checkClearAtt.getAndResetClearCalled());
-    
+
     if (finalOffset != null) {
       assertEquals("finalOffset", finalOffset.intValue(), offsetAtt.endOffset());
     }
@@ -312,11 +323,16 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
 
     ts.close();
   }
-  
+
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[],
                                                int posLengths[], Integer finalOffset, boolean[] keywordAtts,
                                                boolean graphOffsetsAreCorrect) throws IOException {
-    assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, posLengths, finalOffset, null, keywordAtts, graphOffsetsAreCorrect, null);
+    assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, posLengths, finalOffset, null, keywordAtts, graphOffsetsAreCorrect, null, null);
+  }
+  public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[],
+                                               int posLengths[], Integer finalOffset, Integer finalPosInc, boolean[] keywordAtts,
+                                               boolean graphOffsetsAreCorrect, byte[][] payloads) throws IOException {
+    assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, posLengths, finalOffset, finalPosInc, keywordAtts, graphOffsetsAreCorrect, payloads, null);
   }
 
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[], Integer finalOffset, boolean graphOffsetsAreCorrect) throws IOException {
@@ -342,23 +358,23 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
   public static void assertTokenStreamContents(TokenStream ts, String[] output) throws IOException {
     assertTokenStreamContents(ts, output, null, null, null, null, null, null);
   }
-  
+
   public static void assertTokenStreamContents(TokenStream ts, String[] output, String[] types) throws IOException {
     assertTokenStreamContents(ts, output, null, null, types, null, null, null);
   }
-  
+
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int[] posIncrements) throws IOException {
     assertTokenStreamContents(ts, output, null, null, null, posIncrements, null, null);
   }
-  
+
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[]) throws IOException {
     assertTokenStreamContents(ts, output, startOffsets, endOffsets, null, null, null, null);
   }
-  
+
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], Integer finalOffset) throws IOException {
     assertTokenStreamContents(ts, output, startOffsets, endOffsets, null, null, null, finalOffset);
   }
-  
+
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], int[] posIncrements) throws IOException {
     assertTokenStreamContents(ts, output, startOffsets, endOffsets, null, posIncrements, null, null);
   }
@@ -366,17 +382,17 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], int[] posIncrements, Integer finalOffset) throws IOException {
     assertTokenStreamContents(ts, output, startOffsets, endOffsets, null, posIncrements, null, finalOffset);
   }
-  
+
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], int[] posIncrements, int[] posLengths, Integer finalOffset) throws IOException {
     assertTokenStreamContents(ts, output, startOffsets, endOffsets, null, posIncrements, posLengths, finalOffset);
   }
-  
+
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[]) throws IOException {
     assertTokenStreamContents(a.tokenStream("dummy", input), output, startOffsets, endOffsets, types, posIncrements, null, input.length());
     checkResetException(a, input);
     checkAnalysisConsistency(random(), a, true, input);
   }
-  
+
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[]) throws IOException {
     assertTokenStreamContents(a.tokenStream("dummy", input), output, startOffsets, endOffsets, types, posIncrements, posLengths, input.length());
     checkResetException(a, input);
@@ -398,11 +414,11 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output) throws IOException {
     assertAnalyzesTo(a, input, output, null, null, null, null, null);
   }
-  
+
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, String[] types) throws IOException {
     assertAnalyzesTo(a, input, output, null, null, types, null, null);
   }
-  
+
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int[] posIncrements) throws IOException {
     assertAnalyzesTo(a, input, output, null, null, null, posIncrements, null);
   }
@@ -414,11 +430,11 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
   public static void assertAnalyzesToPositions(Analyzer a, String input, String[] output, String[] types, int[] posIncrements, int[] posLengths) throws IOException {
     assertAnalyzesTo(a, input, output, null, null, types, posIncrements, posLengths);
   }
-  
+
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[]) throws IOException {
     assertAnalyzesTo(a, input, output, startOffsets, endOffsets, null, null, null);
   }
-  
+
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], int[] posIncrements) throws IOException {
     assertAnalyzesTo(a, input, output, startOffsets, endOffsets, null, posIncrements, null);
   }
@@ -442,7 +458,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
       ts.end();
       ts.close();
     }
-    
+
     // check for a missing close()
     ts = a.tokenStream("bogus", input);
     ts.reset();
@@ -459,11 +475,11 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
   }
 
   // simple utility method for testing stemmers
-  
+
   public static void checkOneTerm(Analyzer a, final String input, final String expected) throws IOException {
     assertAnalyzesTo(a, input, new String[]{expected});
   }
-  
+
   /** utility method for blasting tokenstreams with data to make sure they don't do anything crazy */
   public static void checkRandomData(Random random, Analyzer a, int iterations) throws IOException {
     checkRandomData(random, a, iterations, 20, false, true);
@@ -473,15 +489,15 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
   public static void checkRandomData(Random random, Analyzer a, int iterations, int maxWordLength) throws IOException {
     checkRandomData(random, a, iterations, maxWordLength, false, true);
   }
-  
-  /** 
-   * utility method for blasting tokenstreams with data to make sure they don't do anything crazy 
+
+  /**
+   * utility method for blasting tokenstreams with data to make sure they don't do anything crazy
    * @param simple true if only ascii strings will be used (try to avoid)
    */
   public static void checkRandomData(Random random, Analyzer a, int iterations, boolean simple) throws IOException {
     checkRandomData(random, a, iterations, 20, simple, true);
   }
-  
+
   /** Asserts that the given stream has expected number of tokens. */
   public static void assertStreamHasNumberOfTokens(TokenStream ts, int expectedCount) throws IOException {
     ts.reset();
@@ -492,7 +508,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
     ts.end();
     assertEquals("wrong number of tokens", expectedCount, count);
   }
-  
+
   static class AnalysisThread extends Thread {
     final int iterations;
     final int maxWordLength;
@@ -508,7 +524,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
     // add memory barriers (ie alter how threads
     // interact)... so this is just "best effort":
     public boolean failed;
-    
+
     AnalysisThread(long seed, CountDownLatch latch, Analyzer a, int iterations, int maxWordLength, boolean useCharFilter, boolean simple, boolean graphOffsetsAreCorrect, RandomIndexWriter iw) {
       this.seed = seed;
       this.a = a;
@@ -520,7 +536,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
       this.iw = iw;
       this.latch = latch;
     }
-    
+
     @Override
     public void run() {
       boolean success = false;
@@ -537,7 +553,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
       }
     }
   };
-  
+
   public static void checkRandomData(Random random, Analyzer a, int iterations, int maxWordLength, boolean simple) throws IOException {
     checkRandomData(random, a, iterations, maxWordLength, simple, true);
   }
@@ -623,7 +639,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
       currentField = field = new Field("dummy", bogus, ft);
       doc.add(currentField);
     }
-    
+
     for (int i = 0; i < iterations; i++) {
       String text = TestUtil.randomAnalysisString(random, maxWordLength, simple);
 
@@ -690,7 +706,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
   public static void checkAnalysisConsistency(Random random, Analyzer a, boolean useCharFilter, String text, boolean graphOffsetsAreCorrect) throws IOException {
     checkAnalysisConsistency(random, a, useCharFilter, text, graphOffsetsAreCorrect, null);
   }
-  
+
   private static void checkAnalysisConsistency(Random random, Analyzer a, boolean useCharFilter, String text, boolean graphOffsetsAreCorrect, Field field) throws IOException {
 
     if (VERBOSE) {
@@ -826,7 +842,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
     ts = a.tokenStream("dummy", useCharFilter ? new MockCharFilter(reader, remainder) : reader);
     if (typeAtt != null && posIncAtt != null && posLengthAtt != null && offsetAtt != null) {
       // offset + pos + posLength + type
-      assertTokenStreamContents(ts, 
+      assertTokenStreamContents(ts,
                                 tokens.toArray(new String[tokens.size()]),
                                 toIntArray(startOffsets),
                                 toIntArray(endOffsets),
@@ -837,7 +853,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
                                 graphOffsetsAreCorrect);
     } else if (typeAtt != null && posIncAtt != null && offsetAtt != null) {
       // offset + pos + type
-      assertTokenStreamContents(ts, 
+      assertTokenStreamContents(ts,
                                 tokens.toArray(new String[tokens.size()]),
                                 toIntArray(startOffsets),
                                 toIntArray(endOffsets),
@@ -848,7 +864,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
                                 graphOffsetsAreCorrect);
     } else if (posIncAtt != null && posLengthAtt != null && offsetAtt != null) {
       // offset + pos + posLength
-      assertTokenStreamContents(ts, 
+      assertTokenStreamContents(ts,
                                 tokens.toArray(new String[tokens.size()]),
                                 toIntArray(startOffsets),
                                 toIntArray(endOffsets),
@@ -859,7 +875,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
                                 graphOffsetsAreCorrect);
     } else if (posIncAtt != null && offsetAtt != null) {
       // offset + pos
-      assertTokenStreamContents(ts, 
+      assertTokenStreamContents(ts,
                                 tokens.toArray(new String[tokens.size()]),
                                 toIntArray(startOffsets),
                                 toIntArray(endOffsets),
@@ -870,7 +886,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
                                 graphOffsetsAreCorrect);
     } else if (offsetAtt != null) {
       // offset
-      assertTokenStreamContents(ts, 
+      assertTokenStreamContents(ts,
                                 tokens.toArray(new String[tokens.size()]),
                                 toIntArray(startOffsets),
                                 toIntArray(endOffsets),
@@ -881,7 +897,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
                                 graphOffsetsAreCorrect);
     } else {
       // terms only
-      assertTokenStreamContents(ts, 
+      assertTokenStreamContents(ts,
                                 tokens.toArray(new String[tokens.size()]));
     }
 
@@ -918,7 +934,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
     new TokenStreamToDot(inputText, ts, new PrintWriter(w)).toDot();
     w.close();
   }
-  
+
   private static int[] toIntArray(List<Integer> list) {
     return list.stream().mapToInt(Integer::intValue).toArray();
   }
@@ -946,7 +962,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
     mockTokenizer.setReader(new StringReader(input));
     return mockTokenizer;
   }
-  
+
   /** Returns a random AttributeFactory impl */
   public static AttributeFactory newAttributeFactory(Random random) {
     switch (random.nextInt(3)) {
@@ -960,7 +976,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
         throw new AssertionError("Please fix the Random.nextInt() call above");
     }
   }
-  
+
   /** Returns a random AttributeFactory impl */
   public static AttributeFactory newAttributeFactory() {
     return newAttributeFactory(random());