You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/12/07 09:57:35 UTC

[20/37] lucene-solr:jira/http2: LUCENE-8509: WordDelimiterGraphFilter no longer adjusts offsets by default

LUCENE-8509: WordDelimiterGraphFilter no longer adjusts offsets by default


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

Branch: refs/heads/jira/http2
Commit: 75a053dd696d6e632755e613380450f22c78c91b
Parents: f5867a1
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Dec 3 13:36:21 2018 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Tue Dec 4 09:47:42 2018 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 ++
 .../miscellaneous/WordDelimiterGraphFilter.java | 17 ++++----
 .../WordDelimiterGraphFilterFactory.java        |  5 ++-
 .../TestWordDelimiterGraphFilter.java           | 42 +++++++++++---------
 4 files changed, 41 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/75a053dd/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index fc609a5..0a41d70 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -142,6 +142,10 @@ Changes in Runtime Behavior
   anymore. This doesn't affect ordering as this is a constant factor which is
   the same for every document. (Luca Cavanna via Adrien Grand)
 
+* LUCENE-8509: WordDelimiterGraphFilter will no longer set the offsets of internal
+  tokens by default, preventing a number of bugs when the filter is chained with
+  tokenfilters that change the length of their tokens (Alan Woodward)
+
 New Features
 
 * LUCENE-8340: LongPoint#newDistanceQuery may be used to boost scores based on

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/75a053dd/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterGraphFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterGraphFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterGraphFilter.java
index a438213..00ace5b 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterGraphFilter.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterGraphFilter.java
@@ -191,6 +191,8 @@ public final class WordDelimiterGraphFilter extends TokenFilter {
   // used for concatenating runs of similar typed subwords (word,number)
   private final WordDelimiterConcatenation concat = new WordDelimiterConcatenation();
 
+  private final boolean adjustInternalOffsets;
+
   // number of subwords last output by concat.
   private int lastConcatCount;
 
@@ -206,10 +208,7 @@ public final class WordDelimiterGraphFilter extends TokenFilter {
   private int savedEndOffset;
   private AttributeSource.State savedState;
   private int lastStartOffset;
-  
-  // if length by start + end offsets doesn't match the term text then assume
-  // this is a synonym and don't adjust the offsets.
-  private boolean hasIllegalOffsets;
+  private boolean adjustingOffsets;
 
   private int wordPos;
 
@@ -217,11 +216,12 @@ public final class WordDelimiterGraphFilter extends TokenFilter {
    * Creates a new WordDelimiterGraphFilter
    *
    * @param in TokenStream to be filtered
+   * @param adjustInternalOffsets if the offsets of partial terms should be adjusted
    * @param charTypeTable table containing character types
    * @param configurationFlags Flags configuring the filter
    * @param protWords If not null is the set of tokens to protect from being delimited
    */
-  public WordDelimiterGraphFilter(TokenStream in, byte[] charTypeTable, int configurationFlags, CharArraySet protWords) {
+  public WordDelimiterGraphFilter(TokenStream in, boolean adjustInternalOffsets, byte[] charTypeTable, int configurationFlags, CharArraySet protWords) {
     super(in);
     if ((configurationFlags &
         ~(GENERATE_WORD_PARTS |
@@ -240,6 +240,7 @@ public final class WordDelimiterGraphFilter extends TokenFilter {
     this.protWords = protWords;
     this.iterator = new WordDelimiterIterator(
         charTypeTable, has(SPLIT_ON_CASE_CHANGE), has(SPLIT_ON_NUMERICS), has(STEM_ENGLISH_POSSESSIVE));
+    this.adjustInternalOffsets = adjustInternalOffsets;
   }
 
   /**
@@ -251,7 +252,7 @@ public final class WordDelimiterGraphFilter extends TokenFilter {
    * @param protWords If not null is the set of tokens to protect from being delimited
    */
   public WordDelimiterGraphFilter(TokenStream in, int configurationFlags, CharArraySet protWords) {
-    this(in, WordDelimiterIterator.DEFAULT_WORD_DELIM_TABLE, configurationFlags, protWords);
+    this(in, false, WordDelimiterIterator.DEFAULT_WORD_DELIM_TABLE, configurationFlags, protWords);
   }
 
   /** Iterates all words parts and concatenations, buffering up the term parts we should return. */
@@ -261,7 +262,7 @@ public final class WordDelimiterGraphFilter extends TokenFilter {
 
     // if length by start + end offsets doesn't match the term's text then set offsets for all our word parts/concats to the incoming
     // offsets.  this can happen if WDGF is applied to an injected synonym, or to a stem'd form, etc:
-    hasIllegalOffsets = (savedEndOffset - savedStartOffset != savedTermLength);
+    adjustingOffsets = adjustInternalOffsets && savedEndOffset - savedStartOffset == savedTermLength;
 
     bufferedLen = 0;
     lastConcatCount = 0;
@@ -391,7 +392,7 @@ public final class WordDelimiterGraphFilter extends TokenFilter {
         int startOffset;
         int endOffset;
 
-        if (hasIllegalOffsets) {
+        if (adjustingOffsets == false) {
           startOffset = savedStartOffset;
           endOffset = savedEndOffset;
         } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/75a053dd/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterGraphFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterGraphFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterGraphFilterFactory.java
index 613aedc..4666c7d 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterGraphFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/WordDelimiterGraphFilterFactory.java
@@ -53,12 +53,14 @@ import static org.apache.lucene.analysis.miscellaneous.WordDelimiterIterator.*;
 public class WordDelimiterGraphFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
   public static final String PROTECTED_TOKENS = "protected";
   public static final String TYPES = "types";
+  public static final String OFFSETS = "adjustOffsets";
 
   private final String wordFiles;
   private final String types;
   private final int flags;
   byte[] typeTable = null;
   private CharArraySet protectedWords = null;
+  private boolean adjustOffsets = false;
   
   /** Creates a new WordDelimiterGraphFilterFactory */
   public WordDelimiterGraphFilterFactory(Map<String, String> args) {
@@ -94,6 +96,7 @@ public class WordDelimiterGraphFilterFactory extends TokenFilterFactory implemen
     wordFiles = get(args, PROTECTED_TOKENS);
     types = get(args, TYPES);
     this.flags = flags;
+    this.adjustOffsets = getBoolean(args, OFFSETS, true);
     if (!args.isEmpty()) {
       throw new IllegalArgumentException("Unknown parameters: " + args);
     }
@@ -117,7 +120,7 @@ public class WordDelimiterGraphFilterFactory extends TokenFilterFactory implemen
 
   @Override
   public TokenFilter create(TokenStream input) {
-    return new WordDelimiterGraphFilter(input, typeTable == null ? WordDelimiterIterator.DEFAULT_WORD_DELIM_TABLE : typeTable,
+    return new WordDelimiterGraphFilter(input, adjustOffsets, typeTable == null ? WordDelimiterIterator.DEFAULT_WORD_DELIM_TABLE : typeTable,
                                         flags, protectedWords);
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/75a053dd/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterGraphFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterGraphFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterGraphFilter.java
index 65d3b02..e3f3f65 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterGraphFilter.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterGraphFilter.java
@@ -64,7 +64,8 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
     int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
     // test that subwords and catenated subwords have
     // the correct offsets.
-    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("foo-bar", 5, 12)), DEFAULT_WORD_DELIM_TABLE, flags, null);
+    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("foo-bar", 5, 12)),
+        true, DEFAULT_WORD_DELIM_TABLE, flags, null);
 
     assertTokenStreamContents(wdf, 
                               new String[] { "foobar", "foo", "bar" },
@@ -72,7 +73,7 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
                               new int[] { 12, 8, 12 });
 
     // with illegal offsets:
-    wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("foo-bar", 5, 6)), DEFAULT_WORD_DELIM_TABLE, flags, null);
+    wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("foo-bar", 5, 6)), true, DEFAULT_WORD_DELIM_TABLE, flags, null);
     assertTokenStreamContents(wdf,
                               new String[] { "foobar", "foo", "bar" },
                               new int[] { 5, 5, 5 },
@@ -81,7 +82,8 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
   
   public void testOffsetChange() throws Exception {
     int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
-    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("übelkeit)", 7, 16)), DEFAULT_WORD_DELIM_TABLE, flags, null);
+    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("übelkeit)", 7, 16)),
+        true, DEFAULT_WORD_DELIM_TABLE, flags, null);
     
     assertTokenStreamContents(wdf,
         new String[] { "übelkeit" },
@@ -91,7 +93,8 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
   
   public void testOffsetChange2() throws Exception {
     int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
-    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("(übelkeit", 7, 17)), DEFAULT_WORD_DELIM_TABLE, flags, null);
+    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("(übelkeit", 7, 17)),
+        true, DEFAULT_WORD_DELIM_TABLE, flags, null);
     // illegal offsets:
     assertTokenStreamContents(wdf,
                               new String[] { "übelkeit" },
@@ -101,7 +104,8 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
   
   public void testOffsetChange3() throws Exception {
     int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
-    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("(übelkeit", 7, 16)), DEFAULT_WORD_DELIM_TABLE, flags, null);
+    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("(übelkeit", 7, 16)),
+        true, DEFAULT_WORD_DELIM_TABLE, flags, null);
     assertTokenStreamContents(wdf,
                               new String[] { "übelkeit" },
                               new int[] { 8 },
@@ -110,7 +114,8 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
   
   public void testOffsetChange4() throws Exception {
     int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
-    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("(foo,bar)", 7, 16)), DEFAULT_WORD_DELIM_TABLE, flags, null);
+    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("(foo,bar)", 7, 16)),
+        true, DEFAULT_WORD_DELIM_TABLE, flags, null);
     
     assertTokenStreamContents(wdf,
         new String[] { "foobar", "foo", "bar"},
@@ -120,7 +125,7 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
 
   public void doSplit(final String input, String... output) throws Exception {
     int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
-    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(keywordMockTokenizer(input),
+    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(keywordMockTokenizer(input), false,
         WordDelimiterIterator.DEFAULT_WORD_DELIM_TABLE, flags, null);
     
     assertTokenStreamContents(wdf, output);
@@ -182,7 +187,7 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
     // the correct offsets.
     Token token = new Token("foo-bar", 5, 12);
     token.setType("mytype");
-    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(token), DEFAULT_WORD_DELIM_TABLE, flags, null);
+    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(token), flags, null);
 
     assertTokenStreamContents(wdf, 
                               new String[] {"foobar", "foo", "bar"},
@@ -235,7 +240,7 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
       public TokenStreamComponents createComponents(String field) {
         Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
         return new TokenStreamComponents(tokenizer, new WordDelimiterGraphFilter(
-            tokenizer,
+            tokenizer, true, DEFAULT_WORD_DELIM_TABLE,
             flags, protWords));
       }
     };
@@ -272,7 +277,7 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
       public TokenStreamComponents createComponents(String field) {
         Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
         return new TokenStreamComponents(tokenizer, new WordDelimiterGraphFilter(
-            new LargePosIncTokenFilter(tokenizer),
+            new LargePosIncTokenFilter(tokenizer), true, DEFAULT_WORD_DELIM_TABLE,
             flags, protWords));
       }
     };
