You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2016/12/22 20:55:07 UTC

[2/2] lucene-solr:branch_6x: LUCENE-6664: add SynonymGraphFilter for correct multi-token synonym handling

LUCENE-6664: add SynonymGraphFilter for correct multi-token synonym handling


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

Branch: refs/heads/branch_6x
Commit: 68db0334089b3cb052d660d143d06aaedd7b922c
Parents: 6c1f085
Author: Mike McCandless <mi...@apache.org>
Authored: Thu Dec 22 15:39:17 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Thu Dec 22 15:39:41 2016 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |    8 +
 .../analysis/synonym/FlattenGraphFilter.java    |  424 ++++
 .../synonym/FlattenGraphFilterFactory.java      |   44 +
 .../lucene/analysis/synonym/SynonymFilter.java  |    4 +
 .../analysis/synonym/SynonymFilterFactory.java  |    4 +
 .../analysis/synonym/SynonymGraphFilter.java    |  586 ++++++
 .../synonym/SynonymGraphFilterFactory.java      |  204 ++
 .../lucene/analysis/synonym/SynonymMap.java     |    7 +-
 .../lucene/analysis/util/CharTokenizer.java     |    6 +-
 ...ache.lucene.analysis.util.TokenFilterFactory |    2 +
 .../miscellaneous/TestWordDelimiterFilter.java  |   46 +-
 .../synonym/TestFlattenGraphFilter.java         |  284 +++
 .../synonym/TestSynonymGraphFilter.java         | 1956 ++++++++++++++++++
 .../apache/lucene/util/automaton/Automaton.java |    5 +-
 .../lucene/util/automaton/Operations.java       |    6 +-
 .../apache/lucene/util/automaton/StatePair.java |    4 +-
 .../analysis/BaseTokenStreamTestCase.java       |   37 +-
 .../apache/lucene/analysis/MockTokenizer.java   |    3 +-
 18 files changed, 3594 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/68db0334/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 2c2908d..e9a0b85 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -23,6 +23,14 @@ New features
   make it simpler to execute drill down when drill sideways counts are
   not needed (Emmanuel Keller via Mike McCandless)
 
