You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2018/03/21 15:07:05 UTC

[1/2] lucene-solr:master: LUCENE-8202: Add FixedShingleFilter

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x 2539578cb -> 230a77ce3
  refs/heads/master d4e69c5cd -> fac84c01c


LUCENE-8202: Add FixedShingleFilter


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

Branch: refs/heads/master
Commit: fac84c01c84b3693a8c1251ae77f349c38497e06
Parents: d4e69c5
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Mar 21 10:35:28 2018 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Wed Mar 21 13:45:03 2018 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../analysis/shingle/FixedShingleFilter.java    | 294 +++++++++++++++++++
 .../shingle/FixedShingleFilterFactory.java      |  52 ++++
 ...ache.lucene.analysis.util.TokenFilterFactory |   1 +
 .../shingle/FixedShingleFilterTest.java         | 200 +++++++++++++
 .../analysis/BaseTokenStreamTestCase.java       |   4 +
 .../java/org/apache/lucene/analysis/Token.java  |   7 +
 7 files changed, 561 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fac84c01/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index dd2a57c..c2cbc07 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -106,6 +106,9 @@ New Features
 * LUCENE-8197: A new FeatureField makes it easy and efficient to integrate
   static relevance signals into the final score. (Adrien Grand, Robert Muir)
 
+* LUCENE-8202: Add a FixedShingleFilter (Alan Woodward, Adrien Grand, Jim
+  Ferenczi)
+
 Other
 
 * LUCENE-8214: Improve selection of testPoint for GeoComplexPolygon.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fac84c01/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilter.java