@@ -317,7 +322,7 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
       public TokenStreamComponents createComponents(String field) {
         Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
         StopFilter filter = new StopFilter(tokenizer, EnglishAnalyzer.ENGLISH_STOP_WORDS_SET);
-        return new TokenStreamComponents(tokenizer, new WordDelimiterGraphFilter(filter, flags, protWords));
+        return new TokenStreamComponents(tokenizer, new WordDelimiterGraphFilter(filter, true, DEFAULT_WORD_DELIM_TABLE, flags, protWords));
       }
     };
 
@@ -350,8 +355,8 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
     assertAnalyzesTo(keywordTestAnalyzer(GENERATE_WORD_PARTS | IGNORE_KEYWORDS),
                      "abc-def klm-nop kpop",
                      new String[] {"abc", "def", "klm-nop", "kpop"},
-                     new int[]{0, 4, 8, 16},
-                     new int[]{3, 7, 15, 20},
+                     new int[]{0, 0, 8, 16},
+                     new int[]{7, 7, 15, 20},
                      null,
                      new int[]{1, 1, 1, 1},
                      null,
@@ -384,7 +389,7 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
       @Override
       public TokenStreamComponents createComponents(String field) {
         Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-        return new TokenStreamComponents(tokenizer, new WordDelimiterGraphFilter(tokenizer, flags, null));
+        return new TokenStreamComponents(tokenizer, new WordDelimiterGraphFilter(tokenizer, true, DEFAULT_WORD_DELIM_TABLE, flags, null));
       }
     };
     