+* LUCENE-6664: A new SynonymGraphFilter outputs a correct graph
+  structure for multi-token synonyms, separating out a
+  FlattenGraphFilter that is hardwired into the current
+  SynonymFilter.  This finally makes it possible to implement
+  correct multi-token synonyms at search time.  See
+  http://blog.mikemccandless.com/2012/04/lucenes-tokenstreams-are-actually.html
+  for details. (Mike McCandless)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/68db0334/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/FlattenGraphFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/FlattenGraphFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/FlattenGraphFilter.java
new file mode 100644
index 0000000..7ede190
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/FlattenGraphFilter.java
@@ -0,0 +1,424 @@
+/*
+ * 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.synonym;
+
+/**
+ * This filter "casts" token graphs down into a "flat" form,
+ * for indexing.   This is an inherently lossy process: nodes (positions)
+ * along side paths are forcefully merged.
+ *
+ * <p>In general this means the output graph will accept token sequences
+ * that the input graph did not accept, and will also fail to accept
+ * token sequences that the input graph did accept.
+ *
+ * <p>This is only necessary at indexing time because Lucene cannot yet index
+ * an arbitrary token graph.  At search time there are better options, e.g.
+ * the experimental <code>TermAutomatonQuery</code> in sandbox.
+ *
+ * @lucene.experimental
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+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.util.AttributeSource;
+import org.apache.lucene.util.RollingBuffer;
+
+/**
+ * Converts an incoming graph token stream, such as one from
+ * {@link SynonymGraphFilter}, into a flat form so that
+ * all nodes form a single linear chain with no side paths.  Every
+ * path through the graph touches every node.
+ *
+ * <p>If the graph was not already flat to start, this
+ * is likely a lossy process, i.e. it will often cause the 
+ * graph to accept token sequences it should not, and to
+ * reject token sequences it should not.
+ *
+ * <p>However, when applying synonyms during indexing, this
+ * is necessary because Lucene already does not index a graph 
+ * and so the indexing process is already lossy
+ * (it ignores the {@link PositionLengthAttribute}).
+ *
+ * @lucene.experimental
+ */
+public final class FlattenGraphFilter extends TokenFilter {
+
+  /** Holds all tokens leaving a given input position. */
+  private final static class InputNode implements RollingBuffer.Resettable {
+    private final List<AttributeSource.State> tokens = new ArrayList<>();
+
+    /** Our input node, or -1 if we haven't been assigned yet */
+    int node = -1;
+
+    /** Maximum to input node for all tokens leaving here; we use this
+     *  to know when we can freeze. */
+    int maxToNode = -1;
+
+    /** Where we currently map to; this changes (can only
+     *  increase as we see more input tokens), until we are finished
+     *  with this position. */
+    int outputNode = -1;
+
+    /** Which token (index into {@link #tokens}) we will next output. */
+    int nextOut;
+
+    @Override
+    public void reset() {
+      tokens.clear();
+      node = -1;
+      outputNode = -1;
+      maxToNode = -1;
+      nextOut = 0;
+    }
+  }
+
+  /** Gathers up merged input positions into a single output position,
+   *  only for the current "frontier" of nodes we've seen but can't yet
+   *  output because they are not frozen. */
+  private final static class OutputNode implements RollingBuffer.Resettable {
+    private final List<Integer> inputNodes = new ArrayList<>();
+
+    /** Node ID for this output, or -1 if we haven't been assigned yet. */
+    int node = -1;
+
+    /** Which input node (index into {@link #inputNodes}) we will next output. */
+    int nextOut;
+    
+    /** Start offset of tokens leaving this node. */
+    int startOffset = -1;
+
+    /** End offset of tokens arriving to this node. */
+    int endOffset = -1;
+
+    @Override
+    public void reset() {
+      inputNodes.clear();
+      node = -1;
+      nextOut = 0;
+      startOffset = -1;
+      endOffset = -1;
+    }
+  }
+
+  private final RollingBuffer<InputNode> inputNodes = new RollingBuffer<InputNode>() {
+    @Override
+    protected InputNode newInstance() {
+      return new InputNode();
+    }
+  };
+
+  private final RollingBuffer<OutputNode> outputNodes = new RollingBuffer<OutputNode>() {
+    @Override
+    protected OutputNode newInstance() {
+      return new OutputNode();
+    }
+  };
+
+  private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+  private final PositionLengthAttribute posLenAtt = addAttribute(PositionLengthAttribute.class);
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+
+  /** Which input node the last seen token leaves from */
+  private int inputFrom;
+
+  /** We are currently releasing tokens leaving from this output node */
+  private int outputFrom;
+
+  // for debugging:
+  //private int retOutputFrom;
+
+  private boolean done;
+
+  private int lastOutputFrom;
+
+  private int finalOffset;
+
+  private int finalPosInc;
+
+  private int maxLookaheadUsed;
+
+  private int lastStartOffset;
+
+  public FlattenGraphFilter(TokenStream in) {
+    super(in);
+  }
+
+  private boolean releaseBufferedToken() {
+
+    // We only need the while loop (retry) if we have a hole (an output node that has no tokens leaving):
+    while (outputFrom < outputNodes.getMaxPos()) {
+      OutputNode output = outputNodes.get(outputFrom);
+      if (output.inputNodes.isEmpty()) {
+        // No tokens arrived to this node, which happens for the first node
+        // after a hole:
+        //System.out.println("    skip empty outputFrom=" + outputFrom);
+        outputFrom++;
+        continue;
+      }
+
+      int maxToNode = -1;
+      for(int inputNodeID : output.inputNodes) {
+        InputNode inputNode = inputNodes.get(inputNodeID);
+        assert inputNode.outputNode == outputFrom;
+        maxToNode = Math.max(maxToNode, inputNode.maxToNode);
+      }
+      //System.out.println("  release maxToNode=" + maxToNode + " vs inputFrom=" + inputFrom);
+
+      // TODO: we could shrink the frontier here somewhat if we
+      // always output posLen=1 as part of our "sausagizing":
+      if (maxToNode <= inputFrom || done) {
+        //System.out.println("  output node merged these inputs: " + output.inputNodes);
+        // These tokens are now frozen
+        assert output.nextOut < output.inputNodes.size(): "output.nextOut=" + output.nextOut + " vs output.inputNodes.size()=" + output.inputNodes.size();
+        InputNode inputNode = inputNodes.get(output.inputNodes.get(output.nextOut));
+        if (done && inputNode.tokens.size() == 0 && outputFrom >= outputNodes.getMaxPos()) {
+          return false;
+        }
+        if (inputNode.tokens.size() == 0) {
+          assert inputNode.nextOut == 0;
+          assert output.nextOut == 0;
+          // Hole dest nodes should never be merged since 1) we always
+          // assign them to a new output position, and 2) since they never
+          // have arriving tokens they cannot be pushed:
+          assert output.inputNodes.size() == 1: output.inputNodes.size();
+          outputFrom++;
+          inputNodes.freeBefore(output.inputNodes.get(0));
+          outputNodes.freeBefore(outputFrom);
+          continue;
+        }
+
+        assert inputNode.nextOut < inputNode.tokens.size();
+
+        restoreState(inputNode.tokens.get(inputNode.nextOut));
+
+        // Correct posInc
+        assert outputFrom >= lastOutputFrom;
+        posIncAtt.setPositionIncrement(outputFrom - lastOutputFrom);
+        int toInputNodeID = inputNode.node + posLenAtt.getPositionLength();
+        InputNode toInputNode = inputNodes.get(toInputNodeID);
+
+        // Correct posLen
+        assert toInputNode.outputNode > outputFrom;
+        posLenAtt.setPositionLength(toInputNode.outputNode - outputFrom);
+        lastOutputFrom = outputFrom;
+        inputNode.nextOut++;
+        //System.out.println("  ret " + this);
+
+        OutputNode outputEndNode = outputNodes.get(toInputNode.outputNode);
+
+        // Correct offsets
+
+        // This is a bit messy; we must do this so offset don't go backwards,
+        // which would otherwise happen if the replacement has more tokens
+        // than the input:
+        int startOffset = Math.max(lastStartOffset, output.startOffset);
+        offsetAtt.setOffset(startOffset, outputEndNode.endOffset);
+        lastStartOffset = startOffset;
+
+        if (inputNode.nextOut == inputNode.tokens.size()) {
+          output.nextOut++;
+          if (output.nextOut == output.inputNodes.size()) {
+            outputFrom++;
+            inputNodes.freeBefore(output.inputNodes.get(0));
+            outputNodes.freeBefore(outputFrom);
+          }
+        }
+
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    //System.out.println("    break false");
+    return false;
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+    //System.out.println("\nF.increment inputFrom=" + inputFrom + " outputFrom=" + outputFrom);
+
+    while (true) {
+      if (releaseBufferedToken()) {
+        //retOutputFrom += posIncAtt.getPositionIncrement();
+        //System.out.println("    return buffered: " + termAtt + " " + retOutputFrom + "-" + (retOutputFrom + posLenAtt.getPositionLength()));
+        //printStates();
+        return true;
+      } else if (done) {
+        //System.out.println("    done, return false");
+        return false;
+      }
+
+      if (input.incrementToken()) {
+        // Input node this token leaves from:
+        inputFrom += posIncAtt.getPositionIncrement();
+
+        int startOffset = offsetAtt.startOffset();
+        int endOffset = offsetAtt.endOffset();
+
+        // Input node this token goes to:
+        int inputTo = inputFrom + posLenAtt.getPositionLength();
+        //System.out.println("  input.inc " + termAtt + ": " + inputFrom + "-" + inputTo);
+
+        InputNode src = inputNodes.get(inputFrom);
+        if (src.node == -1) {
+          // This means the "from" node of this token was never seen as a "to" node,
+          // which should only happen if we just crossed a hole.  This is a challenging
+          // case for us because we normally rely on the full dependencies expressed
+          // by the arcs to assign outgoing node IDs.  It would be better if tokens
+          // were never dropped but instead just marked deleted with a new
+          // TermDeletedAttribute (boolean valued) ... but until that future, we have
+          // a hack here to forcefully jump the output node ID:
+          assert src.outputNode == -1;
+          src.node = inputFrom;
+
+          src.outputNode = outputNodes.getMaxPos() + 1;
+          //System.out.println("    hole: force to outputNode=" + src.outputNode);
+          OutputNode outSrc = outputNodes.get(src.outputNode);
+
+          // Not assigned yet:
+          assert outSrc.node == -1;
+          outSrc.node = src.outputNode;
+          outSrc.inputNodes.add(inputFrom);
+          outSrc.startOffset = startOffset;
+        } else {
+          OutputNode outSrc = outputNodes.get(src.outputNode);
+          if (outSrc.startOffset == -1 || startOffset > outSrc.startOffset) {
+            // "shrink wrap" the offsets so the original tokens (with most
+            // restrictive offsets) win:
+            outSrc.startOffset = Math.max(startOffset, outSrc.startOffset);
+          }
+        }
+
+        // Buffer this token:
+        src.tokens.add(captureState());
+        src.maxToNode = Math.max(src.maxToNode, inputTo);
+        maxLookaheadUsed = Math.max(maxLookaheadUsed, inputNodes.getBufferSize());
+
+        InputNode dest = inputNodes.get(inputTo);
+        if (dest.node == -1) {
+          // Common case: first time a token is arriving to this input position:
+          dest.node = inputTo;
+        }
+
+        // Always number output nodes sequentially:
+        int outputEndNode = src.outputNode + 1;
+
+        if (outputEndNode > dest.outputNode) {
+          if (dest.outputNode != -1) {
+            boolean removed = outputNodes.get(dest.outputNode).inputNodes.remove(Integer.valueOf(inputTo));
+            assert removed;
+          }
+          //System.out.println("    increase output node: " + dest.outputNode + " vs " + outputEndNode);
+          outputNodes.get(outputEndNode).inputNodes.add(inputTo);
+          dest.outputNode = outputEndNode;
+
+          // Since all we ever do is merge incoming nodes together, and then renumber
+          // the merged nodes sequentially, we should only ever assign smaller node
+          // numbers:
+          assert outputEndNode <= inputTo: "outputEndNode=" + outputEndNode + " vs inputTo=" + inputTo;
+        }
+
+        OutputNode outDest = outputNodes.get(dest.outputNode);
+        // "shrink wrap" the offsets so the original tokens (with most
+        // restrictive offsets) win:
+        if (outDest.endOffset == -1 || endOffset < outDest.endOffset) {
+          outDest.endOffset = endOffset;
+        }
+
+      } else {
+        //System.out.println("  got false from input");
+        input.end();
+        finalPosInc = posIncAtt.getPositionIncrement();
+        finalOffset = offsetAtt.endOffset();
+        done = true;
+        // Don't return false here: we need to force release any buffered tokens now
+      }
+    }
+  }
+
+  // Only for debugging:
+  /*
+  private void printStates() {
+    System.out.println("states:");
+    for(int i=outputFrom;i<outputNodes.getMaxPos();i++) {
+      OutputNode outputNode = outputNodes.get(i);
+      System.out.println("  output " + i + ": inputs " + outputNode.inputNodes);
+      for(int inputNodeID : outputNode.inputNodes) {
+        InputNode inputNode = inputNodes.get(inputNodeID);
+        assert inputNode.outputNode == i;
+      }
+    }
+  }
+  */
+
+  @Override
+  public void end() throws IOException {
+    if (done == false) {
+      super.end();
+    } else {
+      // NOTE, shady: don't call super.end, because we did already from incrementToken
+    }
+
+   clearAttributes();
+    if (done) {
+      // On exc, done is false, and we will not have set these:
+      posIncAtt.setPositionIncrement(finalPosInc);
+      offsetAtt.setOffset(finalOffset, finalOffset);
+    } else {
+      super.end();
+    }
+  }
+  
+  @Override
+  public void reset() throws IOException {
+    //System.out.println("F: reset");
+    super.reset();
+    inputFrom = -1;
+    inputNodes.reset();
+    InputNode in = inputNodes.get(0);
+    in.node = 0;
+    in.outputNode = 0;
+
+    outputNodes.reset();
+    OutputNode out = outputNodes.get(0);
+    out.node = 0;
+    out.inputNodes.add(0);
+    out.startOffset = 0;
+    outputFrom = 0;
+    //retOutputFrom = -1;
+    lastOutputFrom = -1;
+    done = false;
+    finalPosInc = -1;
+    finalOffset = -1;
+    lastStartOffset = 0;
+    maxLookaheadUsed = 0;
+  }
+
+  // for testing
+  int getMaxLookaheadUsed() {
+    return maxLookaheadUsed;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/68db0334/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/FlattenGraphFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/FlattenGraphFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/FlattenGraphFilterFactory.java
new file mode 100644
index 0000000..a6cba97
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/FlattenGraphFilterFactory.java
@@ -0,0 +1,44 @@
+/*
+ * 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.synonym;
+
+import java.util.Map;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.util.TokenFilterFactory;
+
+/** 
+ * Factory for {@link FlattenGraphFilter}. 
+ *
+ * @lucene.experimental
+ */
+public class FlattenGraphFilterFactory extends TokenFilterFactory {
+
+  /** Creates a new FlattenGraphFilterFactory */
+  public FlattenGraphFilterFactory(Map<String,String> args) {
+    super(args);
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+  
+  @Override
+  public TokenStream create(TokenStream input) {
+    return new FlattenGraphFilter(input);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/68db0334/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java
index 6a72920..29f6e1c 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilter.java
@@ -81,6 +81,9 @@ import org.apache.lucene.util.fst.FST;
  * used for parsing.  Subsequent tokens simply pass through
  * and are not parsed.  A future improvement would be to
  * allow these tokens to also be matched.</p>
+ *
+ * @deprecated Use {@link SynonymGraphFilter} instead, but be sure to also
+ * use {@link FlattenGraphFilter} at index time (not at search time) as well.
  */ 
 
 // TODO: maybe we should resolve token -> wordID then run
@@ -105,6 +108,7 @@ import org.apache.lucene.util.fst.FST;
 //
 // Another possible solution is described at http://www.cis.uni-muenchen.de/people/Schulz/Pub/dictle5.ps
 
+@Deprecated
 public final class SynonymFilter extends TokenFilter {
 
   public static final String TYPE_SYNONYM = "SYNONYM";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/68db0334/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilterFactory.java
index 8bab9a7..df10e9b 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymFilterFactory.java
@@ -72,7 +72,11 @@ import org.apache.lucene.analysis.util.TokenizerFactory;
  *   <li><code>{@link Analyzer} analyzer</code> - an analyzer used for each raw synonym</li>
  * </ul>
  * @see SolrSynonymParser SolrSynonymParser: default format
+ *
+ * @deprecated Use {@link SynonymGraphFilterFactory} instead, but be sure to also
+ * use {@link FlattenGraphFilterFactory} at index time (not at search time) as well.
  */
+@Deprecated
 public class SynonymFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
   private final boolean ignoreCase;
   private final String tokenizerFactory;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/68db0334/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymGraphFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymGraphFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymGraphFilter.java
new file mode 100644
index 0000000..3d50e08
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymGraphFilter.java
@@ -0,0 +1,586 @@
+/*
+ * 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.synonym;
+
+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.store.ByteArrayDataInput;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CharsRefBuilder;
+import org.apache.lucene.util.RollingBuffer;
+import org.apache.lucene.util.fst.FST;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+// TODO: maybe we should resolve token -> wordID then run
+// FST on wordIDs, for better perf?
+ 
+// TODO: a more efficient approach would be Aho/Corasick's
+// algorithm
+// http://en.wikipedia.org/wiki/Aho%E2%80%93Corasick_string_matching_algorithm
+// It improves over the current approach here
+// because it does not fully re-start matching at every
+// token.  For example if one pattern is "a b c x"
+// and another is "b c d" and the input is "a b c d", on
+// trying to parse "a b c x" but failing when you got to x,
+// rather than starting over again your really should
+// immediately recognize that "b c d" matches at the next
+// input.  I suspect this won't matter that much in
+// practice, but it's possible on some set of synonyms it
+// will.  We'd have to modify Aho/Corasick to enforce our
+// conflict resolving (eg greedy matching) because that algo
+// finds all matches.  This really amounts to adding a .*
+// closure to the FST and then determinizing it.
+//
+// Another possible solution is described at http://www.cis.uni-muenchen.de/people/Schulz/Pub/dictle5.ps
+
+/** Applies single- or multi-token synonyms from a {@link SynonymMap}
+ *  to an incoming {@link TokenStream}, producing a fully correct graph
+ *  output.  This is a replacement for {@link SynonymFilter}, which produces
+ *  incorrect graphs for multi-token synonyms.
+ *
+ *  <p>However, if you use this during indexing, you must follow it with
+ *  {@link FlattenGraphFilter} to squash tokens on top of one another
+ *  like {@link SynonymFilter}, because the indexer can't directly
+ *  consume a graph.  To get fully correct positional queries when your
+ *  synonym replacements are multiple tokens, you should instead apply
+ *  synonyms using this {@code TokenFilter} at query time and translate
+ *  the resulting graph to a {@code TermAutomatonQuery} e.g. using
+ *  {@code TokenStreamToTermAutomatonQuery}.
+ *
+ *  <p><b>NOTE</b>: this cannot consume an incoming graph; results will
+ *  be undefined.
+ *
+ *  @lucene.experimental */
+
+public final class SynonymGraphFilter extends TokenFilter {
+
+  public static final String TYPE_SYNONYM = "SYNONYM";
+
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
+  private final PositionLengthAttribute posLenAtt = addAttribute(PositionLengthAttribute.class);
+  private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+
+  private final SynonymMap synonyms;
+  private final boolean ignoreCase;
+
+  private final FST<BytesRef> fst;
+
+  private final FST.BytesReader fstReader;
+  private final FST.Arc<BytesRef> scratchArc;
+  private final ByteArrayDataInput bytesReader = new ByteArrayDataInput();
+  private final BytesRef scratchBytes = new BytesRef();
+  private final CharsRefBuilder scratchChars = new CharsRefBuilder();
+  private final LinkedList<BufferedOutputToken> outputBuffer = new LinkedList<>();
+
+  private int nextNodeOut;
+  private int lastNodeOut;
+  private int maxLookaheadUsed;
+
+  // For testing:
+  private int captureCount;
+
+  private boolean liveToken;
+
+  // Start/end offset of the current match:
+  private int matchStartOffset;
+  private int matchEndOffset;
+
+  // True once the input TokenStream is exhausted:
+  private boolean finished;
+
+  private int lookaheadNextRead;
+  private int lookaheadNextWrite;
+
+  private RollingBuffer<BufferedInputToken> lookahead = new RollingBuffer<BufferedInputToken>() {
+    @Override
+    protected BufferedInputToken newInstance() {
+      return new BufferedInputToken();
+    }
+  };
+
+  static class BufferedInputToken implements RollingBuffer.Resettable {
+    final CharsRefBuilder term = new CharsRefBuilder();
+    AttributeSource.State state;
+    int startOffset = -1;
+    int endOffset = -1;
+
+    @Override
+    public void reset() {
+      state = null;
+      term.clear();
+
+      // Intentionally invalid to ferret out bugs:
+      startOffset = -1;
+      endOffset = -1;
+    }
+  }
+
+  static class BufferedOutputToken {
+    final String term;
+
+    // Non-null if this was an incoming token:
+    final State state;
+
+    final int startNode;
+    final int endNode;
+
+    public BufferedOutputToken(State state, String term, int startNode, int endNode) {
+      this.state = state;
+      this.term = term;
+      this.startNode = startNode;
+      this.endNode = endNode;
+    }
+  }
+
+  public SynonymGraphFilter(TokenStream input, SynonymMap synonyms, boolean ignoreCase) {
+    super(input);
+    this.synonyms = synonyms;
+    this.fst = synonyms.fst;
+    if (fst == null) {
+      throw new IllegalArgumentException("fst must be non-null");
+    }
+    this.fstReader = fst.getBytesReader();
+    scratchArc = new FST.Arc<>();
+    this.ignoreCase = ignoreCase;
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+    //System.out.println("\nS: incrToken lastNodeOut=" + lastNodeOut + " nextNodeOut=" + nextNodeOut);
+
+    assert lastNodeOut <= nextNodeOut;
+      
+    if (outputBuffer.isEmpty() == false) {
+      // We still have pending outputs from a prior synonym match:
+      releaseBufferedToken();
+      //System.out.println("  syn: ret buffered=" + this);
+      assert liveToken == false;
+      return true;
+    }
+
+    // Try to parse a new synonym match at the current token:
+
+    if (parse()) {
+      // A new match was found:
+      releaseBufferedToken();
+      //System.out.println("  syn: after parse, ret buffered=" + this);
+      assert liveToken == false;
+      return true;
+    }
+
+    if (lookaheadNextRead == lookaheadNextWrite) {
+
+      // Fast path: parse pulled one token, but it didn't match
+      // the start for any synonym, so we now return it "live" w/o having
+      // cloned all of its atts:
+      if (finished) {
+        //System.out.println("  syn: ret END");
+        return false;
+      }
+
+      assert liveToken;
+      liveToken = false;
+
+      // NOTE: no need to change posInc since it's relative, i.e. whatever
+      // node our output is upto will just increase by the incoming posInc.
+      // We also don't need to change posLen, but only because we cannot
+      // consume a graph, so the incoming token can never span a future
+      // synonym match.
+
+    } else {
+      // We still have buffered lookahead tokens from a previous
+      // parse attempt that required lookahead; just replay them now:
+      //System.out.println("  restore buffer");
+      assert lookaheadNextRead < lookaheadNextWrite: "read=" + lookaheadNextRead + " write=" + lookaheadNextWrite;
+      BufferedInputToken token = lookahead.get(lookaheadNextRead);
+      lookaheadNextRead++;
+
+      restoreState(token.state);
+
+      lookahead.freeBefore(lookaheadNextRead);
+
+      //System.out.println("  after restore offset=" + offsetAtt.startOffset() + "-" + offsetAtt.endOffset());
+      assert liveToken == false;
+    }
+
+    lastNodeOut += posIncrAtt.getPositionIncrement();
+    nextNodeOut = lastNodeOut + posLenAtt.getPositionLength();
+
+    //System.out.println("  syn: ret lookahead=" + this);
+
+    return true;
+  }
+
+  private void releaseBufferedToken() throws IOException {
+    //System.out.println("  releaseBufferedToken");
+
+    BufferedOutputToken token = outputBuffer.pollFirst();
+
+    if (token.state != null) {
+      // This is an original input token (keepOrig=true case):
+      //System.out.println("    hasState");
+      restoreState(token.state);
+      //System.out.println("    startOffset=" + offsetAtt.startOffset() + " endOffset=" + offsetAtt.endOffset());
+    } else {
+      clearAttributes();
+      //System.out.println("    no state");
+      termAtt.append(token.term);
+
+      // We better have a match already:
+      assert matchStartOffset != -1;
+
+      offsetAtt.setOffset(matchStartOffset, matchEndOffset);
+      //System.out.println("    startOffset=" + matchStartOffset + " endOffset=" + matchEndOffset);
+      typeAtt.setType(TYPE_SYNONYM);
+    }
+
+    //System.out.println("    lastNodeOut=" + lastNodeOut);
+    //System.out.println("    term=" + termAtt);
+
+    posIncrAtt.setPositionIncrement(token.startNode - lastNodeOut);
+    lastNodeOut = token.startNode;
+    posLenAtt.setPositionLength(token.endNode - token.startNode);
+  }
+
+  /** Scans the next input token(s) to see if a synonym matches.  Returns true
+   *  if a match was found. */
+  private boolean parse() throws IOException {
+    // System.out.println(Thread.currentThread().getName() + ": S: parse: " + System.identityHashCode(this));
+
+    // Holds the longest match we've seen so far:
+    BytesRef matchOutput = null;
+    int matchInputLength = 0;
+
+    BytesRef pendingOutput = fst.outputs.getNoOutput();
+    fst.getFirstArc(scratchArc);
+
+    assert scratchArc.output == fst.outputs.getNoOutput();
+
+    // How many tokens in the current match
+    int matchLength = 0;
+    boolean doFinalCapture = false;
+
+    int lookaheadUpto = lookaheadNextRead;
+    matchStartOffset = -1;
+
+    byToken:
+    while (true) {
+      //System.out.println("  cycle lookaheadUpto=" + lookaheadUpto + " maxPos=" + lookahead.getMaxPos());
+      
+      // Pull next token's chars:
+      final char[] buffer;
+      final int bufferLen;
+      final int inputEndOffset;
+
+      if (lookaheadUpto <= lookahead.getMaxPos()) {
+        // Still in our lookahead buffer
+        BufferedInputToken token = lookahead.get(lookaheadUpto);
+        lookaheadUpto++;
+        buffer = token.term.chars();
+        bufferLen = token.term.length();
+        inputEndOffset = token.endOffset;
+        //System.out.println("    use buffer now max=" + lookahead.getMaxPos());
+        if (matchStartOffset == -1) {
+          matchStartOffset = token.startOffset;
+        }
+      } else {
+
+        // We used up our lookahead buffer of input tokens
+        // -- pull next real input token:
+
+        assert finished || liveToken == false;
+
+        if (finished) {
+          //System.out.println("    break: finished");
+          break;
+        } else if (input.incrementToken()) {
+          //System.out.println("    input.incrToken");
+          liveToken = true;
+          buffer = termAtt.buffer();
+          bufferLen = termAtt.length();
+          if (matchStartOffset == -1) {
+            matchStartOffset = offsetAtt.startOffset();
+          }
+          inputEndOffset = offsetAtt.endOffset();
+
+          lookaheadUpto++;
+        } else {
+          // No more input tokens
+          finished = true;
+          //System.out.println("    break: now set finished");
+          break;
+        }
+      }
+
+      matchLength++;
+      //System.out.println("    cycle term=" + new String(buffer, 0, bufferLen));
+
+      // Run each char in this token through the FST:
+      int bufUpto = 0;
+      while (bufUpto < bufferLen) {
+        final int codePoint = Character.codePointAt(buffer, bufUpto, bufferLen);
+        if (fst.findTargetArc(ignoreCase ? Character.toLowerCase(codePoint) : codePoint, scratchArc, scratchArc, fstReader) == null) {
+          break byToken;
+        }
+
+        // Accum the output
+        pendingOutput = fst.outputs.add(pendingOutput, scratchArc.output);
+        bufUpto += Character.charCount(codePoint);
+      }
+
+      assert bufUpto == bufferLen;
+
+      // OK, entire token matched; now see if this is a final
+      // state in the FST (a match):
+      if (scratchArc.isFinal()) {
+        matchOutput = fst.outputs.add(pendingOutput, scratchArc.nextFinalOutput);
+        matchInputLength = matchLength;
+        matchEndOffset = inputEndOffset;
+        //System.out.println("    ** match");
+      }
+
+      // See if the FST can continue matching (ie, needs to
+      // see the next input token):
+      if (fst.findTargetArc(SynonymMap.WORD_SEPARATOR, scratchArc, scratchArc, fstReader) == null) {
+        // No further rules can match here; we're done
+        // searching for matching rules starting at the
+        // current input position.
+        break;
+      } else {
+        // More matching is possible -- accum the output (if
+        // any) of the WORD_SEP arc:
+        pendingOutput = fst.outputs.add(pendingOutput, scratchArc.output);
+        doFinalCapture = true;
+        if (liveToken) {
+          capture();
+        }
+      }
+    }
+
+    if (doFinalCapture && liveToken && finished == false) {
+      // Must capture the final token if we captured any prior tokens:
+      capture();
+    }
+
+    if (matchOutput != null) {
+
+      if (liveToken) {
+        // Single input token synonym; we must buffer it now:
+        capture();
+      }
+
+      // There is a match!
+      bufferOutputTokens(matchOutput, matchInputLength);
+      lookaheadNextRead += matchInputLength;
+      //System.out.println("  precmatch; set lookaheadNextRead=" + lookaheadNextRead + " now max=" + lookahead.getMaxPos());
+      lookahead.freeBefore(lookaheadNextRead);
+      //System.out.println("  match; set lookaheadNextRead=" + lookaheadNextRead + " now max=" + lookahead.getMaxPos());
+      return true;
+    } else {
+      //System.out.println("  no match; lookaheadNextRead=" + lookaheadNextRead);
+      return false;
+    }
+
+    //System.out.println("  parse done inputSkipCount=" + inputSkipCount + " nextRead=" + nextRead + " nextWrite=" + nextWrite);
+  }
+
+  /** Expands the output graph into the necessary tokens, adding
+   *  synonyms as side paths parallel to the input tokens, and
+   *  buffers them in the output token buffer. */
+  private void bufferOutputTokens(BytesRef bytes, int matchInputLength) {
+    bytesReader.reset(bytes.bytes, bytes.offset, bytes.length);
+
+    final int code = bytesReader.readVInt();
+    final boolean keepOrig = (code & 0x1) == 0;
+    //System.out.println("  buffer: keepOrig=" + keepOrig + " matchInputLength=" + matchInputLength);
+
+    // How many nodes along all paths; we need this to assign the
+    // node ID for the final end node where all paths merge back:
+    int totalPathNodes;
+    if (keepOrig) {
+      assert matchInputLength > 0;
+      totalPathNodes = matchInputLength - 1;
+    } else {
+      totalPathNodes = 0;
+    }
+
+    // How many synonyms we will insert over this match:
+    final int count = code >>> 1;
+
+    // TODO: we could encode this instead into the FST:
+
+    // 1st pass: count how many new nodes we need
+    List<List<String>> paths = new ArrayList<>();
+    for(int outputIDX=0;outputIDX<count;outputIDX++) {
+      int wordID = bytesReader.readVInt();
+      synonyms.words.get(wordID, scratchBytes);
+      scratchChars.copyUTF8Bytes(scratchBytes);
+      int lastStart = 0;
+
+      List<String> path = new ArrayList<>();
+      paths.add(path);
+      int chEnd = scratchChars.length();
+      for(int chUpto=0; chUpto<=chEnd; chUpto++) {
+        if (chUpto == chEnd || scratchChars.charAt(chUpto) == SynonymMap.WORD_SEPARATOR) {
+          path.add(new String(scratchChars.chars(), lastStart, chUpto - lastStart));
+          lastStart = 1 + chUpto;
+        }
+      }
+
+      assert path.size() > 0;
+      totalPathNodes += path.size() - 1;
+    }
+    //System.out.println("  totalPathNodes=" + totalPathNodes);
+
+    // 2nd pass: buffer tokens for the graph fragment
+
+    // NOTE: totalPathNodes will be 0 in the case where the matched
+    // input is a single token and all outputs are also a single token
+
+    // We "spawn" a side-path for each of the outputs for this matched
+    // synonym, all ending back at this end node:
+
+    int startNode = nextNodeOut;
+
+    int endNode = startNode + totalPathNodes + 1;
+    //System.out.println("  " + paths.size() + " new side-paths");
+
+    // First, fanout all tokens departing start node for these new side paths:
+    int newNodeCount = 0;
+    for(List<String> path : paths) {
+      int pathEndNode;
+      //System.out.println("    path size=" + path.size());
+      if (path.size() == 1) {
+        // Single token output, so there are no intermediate nodes:
+        pathEndNode = endNode;
+      } else {
+        pathEndNode = nextNodeOut + newNodeCount + 1;
+        newNodeCount += path.size() - 1;
+      }
+      outputBuffer.add(new BufferedOutputToken(null, path.get(0), startNode, pathEndNode));
+    }
+
+    // We must do the original tokens last, else the offsets "go backwards":
+    if (keepOrig) {
+      BufferedInputToken token = lookahead.get(lookaheadNextRead);
+      int inputEndNode;
+      if (matchInputLength == 1) {
+        // Single token matched input, so there are no intermediate nodes:
+        inputEndNode = endNode;
+      } else {
+        inputEndNode = nextNodeOut + newNodeCount + 1;
+      }
+
+      //System.out.println("    keepOrig first token: " + token.term);
+
+      outputBuffer.add(new BufferedOutputToken(token.state, token.term.toString(), startNode, inputEndNode));
+    }
+
+    nextNodeOut = endNode;
+
+    // Do full side-path for each syn output:
+    for(int pathID=0;pathID<paths.size();pathID++) {
+      List<String> path = paths.get(pathID);
+      if (path.size() > 1) {
+        int lastNode = outputBuffer.get(pathID).endNode;
+        for(int i=1;i<path.size()-1;i++) {
+          outputBuffer.add(new BufferedOutputToken(null, path.get(i), lastNode, lastNode+1));
+          lastNode++;
+        }
+        outputBuffer.add(new BufferedOutputToken(null, path.get(path.size()-1), lastNode, endNode));
+      }
+    }
+
+    if (keepOrig && matchInputLength > 1) {
+      // Do full "side path" with the original tokens:
+      int lastNode = outputBuffer.get(paths.size()).endNode;
+      for(int i=1;i<matchInputLength-1;i++) {
+        BufferedInputToken token = lookahead.get(lookaheadNextRead + i);
+        outputBuffer.add(new BufferedOutputToken(token.state, token.term.toString(), lastNode, lastNode+1));
+        lastNode++;
+      }
+      BufferedInputToken token = lookahead.get(lookaheadNextRead + matchInputLength - 1);
+      outputBuffer.add(new BufferedOutputToken(token.state, token.term.toString(), lastNode, endNode));
+    }
+
+    /*
+    System.out.println("  after buffer: " + outputBuffer.size() + " tokens:");
+    for(BufferedOutputToken token : outputBuffer) {
+      System.out.println("    tok: " + token.term + " startNode=" + token.startNode + " endNode=" + token.endNode);
+    }
+    */
+  }
+
+  /** Buffers the current input token into lookahead buffer. */
+  private void capture() {
+    assert liveToken;
+    liveToken = false;
+    BufferedInputToken token = lookahead.get(lookaheadNextWrite);
+    lookaheadNextWrite++;
+
+    token.state = captureState();
+    token.startOffset = offsetAtt.startOffset();
+    token.endOffset = offsetAtt.endOffset();
+    assert token.term.length() == 0;
+    token.term.append(termAtt);
+
+    captureCount++;
+    maxLookaheadUsed = Math.max(maxLookaheadUsed, lookahead.getBufferSize());
+    //System.out.println("  maxLookaheadUsed=" + maxLookaheadUsed);
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    lookahead.reset();
+    lookaheadNextWrite = 0;
+    lookaheadNextRead = 0;
+    captureCount = 0;
+    lastNodeOut = -1;
+    nextNodeOut = 0;
+    matchStartOffset = -1;
+    matchEndOffset = -1;
+    finished = false;
+    liveToken = false;
+    outputBuffer.clear();
+    maxLookaheadUsed = 0;
+    //System.out.println("S: reset");
+  }
+
+  // for testing
+  int getCaptureCount() {
+    return captureCount;
+  }
+
+  // for testing
+  int getMaxLookaheadUsed() {
+    return maxLookaheadUsed;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/68db0334/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymGraphFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymGraphFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymGraphFilterFactory.java
new file mode 100644
index 0000000..91e06e4
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymGraphFilterFactory.java
@@ -0,0 +1,204 @@
+/*
+ * 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.synonym;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CodingErrorAction;
+import java.nio.charset.StandardCharsets;
+import java.text.ParseException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.core.LowerCaseFilter;
+import org.apache.lucene.analysis.core.WhitespaceTokenizer;
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
+import org.apache.lucene.analysis.util.TokenFilterFactory;
+import org.apache.lucene.analysis.util.TokenizerFactory;
+
+/**
+ * Factory for {@link SynonymGraphFilter}.
+ * <pre class="prettyprint">
+ * &lt;fieldType name="text_synonym" class="solr.TextField" positionIncrementGap="100"&gt;
+ *   &lt;analyzer&gt;
+ *     &lt;tokenizer class="solr.WhitespaceTokenizerFactory"/&gt;
+ *     &lt;filter class="solr.SynonymGraphFilterFactory" synonyms="synonyms.txt" 
+ *             format="solr" ignoreCase="false" expand="true" 
+ *             tokenizerFactory="solr.WhitespaceTokenizerFactory"
+ *             [optional tokenizer factory parameters]/&gt;
+ *   &lt;/analyzer&gt;
+ * &lt;/fieldType&gt;</pre>
+ * 
+ * <p>
+ * An optional param name prefix of "tokenizerFactory." may be used for any 
+ * init params that the SynonymGraphFilterFactory needs to pass to the specified 
+ * TokenizerFactory.  If the TokenizerFactory expects an init parameters with 
+ * the same name as an init param used by the SynonymGraphFilterFactory, the prefix 
+ * is mandatory.
+ * </p>
+ * 
+ * <p>
+ * The optional {@code format} parameter controls how the synonyms will be parsed:
+ * It supports the short names of {@code solr} for {@link SolrSynonymParser} 
+ * and {@code wordnet} for and {@link WordnetSynonymParser}, or your own 
+ * {@code SynonymMap.Parser} class name. The default is {@code solr}.
+ * A custom {@link SynonymMap.Parser} is expected to have a constructor taking:
+ * <ul>
+ *   <li><code>boolean dedup</code> - true if duplicates should be ignored, false otherwise</li>
+ *   <li><code>boolean expand</code> - true if conflation groups should be expanded, false if they are one-directional</li>
+ *   <li><code>{@link Analyzer} analyzer</code> - an analyzer used for each raw synonym</li>
+ * </ul>
+ * @see SolrSynonymParser SolrSynonymParser: default format
+ *
+ * @lucene.experimental
+ */
+public class SynonymGraphFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
+  private final boolean ignoreCase;
+  private final String tokenizerFactory;
+  private final String synonyms;
+  private final String format;
+  private final boolean expand;
+  private final String analyzerName;
+  private final Map<String, String> tokArgs = new HashMap<>();
+
+  private SynonymMap map;
+  
+  public SynonymGraphFilterFactory(Map<String,String> args) {
+    super(args);
+    ignoreCase = getBoolean(args, "ignoreCase", false);
+    synonyms = require(args, "synonyms");
+    format = get(args, "format");
+    expand = getBoolean(args, "expand", true);
+
+    analyzerName = get(args, "analyzer");
+    tokenizerFactory = get(args, "tokenizerFactory");
+    if (analyzerName != null && tokenizerFactory != null) {
+      throw new IllegalArgumentException("Analyzer and TokenizerFactory can't be specified both: " +
+                                         analyzerName + " and " + tokenizerFactory);
+    }
+
+    if (tokenizerFactory != null) {
+      tokArgs.put("luceneMatchVersion", getLuceneMatchVersion().toString());
+      for (Iterator<String> itr = args.keySet().iterator(); itr.hasNext();) {
+        String key = itr.next();
+        tokArgs.put(key.replaceAll("^tokenizerFactory\\.",""), args.get(key));
+        itr.remove();
+      }
+    }
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+  
+  @Override
+  public TokenStream create(TokenStream input) {
+    // if the fst is null, it means there's actually no synonyms... just return the original stream
+    // as there is nothing to do here.
+    return map.fst == null ? input : new SynonymGraphFilter(input, map, ignoreCase);
+  }
+
+  @Override
+  public void inform(ResourceLoader loader) throws IOException {
+    final TokenizerFactory factory = tokenizerFactory == null ? null : loadTokenizerFactory(loader, tokenizerFactory);
+    Analyzer analyzer;
+    
+    if (analyzerName != null) {
+      analyzer = loadAnalyzer(loader, analyzerName);
+    } else {
+      analyzer = new Analyzer() {
+        @Override
+        protected TokenStreamComponents createComponents(String fieldName) {
+          Tokenizer tokenizer = factory == null ? new WhitespaceTokenizer() : factory.create();
+          TokenStream stream = ignoreCase ? new LowerCaseFilter(tokenizer) : tokenizer;
+          return new TokenStreamComponents(tokenizer, stream);
+        }
+      };
+    }
+
+    try (Analyzer a = analyzer) {
+      String formatClass = format;
+      if (format == null || format.equals("solr")) {
+        formatClass = SolrSynonymParser.class.getName();
+      } else if (format.equals("wordnet")) {
+        formatClass = WordnetSynonymParser.class.getName();
+      }
+      // TODO: expose dedup as a parameter?
+      map = loadSynonyms(loader, formatClass, true, a);
+    } catch (ParseException e) {
+      throw new IOException("Error parsing synonyms file:", e);
+    }
+  }
+
+  /**
+   * Load synonyms with the given {@link SynonymMap.Parser} class.
+   */
+  protected SynonymMap loadSynonyms(ResourceLoader loader, String cname, boolean dedup, Analyzer analyzer) throws IOException, ParseException {
+    CharsetDecoder decoder = StandardCharsets.UTF_8.newDecoder()
+        .onMalformedInput(CodingErrorAction.REPORT)
+        .onUnmappableCharacter(CodingErrorAction.REPORT);
+
+    SynonymMap.Parser parser;
+    Class<? extends SynonymMap.Parser> clazz = loader.findClass(cname, SynonymMap.Parser.class);
+    try {
+      parser = clazz.getConstructor(boolean.class, boolean.class, Analyzer.class).newInstance(dedup, expand, analyzer);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    List<String> files = splitFileNames(synonyms);
+    for (String file : files) {
+      decoder.reset();
+      parser.parse(new InputStreamReader(loader.openResource(file), decoder));
+    }
+    return parser.build();
+  }
+  
+  // (there are no tests for this functionality)
+  private TokenizerFactory loadTokenizerFactory(ResourceLoader loader, String cname) throws IOException {
+    Class<? extends TokenizerFactory> clazz = loader.findClass(cname, TokenizerFactory.class);
+    try {
+      TokenizerFactory tokFactory = clazz.getConstructor(Map.class).newInstance(tokArgs);
+      if (tokFactory instanceof ResourceLoaderAware) {
+        ((ResourceLoaderAware) tokFactory).inform(loader);
+      }
+      return tokFactory;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private Analyzer loadAnalyzer(ResourceLoader loader, String cname) throws IOException {
+    Class<? extends Analyzer> clazz = loader.findClass(cname, Analyzer.class);
+    try {
+      Analyzer analyzer = clazz.getConstructor().newInstance();
+      if (analyzer instanceof ResourceLoaderAware) {
+        ((ResourceLoaderAware) analyzer).inform(loader);
+      }
+      return analyzer;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/68db0334/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java
index fc8703f..7371e23 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/SynonymMap.java
@@ -74,6 +74,11 @@ public class SynonymMap {
     private int maxHorizontalContext;
     private final boolean dedup;
 
+    /** Default constructor, passes {@code dedup=true}. */
+    public Builder() {
+      this(true);
+    }
+
     /** If dedup is true then identical rules (same input,
      *  same output) will be added only once. */
     public Builder(boolean dedup) {
@@ -109,8 +114,6 @@ public class SynonymMap {
       reuse.setLength(upto);
       return reuse.get();
     }
-    
-
 
     /** only used for asserting! */
     private boolean hasHoles(CharsRef chars) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/68db0334/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharTokenizer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharTokenizer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharTokenizer.java
index 9100345..13289be 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharTokenizer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/CharTokenizer.java
@@ -256,10 +256,12 @@ public abstract class CharTokenizer extends Tokenizer {
         }
         end += charCount;
         length += Character.toChars(normalize(c), buffer, length); // buffer it, normalized
-        if (length >= MAX_WORD_LEN) // buffer overflow! make sure to check for >= surrogate pair could break == test
+        if (length >= MAX_WORD_LEN) { // buffer overflow! make sure to check for >= surrogate pair could break == test
           break;
-      } else if (length > 0)             // at non-Letter w/ chars
+        }
+      } else if (length > 0) {           // at non-Letter w/ chars
         break;                           // return 'em
+      }
     }
 
     termAtt.setLength(length);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/68db0334/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 70120c5..73986d7 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,5 +101,7 @@ org.apache.lucene.analysis.standard.ClassicFilterFactory
 org.apache.lucene.analysis.standard.StandardFilterFactory
 org.apache.lucene.analysis.sv.SwedishLightStemFilterFactory
 org.apache.lucene.analysis.synonym.SynonymFilterFactory
+org.apache.lucene.analysis.synonym.SynonymGraphFilterFactory
+org.apache.lucene.analysis.synonym.FlattenGraphFilterFactory
 org.apache.lucene.analysis.tr.TurkishLowerCaseFilterFactory
 org.apache.lucene.analysis.util.ElisionFilterFactory

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/68db0334/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterFilter.java
index a22d9c9..580b17e 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterFilter.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterFilter.java
@@ -224,18 +224,27 @@ public class TestWordDelimiterFilter extends BaseTokenStreamTestCase {
     assertAnalyzesTo(a, "LUCENE / SOLR", new String[] { "LUCENE", "SOLR" },
         new int[] { 0, 9 },
         new int[] { 6, 13 },
-        new int[] { 1, 1 });
+        null,
+        new int[] { 1, 1 },
+        null,
+        false);
     
     /* only in this case, posInc of 2 ?! */
     assertAnalyzesTo(a, "LUCENE / solR", new String[] { "LUCENE", "sol", "solR", "R" },
         new int[] { 0, 9, 9, 12 },
         new int[] { 6, 12, 13, 13 },
-        new int[] { 1, 1, 0, 1 });
+        null,                     
+        new int[] { 1, 1, 0, 1 },
+        null,
+        false);
     
     assertAnalyzesTo(a, "LUCENE / NUTCH SOLR", new String[] { "LUCENE", "NUTCH", "SOLR" },
         new int[] { 0, 9, 15 },
         new int[] { 6, 14, 19 },
-        new int[] { 1, 1, 1 });
+        null,
+        new int[] { 1, 1, 1 },
+        null,
+        false);
     
     /* analyzer that will consume tokens with large position increments */
     Analyzer a2 = new Analyzer() {
@@ -252,24 +261,36 @@ public class TestWordDelimiterFilter extends BaseTokenStreamTestCase {
     assertAnalyzesTo(a2, "LUCENE largegap SOLR", new String[] { "LUCENE", "largegap", "SOLR" },
         new int[] { 0, 7, 16 },
         new int[] { 6, 15, 20 },
-        new int[] { 1, 10, 1 });
+        null,
+        new int[] { 1, 10, 1 },
+        null,
+        false);
     
     /* the "/" had a position increment of 10, where did it go?!?!! */
     assertAnalyzesTo(a2, "LUCENE / SOLR", new String[] { "LUCENE", "SOLR" },
         new int[] { 0, 9 },
         new int[] { 6, 13 },
-        new int[] { 1, 11 });
+        null,
+        new int[] { 1, 11 },
+        null,
+        false);
     
     /* in this case, the increment of 10 from the "/" is carried over */
     assertAnalyzesTo(a2, "LUCENE / solR", new String[] { "LUCENE", "sol", "solR", "R" },
         new int[] { 0, 9, 9, 12 },
         new int[] { 6, 12, 13, 13 },
-        new int[] { 1, 11, 0, 1 });
+        null,
+        new int[] { 1, 11, 0, 1 },
+        null,
+        false);
     
     assertAnalyzesTo(a2, "LUCENE / NUTCH SOLR", new String[] { "LUCENE", "NUTCH", "SOLR" },
         new int[] { 0, 9, 15 },
         new int[] { 6, 14, 19 },
-        new int[] { 1, 11, 1 });
+        null,
+        new int[] { 1, 11, 1 },
+        null,
+        false);
 
     Analyzer a3 = new Analyzer() {
       @Override
@@ -284,14 +305,21 @@ public class TestWordDelimiterFilter extends BaseTokenStreamTestCase {
         new String[] { "lucene", "lucenesolr", "solr" },
         new int[] { 0, 0, 7 },
         new int[] { 6, 11, 11 },
-        new int[] { 1, 0, 1 });
+        null,
+        new int[] { 1, 0, 1 },
+        null,
+        false);
 
     /* the stopword should add a gap here */
     assertAnalyzesTo(a3, "the lucene.solr", 
         new String[] { "lucene", "lucenesolr", "solr" }, 
         new int[] { 4, 4, 11 }, 
         new int[] { 10, 15, 15 },
-        new int[] { 2, 0, 1 });
+        null,
+        new int[] { 2, 0, 1 },
+        null,
+        false);
+
     IOUtils.close(a, a2, a3);
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/68db0334/lucene/analysis/common/src/test/org/apache/lucene/analysis/synonym/TestFlattenGraphFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/synonym/TestFlattenGraphFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/synonym/TestFlattenGraphFilter.java
new file mode 100644
index 0000000..d61fa96
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/synonym/TestFlattenGraphFilter.java
@@ -0,0 +1,284 @@
+/*
+ * 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.synonym;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.CannedTokenStream;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+
+public class TestFlattenGraphFilter extends BaseTokenStreamTestCase {
+  
+  private static Token token(String term, int posInc, int posLength, int startOffset, int endOffset) {
+    final Token t = new Token(term, startOffset, endOffset);
+    t.setPositionIncrement(posInc);
+    t.setPositionLength(posLength);
+    return t;
+  }
+
+  public void testSimpleMock() throws Exception {
+    Analyzer a = new Analyzer() {
+        @Override
+        protected TokenStreamComponents createComponents(String fieldName) {
+          Tokenizer tokenizer = new MockTokenizer(MockTokenizer.SIMPLE, true);
+          TokenStream ts = new FlattenGraphFilter(tokenizer);
+          return new TokenStreamComponents(tokenizer, ts);
+        }
+      };
+
+    assertAnalyzesTo(a, "wtf happened",
+                     new String[] {"wtf", "happened"},
+                     new int[]    {    0,          4},
+                     new int[]    {    3,         12},
+                     null,
+                     new int[]    {    1,          1},
+                     new int[]    {    1,          1},
+                     true);
+  }
+
+  // Make sure graph is unchanged if it's already flat
+  public void testAlreadyFlatten() throws Exception {
+    TokenStream in = new CannedTokenStream(0, 12, new Token[] {
+        token("wtf", 1, 1, 0, 3),
+        token("what", 0, 1, 0, 3),
+        token("wow", 0, 1, 0, 3),
+        token("the", 1, 1, 0, 3),
+        token("that's", 0, 1, 0, 3),
+        token("fudge", 1, 1, 0, 3),
+        token("funny", 0, 1, 0, 3),
+        token("happened", 1, 1, 4, 12)
+      });
+
+    TokenStream out = new FlattenGraphFilter(in);
+
+    // ... but on output, it's flattened to wtf/what/wow that's/the fudge/funny happened:
+    assertTokenStreamContents(out,
+                              new String[] {"wtf", "what", "wow", "the", "that's", "fudge", "funny", "happened"},
+                              new int[] {0, 0, 0, 0, 0, 0, 0, 4},
+                              new int[] {3, 3, 3, 3, 3, 3, 3, 12},
+                              new int[] {1, 0, 0, 1, 0, 1, 0, 1},
+                              new int[] {1, 1, 1, 1, 1, 1, 1, 1},
+                              12);
+  }
+
+  public void testWTF1() throws Exception {
+
+    // "wow that's funny" and "what the fudge" are separate side paths, in parallel with "wtf", on input:
+    TokenStream in = new CannedTokenStream(0, 12, new Token[] {
+        token("wtf", 1, 5, 0, 3),
+        token("what", 0, 1, 0, 3),
+        token("wow", 0, 3, 0, 3),
+        token("the", 1, 1, 0, 3),
+        token("fudge", 1, 3, 0, 3),
+        token("that's", 1, 1, 0, 3),
+        token("funny", 1, 1, 0, 3),
+        token("happened", 1, 1, 4, 12)
+      });
+
+
+    TokenStream out = new FlattenGraphFilter(in);
+
+    // ... but on output, it's flattened to wtf/what/wow that's/the fudge/funny happened:
+    assertTokenStreamContents(out,
+                              new String[] {"wtf", "what", "wow", "the", "that's", "fudge", "funny", "happened"},
+                              new int[] {0, 0, 0, 0, 0, 0, 0, 4},
+                              new int[] {3, 3, 3, 3, 3, 3, 3, 12},
+                              new int[] {1, 0, 0, 1, 0, 1, 0, 1},
+                              new int[] {3, 1, 1, 1, 1, 1, 1, 1},
+                              12);
+    
+  }
+
+  /** Same as testWTF1 except the "wtf" token comes out later */
+  public void testWTF2() throws Exception {
+
+    // "wow that's funny" and "what the fudge" are separate side paths, in parallel with "wtf", on input:
+    TokenStream in = new CannedTokenStream(0, 12, new Token[] {
+        token("what", 1, 1, 0, 3),
+        token("wow", 0, 3, 0, 3),
+        token("wtf", 0, 5, 0, 3),
+        token("the", 1, 1, 0, 3),
+        token("fudge", 1, 3, 0, 3),
+        token("that's", 1, 1, 0, 3),
+        token("funny", 1, 1, 0, 3),
+        token("happened", 1, 1, 4, 12)
+      });
+
+
+    TokenStream out = new FlattenGraphFilter(in);
+
+    // ... but on output, it's flattened to wtf/what/wow that's/the fudge/funny happened:
+    assertTokenStreamContents(out,
+                              new String[] {"what", "wow", "wtf", "the", "that's", "fudge", "funny", "happened"},
+                              new int[] {0, 0, 0, 0, 0, 0, 0, 4},
+                              new int[] {3, 3, 3, 3, 3, 3, 3, 12},
+                              new int[] {1, 0, 0, 1, 0, 1, 0, 1},
+                              new int[] {1, 1, 3, 1, 1, 1, 1, 1},
+                              12);
+    
+  }
+
+  public void testNonGreedySynonyms() throws Exception {
+    // This is just "hypothetical" for Lucene today, because SynFilter is
+    // greedy: when two syn rules match on overlapping tokens, only one
+    // (greedily) wins.  This test pretends all syn matches could match:
+
+    TokenStream in = new CannedTokenStream(0, 20, new Token[] {
+        token("wizard", 1, 1, 0, 6),
+        token("wizard_of_oz", 0, 3, 0, 12),
+        token("of", 1, 1, 7, 9),
+        token("oz", 1, 1, 10, 12),
+        token("oz_screams", 0, 2, 10, 20),
+        token("screams", 1, 1, 13, 20),
+      });
+
+
+    TokenStream out = new FlattenGraphFilter(in);
+
+    // ... but on output, it's flattened to wtf/what/wow that's/the fudge/funny happened:
+    assertTokenStreamContents(out,
+                              new String[] {"wizard", "wizard_of_oz", "of", "oz", "oz_screams", "screams"},
+                              new int[] {0, 0, 7, 10, 10, 13},
+                              new int[] {6, 12, 9, 12, 20, 20},
+                              new int[] {1, 0, 1, 1, 0, 1},
+                              new int[] {1, 3, 1, 1, 2, 1},
+                              20);
+    
+  }
+
+  public void testNonGraph() throws Exception {
+    TokenStream in = new CannedTokenStream(0, 22, new Token[] {
+        token("hello", 1, 1, 0, 5),
+        token("pseudo", 1, 1, 6, 12),
+        token("world", 1, 1, 13, 18),
+        token("fun", 1, 1, 19, 22),
+      });
+
+
+    TokenStream out = new FlattenGraphFilter(in);
+
+    // ... but on output, it's flattened to wtf/what/wow that's/the fudge/funny happened:
+    assertTokenStreamContents(out,
+                              new String[] {"hello", "pseudo", "world", "fun"},
+                              new int[] {0, 6, 13, 19},
+                              new int[] {5, 12, 18, 22},
+                              new int[] {1, 1, 1, 1},
+                              new int[] {1, 1, 1, 1},
+                              22);
+  }
+
+  public void testSimpleHole() throws Exception {
+    TokenStream in = new CannedTokenStream(0, 13, new Token[] {
+        token("hello", 1, 1, 0, 5),
+        token("hole", 2, 1, 6, 10),
+        token("fun", 1, 1, 11, 13),
+      });
+
+
+    TokenStream out = new FlattenGraphFilter(in);
+
+    // ... but on output, it's flattened to wtf/what/wow that's/the fudge/funny happened:
+    assertTokenStreamContents(out,
+                              new String[] {"hello", "hole", "fun"},
+                              new int[] {0, 6, 11},
+                              new int[] {5, 10, 13},
+                              new int[] {1, 2, 1},
+                              new int[] {1, 1, 1},
+                              13);
+  }
+
+  public void testHoleUnderSyn() throws Exception {
+    // Tests a StopFilter after SynFilter where a stopword in a syn is removed
+    //
+    //   wizard of oz -> woz syn, but then "of" becomes a hole
+
+    TokenStream in = new CannedTokenStream(0, 12, new Token[] {
+        token("wizard", 1, 1, 0, 6),
+        token("woz", 0, 3, 0, 12),
+        token("oz", 2, 1, 10, 12),
+      });
+
+
+    TokenStream out = new FlattenGraphFilter(in);
+
+    assertTokenStreamContents(out,
+                              new String[] {"wizard", "woz", "oz"},
+                              new int[] {0, 0, 10},
+                              new int[] {6, 12, 12},
+                              new int[] {1, 0, 2},
+                              new int[] {1, 3, 1},
+                              12);
+  }
+
+  public void testStrangelyNumberedNodes() throws Exception {
+
+    // Uses only nodes 0, 2, 3, i.e. 1 is just never used (it is not a hole!!)
+    TokenStream in = new CannedTokenStream(0, 27, new Token[] {
+        token("dog", 1, 3, 0, 5),
+        token("puppy", 0, 3, 0, 5),
+        token("flies", 3, 1, 6, 11),
+      });
+
+    TokenStream out = new FlattenGraphFilter(in);
+
+    assertTokenStreamContents(out,
+                              new String[] {"dog", "puppy", "flies"},
+                              new int[] {0, 0, 6},
+                              new int[] {5, 5, 11},
+                              new int[] {1, 0, 1},
+                              new int[] {1, 1, 1},
+                              27);
+  }
+
+  public void testTwoLongParallelPaths() throws Exception {
+
+    // "a a a a a a" in parallel with "b b b b b b"
+    TokenStream in = new CannedTokenStream(0, 11, new Token[] {
+        token("a", 1, 1, 0, 1),
+        token("b", 0, 2, 0, 1),
+        token("a", 1, 2, 2, 3),
+        token("b", 1, 2, 2, 3),
+        token("a", 1, 2, 4, 5),
+        token("b", 1, 2, 4, 5),
+        token("a", 1, 2, 6, 7),
+        token("b", 1, 2, 6, 7),
+        token("a", 1, 2, 8, 9),
+        token("b", 1, 2, 8, 9),
+        token("a", 1, 2, 10, 11),
+        token("b", 1, 2, 10, 11),
+      });
+
+
+    TokenStream out = new FlattenGraphFilter(in);
+    
+    // ... becomes flattened to a single path with overlapping a/b token between each node:
+    assertTokenStreamContents(out,
+                              new String[] {"a", "b", "a", "b", "a", "b", "a", "b", "a", "b", "a", "b"},
+                              new int[] {0, 0, 2, 2, 4, 4, 6, 6, 8, 8, 10, 10},
+                              new int[] {1, 1, 3, 3, 5, 5, 7, 7, 9, 9, 11, 11},
+                              new int[] {1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0},
+                              new int[] {1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1},
+                              11);
+    
+  }
+
+  // NOTE: TestSynonymGraphFilter's testRandomSyns also tests FlattenGraphFilter
+}