new file mode 100644
index 0000000..a223cd8
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilter.java
@@ -0,0 +1,294 @@
+/*
+ * 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.shingle;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Deque;
+
+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.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
+import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
+import org.apache.lucene.util.AttributeSource;
+
+/**
+ * A FixedShingleFilter constructs shingles (token n-grams) from a token stream.
+ * In other words, it creates combinations of tokens as a single token.
+ *
+ * Unlike the {@link ShingleFilter}, FixedShingleFilter only emits shingles of a
+ * fixed size, and never emits unigrams, even at the end of a TokenStream. In
+ * addition, if the filter encounters stacked tokens (eg synonyms), then it will
+ * output stacked shingles
+ *
+ * For example, the sentence "please divide this sentence into shingles"
+ * might be tokenized into shingles "please divide", "divide this",
+ * "this sentence", "sentence into", and "into shingles".
+ *
+ * This filter handles position increments &gt; 1 by inserting filler tokens
+ * (tokens with termtext "_").
+ *
+ * @lucene.experimental
+ */
+public final class FixedShingleFilter extends TokenFilter {
+
+  private final Deque<Token> tokenPool = new ArrayDeque<>();
+
+  private final int shingleSize;
+  private final String tokenSeparator;
+  private final Token gapToken = new Token(new AttributeSource());
+  private final Token endToken = new Token(new AttributeSource());
+
+  private final PositionIncrementAttribute incAtt = addAttribute(PositionIncrementAttribute.class);
+  private final PositionLengthAttribute posLenAtt = addAttribute(PositionLengthAttribute.class);
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
+
+  private Token[] currentShingleTokens;
+  private boolean inputStreamExhausted = false;
+
+  public FixedShingleFilter(TokenStream input, int shingleSize) {
+    this(input, shingleSize, " ", "_");
+  }
+
+  public FixedShingleFilter(TokenStream input, int shingleSize, String tokenSeparator, String fillerToken) {
+    super(input);
+    this.shingleSize = shingleSize;
+    this.tokenSeparator = tokenSeparator;
+
+    this.gapToken.termAtt.setEmpty().append(fillerToken);
+
+    this.currentShingleTokens = new Token[shingleSize];
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+    int posInc = 0;
+    if (nextShingle() == false) {
+      Token nextRoot = nextTokenInStream(currentShingleTokens[0]);
+      if (nextRoot == endToken)
+        return false;
+      recycleToken(currentShingleTokens[0]);
+      if (resetShingleRoot(nextRoot) == false) {
+        return false;
+      }
+      posInc = currentShingleTokens[0].posInc();
+    }
+    clearAttributes();
+    incAtt.setPositionIncrement(posInc);
+    offsetAtt.setOffset(currentShingleTokens[0].startOffset(), lastTokenInShingle().endOffset());
+    termAtt.setEmpty();
+    termAtt.append(currentShingleTokens[0].term());
+    typeAtt.setType("shingle");
+    posLenAtt.setPositionLength(shingleSize);
+    for (int i = 1; i < shingleSize; i++) {
+      termAtt.append(tokenSeparator).append(currentShingleTokens[i].term());
+    }
+    return true;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    this.tokenPool.clear();
+    this.currentShingleTokens[0] = null;
+    this.inputStreamExhausted = false;
+  }
+
+  @Override
+  public void end() throws IOException {
+    if (inputStreamExhausted == false) {
+      finishInnerStream();
+    }
+    clearAttributes();
+    this.offsetAtt.setOffset(0, endToken.endOffset());
+  }
+
+  private void finishInnerStream() throws IOException {
+    input.end();
+    inputStreamExhausted = true;
+    // check for gaps at the end of the tokenstream
+    endToken.posIncAtt.setPositionIncrement(this.incAtt.getPositionIncrement());
+    OffsetAttribute inputOffsets = input.getAttribute(OffsetAttribute.class);
+    endToken.offsetAtt.setOffset(inputOffsets.startOffset(), inputOffsets.endOffset());
+  }
+
+  private Token lastTokenInShingle() {
+    int lastTokenIndex = shingleSize - 1;
+    while (currentShingleTokens[lastTokenIndex] == gapToken) {
+      lastTokenIndex--;
+    }
+    return currentShingleTokens[lastTokenIndex];
+  }
+
+  private boolean resetShingleRoot(Token token) throws IOException {
+    this.currentShingleTokens[0] = token;
+    for (int i = 1; i < shingleSize; i++) {
+      Token current = nextTokenInGraph(this.currentShingleTokens[i - 1]);
+      if (current == endToken) {
+        if (endToken.posInc() + i >= shingleSize) {
+          // end tokens are a special case, because their posIncs are always
+          // due to stopwords.  Therefore, we can happily append gap tokens
+          // to the end of the current shingle
+          for (int j = i; j < shingleSize; j++) {
+            this.currentShingleTokens[i] = gapToken;
+            i++;
+          }
+          return true;
+        }
+        return false;
+      }
+      if (current.posInc() > 1) {
+        // insert gaps into the shingle list
+        for (int j = 1; j < current.posInc(); j++) {
+          this.currentShingleTokens[i] = gapToken;
+          i++;
+          if (i >= shingleSize)
+            return true;
+        }
+      }
+      this.currentShingleTokens[i] = current;
+    }
+    return true;
+  }
+
+  private boolean nextShingle() throws IOException {
+    return currentShingleTokens[0] != null && advanceStack();
+  }
+
+  // check if the next token in the tokenstream is at the same position as this one
+  private boolean lastInStack(Token token) throws IOException {
+    Token next = nextTokenInStream(token);
+    return next == endToken || next.posInc() != 0;
+  }
+  
+  private boolean advanceStack() throws IOException {
+    for (int i = shingleSize - 1; i >= 1; i--) {
+      if (currentShingleTokens[i] != gapToken && lastInStack(currentShingleTokens[i]) == false) {
+        currentShingleTokens[i] = nextTokenInStream(currentShingleTokens[i]);
+        for (int j = i + 1; j < shingleSize; j++) {
+          currentShingleTokens[j] = nextTokenInGraph(currentShingleTokens[j - 1]);
+        }
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private Token newToken() {
+    Token token = tokenPool.size() == 0 ? new Token(this.cloneAttributes()) : tokenPool.removeFirst();
+    token.reset(this);
+    return token;
+  }
+
+  private void recycleToken(Token token) {
+    if (token == null)
+      return;
+    token.nextToken = null;
+    tokenPool.add(token);
+  }
+
+  // for testing
+  int instantiatedTokenCount() {
+    int tokenCount = tokenPool.size() + 1;
+    if (currentShingleTokens[0] == endToken || currentShingleTokens[0] == null)
+      return tokenCount;
+    for (Token t = currentShingleTokens[0]; t != endToken && t != null; t = t.nextToken) {
+      tokenCount++;
+    }
+    return tokenCount;
+  }
+
+  private Token nextTokenInGraph(Token token) throws IOException {
+    do {
+      token = nextTokenInStream(token);
+      if (token == endToken) {
+        return endToken;
+      }
+    } while (token.posInc() == 0);
+    return token;
+  }
+
+  private Token nextTokenInStream(Token token) throws IOException {
+    if (token != null && token.nextToken != null) {
+      return token.nextToken;
+    }
+    if (input.incrementToken() == false) {
+      finishInnerStream();
+      if (token == null) {
+        return endToken;
+      }
+      else {
+        token.nextToken = endToken;
+        return endToken;
+      }
+    }
+    if (token == null) {
+      return newToken();
+    }
+    token.nextToken = newToken();
+    return token.nextToken;
+  }
+
+  private static class Token {
+    final AttributeSource attSource;
+    final PositionIncrementAttribute posIncAtt;
+    final CharTermAttribute termAtt;
+    final OffsetAttribute offsetAtt;
+
+    Token nextToken;
+
+    Token(AttributeSource attSource) {
+      this.attSource = attSource;
+      this.posIncAtt = attSource.addAttribute(PositionIncrementAttribute.class);
+      this.termAtt = attSource.addAttribute(CharTermAttribute.class);
+      this.offsetAtt = attSource.addAttribute(OffsetAttribute.class);
+    }
+
+    int posInc() {
+      return this.posIncAtt.getPositionIncrement();
+    }
+
+    CharSequence term() {
+      return this.termAtt;
+    }
+
+    int startOffset() {
+      return this.offsetAtt.startOffset();
+    }
+
+    int endOffset() {
+      return this.offsetAtt.endOffset();
+    }
+
+    void reset(AttributeSource attSource) {
+      attSource.copyTo(this.attSource);
+      this.nextToken = null;
+    }
+
+    @Override
+    public String toString() {
+      return term() + "(" + startOffset() + "," + endOffset() + ") " + posInc();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fac84c01/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilterFactory.java
new file mode 100644
index 0000000..de824b8
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilterFactory.java
@@ -0,0 +1,52 @@
+/*
+ * 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.shingle;
+
+import java.util.Map;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.util.TokenFilterFactory;
+
+/**
+ * Factory for {@link FixedShingleFilter}
+ *
+ * Parameters are:
+ * <ul>
+ *   <li>shingleSize - how many tokens should be combined into each shingle (default: 2)
+ *   <li>tokenSeparator - how tokens should be joined together in the shingle (default: space)
+ *   <li>fillerToken - what should be added in place of stop words (default: _ )
+ * </ul>
+ */
+public class FixedShingleFilterFactory extends TokenFilterFactory {
+
+  private final int shingleSize;
+  private final String tokenSeparator;
+  private final String fillerToken;
+
+  public FixedShingleFilterFactory(Map<String, String> args) {
+    super(args);
+    this.shingleSize = getInt(args, "shingleSize", 2);
+    this.tokenSeparator = get(args, "tokenSeparator", " ");
+    this.fillerToken = get(args, "fillerToken", "_");
+  }
+
+  @Override
+  public TokenStream create(TokenStream input) {
+    return new FixedShingleFilter(input, shingleSize, tokenSeparator, fillerToken);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fac84c01/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory b/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
index 6dcc81c..b46de18 100644
--- a/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
+++ b/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
@@ -101,6 +101,7 @@ org.apache.lucene.analysis.pt.PortugueseStemFilterFactory
 org.apache.lucene.analysis.reverse.ReverseStringFilterFactory
 org.apache.lucene.analysis.ru.RussianLightStemFilterFactory
 org.apache.lucene.analysis.shingle.ShingleFilterFactory
+org.apache.lucene.analysis.shingle.FixedShingleFilterFactory
 org.apache.lucene.analysis.snowball.SnowballPorterFilterFactory
 org.apache.lucene.analysis.sr.SerbianNormalizationFilterFactory
 org.apache.lucene.analysis.standard.ClassicFilterFactory

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fac84c01/lucene/analysis/common/src/test/org/apache/lucene/analysis/shingle/FixedShingleFilterTest.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/shingle/FixedShingleFilterTest.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/shingle/FixedShingleFilterTest.java
new file mode 100644
index 0000000..d7d2825
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/shingle/FixedShingleFilterTest.java
@@ -0,0 +1,200 @@
+/*
+ * 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.shingle;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.CannedTokenStream;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+
+public class FixedShingleFilterTest extends BaseTokenStreamTestCase {
+
+  public void testBiGramFilter() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(
+        new Token("please", 0, 6),
+        new Token("divide", 7, 13),
+        new Token("this", 14, 18),
+        new Token("sentence", 19, 27),
+        new Token("into", 28, 32),
+        new Token("shingles", 33, 41)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 2),
+        new String[]{"please divide", "divide this", "this sentence", "sentence into", "into shingles"},
+        new int[]{0, 7, 14, 19, 28,},
+        new int[]{13, 18, 27, 32, 41,},
+        new String[]{"shingle", "shingle", "shingle", "shingle", "shingle",},
+        new int[]{1, 1, 1, 1, 1,},
+        new int[]{2, 2, 2, 2, 2});
+
+  }
+
+  public void testBiGramFilterWithAltSeparator() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(
+        new Token("please", 0, 6),
+        new Token("divide", 7, 13),
+        new Token("this", 14, 18),
+        new Token("sentence", 19, 27),
+        new Token("into", 28, 32),
+        new Token("shingles", 33, 41)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 2, "<SEP>", "_"),
+        new String[]{"please<SEP>divide", "divide<SEP>this", "this<SEP>sentence", "sentence<SEP>into", "into<SEP>shingles"},
+        new int[]{0, 7, 14, 19, 28},
+        new int[]{13, 18, 27, 32, 41},
+        new String[]{"shingle", "shingle", "shingle", "shingle", "shingle"},
+        new int[]{1, 1, 1, 1, 1});
+
+  }
+
+  public void testTriGramFilter() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(
+        new Token("please", 0, 6),
+        new Token("divide", 7, 13),
+        new Token("this", 14, 18),
+        new Token("sentence", 19, 27),
+        new Token("into", 28, 32),
+        new Token("shingles", 33, 41)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 3),
+        new String[]{"please divide this", "divide this sentence", "this sentence into", "sentence into shingles"});
+  }
+
+  public void testShingleSizeGreaterThanTokenstreamLength() throws IOException {
+
+    TokenStream ts = new FixedShingleFilter(new CannedTokenStream(
+        new Token("please", 0, 6),
+        new Token("divide", 7, 13)
+    ), 3);
+
+    ts.reset();
+    assertFalse(ts.incrementToken());
+
+  }
+
+  public void testWithStopwords() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(
+        new Token("please", 0, 6),
+        new Token("divide", 7, 13),
+        new Token("sentence", 2, 19, 27),
+        new Token("shingles", 2, 33, 41)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 3),
+        new String[]{"please divide _", "divide _ sentence", "sentence _ shingles"},
+        new int[]{0, 7, 19,},
+        new int[]{13, 27, 41,},
+        new String[]{"shingle", "shingle", "shingle",},
+        new int[]{1, 1, 2,});
+
+  }
+
+  public void testConsecutiveStopwords() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(
+        new Token("b", 2, 2, 3),
+        new Token("c", 4, 5),
+        new Token("d", 6, 7),
+        new Token("b", 3, 12, 13),
+        new Token("c", 14, 15)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 4),
+        new String[]{"b c d _", "c d _ _", "d _ _ b"},
+        new int[]{2, 4, 6,},
+        new int[]{7, 7, 13,},
+        new int[]{2, 1, 1,});
+  }
+
+  public void testTrailingStopwords() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(1, 7,
+        new Token("b", 0, 1),
+        new Token("c", 2, 3),
+        new Token("d", 4, 5)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 3),
+          new String[] { "b c d", "c d _" },
+          new int[] {    0,         2,    },
+          new int[] {    5,         5,    },
+          new int[] {    1,         1,    });
+
+
+  }
+
+  public void testMultipleTrailingStopwords() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(2, 9,
+        new Token("b", 0, 1),
+        new Token("c", 2, 3),
+        new Token("d", 4, 5)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 3),
+          new String[] { "b c d", "c d _", "d _ _" },
+          new int[] {    0,         2,      4 },
+          new int[] {    5,         5,      5 },
+          new int[] {    1,         1,      1 });
+  }
+
+  public void testIncomingGraphs() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(
+        new Token("b", 0, 1),
+        new Token("a", 0, 0, 1),
+        new Token("c", 2, 3),
+        new Token("b", 4, 5),
+        new Token("a", 0, 4, 5),
+        new Token("d", 6, 7)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 2),
+          new String[] { "b c", "a c", "c b", "c a", "b d", "a d" },
+          new int[] {    0,     0,     2,     2,     4,     4 },
+          new int[] {    3,     3,     5,     5,     7,     7 },
+          new int[] {    1,     0,     1,     0,     1,     0 });
+  }
+
+  public void testShinglesSpanningGraphs() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(
+        new Token("b", 0, 1),
+        new Token("a", 0, 0, 1),
+        new Token("c", 2, 3),
+        new Token("b", 4, 5),
+        new Token("a", 0, 4, 5),
+        new Token("d", 6, 7)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 3),
+          new String[] { "b c b", "b c a", "a c b", "a c a", "c b d", "c a d" },
+          new int[] {    0,        0,      0,       0,       2,        2,     },
+          new int[] {    5,        5,      5,       5,       7,        7,     },
+          new int[] {    1,        0,      0,       0,       1,        0,     });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fac84c01/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
----------------------------------------------------------------------
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 3e1e375..8c0a295 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
@@ -341,6 +341,10 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
     assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, null, null);
   }
 
+  public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int[] posLengths) throws IOException {
+    assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, posLengths, null);
+  }
+
   public static void assertTokenStreamContents(TokenStream ts, String[] output) throws IOException {
     assertTokenStreamContents(ts, output, null, null, null, null, null, null);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fac84c01/lucene/test-framework/src/java/org/apache/lucene/analysis/Token.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/analysis/Token.java b/lucene/test-framework/src/java/org/apache/lucene/analysis/Token.java
index 04b1df8..9994175 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/analysis/Token.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/analysis/Token.java
@@ -75,6 +75,13 @@ public class Token extends PackedTokenAttributeImpl implements FlagsAttribute, P
     setOffset(start, end);
   }
 
+  /** Constructs a Token with the given term text, position increment, start and end offsets */
+  public Token(CharSequence text, int posInc, int start, int end) {
+    append(text);
+    setOffset(start, end);
+    setPositionIncrement(posInc);
+  }
+
   /**
    * {@inheritDoc}
    * @see FlagsAttribute


[2/2] lucene-solr:branch_7x: LUCENE-8202: Add FixedShingleFilter

Posted by ro...@apache.org.
LUCENE-8202: Add FixedShingleFilter


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

Branch: refs/heads/branch_7x
Commit: 230a77ce38ebe6294a06aebf23d85b68223b6ec2
Parents: 2539578
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Mar 21 10:35:28 2018 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Wed Mar 21 13:45:24 2018 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../analysis/shingle/FixedShingleFilter.java    | 294 +++++++++++++++++++
 .../shingle/FixedShingleFilterFactory.java      |  52 ++++
 ...ache.lucene.analysis.util.TokenFilterFactory |   1 +
 .../shingle/FixedShingleFilterTest.java         | 200 +++++++++++++
 .../analysis/BaseTokenStreamTestCase.java       |   4 +
 .../java/org/apache/lucene/analysis/Token.java  |   7 +
 7 files changed, 561 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/230a77ce/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 048b00d..d9ad6c9 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -16,6 +16,9 @@ New Features
 * LUCENE-8197: A new FeatureField makes it easy and efficient to integrate
   static relevance signals into the final score. (Adrien Grand, Robert Muir)
 
+* LUCENE-8202: Add a FixedShingleFilter (Alan Woodward, Adrien Grand, Jim
+  Ferenczi)
+
 Other
 
 * LUCENE-8214: Improve selection of testPoint for GeoComplexPolygon.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/230a77ce/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilter.java
new file mode 100644
index 0000000..a223cd8
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilter.java
@@ -0,0 +1,294 @@
+/*
+ * 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.shingle;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Deque;
+
+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.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
+import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
+import org.apache.lucene.util.AttributeSource;
+
+/**
+ * A FixedShingleFilter constructs shingles (token n-grams) from a token stream.
+ * In other words, it creates combinations of tokens as a single token.
+ *
+ * Unlike the {@link ShingleFilter}, FixedShingleFilter only emits shingles of a
+ * fixed size, and never emits unigrams, even at the end of a TokenStream. In
+ * addition, if the filter encounters stacked tokens (eg synonyms), then it will
+ * output stacked shingles
+ *
+ * For example, the sentence "please divide this sentence into shingles"
+ * might be tokenized into shingles "please divide", "divide this",
+ * "this sentence", "sentence into", and "into shingles".
+ *
+ * This filter handles position increments &gt; 1 by inserting filler tokens
+ * (tokens with termtext "_").
+ *
+ * @lucene.experimental
+ */
+public final class FixedShingleFilter extends TokenFilter {
+
+  private final Deque<Token> tokenPool = new ArrayDeque<>();
+
+  private final int shingleSize;
+  private final String tokenSeparator;
+  private final Token gapToken = new Token(new AttributeSource());
+  private final Token endToken = new Token(new AttributeSource());
+
+  private final PositionIncrementAttribute incAtt = addAttribute(PositionIncrementAttribute.class);
+  private final PositionLengthAttribute posLenAtt = addAttribute(PositionLengthAttribute.class);
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
+
+  private Token[] currentShingleTokens;
+  private boolean inputStreamExhausted = false;
+
+  public FixedShingleFilter(TokenStream input, int shingleSize) {
+    this(input, shingleSize, " ", "_");
+  }
+
+  public FixedShingleFilter(TokenStream input, int shingleSize, String tokenSeparator, String fillerToken) {
+    super(input);
+    this.shingleSize = shingleSize;
+    this.tokenSeparator = tokenSeparator;
+
+    this.gapToken.termAtt.setEmpty().append(fillerToken);
+
+    this.currentShingleTokens = new Token[shingleSize];
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+    int posInc = 0;
+    if (nextShingle() == false) {
+      Token nextRoot = nextTokenInStream(currentShingleTokens[0]);
+      if (nextRoot == endToken)
+        return false;
+      recycleToken(currentShingleTokens[0]);
+      if (resetShingleRoot(nextRoot) == false) {
+        return false;
+      }
+      posInc = currentShingleTokens[0].posInc();
+    }
+    clearAttributes();
+    incAtt.setPositionIncrement(posInc);
+    offsetAtt.setOffset(currentShingleTokens[0].startOffset(), lastTokenInShingle().endOffset());
+    termAtt.setEmpty();
+    termAtt.append(currentShingleTokens[0].term());
+    typeAtt.setType("shingle");
+    posLenAtt.setPositionLength(shingleSize);
+    for (int i = 1; i < shingleSize; i++) {
+      termAtt.append(tokenSeparator).append(currentShingleTokens[i].term());
+    }
+    return true;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    this.tokenPool.clear();
+    this.currentShingleTokens[0] = null;
+    this.inputStreamExhausted = false;
+  }
+
+  @Override
+  public void end() throws IOException {
+    if (inputStreamExhausted == false) {
+      finishInnerStream();
+    }
+    clearAttributes();
+    this.offsetAtt.setOffset(0, endToken.endOffset());
+  }
+
+  private void finishInnerStream() throws IOException {
+    input.end();
+    inputStreamExhausted = true;
+    // check for gaps at the end of the tokenstream
+    endToken.posIncAtt.setPositionIncrement(this.incAtt.getPositionIncrement());
+    OffsetAttribute inputOffsets = input.getAttribute(OffsetAttribute.class);
+    endToken.offsetAtt.setOffset(inputOffsets.startOffset(), inputOffsets.endOffset());
+  }
+
+  private Token lastTokenInShingle() {
+    int lastTokenIndex = shingleSize - 1;
+    while (currentShingleTokens[lastTokenIndex] == gapToken) {
+      lastTokenIndex--;
+    }
+    return currentShingleTokens[lastTokenIndex];
+  }
+
+  private boolean resetShingleRoot(Token token) throws IOException {
+    this.currentShingleTokens[0] = token;
+    for (int i = 1; i < shingleSize; i++) {
+      Token current = nextTokenInGraph(this.currentShingleTokens[i - 1]);
+      if (current == endToken) {
+        if (endToken.posInc() + i >= shingleSize) {
+          // end tokens are a special case, because their posIncs are always
+          // due to stopwords.  Therefore, we can happily append gap tokens
+          // to the end of the current shingle
+          for (int j = i; j < shingleSize; j++) {
+            this.currentShingleTokens[i] = gapToken;
+            i++;
+          }
+          return true;
+        }
+        return false;
+      }
+      if (current.posInc() > 1) {
+        // insert gaps into the shingle list
+        for (int j = 1; j < current.posInc(); j++) {
+          this.currentShingleTokens[i] = gapToken;
+          i++;
+          if (i >= shingleSize)
+            return true;
+        }
+      }
+      this.currentShingleTokens[i] = current;
+    }
+    return true;
+  }
+
+  private boolean nextShingle() throws IOException {
+    return currentShingleTokens[0] != null && advanceStack();
+  }
+
+  // check if the next token in the tokenstream is at the same position as this one
+  private boolean lastInStack(Token token) throws IOException {
+    Token next = nextTokenInStream(token);
+    return next == endToken || next.posInc() != 0;
+  }
+  
+  private boolean advanceStack() throws IOException {
+    for (int i = shingleSize - 1; i >= 1; i--) {
+      if (currentShingleTokens[i] != gapToken && lastInStack(currentShingleTokens[i]) == false) {
+        currentShingleTokens[i] = nextTokenInStream(currentShingleTokens[i]);
+        for (int j = i + 1; j < shingleSize; j++) {
+          currentShingleTokens[j] = nextTokenInGraph(currentShingleTokens[j - 1]);
+        }
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private Token newToken() {
+    Token token = tokenPool.size() == 0 ? new Token(this.cloneAttributes()) : tokenPool.removeFirst();
+    token.reset(this);
+    return token;
+  }
+
+  private void recycleToken(Token token) {
+    if (token == null)
+      return;
+    token.nextToken = null;
+    tokenPool.add(token);
+  }
+
+  // for testing
+  int instantiatedTokenCount() {
+    int tokenCount = tokenPool.size() + 1;
+    if (currentShingleTokens[0] == endToken || currentShingleTokens[0] == null)
+      return tokenCount;
+    for (Token t = currentShingleTokens[0]; t != endToken && t != null; t = t.nextToken) {
+      tokenCount++;
+    }
+    return tokenCount;
+  }
+
+  private Token nextTokenInGraph(Token token) throws IOException {
+    do {
+      token = nextTokenInStream(token);
+      if (token == endToken) {
+        return endToken;
+      }
+    } while (token.posInc() == 0);
+    return token;
+  }
+
+  private Token nextTokenInStream(Token token) throws IOException {
+    if (token != null && token.nextToken != null) {
+      return token.nextToken;
+    }
+    if (input.incrementToken() == false) {
+      finishInnerStream();
+      if (token == null) {
+        return endToken;
+      }
+      else {
+        token.nextToken = endToken;
+        return endToken;
+      }
+    }
+    if (token == null) {
+      return newToken();
+    }
+    token.nextToken = newToken();
+    return token.nextToken;
+  }
+
+  private static class Token {
+    final AttributeSource attSource;
+    final PositionIncrementAttribute posIncAtt;
+    final CharTermAttribute termAtt;
+    final OffsetAttribute offsetAtt;
+
+    Token nextToken;
+
+    Token(AttributeSource attSource) {
+      this.attSource = attSource;
+      this.posIncAtt = attSource.addAttribute(PositionIncrementAttribute.class);
+      this.termAtt = attSource.addAttribute(CharTermAttribute.class);
+      this.offsetAtt = attSource.addAttribute(OffsetAttribute.class);
+    }
+
+    int posInc() {
+      return this.posIncAtt.getPositionIncrement();
+    }
+
+    CharSequence term() {
+      return this.termAtt;
+    }
+
+    int startOffset() {
+      return this.offsetAtt.startOffset();
+    }
+
+    int endOffset() {
+      return this.offsetAtt.endOffset();
+    }
+
+    void reset(AttributeSource attSource) {
+      attSource.copyTo(this.attSource);
+      this.nextToken = null;
+    }
+
+    @Override
+    public String toString() {
+      return term() + "(" + startOffset() + "," + endOffset() + ") " + posInc();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/230a77ce/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilterFactory.java
new file mode 100644
index 0000000..de824b8
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/FixedShingleFilterFactory.java
@@ -0,0 +1,52 @@
+/*
+ * 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.shingle;
+
+import java.util.Map;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.util.TokenFilterFactory;
+
+/**
+ * Factory for {@link FixedShingleFilter}
+ *
+ * Parameters are:
+ * <ul>
+ *   <li>shingleSize - how many tokens should be combined into each shingle (default: 2)
+ *   <li>tokenSeparator - how tokens should be joined together in the shingle (default: space)
+ *   <li>fillerToken - what should be added in place of stop words (default: _ )
+ * </ul>
+ */
+public class FixedShingleFilterFactory extends TokenFilterFactory {
+
+  private final int shingleSize;
+  private final String tokenSeparator;
+  private final String fillerToken;
+
+  public FixedShingleFilterFactory(Map<String, String> args) {
+    super(args);
+    this.shingleSize = getInt(args, "shingleSize", 2);
+    this.tokenSeparator = get(args, "tokenSeparator", " ");
+    this.fillerToken = get(args, "fillerToken", "_");
+  }
+
+  @Override
+  public TokenStream create(TokenStream input) {
+    return new FixedShingleFilter(input, shingleSize, tokenSeparator, fillerToken);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/230a77ce/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory b/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
index 6dcc81c..b46de18 100644
--- a/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
+++ b/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
@@ -101,6 +101,7 @@ org.apache.lucene.analysis.pt.PortugueseStemFilterFactory
 org.apache.lucene.analysis.reverse.ReverseStringFilterFactory
 org.apache.lucene.analysis.ru.RussianLightStemFilterFactory
 org.apache.lucene.analysis.shingle.ShingleFilterFactory
+org.apache.lucene.analysis.shingle.FixedShingleFilterFactory
 org.apache.lucene.analysis.snowball.SnowballPorterFilterFactory
 org.apache.lucene.analysis.sr.SerbianNormalizationFilterFactory
 org.apache.lucene.analysis.standard.ClassicFilterFactory

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/230a77ce/lucene/analysis/common/src/test/org/apache/lucene/analysis/shingle/FixedShingleFilterTest.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/shingle/FixedShingleFilterTest.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/shingle/FixedShingleFilterTest.java
new file mode 100644
index 0000000..d7d2825
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/shingle/FixedShingleFilterTest.java
@@ -0,0 +1,200 @@
+/*
+ * 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.shingle;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.CannedTokenStream;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+
+public class FixedShingleFilterTest extends BaseTokenStreamTestCase {
+
+  public void testBiGramFilter() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(
+        new Token("please", 0, 6),
+        new Token("divide", 7, 13),
+        new Token("this", 14, 18),
+        new Token("sentence", 19, 27),
+        new Token("into", 28, 32),
+        new Token("shingles", 33, 41)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 2),
+        new String[]{"please divide", "divide this", "this sentence", "sentence into", "into shingles"},
+        new int[]{0, 7, 14, 19, 28,},
+        new int[]{13, 18, 27, 32, 41,},
+        new String[]{"shingle", "shingle", "shingle", "shingle", "shingle",},
+        new int[]{1, 1, 1, 1, 1,},
+        new int[]{2, 2, 2, 2, 2});
+
+  }
+
+  public void testBiGramFilterWithAltSeparator() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(
+        new Token("please", 0, 6),
+        new Token("divide", 7, 13),
+        new Token("this", 14, 18),
+        new Token("sentence", 19, 27),
+        new Token("into", 28, 32),
+        new Token("shingles", 33, 41)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 2, "<SEP>", "_"),
+        new String[]{"please<SEP>divide", "divide<SEP>this", "this<SEP>sentence", "sentence<SEP>into", "into<SEP>shingles"},
+        new int[]{0, 7, 14, 19, 28},
+        new int[]{13, 18, 27, 32, 41},
+        new String[]{"shingle", "shingle", "shingle", "shingle", "shingle"},
+        new int[]{1, 1, 1, 1, 1});
+
+  }
+
+  public void testTriGramFilter() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(
+        new Token("please", 0, 6),
+        new Token("divide", 7, 13),
+        new Token("this", 14, 18),
+        new Token("sentence", 19, 27),
+        new Token("into", 28, 32),
+        new Token("shingles", 33, 41)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 3),
+        new String[]{"please divide this", "divide this sentence", "this sentence into", "sentence into shingles"});
+  }
+
+  public void testShingleSizeGreaterThanTokenstreamLength() throws IOException {
+
+    TokenStream ts = new FixedShingleFilter(new CannedTokenStream(
+        new Token("please", 0, 6),
+        new Token("divide", 7, 13)
+    ), 3);
+
+    ts.reset();
+    assertFalse(ts.incrementToken());
+
+  }
+
+  public void testWithStopwords() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(
+        new Token("please", 0, 6),
+        new Token("divide", 7, 13),
+        new Token("sentence", 2, 19, 27),
+        new Token("shingles", 2, 33, 41)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 3),
+        new String[]{"please divide _", "divide _ sentence", "sentence _ shingles"},
+        new int[]{0, 7, 19,},
+        new int[]{13, 27, 41,},
+        new String[]{"shingle", "shingle", "shingle",},
+        new int[]{1, 1, 2,});
+
+  }
+
+  public void testConsecutiveStopwords() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(
+        new Token("b", 2, 2, 3),
+        new Token("c", 4, 5),
+        new Token("d", 6, 7),
+        new Token("b", 3, 12, 13),
+        new Token("c", 14, 15)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 4),
+        new String[]{"b c d _", "c d _ _", "d _ _ b"},
+        new int[]{2, 4, 6,},
+        new int[]{7, 7, 13,},
+        new int[]{2, 1, 1,});
+  }
+
+  public void testTrailingStopwords() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(1, 7,
+        new Token("b", 0, 1),
+        new Token("c", 2, 3),
+        new Token("d", 4, 5)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 3),
+          new String[] { "b c d", "c d _" },
+          new int[] {    0,         2,    },
+          new int[] {    5,         5,    },
+          new int[] {    1,         1,    });
+
+
+  }
+
+  public void testMultipleTrailingStopwords() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(2, 9,
+        new Token("b", 0, 1),
+        new Token("c", 2, 3),
+        new Token("d", 4, 5)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 3),
+          new String[] { "b c d", "c d _", "d _ _" },
+          new int[] {    0,         2,      4 },
+          new int[] {    5,         5,      5 },
+          new int[] {    1,         1,      1 });
+  }
+
+  public void testIncomingGraphs() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(
+        new Token("b", 0, 1),
+        new Token("a", 0, 0, 1),
+        new Token("c", 2, 3),
+        new Token("b", 4, 5),
+        new Token("a", 0, 4, 5),
+        new Token("d", 6, 7)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 2),
+          new String[] { "b c", "a c", "c b", "c a", "b d", "a d" },
+          new int[] {    0,     0,     2,     2,     4,     4 },
+          new int[] {    3,     3,     5,     5,     7,     7 },
+          new int[] {    1,     0,     1,     0,     1,     0 });
+  }
+
+  public void testShinglesSpanningGraphs() throws IOException {
+
+    TokenStream ts = new CannedTokenStream(
+        new Token("b", 0, 1),
+        new Token("a", 0, 0, 1),
+        new Token("c", 2, 3),
+        new Token("b", 4, 5),
+        new Token("a", 0, 4, 5),
+        new Token("d", 6, 7)
+    );
+
+    assertTokenStreamContents(new FixedShingleFilter(ts, 3),
+          new String[] { "b c b", "b c a", "a c b", "a c a", "c b d", "c a d" },
+          new int[] {    0,        0,      0,       0,       2,        2,     },
+          new int[] {    5,        5,      5,       5,       7,        7,     },
+          new int[] {    1,        0,      0,       0,       1,        0,     });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/230a77ce/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
----------------------------------------------------------------------
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 3e1e375..8c0a295 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
@@ -341,6 +341,10 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
     assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, null, null);
   }
 
+  public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int[] posLengths) throws IOException {
+    assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, posLengths, null);
+  }
+
   public static void assertTokenStreamContents(TokenStream ts, String[] output) throws IOException {
     assertTokenStreamContents(ts, output, null, null, null, null, null, null);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/230a77ce/lucene/test-framework/src/java/org/apache/lucene/analysis/Token.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/analysis/Token.java b/lucene/test-framework/src/java/org/apache/lucene/analysis/Token.java
index 04b1df8..9994175 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/analysis/Token.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/analysis/Token.java
@@ -75,6 +75,13 @@ public class Token extends PackedTokenAttributeImpl implements FlagsAttribute, P
     setOffset(start, end);
   }
 
+  /** Constructs a Token with the given term text, position increment, start and end offsets */
+  public Token(CharSequence text, int posInc, int start, int end) {
+    append(text);
+    setOffset(start, end);
+    setPositionIncrement(posInc);
+  }
+
   /**
    * {@inheritDoc}
    * @see FlagsAttribute