@@ -414,8 +419,8 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
     
     assertAnalyzesTo(a, "abc-def-123-456", 
                      new String[] { "abcdef123456", "abc-def-123-456", "abcdef", "abc", "def", "123456", "123", "456" }, 
-                     new int[] { 0, 0, 0, 0, 4, 8, 8, 12 }, 
-                     new int[] { 15, 15, 7, 3, 7, 15, 11, 15 },
+                     new int[] { 0, 0, 0, 0, 0, 0, 0, 0 },
+                     new int[] { 15, 15, 15, 15, 15, 15, 15, 15 },
                      null,
                      new int[] { 1, 0, 0, 0, 1, 1, 0, 1 },
                      null,
@@ -954,7 +959,8 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
   }
 
   public void testEmptyString() throws Exception {
-    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("", 0, 0)), DEFAULT_WORD_DELIM_TABLE, GENERATE_WORD_PARTS | CATENATE_ALL | PRESERVE_ORIGINAL, null);
+    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("", 0, 0)),
+        GENERATE_WORD_PARTS | CATENATE_ALL | PRESERVE_ORIGINAL, null);
     wdf.reset();
     assertTrue(wdf.incrementToken());
     assertFalse(wdf.incrementToken());
@@ -967,7 +973,7 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
                                                new Token("foo-bar", 0, 7));
 
     CharArraySet protectedWords = new CharArraySet(new HashSet<>(Arrays.asList("foo17-BAR")), true);
-    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(tokens, DEFAULT_WORD_DELIM_TABLE, GENERATE_WORD_PARTS | PRESERVE_ORIGINAL | CATENATE_ALL, protectedWords);
+    WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(tokens, GENERATE_WORD_PARTS | PRESERVE_ORIGINAL | CATENATE_ALL, protectedWords);
     assertGraphStrings(wdf,
                        "foo17-bar foo bar",
                        "foo17-bar foo-bar",