Browse Source

Synonym Graph Support (LUCENE-6664) (#21517)

Integrate the patch from LUCENE-6664 into elasticsearch and
add support for handling a graph token stream in match/multi-match
queries.

This fixes longstanding bugs with multi-token synonyms returning
incorrect results with proximity queries.
Matt Weber 9 years ago
parent
commit
04e07bcdb6

+ 291 - 0
core/src/main/java/org/apache/lucene/analysis/synonym/GraphTokenStreamFiniteStrings.java

@@ -0,0 +1,291 @@
+/*
+ * 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 static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.BytesTermAttribute;
+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.TermToBytesRefAttribute;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.FiniteStringsIterator;
+import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.Transition;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Creates a list of {@link TokenStream} where each stream is the tokens that make up a finite string in graph token stream.  To do this,
+ * the graph token stream is converted to an {@link Automaton} and from there we use a {@link FiniteStringsIterator} to collect the various
+ * token streams for each finite string.
+ */
+public class GraphTokenStreamFiniteStrings {
+    private final Automaton.Builder builder;
+    Automaton det;
+    private final Map<BytesRef, Integer> termToID = new HashMap<>();
+    private final Map<Integer, BytesRef> idToTerm = new HashMap<>();
+    private int anyTermID = -1;
+
+    public GraphTokenStreamFiniteStrings() {
+        this.builder = new Automaton.Builder();
+    }
+
+    private static class BytesRefArrayTokenStream extends TokenStream {
+        private final BytesTermAttribute termAtt = addAttribute(BytesTermAttribute.class);
+        private final BytesRef[] terms;
+        private int offset;
+
+        BytesRefArrayTokenStream(BytesRef[] terms) {
+            this.terms = terms;
+            offset = 0;
+        }
+
+        @Override
+        public boolean incrementToken() throws IOException {
+            if (offset < terms.length) {
+                clearAttributes();
+                termAtt.setBytesRef(terms[offset]);
+                offset = offset + 1;
+                return true;
+            }
+
+            return false;
+        }
+    }
+
+    /**
+     * Gets
+     */
+    public List<TokenStream> getTokenStreams(final TokenStream in) throws IOException {
+        // build automation
+        build(in);
+
+        List<TokenStream> tokenStreams = new ArrayList<>();
+        final FiniteStringsIterator finiteStrings = new FiniteStringsIterator(det);
+        for (IntsRef string; (string = finiteStrings.next()) != null; ) {
+            final BytesRef[] tokens = new BytesRef[string.length];
+            for (int idx = string.offset, len = string.offset + string.length; idx < len; idx++) {
+                tokens[idx - string.offset] = idToTerm.get(string.ints[idx]);
+            }
+
+            tokenStreams.add(new BytesRefArrayTokenStream(tokens));
+        }
+
+        return tokenStreams;
+    }
+
+    private void build(final TokenStream in) throws IOException {
+        if (det != null) {
+            throw new IllegalStateException("Automation already built");
+        }
+
+        final TermToBytesRefAttribute termBytesAtt = in.addAttribute(TermToBytesRefAttribute.class);
+        final PositionIncrementAttribute posIncAtt = in.addAttribute(PositionIncrementAttribute.class);
+        final PositionLengthAttribute posLengthAtt = in.addAttribute(PositionLengthAttribute.class);
+        final OffsetAttribute offsetAtt = in.addAttribute(OffsetAttribute.class);
+
+        in.reset();
+
+        int pos = -1;
+        int lastPos = 0;
+        int maxOffset = 0;
+        int maxPos = -1;
+        int state = -1;
+        while (in.incrementToken()) {
+            int posInc = posIncAtt.getPositionIncrement();
+            assert pos > -1 || posInc > 0;
+
+            if (posInc > 1) {
+                throw new IllegalArgumentException("cannot handle holes; to accept any term, use '*' term");
+            }
+
+            if (posInc > 0) {
+                // New node:
+                pos += posInc;
+            }
+
+            int endPos = pos + posLengthAtt.getPositionLength();
+            while (state < endPos) {
+                state = createState();
+            }
+
+            BytesRef term = termBytesAtt.getBytesRef();
+            //System.out.println(pos + "-" + endPos + ": " + term.utf8ToString() + ": posInc=" + posInc);
+            if (term.length == 1 && term.bytes[term.offset] == (byte) '*') {
+                addAnyTransition(pos, endPos);
+            } else {
+                addTransition(pos, endPos, term);
+            }
+
+            maxOffset = Math.max(maxOffset, offsetAtt.endOffset());
+            maxPos = Math.max(maxPos, endPos);
+        }
+
+        in.end();
+
+        // TODO: look at endOffset?  ts2a did...
+
+        // TODO: this (setting "last" state as the only accept state) may be too simplistic?
+        setAccept(state, true);
+        finish();
+    }
+
+    /**
+     * Returns a new state; state 0 is always the initial state.
+     */
+    private int createState() {
+        return builder.createState();
+    }
+
+    /**
+     * Marks the specified state as accept or not.
+     */
+    private void setAccept(int state, boolean accept) {
+        builder.setAccept(state, accept);
+    }
+
+    /**
+     * Adds a transition to the automaton.
+     */
+    private void addTransition(int source, int dest, String term) {
+        addTransition(source, dest, new BytesRef(term));
+    }
+
+    /**
+     * Adds a transition to the automaton.
+     */
+    private void addTransition(int source, int dest, BytesRef term) {
+        if (term == null) {
+            throw new NullPointerException("term should not be null");
+        }
+        builder.addTransition(source, dest, getTermID(term));
+    }
+
+    /**
+     * Adds a transition matching any term.
+     */
+    private void addAnyTransition(int source, int dest) {
+        builder.addTransition(source, dest, getTermID(null));
+    }
+
+    /**
+     * Call this once you are done adding states/transitions.
+     */
+    private void finish() {
+        finish(DEFAULT_MAX_DETERMINIZED_STATES);
+    }
+
+    /**
+     * Call this once you are done adding states/transitions.
+     *
+     * @param maxDeterminizedStates Maximum number of states created when determinizing the automaton.  Higher numbers allow this operation
+     *                              to consume more memory but allow more complex automatons.
+     */
+    private void finish(int maxDeterminizedStates) {
+        Automaton automaton = builder.finish();
+
+        // System.out.println("before det:\n" + automaton.toDot());
+
+        Transition t = new Transition();
+
+        // TODO: should we add "eps back to initial node" for all states,
+        // and det that?  then we don't need to revisit initial node at
+        // every position?  but automaton could blow up?  And, this makes it
+        // harder to skip useless positions at search time?
+
+        if (anyTermID != -1) {
+
+            // Make sure there are no leading or trailing ANY:
+            int count = automaton.initTransition(0, t);
+            for (int i = 0; i < count; i++) {
+                automaton.getNextTransition(t);
+                if (anyTermID >= t.min && anyTermID <= t.max) {
+                    throw new IllegalStateException("automaton cannot lead with an ANY transition");
+                }
+            }
+
+            int numStates = automaton.getNumStates();
+            for (int i = 0; i < numStates; i++) {
+                count = automaton.initTransition(i, t);
+                for (int j = 0; j < count; j++) {
+                    automaton.getNextTransition(t);
+                    if (automaton.isAccept(t.dest) && anyTermID >= t.min && anyTermID <= t.max) {
+                        throw new IllegalStateException("automaton cannot end with an ANY transition");
+                    }
+                }
+            }
+
+            int termCount = termToID.size();
+
+            // We have to carefully translate these transitions so automaton
+            // realizes they also match all other terms:
+            Automaton newAutomaton = new Automaton();
+            for (int i = 0; i < numStates; i++) {
+                newAutomaton.createState();
+                newAutomaton.setAccept(i, automaton.isAccept(i));
+            }
+
+            for (int i = 0; i < numStates; i++) {
+                count = automaton.initTransition(i, t);
+                for (int j = 0; j < count; j++) {
+                    automaton.getNextTransition(t);
+                    int min, max;
+                    if (t.min <= anyTermID && anyTermID <= t.max) {
+                        // Match any term
+                        min = 0;
+                        max = termCount - 1;
+                    } else {
+                        min = t.min;
+                        max = t.max;
+                    }
+                    newAutomaton.addTransition(t.source, t.dest, min, max);
+                }
+            }
+            newAutomaton.finishState();
+            automaton = newAutomaton;
+        }
+
+        det = Operations.removeDeadStates(Operations.determinize(automaton, maxDeterminizedStates));
+    }
+
+    private int getTermID(BytesRef term) {
+        Integer id = termToID.get(term);
+        if (id == null) {
+            id = termToID.size();
+            if (term != null) {
+                term = BytesRef.deepCopyOf(term);
+            }
+            termToID.put(term, id);
+            idToTerm.put(id, term);
+            if (term == null) {
+                anyTermID = id;
+            }
+        }
+
+        return id;
+    }
+}

+ 588 - 0
core/src/main/java/org/apache/lucene/analysis/synonym/SynonymGraphFilter.java

@@ -0,0 +1,588 @@
+/*
+ * 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.FlagsAttribute;
+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.XRollingBuffer;
+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.
+ *
+ * <b>NOTE</b>: this cannot consume an incoming graph; results will
+ * be undefined.
+ */
+public final class SynonymGraphFilter extends TokenFilter {
+
+    public static final String TYPE_SYNONYM = "SYNONYM";
+    public static final int GRAPH_FLAG = 8;
+
+    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+    private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
+    private final PositionLengthAttribute posLenAtt = addAttribute(PositionLengthAttribute.class);
+    private final FlagsAttribute flagsAtt = addAttribute(FlagsAttribute.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 XRollingBuffer<BufferedInputToken> lookahead = new XRollingBuffer<BufferedInputToken>() {
+        @Override
+        protected BufferedInputToken newInstance() {
+            return new BufferedInputToken();
+        }
+    };
+
+    static class BufferedInputToken implements XRollingBuffer.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);
+        flagsAtt.setFlags(flagsAtt.getFlags() | GRAPH_FLAG);  // set the graph flag
+    }
+
+    /**
+     * 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;
+    }
+}

+ 115 - 0
core/src/main/java/org/apache/lucene/search/GraphQuery.java

@@ -0,0 +1,115 @@
+/*
+ * 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.search;
+
+import org.apache.lucene.index.IndexReader;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * A query that wraps multiple sub-queries generated from a graph token stream.
+ */
+public final class GraphQuery extends Query {
+    private final Query[] queries;
+    private final boolean hasBoolean;
+
+    /**
+     * Constructor sets the queries and checks if any of them are
+     * a boolean query.
+     *
+     * @param queries the non-null array of queries
+     */
+    public GraphQuery(Query... queries) {
+        this.queries = Objects.requireNonNull(queries).clone();
+        for (Query query : queries) {
+            if (query instanceof BooleanQuery) {
+                hasBoolean = true;
+                return;
+            }
+        }
+        hasBoolean = false;
+    }
+
+    /**
+     * Gets the queries
+     *
+     * @return unmodifiable list of Query
+     */
+    public List<Query> getQueries() {
+        return Collections.unmodifiableList(Arrays.asList(queries));
+    }
+
+    /**
+     * If there is at least one boolean query or not.
+     *
+     * @return true if there is a boolean, false if not
+     */
+    public boolean hasBoolean() {
+        return hasBoolean;
+    }
+
+    /**
+     * Rewrites to a single query or a boolean query where each query is a SHOULD clause.
+     */
+    @Override
+    public Query rewrite(IndexReader reader) throws IOException {
+        if (queries.length == 0) {
+            return new BooleanQuery.Builder().build();
+        }
+
+        if (queries.length == 1) {
+            return queries[0];
+        }
+
+        BooleanQuery.Builder q = new BooleanQuery.Builder();
+        q.setDisableCoord(true);
+        for (Query clause : queries) {
+            q.add(clause, BooleanClause.Occur.SHOULD);
+        }
+
+        return q.build();
+    }
+
+    @Override
+    public String toString(String field) {
+        StringBuilder builder = new StringBuilder("Graph(");
+        for (int i = 0; i < queries.length; i++) {
+            if (i != 0) {
+                builder.append(", ");
+            }
+            builder.append(Objects.toString(queries[i]));
+        }
+        builder.append(")");
+        return builder.toString();
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        return sameClassAs(other) &&
+            Arrays.equals(queries, ((GraphQuery) other).queries);
+    }
+
+    @Override
+    public int hashCode() {
+        return 31 * classHash() + Arrays.hashCode(queries);
+    }
+}

+ 435 - 0
core/src/main/java/org/apache/lucene/util/XQueryBuilder.java

@@ -0,0 +1,435 @@
+/*
+ * 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.util;
+
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.CachingTokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.MultiPhraseQuery;
+import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SynonymQuery;
+import org.apache.lucene.search.TermQuery;
+
+/**
+ * Creates queries from the {@link Analyzer} chain.
+ * <p>
+ * Example usage:
+ * <pre class="prettyprint">
+ *   QueryBuilder builder = new QueryBuilder(analyzer);
+ *   Query a = builder.createBooleanQuery("body", "just a test");
+ *   Query b = builder.createPhraseQuery("body", "another test");
+ *   Query c = builder.createMinShouldMatchQuery("body", "another test", 0.5f);
+ * </pre>
+ * <p>
+ * This can also be used as a subclass for query parsers to make it easier
+ * to interact with the analysis chain. Factory methods such as {@code newTermQuery}
+ * are provided so that the generated queries can be customized.
+ *
+ * TODO: un-fork once we are on Lucene 6.4.0
+ * This is only forked due to `createFieldQuery` being final and the analyze* methods being private.  Lucene 6.4.0 removes final and will
+ * make the private methods protected allowing us to override it.
+ */
+public class XQueryBuilder {
+    private Analyzer analyzer;
+    private boolean enablePositionIncrements = true;
+
+    /** Creates a new QueryBuilder using the given analyzer. */
+    public XQueryBuilder(Analyzer analyzer) {
+        this.analyzer = analyzer;
+    }
+
+    /**
+     * Creates a boolean query from the query text.
+     * <p>
+     * This is equivalent to {@code createBooleanQuery(field, queryText, Occur.SHOULD)}
+     * @param field field name
+     * @param queryText text to be passed to the analyzer
+     * @return {@code TermQuery} or {@code BooleanQuery}, based on the analysis
+     *         of {@code queryText}
+     */
+    public Query createBooleanQuery(String field, String queryText) {
+        return createBooleanQuery(field, queryText, BooleanClause.Occur.SHOULD);
+    }
+
+    /**
+     * Creates a boolean query from the query text.
+     * <p>
+     * @param field field name
+     * @param queryText text to be passed to the analyzer
+     * @param operator operator used for clauses between analyzer tokens.
+     * @return {@code TermQuery} or {@code BooleanQuery}, based on the analysis
+     *         of {@code queryText}
+     */
+    public Query createBooleanQuery(String field, String queryText, BooleanClause.Occur operator) {
+        if (operator != BooleanClause.Occur.SHOULD && operator != BooleanClause.Occur.MUST) {
+            throw new IllegalArgumentException("invalid operator: only SHOULD or MUST are allowed");
+        }
+        return createFieldQuery(analyzer, operator, field, queryText, false, 0);
+    }
+
+    /**
+     * Creates a phrase query from the query text.
+     * <p>
+     * This is equivalent to {@code createPhraseQuery(field, queryText, 0)}
+     * @param field field name
+     * @param queryText text to be passed to the analyzer
+     * @return {@code TermQuery}, {@code BooleanQuery}, {@code PhraseQuery}, or
+     *         {@code MultiPhraseQuery}, based on the analysis of {@code queryText}
+     */
+    public Query createPhraseQuery(String field, String queryText) {
+        return createPhraseQuery(field, queryText, 0);
+    }
+
+    /**
+     * Creates a phrase query from the query text.
+     * <p>
+     * @param field field name
+     * @param queryText text to be passed to the analyzer
+     * @param phraseSlop number of other words permitted between words in query phrase
+     * @return {@code TermQuery}, {@code BooleanQuery}, {@code PhraseQuery}, or
+     *         {@code MultiPhraseQuery}, based on the analysis of {@code queryText}
+     */
+    public Query createPhraseQuery(String field, String queryText, int phraseSlop) {
+        return createFieldQuery(analyzer, BooleanClause.Occur.MUST, field, queryText, true, phraseSlop);
+    }
+
+    /**
+     * Creates a minimum-should-match query from the query text.
+     * <p>
+     * @param field field name
+     * @param queryText text to be passed to the analyzer
+     * @param fraction of query terms {@code [0..1]} that should match
+     * @return {@code TermQuery} or {@code BooleanQuery}, based on the analysis
+     *         of {@code queryText}
+     */
+    public Query createMinShouldMatchQuery(String field, String queryText, float fraction) {
+        if (Float.isNaN(fraction) || fraction < 0 || fraction > 1) {
+            throw new IllegalArgumentException("fraction should be >= 0 and <= 1");
+        }
+
+        // TODO: wierd that BQ equals/rewrite/scorer doesn't handle this?
+        if (fraction == 1) {
+            return createBooleanQuery(field, queryText, BooleanClause.Occur.MUST);
+        }
+
+        Query query = createFieldQuery(analyzer, BooleanClause.Occur.SHOULD, field, queryText, false, 0);
+        if (query instanceof BooleanQuery) {
+            BooleanQuery bq = (BooleanQuery) query;
+            BooleanQuery.Builder builder = new BooleanQuery.Builder();
+            builder.setDisableCoord(bq.isCoordDisabled());
+            builder.setMinimumNumberShouldMatch((int) (fraction * bq.clauses().size()));
+            for (BooleanClause clause : bq) {
+                builder.add(clause);
+            }
+            query = builder.build();
+        }
+        return query;
+    }
+
+    /**
+     * Returns the analyzer.
+     * @see #setAnalyzer(Analyzer)
+     */
+    public Analyzer getAnalyzer() {
+        return analyzer;
+    }
+
+    /**
+     * Sets the analyzer used to tokenize text.
+     */
+    public void setAnalyzer(Analyzer analyzer) {
+        this.analyzer = analyzer;
+    }
+
+    /**
+     * Returns true if position increments are enabled.
+     * @see #setEnablePositionIncrements(boolean)
+     */
+    public boolean getEnablePositionIncrements() {
+        return enablePositionIncrements;
+    }
+
+    /**
+     * Set to <code>true</code> to enable position increments in result query.
+     * <p>
+     * When set, result phrase and multi-phrase queries will
+     * be aware of position increments.
+     * Useful when e.g. a StopFilter increases the position increment of
+     * the token that follows an omitted token.
+     * <p>
+     * Default: true.
+     */
+    public void setEnablePositionIncrements(boolean enable) {
+        this.enablePositionIncrements = enable;
+    }
+
+    /**
+     * Creates a query from the analysis chain.
+     * <p>
+     * Expert: this is more useful for subclasses such as queryparsers.
+     * If using this class directly, just use {@link #createBooleanQuery(String, String)}
+     * and {@link #createPhraseQuery(String, String)}
+     * @param analyzer analyzer used for this query
+     * @param operator default boolean operator used for this query
+     * @param field field to create queries against
+     * @param queryText text to be passed to the analysis chain
+     * @param quoted true if phrases should be generated when terms occur at more than one position
+     * @param phraseSlop slop factor for phrase/multiphrase queries
+     */
+    protected Query createFieldQuery(Analyzer analyzer, BooleanClause.Occur operator, String field, String queryText, boolean quoted,
+                                     int phraseSlop) {
+        assert operator == BooleanClause.Occur.SHOULD || operator == BooleanClause.Occur.MUST;
+
+        // Use the analyzer to get all the tokens, and then build an appropriate
+        // query based on the analysis chain.
+
+        try (TokenStream source = analyzer.tokenStream(field, queryText);
+             CachingTokenFilter stream = new CachingTokenFilter(source)) {
+
+            TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
+            PositionIncrementAttribute posIncAtt = stream.addAttribute(PositionIncrementAttribute.class);
+
+            if (termAtt == null) {
+                return null;
+            }
+
+            // phase 1: read through the stream and assess the situation:
+            // counting the number of tokens/positions and marking if we have any synonyms.
+
+            int numTokens = 0;
+            int positionCount = 0;
+            boolean hasSynonyms = false;
+
+            stream.reset();
+            while (stream.incrementToken()) {
+                numTokens++;
+                int positionIncrement = posIncAtt.getPositionIncrement();
+                if (positionIncrement != 0) {
+                    positionCount += positionIncrement;
+                } else {
+                    hasSynonyms = true;
+                }
+            }
+
+            // phase 2: based on token count, presence of synonyms, and options
+            // formulate a single term, boolean, or phrase.
+
+            if (numTokens == 0) {
+                return null;
+            } else if (numTokens == 1) {
+                // single term
+                return analyzeTerm(field, stream);
+            } else if (quoted && positionCount > 1) {
+                // phrase
+                if (hasSynonyms) {
+                    // complex phrase with synonyms
+                    return analyzeMultiPhrase(field, stream, phraseSlop);
+                } else {
+                    // simple phrase
+                    return analyzePhrase(field, stream, phraseSlop);
+                }
+            } else {
+                // boolean
+                if (positionCount == 1) {
+                    // only one position, with synonyms
+                    return analyzeBoolean(field, stream);
+                } else {
+                    // complex case: multiple positions
+                    return analyzeMultiBoolean(field, stream, operator);
+                }
+            }
+        } catch (IOException e) {
+            throw new RuntimeException("Error analyzing query text", e);
+        }
+    }
+
+    /**
+     * Creates simple term query from the cached tokenstream contents
+     */
+    protected Query analyzeTerm(String field, TokenStream stream) throws IOException {
+        TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
+
+        stream.reset();
+        if (!stream.incrementToken()) {
+            throw new AssertionError();
+        }
+
+        return newTermQuery(new Term(field, termAtt.getBytesRef()));
+    }
+
+    /**
+     * Creates simple boolean query from the cached tokenstream contents
+     */
+    protected Query analyzeBoolean(String field, TokenStream stream) throws IOException {
+        TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
+
+        stream.reset();
+        List<Term> terms = new ArrayList<>();
+        while (stream.incrementToken()) {
+            terms.add(new Term(field, termAtt.getBytesRef()));
+        }
+
+        return newSynonymQuery(terms.toArray(new Term[terms.size()]));
+    }
+
+    protected void add(BooleanQuery.Builder q, List<Term> current, BooleanClause.Occur operator) {
+        if (current.isEmpty()) {
+            return;
+        }
+        if (current.size() == 1) {
+            q.add(newTermQuery(current.get(0)), operator);
+        } else {
+            q.add(newSynonymQuery(current.toArray(new Term[current.size()])), operator);
+        }
+    }
+
+    /**
+     * Creates complex boolean query from the cached tokenstream contents
+     */
+    protected Query analyzeMultiBoolean(String field, TokenStream stream, BooleanClause.Occur operator) throws IOException {
+        BooleanQuery.Builder q = newBooleanQuery();
+        List<Term> currentQuery = new ArrayList<>();
+
+        TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
+        PositionIncrementAttribute posIncrAtt = stream.getAttribute(PositionIncrementAttribute.class);
+
+        stream.reset();
+        while (stream.incrementToken()) {
+            if (posIncrAtt.getPositionIncrement() != 0) {
+                add(q, currentQuery, operator);
+                currentQuery.clear();
+            }
+            currentQuery.add(new Term(field, termAtt.getBytesRef()));
+        }
+        add(q, currentQuery, operator);
+
+        return q.build();
+    }
+
+    /**
+     * Creates simple phrase query from the cached tokenstream contents
+     */
+    protected Query analyzePhrase(String field, TokenStream stream, int slop) throws IOException {
+        PhraseQuery.Builder builder = new PhraseQuery.Builder();
+        builder.setSlop(slop);
+
+        TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
+        PositionIncrementAttribute posIncrAtt = stream.getAttribute(PositionIncrementAttribute.class);
+        int position = -1;
+
+        stream.reset();
+        while (stream.incrementToken()) {
+            if (enablePositionIncrements) {
+                position += posIncrAtt.getPositionIncrement();
+            } else {
+                position += 1;
+            }
+            builder.add(new Term(field, termAtt.getBytesRef()), position);
+        }
+
+        return builder.build();
+    }
+
+    /**
+     * Creates complex phrase query from the cached tokenstream contents
+     */
+    protected Query analyzeMultiPhrase(String field, TokenStream stream, int slop) throws IOException {
+        MultiPhraseQuery.Builder mpqb = newMultiPhraseQueryBuilder();
+        mpqb.setSlop(slop);
+
+        TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
+
+        PositionIncrementAttribute posIncrAtt = stream.getAttribute(PositionIncrementAttribute.class);
+        int position = -1;
+
+        List<Term> multiTerms = new ArrayList<>();
+        stream.reset();
+        while (stream.incrementToken()) {
+            int positionIncrement = posIncrAtt.getPositionIncrement();
+
+            if (positionIncrement > 0 && multiTerms.size() > 0) {
+                if (enablePositionIncrements) {
+                    mpqb.add(multiTerms.toArray(new Term[0]), position);
+                } else {
+                    mpqb.add(multiTerms.toArray(new Term[0]));
+                }
+                multiTerms.clear();
+            }
+            position += positionIncrement;
+            multiTerms.add(new Term(field, termAtt.getBytesRef()));
+        }
+
+        if (enablePositionIncrements) {
+            mpqb.add(multiTerms.toArray(new Term[0]), position);
+        } else {
+            mpqb.add(multiTerms.toArray(new Term[0]));
+        }
+        return mpqb.build();
+    }
+
+    /**
+     * Builds a new BooleanQuery instance.
+     * <p>
+     * This is intended for subclasses that wish to customize the generated queries.
+     * @return new BooleanQuery instance
+     */
+    protected BooleanQuery.Builder newBooleanQuery() {
+        return new BooleanQuery.Builder();
+    }
+
+    /**
+     * Builds a new SynonymQuery instance.
+     * <p>
+     * This is intended for subclasses that wish to customize the generated queries.
+     * @return new Query instance
+     */
+    protected Query newSynonymQuery(Term terms[]) {
+        return new SynonymQuery(terms);
+    }
+
+    /**
+     * Builds a new TermQuery instance.
+     * <p>
+     * This is intended for subclasses that wish to customize the generated queries.
+     * @param term term
+     * @return new TermQuery instance
+     */
+    protected Query newTermQuery(Term term) {
+        return new TermQuery(term);
+    }
+
+    /**
+     * Builds a new MultiPhraseQuery instance.
+     * <p>
+     * This is intended for subclasses that wish to customize the generated queries.
+     * @return new MultiPhraseQuery instance
+     */
+    protected MultiPhraseQuery.Builder newMultiPhraseQueryBuilder() {
+        return new MultiPhraseQuery.Builder();
+    }
+}

+ 151 - 0
core/src/main/java/org/apache/lucene/util/XRollingBuffer.java

@@ -0,0 +1,151 @@
+/*
+ * 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.util;
+
+
+/**
+ * Acts like forever growing T[], but internally uses a
+ * circular buffer to reuse instances of T.
+ *
+ * TODO: un-fork once we are on Lucene 6.4.0
+ */
+public abstract class XRollingBuffer<T extends XRollingBuffer.Resettable> {
+
+    /**
+     * Implement to reset an instance
+     */
+    public interface Resettable {
+        void reset();
+    }
+
+    @SuppressWarnings("unchecked")
+    private T[] buffer = (T[]) new XRollingBuffer.Resettable[8];
+
+    // Next array index to write to:
+    private int nextWrite;
+
+    // Next position to write:
+    private int nextPos;
+
+    // How many valid Position are held in the
+    // array:
+    private int count;
+
+    public XRollingBuffer() {
+        for (int idx = 0; idx < buffer.length; idx++) {
+            buffer[idx] = newInstance();
+        }
+    }
+
+    protected abstract T newInstance();
+
+    public void reset() {
+        nextWrite--;
+        while (count > 0) {
+            if (nextWrite == -1) {
+                nextWrite = buffer.length - 1;
+            }
+            buffer[nextWrite--].reset();
+            count--;
+        }
+        nextWrite = 0;
+        nextPos = 0;
+        count = 0;
+    }
+
+    // For assert:
+    private boolean inBounds(int pos) {
+        return pos < nextPos && pos >= nextPos - count;
+    }
+
+    private int getIndex(int pos) {
+        int index = nextWrite - (nextPos - pos);
+        if (index < 0) {
+            index += buffer.length;
+        }
+        return index;
+    }
+
+    /**
+     * Get T instance for this absolute position;
+     * this is allowed to be arbitrarily far "in the
+     * future" but cannot be before the last freeBefore.
+     */
+    public T get(int pos) {
+        //System.out.println("RA.get pos=" + pos + " nextPos=" + nextPos + " nextWrite=" + nextWrite + " count=" + count);
+        while (pos >= nextPos) {
+            if (count == buffer.length) {
+                @SuppressWarnings("unchecked") T[] newBuffer = (T[]) new Resettable[ArrayUtil.oversize(1 + count, RamUsageEstimator
+                    .NUM_BYTES_OBJECT_REF)];
+                //System.out.println("  grow length=" + newBuffer.length);
+                System.arraycopy(buffer, nextWrite, newBuffer, 0, buffer.length - nextWrite);
+                System.arraycopy(buffer, 0, newBuffer, buffer.length - nextWrite, nextWrite);
+                for (int i = buffer.length; i < newBuffer.length; i++) {
+                    newBuffer[i] = newInstance();
+                }
+                nextWrite = buffer.length;
+                buffer = newBuffer;
+            }
+            if (nextWrite == buffer.length) {
+                nextWrite = 0;
+            }
+            // Should have already been reset:
+            nextWrite++;
+            nextPos++;
+            count++;
+        }
+        assert inBounds(pos) : "pos=" + pos + " nextPos=" + nextPos + " count=" + count;
+        final int index = getIndex(pos);
+        //System.out.println("  pos=" + pos + " nextPos=" + nextPos + " -> index=" + index);
+        //assert buffer[index].pos == pos;
+        return buffer[index];
+    }
+
+    /**
+     * Returns the maximum position looked up, or -1 if no
+     * position has been looked up since reset/init.
+     */
+    public int getMaxPos() {
+        return nextPos - 1;
+    }
+
+    /**
+     * Returns how many active positions are in the buffer.
+     */
+    public int getBufferSize() {
+        return count;
+    }
+
+    public void freeBefore(int pos) {
+        final int toFree = count - (nextPos - pos);
+        assert toFree >= 0;
+        assert toFree <= count : "toFree=" + toFree + " count=" + count;
+        int index = nextWrite - count;
+        if (index < 0) {
+            index += buffer.length;
+        }
+        for (int i = 0; i < toFree; i++) {
+            if (index == buffer.length) {
+                index = 0;
+            }
+            //System.out.println("  fb idx=" + index);
+            buffer[index].reset();
+            index++;
+        }
+        count -= toFree;
+    }
+}

+ 5 - 2
core/src/main/java/org/elasticsearch/index/analysis/AnalysisRegistry.java

@@ -158,11 +158,12 @@ public final class AnalysisRegistry implements Closeable {
         final Map<String, Settings> tokenFiltersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_FILTER);
         Map<String, AnalysisModule.AnalysisProvider<TokenFilterFactory>> tokenFilters = new HashMap<>(this.tokenFilters);
         /*
-         * synonym is different than everything else since it needs access to the tokenizer factories for this index.
+         * synonym and synonym_graph are different than everything else since they need access to the tokenizer factories for the index.
          * instead of building the infrastructure for plugins we rather make it a real exception to not pollute the general interface and
          * hide internal data-structures as much as possible.
          */
         tokenFilters.put("synonym", requriesAnalysisSettings((is, env, name, settings) -> new SynonymTokenFilterFactory(is, env, this, name, settings)));
+        tokenFilters.put("synonym_graph", requriesAnalysisSettings((is, env, name, settings) -> new SynonymGraphFilterFactory(is, env, this, name, settings)));
         return buildMapping(false, "tokenfilter", indexSettings, tokenFiltersSettings, Collections.unmodifiableMap(tokenFilters), prebuiltAnalysis.tokenFilterFactories);
     }
 
@@ -213,12 +214,14 @@ public final class AnalysisRegistry implements Closeable {
             Settings currentSettings = tokenFilterSettings.get(tokenFilter);
             String typeName = currentSettings.get("type");
             /*
-             * synonym is different than everything else since it needs access to the tokenizer factories for this index.
+             * synonym and synonym_graph are different than everything else since they need access to the tokenizer factories for the index.
              * instead of building the infrastructure for plugins we rather make it a real exception to not pollute the general interface and
              * hide internal data-structures as much as possible.
              */
             if ("synonym".equals(typeName)) {
                 return requriesAnalysisSettings((is, env, name, settings) -> new SynonymTokenFilterFactory(is, env, this, name, settings));
+            } else if ("synonym_graph".equals(typeName)) {
+                return requriesAnalysisSettings((is, env, name, settings) -> new SynonymGraphFilterFactory(is, env, this, name, settings));
             } else {
                 return getAnalysisProvider("tokenfilter", tokenFilters, tokenFilter, typeName);
             }

+ 41 - 0
core/src/main/java/org/elasticsearch/index/analysis/SynonymGraphFilterFactory.java

@@ -0,0 +1,41 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch 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.elasticsearch.index.analysis;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.synonym.SynonymGraphFilter;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.env.Environment;
+import org.elasticsearch.index.IndexSettings;
+
+import java.io.IOException;
+
+public class SynonymGraphFilterFactory extends SynonymTokenFilterFactory {
+    public SynonymGraphFilterFactory(IndexSettings indexSettings, Environment env, AnalysisRegistry analysisRegistry,
+                                     String name, Settings settings) throws IOException {
+        super(indexSettings, env, analysisRegistry, name, settings);
+    }
+
+    @Override
+    public TokenStream create(TokenStream tokenStream) {
+        // fst is null means no synonyms
+        return synonymMap.fst == null ? tokenStream : new SynonymGraphFilter(tokenStream, synonymMap, ignoreCase);
+    }
+}

+ 2 - 2
core/src/main/java/org/elasticsearch/index/analysis/SynonymTokenFilterFactory.java

@@ -40,8 +40,8 @@ import java.util.List;
 
 public class SynonymTokenFilterFactory extends AbstractTokenFilterFactory {
 
-    private final SynonymMap synonymMap;
-    private final boolean ignoreCase;
+    protected final SynonymMap synonymMap;
+    protected final boolean ignoreCase;
 
     public SynonymTokenFilterFactory(IndexSettings indexSettings, Environment env, AnalysisRegistry analysisRegistry,
                                       String name, Settings settings) throws IOException {

+ 18 - 0
core/src/main/java/org/elasticsearch/index/query/MatchQueryBuilder.java

@@ -22,6 +22,7 @@ package org.elasticsearch.index.query;
 import org.apache.lucene.queries.ExtendedCommonTermsQuery;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.FuzzyQuery;
+import org.apache.lucene.search.GraphQuery;
 import org.apache.lucene.search.Query;
 import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.ParsingException;
@@ -36,6 +37,7 @@ import org.elasticsearch.index.search.MatchQuery;
 import org.elasticsearch.index.search.MatchQuery.ZeroTermsQuery;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.Locale;
 import java.util.Objects;
 import java.util.Optional;
@@ -471,9 +473,25 @@ public class MatchQueryBuilder extends AbstractQueryBuilder<MatchQueryBuilder> {
         // and multiple variations of the same word in the query (synonyms for instance).
         if (query instanceof BooleanQuery && !((BooleanQuery) query).isCoordDisabled()) {
             query = Queries.applyMinimumShouldMatch((BooleanQuery) query, minimumShouldMatch);
+        } else if (query instanceof GraphQuery && ((GraphQuery) query).hasBoolean()) {
+            // we have a graph query that has at least one boolean sub-query
+            // re-build and set minimum should match value on all boolean queries
+            List<Query> oldQueries = ((GraphQuery) query).getQueries();
+            Query[] queries = new Query[oldQueries.size()];
+            for (int i = 0; i < queries.length; i++) {
+                Query oldQuery = oldQueries.get(i);
+                if (oldQuery instanceof BooleanQuery) {
+                    queries[i] = Queries.applyMinimumShouldMatch((BooleanQuery) oldQuery, minimumShouldMatch);
+                } else {
+                    queries[i] = oldQuery;
+                }
+            }
+
+            query = new GraphQuery(queries);
         } else if (query instanceof ExtendedCommonTermsQuery) {
             ((ExtendedCommonTermsQuery)query).setLowFreqMinimumNumberShouldMatch(minimumShouldMatch);
         }
+
         return query;
     }
 

+ 162 - 8
core/src/main/java/org/elasticsearch/index/search/MatchQuery.java

@@ -19,7 +19,16 @@
 
 package org.elasticsearch.index.search;
 
+import static org.apache.lucene.analysis.synonym.SynonymGraphFilter.GRAPH_FLAG;
+
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.CachingTokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.synonym.GraphTokenStreamFiniteStrings;
+import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.ExtendedCommonTermsQuery;
 import org.apache.lucene.search.BooleanClause;
@@ -27,13 +36,14 @@ import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.FuzzyQuery;
+import org.apache.lucene.search.GraphQuery;
 import org.apache.lucene.search.MultiPhraseQuery;
 import org.apache.lucene.search.MultiTermQuery;
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SynonymQuery;
 import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.util.QueryBuilder;
+import org.apache.lucene.util.XQueryBuilder;
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.io.stream.StreamInput;
@@ -48,6 +58,8 @@ import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.index.query.support.QueryParsers;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
 public class MatchQuery {
 
@@ -113,13 +125,19 @@ public class MatchQuery {
         }
     }
 
-    /** the default phrase slop */
+    /**
+     * the default phrase slop
+     */
     public static final int DEFAULT_PHRASE_SLOP = 0;
 
-    /** the default leniency setting */
+    /**
+     * the default leniency setting
+     */
     public static final boolean DEFAULT_LENIENCY = false;
 
-    /** the default zero terms query */
+    /**
+     * the default zero terms query
+     */
     public static final ZeroTermsQuery DEFAULT_ZERO_TERMS_QUERY = ZeroTermsQuery.NONE;
 
     protected final QueryShardContext context;
@@ -286,7 +304,7 @@ public class MatchQuery {
         return Queries.newMatchAllQuery();
     }
 
-    private class MatchQueryBuilder extends QueryBuilder {
+    private class MatchQueryBuilder extends XQueryBuilder {
 
         private final MappedFieldType mapper;
 
@@ -298,6 +316,116 @@ public class MatchQuery {
             this.mapper = mapper;
         }
 
+        /**
+         * Creates a query from the analysis chain.  Overrides original so all it does is create the token stream and pass that into the
+         * new {@link #createFieldQuery(TokenStream, Occur, String, boolean, int)} method which has all the original query generation logic.
+         *
+         * @param analyzer   analyzer used for this query
+         * @param operator   default boolean operator used for this query
+         * @param field      field to create queries against
+         * @param queryText  text to be passed to the analysis chain
+         * @param quoted     true if phrases should be generated when terms occur at more than one position
+         * @param phraseSlop slop factor for phrase/multiphrase queries
+         */
+        @Override
+        protected final Query createFieldQuery(Analyzer analyzer, BooleanClause.Occur operator, String field, String queryText,
+                                               boolean quoted, int phraseSlop) {
+            assert operator == BooleanClause.Occur.SHOULD || operator == BooleanClause.Occur.MUST;
+
+            // Use the analyzer to get all the tokens, and then build an appropriate
+            // query based on the analysis chain.
+            try (TokenStream source = analyzer.tokenStream(field, queryText)) {
+                return createFieldQuery(source, operator, field, quoted, phraseSlop);
+            } catch (IOException e) {
+                throw new RuntimeException("Error analyzing query text", e);
+            }
+        }
+
+        /**
+         * Creates a query from a token stream.  Same logic as {@link #createFieldQuery(Analyzer, Occur, String, String, boolean, int)}
+         * with additional graph token stream detection.
+         *
+         * @param source     the token stream to create the query from
+         * @param operator   default boolean operator used for this query
+         * @param field      field to create queries against
+         * @param quoted     true if phrases should be generated when terms occur at more than one position
+         * @param phraseSlop slop factor for phrase/multiphrase queries
+         */
+        protected final Query createFieldQuery(TokenStream source, BooleanClause.Occur operator, String field, boolean quoted,
+                                               int phraseSlop) {
+            assert operator == BooleanClause.Occur.SHOULD || operator == BooleanClause.Occur.MUST;
+
+            // Build an appropriate query based on the analysis chain.
+            try (CachingTokenFilter stream = new CachingTokenFilter(source)) {
+
+                TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
+                PositionIncrementAttribute posIncAtt = stream.addAttribute(PositionIncrementAttribute.class);
+                PositionLengthAttribute posLenAtt = stream.addAttribute(PositionLengthAttribute.class);
+                FlagsAttribute flagsAtt = stream.addAttribute(FlagsAttribute.class);
+
+                if (termAtt == null) {
+                    return null;
+                }
+
+                // phase 1: read through the stream and assess the situation:
+                // counting the number of tokens/positions and marking if we have any synonyms.
+
+                int numTokens = 0;
+                int positionCount = 0;
+                boolean hasSynonyms = false;
+                boolean isGraph = false;
+
+                stream.reset();
+                while (stream.incrementToken()) {
+                    numTokens++;
+                    int positionIncrement = posIncAtt.getPositionIncrement();
+                    if (positionIncrement != 0) {
+                        positionCount += positionIncrement;
+                    } else {
+                        hasSynonyms = true;
+                    }
+
+                    int positionLength = posLenAtt.getPositionLength();
+                    if (!isGraph && positionLength > 1 && ((flagsAtt.getFlags() & GRAPH_FLAG) == GRAPH_FLAG)) {
+                        isGraph = true;
+                    }
+                }
+
+                // phase 2: based on token count, presence of synonyms, and options
+                // formulate a single term, boolean, or phrase.
+
+                if (numTokens == 0) {
+                    return null;
+                } else if (numTokens == 1) {
+                    // single term
+                    return analyzeTerm(field, stream);
+                } else if (isGraph) {
+                    // graph
+                    return analyzeGraph(stream, operator, field, quoted, phraseSlop);
+                } else if (quoted && positionCount > 1) {
+                    // phrase
+                    if (hasSynonyms) {
+                        // complex phrase with synonyms
+                        return analyzeMultiPhrase(field, stream, phraseSlop);
+                    } else {
+                        // simple phrase
+                        return analyzePhrase(field, stream, phraseSlop);
+                    }
+                } else {
+                    // boolean
+                    if (positionCount == 1) {
+                        // only one position, with synonyms
+                        return analyzeBoolean(field, stream);
+                    } else {
+                        // complex case: multiple positions
+                        return analyzeMultiBoolean(field, stream, operator);
+                    }
+                }
+            } catch (IOException e) {
+                throw new RuntimeException("Error analyzing query text", e);
+            }
+        }
+
         @Override
         protected Query newTermQuery(Term term) {
             return blendTermQuery(term, mapper);
@@ -325,7 +453,7 @@ public class MatchQuery {
                 Term[] terms = pq.getTerms();
                 int[] positions = pq.getPositions();
                 for (int i = 0; i < terms.length; i++) {
-                    prefixQuery.add(new Term[] {terms[i]}, positions[i]);
+                    prefixQuery.add(new Term[]{terms[i]}, positions[i]);
                 }
                 return boost == 1 ? prefixQuery : new BoostQuery(prefixQuery, boost);
             } else if (innerQuery instanceof MultiPhraseQuery) {
@@ -346,11 +474,13 @@ public class MatchQuery {
             return query;
         }
 
-        public Query createCommonTermsQuery(String field, String queryText, Occur highFreqOccur, Occur lowFreqOccur, float maxTermFrequency, MappedFieldType fieldType) {
+        public Query createCommonTermsQuery(String field, String queryText, Occur highFreqOccur, Occur lowFreqOccur, float
+            maxTermFrequency, MappedFieldType fieldType) {
             Query booleanQuery = createBooleanQuery(field, queryText, lowFreqOccur);
             if (booleanQuery != null && booleanQuery instanceof BooleanQuery) {
                 BooleanQuery bq = (BooleanQuery) booleanQuery;
-                ExtendedCommonTermsQuery query = new ExtendedCommonTermsQuery(highFreqOccur, lowFreqOccur, maxTermFrequency, ((BooleanQuery)booleanQuery).isCoordDisabled(), fieldType);
+                ExtendedCommonTermsQuery query = new ExtendedCommonTermsQuery(highFreqOccur, lowFreqOccur, maxTermFrequency, (
+                    (BooleanQuery) booleanQuery).isCoordDisabled(), fieldType);
                 for (BooleanClause clause : bq.clauses()) {
                     if (!(clause.getQuery() instanceof TermQuery)) {
                         return booleanQuery;
@@ -362,6 +492,30 @@ public class MatchQuery {
             return booleanQuery;
 
         }
+
+        /**
+         * Creates a query from a graph token stream by extracting all the finite strings from the graph and using them to create the query.
+         */
+        protected Query analyzeGraph(TokenStream source, BooleanClause.Occur operator, String field, boolean quoted, int phraseSlop)
+            throws IOException {
+            source.reset();
+            GraphTokenStreamFiniteStrings graphTokenStreams = new GraphTokenStreamFiniteStrings();
+            List<TokenStream> tokenStreams = graphTokenStreams.getTokenStreams(source);
+
+            if (tokenStreams.isEmpty()) {
+                return null;
+            }
+
+            List<Query> queries = new ArrayList<>(tokenStreams.size());
+            for (TokenStream ts : tokenStreams) {
+                Query query = createFieldQuery(ts, operator, field, quoted, phraseSlop);
+                if (query != null) {
+                    queries.add(query);
+                }
+            }
+
+            return new GraphQuery(queries.toArray(new Query[0]));
+        }
     }
 
     protected Query blendTermsQuery(Term[] terms, MappedFieldType fieldType) {

+ 17 - 16
core/src/main/java/org/elasticsearch/indices/analysis/AnalysisModule.java

@@ -152,13 +152,12 @@ import java.util.List;
  */
 public final class AnalysisModule {
     static {
-        Settings build = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
-                .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
-                .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
-                .build();
+        Settings build = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).put(IndexMetaData
+            .SETTING_NUMBER_OF_REPLICAS, 1).put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).build();
         IndexMetaData metaData = IndexMetaData.builder("_na_").settings(build).build();
         NA_INDEX_SETTINGS = new IndexSettings(metaData, Settings.EMPTY);
     }
+
     private static final IndexSettings NA_INDEX_SETTINGS;
 
     private final HunspellService hunspellService;
@@ -171,8 +170,8 @@ public final class AnalysisModule {
         NamedRegistry<AnalysisProvider<TokenFilterFactory>> tokenFilters = setupTokenFilters(plugins, hunspellService);
         NamedRegistry<AnalysisProvider<TokenizerFactory>> tokenizers = setupTokenizers(plugins);
         NamedRegistry<AnalysisProvider<AnalyzerProvider<?>>> analyzers = setupAnalyzers(plugins);
-        analysisRegistry = new AnalysisRegistry(environment, charFilters.getRegistry(), tokenFilters.getRegistry(),
-                tokenizers.getRegistry(), analyzers.getRegistry());
+        analysisRegistry = new AnalysisRegistry(environment, charFilters.getRegistry(), tokenFilters.getRegistry(), tokenizers
+            .getRegistry(), analyzers.getRegistry());
     }
 
     HunspellService getHunspellService() {
@@ -198,8 +197,8 @@ public final class AnalysisModule {
         return hunspellDictionaries;
     }
 
-    private NamedRegistry<AnalysisProvider<TokenFilterFactory>> setupTokenFilters(List<AnalysisPlugin> plugins,
-            HunspellService hunspellService) {
+    private NamedRegistry<AnalysisProvider<TokenFilterFactory>> setupTokenFilters(List<AnalysisPlugin> plugins, HunspellService
+        hunspellService) {
         NamedRegistry<AnalysisProvider<TokenFilterFactory>> tokenFilters = new NamedRegistry<>("token_filter");
         tokenFilters.register("stop", StopTokenFilterFactory::new);
         tokenFilters.register("reverse", ReverseTokenFilterFactory::new);
@@ -251,8 +250,8 @@ public final class AnalysisModule {
         tokenFilters.register("scandinavian_folding", ScandinavianFoldingFilterFactory::new);
         tokenFilters.register("serbian_normalization", SerbianNormalizationFilterFactory::new);
 
-        tokenFilters.register("hunspell", requriesAnalysisSettings(
-                (indexSettings, env, name, settings) -> new HunspellTokenFilterFactory(indexSettings, name, settings, hunspellService)));
+        tokenFilters.register("hunspell", requriesAnalysisSettings((indexSettings, env, name, settings) -> new HunspellTokenFilterFactory
+            (indexSettings, name, settings, hunspellService)));
         tokenFilters.register("cjk_bigram", CJKBigramFilterFactory::new);
         tokenFilters.register("cjk_width", CJKWidthFilterFactory::new);
 
@@ -341,6 +340,7 @@ public final class AnalysisModule {
             public T get(IndexSettings indexSettings, Environment environment, String name, Settings settings) throws IOException {
                 return provider.get(indexSettings, environment, name, settings);
             }
+
             @Override
             public boolean requiresAnalysisSettings() {
                 return true;
@@ -355,10 +355,11 @@ public final class AnalysisModule {
 
         /**
          * Creates a new analysis provider.
+         *
          * @param indexSettings the index settings for the index this provider is created for
-         * @param environment the nodes environment to load resources from persistent storage
-         * @param name the name of the analysis component
-         * @param settings the component specific settings without context prefixes
+         * @param environment   the nodes environment to load resources from persistent storage
+         * @param name          the name of the analysis component
+         * @param settings      the component specific settings without context prefixes
          * @return a new provider instance
          * @throws IOException if an {@link IOException} occurs
          */
@@ -369,11 +370,11 @@ public final class AnalysisModule {
          * This can be used to get a default instance of an analysis factory without binding to an index.
          *
          * @param environment the nodes environment to load resources from persistent storage
-         * @param name the name of the analysis component
+         * @param name        the name of the analysis component
          * @return a new provider instance
-         * @throws IOException if an {@link IOException} occurs
+         * @throws IOException              if an {@link IOException} occurs
          * @throws IllegalArgumentException if the provider requires analysis settings ie. if {@link #requiresAnalysisSettings()} returns
-         *  <code>true</code>
+         *                                  <code>true</code>
          */
         default T get(Environment environment, String name) throws IOException {
             if (requiresAnalysisSettings()) {

+ 1 - 1
core/src/main/java/org/elasticsearch/indices/analysis/PreBuiltTokenFilters.java

@@ -386,7 +386,7 @@ public enum PreBuiltTokenFilters {
         public TokenStream create(TokenStream tokenStream, Version version) {
             return new LimitTokenCountFilter(tokenStream, LimitTokenCountFilterFactory.DEFAULT_MAX_TOKEN_COUNT, LimitTokenCountFilterFactory.DEFAULT_CONSUME_ALL_TOKENS);
         }
-    }
+    },
 
     ;
 

+ 1074 - 0
core/src/test/java/org/apache/lucene/analysis/synonym/SynonymGraphFilterTests.java

@@ -0,0 +1,1074 @@
+/*
+ * 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.MockAnalyzer;
+import org.apache.lucene.analysis.MockGraphTokenFilter;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.TokenStreamToAutomaton;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TokenStreamToTermAutomatonQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.CharsRefBuilder;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.AutomatonTestUtil;
+import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.TooComplexToDeterminizeException;
+import org.apache.lucene.util.automaton.Transition;
+import org.apache.lucene.util.fst.Util;
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class SynonymGraphFilterTests extends BaseTokenStreamTestCase {
+
+    /**
+     * Set a side effect by {@link #getAnalyzer}.
+     */
+    private SynonymGraphFilter synFilter;
+
+    // LUCENE-6664
+    public static void assertAnalyzesToPositions(Analyzer a, String input, String[] output, String[] types, int[] posIncrements, int[]
+        posLengths) throws IOException {
+        assertAnalyzesTo(a, input, output, null, null, types, posIncrements, posLengths);
+    }
+
+    public void testBasicKeepOrigOneOutput() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b", "x", true);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "c a b", new String[]{"c", "x", "a", "b"}, new int[]{0, 2, 2, 4}, new int[]{1, 5, 3, 5}, new String[]{"word",
+            "SYNONYM", "word", "word"}, new int[]{1, 1, 0, 1}, new int[]{1, 2, 1, 1});
+        a.close();
+    }
+
+    public void testMixedKeepOrig() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b", "x", true);
+        add(b, "e f", "y", false);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "c a b c e f g", new String[]{"c", "x", "a", "b", "c", "y", "g"}, new int[]{0, 2, 2, 4, 6, 8, 12}, new
+            int[]{1, 5, 3, 5, 7, 11, 13}, new String[]{"word", "SYNONYM", "word", "word", "word", "SYNONYM", "word"}, new
+            int[]{1, 1, 0,
+            1, 1, 1, 1}, new int[]{1, 2, 1, 1, 1, 1, 1});
+        a.close();
+    }
+
+    public void testNoParseAfterBuffer() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "b a", "x", true);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "b b b", new String[]{"b", "b", "b"}, new int[]{0, 2, 4}, new int[]{1, 3, 5}, new String[]{"word", "word",
+            "word"}, new int[]{1, 1, 1}, new int[]{1, 1, 1});
+        a.close();
+    }
+
+    public void testOneInputMultipleOutputKeepOrig() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b", "x", true);
+        add(b, "a b", "y", true);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "c a b c", new String[]{"c", "x", "y", "a", "b", "c"}, new int[]{0, 2, 2, 2, 4, 6}, new int[]{1, 5, 5, 3, 5,
+            7}, new String[]{"word", "SYNONYM", "SYNONYM", "word", "word", "word"}, new int[]{1, 1, 0, 0, 1, 1, 1, 1}, new
+            int[]{1, 2, 2,
+            1, 1, 1, 1, 1});
+        a.close();
+    }
+
+    /**
+     * parse a syn file with bad syntax
+     */
+    public void testInvalidAnalyzesToNothingOutput() throws Exception {
+        String testFile = "a => 1";
+        Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, false);
+        SolrSynonymParser parser = new SolrSynonymParser(true, true, analyzer);
+        try {
+            parser.parse(new StringReader(testFile));
+            fail("didn't get expected exception");
+        } catch (ParseException expected) {
+            // expected exc
+        }
+        analyzer.close();
+    }
+
+    /**
+     * parse a syn file with bad syntax
+     */
+    public void testInvalidDoubleMap() throws Exception {
+        String testFile = "a => b => c";
+        Analyzer analyzer = new MockAnalyzer(random());
+        SolrSynonymParser parser = new SolrSynonymParser(true, true, analyzer);
+        try {
+            parser.parse(new StringReader(testFile));
+            fail("didn't get expected exception");
+        } catch (ParseException expected) {
+            // expected exc
+        }
+        analyzer.close();
+    }
+
+    public void testMoreThanOneLookAhead() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b c d", "x", true);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "a b c e", new String[]{"a", "b", "c", "e"}, new int[]{0, 2, 4, 6}, new int[]{1, 3, 5, 7}, new
+            String[]{"word", "word", "word", "word"}, new int[]{1, 1, 1, 1}, new int[]{1, 1, 1, 1});
+        a.close();
+    }
+
+    public void testLookaheadAfterParse() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "b b", "x", true);
+        add(b, "b", "y", true);
+
+        Analyzer a = getAnalyzer(b, true);
+
+        assertAnalyzesTo(a, "b a b b", new String[]{"y", "b", "a", "x", "b", "b"}, new int[]{0, 0, 2, 4, 4, 6}, new int[]{1, 1, 3, 7, 5,
+            7}, null, new int[]{1, 0, 1, 1, 0, 1}, new int[]{1, 1, 1, 2, 1, 1}, true);
+    }
+
+    public void testLookaheadSecondParse() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "b b b", "x", true);
+        add(b, "b", "y", true);
+
+        Analyzer a = getAnalyzer(b, true);
+
+        assertAnalyzesTo(a, "b b", new String[]{"y", "b", "y", "b"}, new int[]{0, 0, 2, 2}, new int[]{1, 1, 3, 3}, null, new int[]{1, 0,
+            1, 0}, new int[]{1, 1, 1, 1}, true);
+    }
+
+    public void testOneInputMultipleOutputNoKeepOrig() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b", "x", false);
+        add(b, "a b", "y", false);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "c a b c", new String[]{"c", "x", "y", "c"}, new int[]{0, 2, 2, 6}, new int[]{1, 5, 5, 7}, new
+            String[]{"word", "SYNONYM", "SYNONYM", "word"}, new int[]{1, 1, 0, 1}, new int[]{1, 1, 1, 1});
+        a.close();
+    }
+
+    public void testOneInputMultipleOutputMixedKeepOrig() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b", "x", true);
+        add(b, "a b", "y", false);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "c a b c", new String[]{"c", "x", "y", "a", "b", "c"}, new int[]{0, 2, 2, 2, 4, 6}, new int[]{1, 5, 5, 3, 5,
+            7}, new String[]{"word", "SYNONYM", "SYNONYM", "word", "word", "word"}, new int[]{1, 1, 0, 0, 1, 1, 1, 1}, new
+            int[]{1, 2, 2,
+            1, 1, 1, 1, 1});
+        a.close();
+    }
+
+    public void testSynAtEnd() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b", "x", true);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "c d e a b", new String[]{"c", "d", "e", "x", "a", "b"}, new int[]{0, 2, 4, 6, 6, 8}, new int[]{1, 3, 5, 9,
+            7, 9}, new String[]{"word", "word", "word", "SYNONYM", "word", "word"}, new int[]{1, 1, 1, 1, 0, 1}, new int[]{1, 1, 1,
+            2, 1,
+            1});
+        a.close();
+    }
+
+    public void testTwoSynsInARow() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a", "x", false);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "c a a b", new String[]{"c", "x", "x", "b"}, new int[]{0, 2, 4, 6}, new int[]{1, 3, 5, 7}, new
+            String[]{"word", "SYNONYM", "SYNONYM", "word"}, new int[]{1, 1, 1, 1}, new int[]{1, 1, 1, 1});
+        a.close();
+    }
+
+    public void testBasicKeepOrigTwoOutputs() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b", "x y", true);
+        add(b, "a b", "m n o", true);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "c a b d", new String[]{"c", "x", "m", "a", "y", "n", "o", "b", "d"}, new int[]{0, 2, 2, 2, 2, 2, 2, 4, 6},
+            new int[]{1, 5, 5, 3, 5, 5, 5, 5, 7}, new String[]{"word", "SYNONYM", "SYNONYM", "word", "SYNONYM",
+                "SYNONYM", "SYNONYM",
+                "word", "word"}, new int[]{1, 1, 0, 0, 1, 1, 1, 1, 1}, new int[]{1, 1, 2, 4, 4, 1, 2, 1, 1});
+        a.close();
+    }
+
+    public void testNoCaptureIfNoMatch() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b", "x y", true);
+
+        Analyzer a = getAnalyzer(b, true);
+
+        assertAnalyzesTo(a, "c d d", new String[]{"c", "d", "d"}, new int[]{0, 2, 4}, new int[]{1, 3, 5}, new String[]{"word", "word",
+            "word"}, new int[]{1, 1, 1}, new int[]{1, 1, 1});
+        assertEquals(0, synFilter.getCaptureCount());
+        a.close();
+    }
+
+    public void testBasicNotKeepOrigOneOutput() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b", "x", false);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "c a b", new String[]{"c", "x"}, new int[]{0, 2}, new int[]{1, 5}, new String[]{"word", "SYNONYM"}, new
+            int[]{1, 1}, new int[]{1, 1});
+        a.close();
+    }
+
+    public void testBasicNoKeepOrigTwoOutputs() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b", "x y", false);
+        add(b, "a b", "m n o", false);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "c a b d", new String[]{"c", "x", "m", "y", "n", "o", "d"}, new int[]{0, 2, 2, 2, 2, 2, 6}, new int[]{1, 5,
+            5, 5, 5, 5, 7}, new String[]{"word", "SYNONYM", "SYNONYM", "SYNONYM", "SYNONYM", "SYNONYM",
+            "word"}, new int[]{1, 1, 0, 1, 1,
+            1, 1}, new int[]{1, 1, 2, 3, 1, 1, 1});
+        a.close();
+    }
+
+    public void testIgnoreCase() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b", "x y", false);
+        add(b, "a b", "m n o", false);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "c A B D", new String[]{"c", "x", "m", "y", "n", "o", "D"}, new int[]{0, 2, 2, 2, 2, 2, 6}, new int[]{1, 5,
+            5, 5, 5, 5, 7}, new String[]{"word", "SYNONYM", "SYNONYM", "SYNONYM", "SYNONYM", "SYNONYM",
+            "word"}, new int[]{1, 1, 0, 1, 1,
+            1, 1}, new int[]{1, 1, 2, 3, 1, 1, 1});
+        a.close();
+    }
+
+    public void testDoNotIgnoreCase() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b", "x y", false);
+        add(b, "a b", "m n o", false);
+
+        Analyzer a = getAnalyzer(b, false);
+        assertAnalyzesTo(a, "c A B D", new String[]{"c", "A", "B", "D"}, new int[]{0, 2, 4, 6}, new int[]{1, 3, 5, 7}, new
+            String[]{"word", "word", "word", "word"}, new int[]{1, 1, 1, 1}, new int[]{1, 1, 1, 1});
+        a.close();
+    }
+
+    public void testBufferedFinish1() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b c", "m n o", false);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "c a b", new String[]{"c", "a", "b"}, new int[]{0, 2, 4}, new int[]{1, 3, 5}, new String[]{"word", "word",
+            "word"}, new int[]{1, 1, 1}, new int[]{1, 1, 1});
+        a.close();
+    }
+
+    public void testBufferedFinish2() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b", "m n o", false);
+        add(b, "d e", "m n o", false);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "c a d", new String[]{"c", "a", "d"}, new int[]{0, 2, 4}, new int[]{1, 3, 5}, new String[]{"word", "word",
+            "word"}, new int[]{1, 1, 1}, new int[]{1, 1, 1});
+        a.close();
+    }
+
+    public void testCanReuse() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b", "x", true);
+        Analyzer a = getAnalyzer(b, true);
+        for (int i = 0; i < 10; i++) {
+            assertAnalyzesTo(a, "c a b", new String[]{"c", "x", "a", "b"}, new int[]{0, 2, 2, 4}, new int[]{1, 5, 3, 5}, new
+                String[]{"word", "SYNONYM", "word", "word"}, new int[]{1, 1, 0, 1}, new int[]{1, 2, 1, 1});
+        }
+        a.close();
+    }
+
+    /**
+     * Multiple input tokens map to a single output token
+     */
+    public void testManyToOne() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b c", "z", true);
+
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "a b c d", new String[]{"z", "a", "b", "c", "d"}, new int[]{0, 0, 2, 4, 6}, new int[]{5, 1, 3, 5, 7}, new
+            String[]{"SYNONYM", "word", "word", "word", "word"}, new int[]{1, 0, 1, 1, 1}, new int[]{3, 1, 1, 1, 1});
+        a.close();
+    }
+
+    public void testBufferAfterMatch() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "a b c d", "x", true);
+        add(b, "a b", "y", false);
+
+        // The 'c' token has to be buffered because SynGraphFilter
+        // needs to know whether a b c d -> x matches:
+        Analyzer a = getAnalyzer(b, true);
+        assertAnalyzesTo(a, "f a b c e", new String[]{"f", "y", "c", "e"}, new int[]{0, 2, 6, 8}, new int[]{1, 5, 7, 9}, new
+            String[]{"word", "SYNONYM", "word", "word"}, new int[]{1, 1, 1, 1}, new int[]{1, 1, 1, 1});
+        a.close();
+    }
+
+    public void testZeroSyns() throws Exception {
+        Tokenizer tokenizer = new MockTokenizer();
+        tokenizer.setReader(new StringReader("aa bb"));
+        try {
+            new SynonymGraphFilter(tokenizer, new SynonymMap.Builder(true).build(), true);
+            fail("did not hit expected exception");
+        } catch (IllegalArgumentException iae) {
+            // expected
+            assertEquals("fst must be non-null", iae.getMessage());
+        }
+    }
+
+    // Needs TermAutomatonQuery, which is in sandbox still:
+    public void testAccurateGraphQuery1() throws Exception {
+        Directory dir = newDirectory();
+        RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+        Document doc = new Document();
+        doc.add(newTextField("field", "wtf happened", Field.Store.NO));
+        w.addDocument(doc);
+        IndexReader r = w.getReader();
+        w.close();
+
+        IndexSearcher s = newSearcher(r);
+
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "what the fudge", "wtf", true);
+
+        SynonymMap map = b.build();
+
+        TokenStreamToTermAutomatonQuery ts2q = new TokenStreamToTermAutomatonQuery();
+
+
+        TokenStream in = new CannedTokenStream(0, 23, token("what", 1, 1, 0, 4), token("the", 1, 1, 5, 8), token("fudge", 1, 1, 9, 14),
+            token("happened", 1, 1, 15, 23));
+
+        assertEquals(1, s.count(ts2q.toQuery("field", new SynonymGraphFilter(in, map, true))));
+
+        in = new CannedTokenStream(0, 12, token("wtf", 1, 1, 0, 3), token("happened", 1, 1, 4, 12));
+
+        assertEquals(1, s.count(ts2q.toQuery("field", new SynonymGraphFilter(in, map, true))));
+
+        // "what happened" should NOT match:
+        in = new CannedTokenStream(0, 13, token("what", 1, 1, 0, 4), token("happened", 1, 1, 5, 13));
+        assertEquals(0, s.count(ts2q.toQuery("field", new SynonymGraphFilter(in, map, true))));
+
+        IOUtils.close(r, dir);
+    }
+
+
+    /**
+     * If we expand synonyms at search time, the results are correct.
+     */
+    // Needs TermAutomatonQuery, which is in sandbox still:
+    public void testAccurateGraphQuery2() throws Exception {
+        Directory dir = newDirectory();
+        RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+        Document doc = new Document();
+        doc.add(newTextField("field", "say wtf happened", Field.Store.NO));
+        w.addDocument(doc);
+        IndexReader r = w.getReader();
+        w.close();
+
+        IndexSearcher s = newSearcher(r);
+
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "fudge", "chocolate", true);
+        add(b, "what the fudge", "wtf", true);
+        add(b, "what the", "wut", true);
+        add(b, "say", "say what", true);
+
+        SynonymMap map = b.build();
+
+        TokenStream in = new CannedTokenStream(0, 26, token("say", 1, 1, 0, 3), token("what", 1, 1, 3, 7), token("the", 1, 1, 8, 11),
+            token("fudge", 1, 1, 12, 17), token("happened", 1, 1, 18, 26));
+
+        TokenStreamToTermAutomatonQuery ts2q = new TokenStreamToTermAutomatonQuery();
+
+        assertEquals(1, s.count(ts2q.toQuery("field", new SynonymGraphFilter(in, map, true))));
+
+        // "what happened" should NOT match:
+        in = new CannedTokenStream(0, 13, token("what", 1, 1, 0, 4), token("happened", 1, 1, 5, 13));
+        assertEquals(0, s.count(ts2q.toQuery("field", new SynonymGraphFilter(in, map, true))));
+
+        IOUtils.close(r, dir);
+    }
+
+
+    // Needs TermAutomatonQuery, which is in sandbox still:
+    public void testAccurateGraphQuery3() throws Exception {
+        Directory dir = newDirectory();
+        RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+        Document doc = new Document();
+        doc.add(newTextField("field", "say what the fudge happened", Field.Store.NO));
+        w.addDocument(doc);
+        IndexReader r = w.getReader();
+        w.close();
+
+        IndexSearcher s = newSearcher(r);
+
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        add(b, "wtf", "what the fudge", true);
+
+        SynonymMap map = b.build();
+
+        TokenStream in = new CannedTokenStream(0, 15, token("say", 1, 1, 0, 3), token("wtf", 1, 1, 3, 6), token("happened", 1, 1, 7, 15));
+
+        TokenStreamToTermAutomatonQuery ts2q = new TokenStreamToTermAutomatonQuery();
+
+        assertEquals(1, s.count(ts2q.toQuery("field", new SynonymGraphFilter(in, map, true))));
+
+        // "what happened" should NOT match:
+        in = new CannedTokenStream(0, 13, token("what", 1, 1, 0, 4), token("happened", 1, 1, 5, 13));
+        assertEquals(0, s.count(ts2q.toQuery("field", new SynonymGraphFilter(in, map, true))));
+
+        IOUtils.close(r, dir);
+    }
+
+    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;
+    }
+
+    private String randomNonEmptyString() {
+        while (true) {
+            String s = TestUtil.randomUnicodeString(random()).trim();
+            //String s = TestUtil.randomSimpleString(random()).trim();
+            if (s.length() != 0 && s.indexOf('\u0000') == -1) {
+                return s;
+            }
+        }
+    }
+
+    // Adds MockGraphTokenFilter after SynFilter:
+    public void testRandomGraphAfter() throws Exception {
+        final int numIters = atLeast(3);
+        for (int i = 0; i < numIters; i++) {
+            SynonymMap.Builder b = new SynonymMap.Builder(random().nextBoolean());
+            final int numEntries = atLeast(10);
+            for (int j = 0; j < numEntries; j++) {
+                add(b, randomNonEmptyString(), randomNonEmptyString(), random().nextBoolean());
+            }
+            final SynonymMap map = b.build();
+            final boolean ignoreCase = random().nextBoolean();
+
+            final Analyzer analyzer = new Analyzer() {
+                @Override
+                protected TokenStreamComponents createComponents(String fieldName) {
+                    Tokenizer tokenizer = new MockTokenizer(MockTokenizer.SIMPLE, true);
+                    TokenStream syns = new SynonymGraphFilter(tokenizer, map, ignoreCase);
+                    TokenStream graph = new MockGraphTokenFilter(random(), syns);
+                    return new TokenStreamComponents(tokenizer, graph);
+                }
+            };
+
+            checkRandomData(random(), analyzer, 100);
+            analyzer.close();
+        }
+    }
+
+    public void testEmptyStringInput() throws IOException {
+        final int numIters = atLeast(10);
+        for (int i = 0; i < numIters; i++) {
+            SynonymMap.Builder b = new SynonymMap.Builder(random().nextBoolean());
+            final int numEntries = atLeast(10);
+            for (int j = 0; j < numEntries; j++) {
+                add(b, randomNonEmptyString(), randomNonEmptyString(), random().nextBoolean());
+            }
+            final boolean ignoreCase = random().nextBoolean();
+
+            Analyzer analyzer = getAnalyzer(b, ignoreCase);
+
+            checkAnalysisConsistency(random(), analyzer, random().nextBoolean(), "");
+            analyzer.close();
+        }
+    }
+
+    /**
+     * simple random test, doesn't verify correctness.
+     * does verify it doesnt throw exceptions, or that the stream doesn't misbehave
+     */
+    public void testRandom2() throws Exception {
+        final int numIters = atLeast(3);
+        for (int i = 0; i < numIters; i++) {
+            SynonymMap.Builder b = new SynonymMap.Builder(random().nextBoolean());
+            final int numEntries = atLeast(10);
+            for (int j = 0; j < numEntries; j++) {
+                add(b, randomNonEmptyString(), randomNonEmptyString(), random().nextBoolean());
+            }
+            final boolean ignoreCase = random().nextBoolean();
+
+            Analyzer analyzer = getAnalyzer(b, ignoreCase);
+            checkRandomData(random(), analyzer, 100);
+            analyzer.close();
+        }
+    }
+
+    /**
+     * simple random test like testRandom2, but for larger docs
+     */
+    public void testRandomHuge() throws Exception {
+        final int numIters = atLeast(3);
+        for (int i = 0; i < numIters; i++) {
+            SynonymMap.Builder b = new SynonymMap.Builder(random().nextBoolean());
+            final int numEntries = atLeast(10);
+            //if (VERBOSE) {
+            //System.out.println("TEST: iter=" + i + " numEntries=" + numEntries);
+            //}
+            for (int j = 0; j < numEntries; j++) {
+                add(b, randomNonEmptyString(), randomNonEmptyString(), random().nextBoolean());
+            }
+            final boolean ignoreCase = random().nextBoolean();
+
+            Analyzer analyzer = getAnalyzer(b, ignoreCase);
+            checkRandomData(random(), analyzer, 100, 1024);
+            analyzer.close();
+        }
+    }
+
+    public void testEmptyTerm() throws IOException {
+        final int numIters = atLeast(10);
+        for (int i = 0; i < numIters; i++) {
+            SynonymMap.Builder b = new SynonymMap.Builder(random().nextBoolean());
+            final int numEntries = atLeast(10);
+            for (int j = 0; j < numEntries; j++) {
+                add(b, randomNonEmptyString(), randomNonEmptyString(), random().nextBoolean());
+            }
+            final boolean ignoreCase = random().nextBoolean();
+
+            final Analyzer analyzer = getAnalyzer(b, ignoreCase);
+
+            checkAnalysisConsistency(random(), analyzer, random().nextBoolean(), "");
+            analyzer.close();
+        }
+    }
+
+    public void testBuilderDedup() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        final boolean keepOrig = false;
+        add(b, "a b", "ab", keepOrig);
+        add(b, "a b", "ab", keepOrig);
+        add(b, "a b", "ab", keepOrig);
+        Analyzer a = getAnalyzer(b, true);
+
+        assertAnalyzesTo(a, "a b", new String[]{"ab"}, new int[]{1});
+        a.close();
+    }
+
+    public void testBuilderNoDedup() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(false);
+        final boolean keepOrig = false;
+        add(b, "a b", "ab", keepOrig);
+        add(b, "a b", "ab", keepOrig);
+        add(b, "a b", "ab", keepOrig);
+        Analyzer a = getAnalyzer(b, true);
+
+        assertAnalyzesTo(a, "a b", new String[]{"ab", "ab", "ab"}, new int[]{1, 0, 0});
+        a.close();
+    }
+
+    public void testRecursion1() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        final boolean keepOrig = false;
+        add(b, "zoo", "zoo", keepOrig);
+        Analyzer a = getAnalyzer(b, true);
+
+        assertAnalyzesTo(a, "zoo zoo $ zoo", new String[]{"zoo", "zoo", "$", "zoo"}, new int[]{1, 1, 1, 1});
+        a.close();
+    }
+
+    public void testRecursion2() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        final boolean keepOrig = false;
+        add(b, "zoo", "zoo", keepOrig);
+        add(b, "zoo", "zoo zoo", keepOrig);
+        Analyzer a = getAnalyzer(b, true);
+
+        // verify("zoo zoo $ zoo", "zoo/zoo zoo/zoo/zoo $/zoo zoo/zoo zoo");
+        assertAnalyzesTo(a, "zoo zoo $ zoo", new String[]{"zoo", "zoo", "zoo", "zoo", "zoo", "zoo", "$", "zoo", "zoo", "zoo"}, new
+            int[]{1, 0, 1, 1, 0, 1, 1, 1, 0, 1});
+        a.close();
+    }
+
+    public void testKeepOrig() throws Exception {
+        SynonymMap.Builder b = new SynonymMap.Builder(true);
+        final boolean keepOrig = true;
+        add(b, "a b", "ab", keepOrig);
+        add(b, "a c", "ac", keepOrig);
+        add(b, "a", "aa", keepOrig);
+        add(b, "b", "bb", keepOrig);
+        add(b, "z x c v", "zxcv", keepOrig);
+        add(b, "x c", "xc", keepOrig);
+        Analyzer a = getAnalyzer(b, true);
+
+        assertAnalyzesTo(a, "$", new String[]{"$"}, new int[]{1});
+        assertAnalyzesTo(a, "a", new String[]{"aa", "a"}, new int[]{1, 0});
+        assertAnalyzesTo(a, "a", new String[]{"aa", "a"}, new int[]{1, 0});
+        assertAnalyzesTo(a, "$ a", new String[]{"$", "aa", "a"}, new int[]{1, 1, 0});
+        assertAnalyzesTo(a, "a $", new String[]{"aa", "a", "$"}, new int[]{1, 0, 1});
+        assertAnalyzesTo(a, "$ a !", new String[]{"$", "aa", "a", "!"}, new int[]{1, 1, 0, 1});
+        assertAnalyzesTo(a, "a a", new String[]{"aa", "a", "aa", "a"}, new int[]{1, 0, 1, 0});
+        assertAnalyzesTo(a, "b", new String[]{"bb", "b"}, new int[]{1, 0});
+        assertAnalyzesTo(a, "z x c v", new String[]{"zxcv", "z", "x", "c", "v"}, new int[]{1, 0, 1, 1, 1});
+        assertAnalyzesTo(a, "z x c $", new String[]{"z", "xc", "x", "c", "$"}, new int[]{1, 1, 0, 1, 1});
+        a.close();
+    }
+
+    private Analyzer getAnalyzer(SynonymMap.Builder b, final boolean ignoreCase) throws IOException {
+        final SynonymMap map = b.build();
+        return new Analyzer() {
+            @Override
+            protected TokenStreamComponents createComponents(String fieldName) {
+                Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+                // Make a local variable so testRandomHuge doesn't share it across threads!
+                SynonymGraphFilter synFilter = new SynonymGraphFilter(tokenizer, map, ignoreCase);
+                SynonymGraphFilterTests.this.synFilter = synFilter;
+                return new TokenStreamComponents(tokenizer, synFilter);
+            }
+        };
+    }
+
+    private void add(SynonymMap.Builder b, String input, String output, boolean keepOrig) {
+        if (VERBOSE) {
+            //System.out.println("  add input=" + input + " output=" + output + " keepOrig=" + keepOrig);
+        }
+        CharsRefBuilder inputCharsRef = new CharsRefBuilder();
+        SynonymMap.Builder.join(input.split(" +"), inputCharsRef);
+
+        CharsRefBuilder outputCharsRef = new CharsRefBuilder();
+        SynonymMap.Builder.join(output.split(" +"), outputCharsRef);
+
+        b.add(inputCharsRef.get(), outputCharsRef.get(), keepOrig);
+    }
+
+    private char[] randomBinaryChars(int minLen, int maxLen, double bias, char base) {
+        int len = TestUtil.nextInt(random(), minLen, maxLen);
+        char[] chars = new char[len];
+        for (int i = 0; i < len; i++) {
+            char ch;
+            if (random().nextDouble() < bias) {
+                ch = base;
+            } else {
+                ch = (char) (base + 1);
+            }
+            chars[i] = ch;
+        }
+
+        return chars;
+    }
+
+    private static String toTokenString(char[] chars) {
+        StringBuilder b = new StringBuilder();
+        for (char c : chars) {
+            if (b.length() > 0) {
+                b.append(' ');
+            }
+            b.append(c);
+        }
+        return b.toString();
+    }
+
+    private static class OneSyn {
+        char[] in;
+        char[] out;
+        boolean keepOrig;
+
+        @Override
+        public String toString() {
+            return toTokenString(in) + " --> " + toTokenString(out) + " (keepOrig=" + keepOrig + ")";
+        }
+    }
+
+    public void testRandomSyns() throws Exception {
+        int synCount = atLeast(10);
+        double bias = random().nextDouble();
+        boolean dedup = random().nextBoolean();
+
+        SynonymMap.Builder b = new SynonymMap.Builder(dedup);
+        List<OneSyn> syns = new ArrayList<>();
+        // Makes random syns from random a / b tokens, mapping to random x / y tokens
+        //if (VERBOSE) {
+        //    System.out.println("TEST: make " + synCount + " syns");
+        //    System.out.println("  bias for a over b=" + bias);
+        //    System.out.println("  dedup=" + dedup);
+        //    System.out.println("  sausage=" + sausage);
+        //}
+
+        int maxSynLength = 0;
+
+        for (int i = 0; i < synCount; i++) {
+            OneSyn syn = new OneSyn();
+            syn.in = randomBinaryChars(1, 5, bias, 'a');
+            syn.out = randomBinaryChars(1, 5, 0.5, 'x');
+            syn.keepOrig = random().nextBoolean();
+            syns.add(syn);
+
+            maxSynLength = Math.max(maxSynLength, syn.in.length);
+
+            //if (VERBOSE) {
+            //    System.out.println("  " + syn);
+            //}
+            add(b, toTokenString(syn.in), toTokenString(syn.out), syn.keepOrig);
+        }
+
+        // Only used w/ VERBOSE:
+        Analyzer aNoSausageed;
+        if (VERBOSE) {
+            aNoSausageed = getAnalyzer(b, true);
+        } else {
+            aNoSausageed = null;
+        }
+
+        Analyzer a = getAnalyzer(b, true);
+        int iters = atLeast(20);
+        for (int iter = 0; iter < iters; iter++) {
+
+            String doc = toTokenString(randomBinaryChars(50, 100, bias, 'a'));
+            //String doc = toTokenString(randomBinaryChars(10, 50, bias, 'a'));
+
+            //if (VERBOSE) {
+            //    System.out.println("TEST: iter=" + iter + " doc=" + doc);
+            //}
+            Automaton expected = slowSynFilter(doc, syns);
+            if (VERBOSE) {
+                //System.out.println("  expected:\n" + expected.toDot());
+            }
+            Automaton actual = toAutomaton(a.tokenStream("field", new StringReader(doc)));
+            //if (VERBOSE) {
+            //    System.out.println("  actual:\n" + actual.toDot());
+            //}
+
+            assertTrue("maxLookaheadUsed=" + synFilter.getMaxLookaheadUsed() + " maxSynLength=" + maxSynLength, synFilter
+                .getMaxLookaheadUsed() <= maxSynLength);
+
+            checkAnalysisConsistency(random(), a, random().nextBoolean(), doc);
+            // We can easily have a non-deterministic automaton at this point, e.g. if
+            // more than one syn matched at given point, or if the syn mapped to an
+            // output token that also happens to be in the input:
+            try {
+                actual = Operations.determinize(actual, 50000);
+            } catch (TooComplexToDeterminizeException tctde) {
+                // Unfortunately the syns can easily create difficult-to-determinize graphs:
+                assertTrue(approxEquals(actual, expected));
+                continue;
+            }
+
+            try {
+                expected = Operations.determinize(expected, 50000);
+            } catch (TooComplexToDeterminizeException tctde) {
+                // Unfortunately the syns can easily create difficult-to-determinize graphs:
+                assertTrue(approxEquals(actual, expected));
+                continue;
+            }
+
+            assertTrue(approxEquals(actual, expected));
+            assertTrue(Operations.sameLanguage(actual, expected));
+        }
+
+        a.close();
+    }
+
+    /**
+     * Only used when true equality is too costly to check!
+     */
+    private boolean approxEquals(Automaton actual, Automaton expected) {
+        // Don't collapse these into one line else the thread stack won't say which direction failed!:
+        boolean b1 = approxSubsetOf(actual, expected);
+        boolean b2 = approxSubsetOf(expected, actual);
+        return b1 && b2;
+    }
+
+    private boolean approxSubsetOf(Automaton a1, Automaton a2) {
+        AutomatonTestUtil.RandomAcceptedStrings ras = new AutomatonTestUtil.RandomAcceptedStrings(a1);
+        for (int i = 0; i < 2000; i++) {
+            int[] ints = ras.getRandomAcceptedString(random());
+            IntsRef path = new IntsRef(ints, 0, ints.length);
+            if (accepts(a2, path) == false) {
+                throw new RuntimeException("a2 does not accept " + path);
+            }
+        }
+
+        // Presumed true
+        return true;
+    }
+
+    /**
+     * Like {@link Operations#run} except the incoming automaton is allowed to be non-deterministic.
+     */
+    private static boolean accepts(Automaton a, IntsRef path) {
+        Set<Integer> states = new HashSet<>();
+        states.add(0);
+        Transition t = new Transition();
+        for (int i = 0; i < path.length; i++) {
+            int digit = path.ints[path.offset + i];
+            Set<Integer> nextStates = new HashSet<>();
+            for (int state : states) {
+                int count = a.initTransition(state, t);
+                for (int j = 0; j < count; j++) {
+                    a.getNextTransition(t);
+                    if (digit >= t.min && digit <= t.max) {
+                        nextStates.add(t.dest);
+                    }
+                }
+            }
+            states = nextStates;
+            if (states.isEmpty()) {
+                return false;
+            }
+        }
+
+        for (int state : states) {
+            if (a.isAccept(state)) {
+                return true;
+            }
+        }
+
+        return false;
+    }
+
+    /**
+     * Stupid, slow brute-force, yet hopefully bug-free, synonym filter.
+     */
+    private Automaton slowSynFilter(String doc, List<OneSyn> syns) {
+        String[] tokens = doc.split(" +");
+        //if (VERBOSE) {
+        //    System.out.println("  doc has " + tokens.length + " tokens");
+        //}
+        int i = 0;
+        Automaton.Builder a = new Automaton.Builder();
+        int lastState = a.createState();
+        while (i < tokens.length) {
+            // Consider all possible syn matches starting at this point:
+            assert tokens[i].length() == 1;
+            //if (VERBOSE) {
+            //    System.out.println("    i=" + i);
+            //}
+
+            List<OneSyn> matches = new ArrayList<>();
+            for (OneSyn syn : syns) {
+                if (i + syn.in.length <= tokens.length) {
+                    boolean match = true;
+                    for (int j = 0; j < syn.in.length; j++) {
+                        if (tokens[i + j].charAt(0) != syn.in[j]) {
+                            match = false;
+                            break;
+                        }
+                    }
+
+                    if (match) {
+                        if (matches.isEmpty() == false) {
+                            if (syn.in.length < matches.get(0).in.length) {
+                                // Greedy matching: we already found longer syns matching here
+                                continue;
+                            } else if (syn.in.length > matches.get(0).in.length) {
+                                // Greedy matching: all previous matches were shorter, so we drop them
+                                matches.clear();
+                            } else {
+                                // Keep the current matches: we allow multiple synonyms matching the same input string
+                            }
+                        }
+
+                        matches.add(syn);
+                    }
+                }
+            }
+
+            int nextState = a.createState();
+
+            if (matches.isEmpty() == false) {
+                // We have match(es) starting at this token
+                //if (VERBOSE) {
+                //    System.out.println("  matches @ i=" + i + ": " + matches);
+                //}
+                // We keepOrig if any of the matches said to:
+                boolean keepOrig = false;
+                for (OneSyn syn : matches) {
+                    keepOrig |= syn.keepOrig;
+                }
+
+                if (keepOrig) {
+                    // Add path for the original tokens
+                    addSidePath(a, lastState, nextState, matches.get(0).in);
+                }
+
+                for (OneSyn syn : matches) {
+                    addSidePath(a, lastState, nextState, syn.out);
+                }
+
+                i += matches.get(0).in.length;
+            } else {
+                a.addTransition(lastState, nextState, tokens[i].charAt(0));
+                i++;
+            }
+
+            lastState = nextState;
+        }
+
+        a.setAccept(lastState, true);
+
+        return topoSort(a.finish());
+    }
+
+    /**
+     * Just creates a side path from startState to endState with the provided tokens.
+     */
+    private static void addSidePath(Automaton.Builder a, int startState, int endState, char[] tokens) {
+        int lastState = startState;
+        for (int i = 0; i < tokens.length; i++) {
+            int nextState;
+            if (i == tokens.length - 1) {
+                nextState = endState;
+            } else {
+                nextState = a.createState();
+            }
+
+            a.addTransition(lastState, nextState, tokens[i]);
+
+            lastState = nextState;
+        }
+    }
+
+    private Automaton toAutomaton(TokenStream ts) throws IOException {
+        PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
+        PositionLengthAttribute posLenAtt = ts.addAttribute(PositionLengthAttribute.class);
+        CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
+        ts.reset();
+        Automaton a = new Automaton();
+        int srcNode = -1;
+        int destNode = -1;
+        int state = a.createState();
+        while (ts.incrementToken()) {
+            assert termAtt.length() == 1;
+            char c = termAtt.charAt(0);
+            int posInc = posIncAtt.getPositionIncrement();
+            if (posInc != 0) {
+                srcNode += posInc;
+                while (state < srcNode) {
+                    state = a.createState();
+                }
+            }
+            destNode = srcNode + posLenAtt.getPositionLength();
+            while (state < destNode) {
+                state = a.createState();
+            }
+            a.addTransition(srcNode, destNode, c);
+        }
+        ts.end();
+        ts.close();
+        a.finishState();
+        a.setAccept(destNode, true);
+        return a;
+    }
+
+    /**
+     * Renumbers nodes according to their topo sort
+     */
+    private Automaton topoSort(Automaton in) {
+        int[] newToOld = Operations.topoSortStates(in);
+        int[] oldToNew = new int[newToOld.length];
+
+        Automaton.Builder a = new Automaton.Builder();
+        //System.out.println("remap:");
+        for (int i = 0; i < newToOld.length; i++) {
+            a.createState();
+            oldToNew[newToOld[i]] = i;
+            //System.out.println("  " + newToOld[i] + " -> " + i);
+            if (in.isAccept(newToOld[i])) {
+                a.setAccept(i, true);
+                //System.out.println("    **");
+            }
+        }
+
+        Transition t = new Transition();
+        for (int i = 0; i < newToOld.length; i++) {
+            int count = in.initTransition(newToOld[i], t);
+            for (int j = 0; j < count; j++) {
+                in.getNextTransition(t);
+                a.addTransition(i, oldToNew[t.dest], t.min, t.max);
+            }
+        }
+
+        return a.finish();
+    }
+
+    /**
+     * Helper method to validate all strings that can be generated from a token stream. Uses {@link
+     * TokenStreamToAutomaton} to create an automaton. Asserts the finite strings of the automaton
+     * are all and only the given valid strings.
+     *
+     * @param analyzer        analyzer containing the SynonymFilter under test.
+     * @param text            text to be analyzed.
+     * @param expectedStrings all expected finite strings.
+     */
+    public void assertAllStrings(Analyzer analyzer, String text, String[] expectedStrings) throws IOException {
+        TokenStream tokenStream = analyzer.tokenStream("dummy", text);
+        try {
+            Automaton automaton = new TokenStreamToAutomaton().toAutomaton(tokenStream);
+            Set<IntsRef> finiteStrings = AutomatonTestUtil.getFiniteStringsRecursive(automaton, -1);
+
+            assertEquals("Invalid resulting strings count. Expected " + expectedStrings.length + " was " + finiteStrings.size(),
+                expectedStrings.length, finiteStrings.size());
+
+            Set<String> expectedStringsSet = new HashSet<>(Arrays.asList(expectedStrings));
+
+            BytesRefBuilder scratchBytesRefBuilder = new BytesRefBuilder();
+            for (IntsRef ir : finiteStrings) {
+                String s = Util.toBytesRef(ir, scratchBytesRefBuilder).utf8ToString().replace((char) TokenStreamToAutomaton.POS_SEP, ' ');
+                assertTrue("Unexpected string found: " + s, expectedStringsSet.contains(s));
+            }
+        } finally {
+            tokenStream.close();
+        }
+    }
+}

+ 154 - 0
core/src/test/java/org/elasticsearch/index/search/MatchQueryIT.java

@@ -0,0 +1,154 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch 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.elasticsearch.index.search;
+
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoSearchHits;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits;
+
+import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
+import org.elasticsearch.action.index.IndexRequestBuilder;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.index.query.Operator;
+import org.elasticsearch.index.query.QueryBuilders;
+import org.elasticsearch.test.ESIntegTestCase;
+import org.junit.Before;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+public class MatchQueryIT extends ESIntegTestCase {
+    private static final String INDEX = "test";
+
+    /**
+     * Test setup.
+     */
+    @Before
+    public void setUp() throws Exception {
+        super.setUp();
+        CreateIndexRequestBuilder builder = prepareCreate(INDEX).setSettings(
+            Settings.builder()
+                .put(indexSettings())
+                .put("index.analysis.filter.syns.type", "synonym")
+                .putArray("index.analysis.filter.syns.synonyms", "wtf, what the fudge", "foo, bar baz")
+                .put("index.analysis.analyzer.lower_syns.type", "custom")
+                .put("index.analysis.analyzer.lower_syns.tokenizer", "standard")
+                .putArray("index.analysis.analyzer.lower_syns.filter", "lowercase", "syns")
+                .put("index.analysis.filter.graphsyns.type", "synonym_graph")
+                .putArray("index.analysis.filter.graphsyns.synonyms", "wtf, what the fudge", "foo, bar baz")
+                .put("index.analysis.analyzer.lower_graphsyns.type", "custom")
+                .put("index.analysis.analyzer.lower_graphsyns.tokenizer", "standard")
+                .putArray("index.analysis.analyzer.lower_graphsyns.filter", "lowercase", "graphsyns")
+        );
+
+        assertAcked(builder.addMapping(INDEX, createMapping()));
+        ensureGreen();
+
+        List<IndexRequestBuilder> builders = new ArrayList<>();
+        builders.add(client().prepareIndex("test", "test", "1").setSource("field", "say wtf happened foo"));
+        builders.add(client().prepareIndex("test", "test", "2").setSource("field", "bar baz what the fudge man"));
+        builders.add(client().prepareIndex("test", "test", "3").setSource("field", "wtf"));
+        builders.add(client().prepareIndex("test", "test", "4").setSource("field", "what is the name for fudge"));
+        builders.add(client().prepareIndex("test", "test", "5").setSource("field", "bar two three"));
+        builders.add(client().prepareIndex("test", "test", "6").setSource("field", "bar baz two three"));
+
+        indexRandom(true, false, builders);
+    }
+
+    /**
+     * Setup the index mappings for the test index.
+     *
+     * @return the json builder with the index mappings
+     * @throws IOException on error creating mapping json
+     */
+    private XContentBuilder createMapping() throws IOException {
+        return XContentFactory.jsonBuilder()
+            .startObject()
+                .startObject(INDEX)
+                    .startObject("properties")
+                        .startObject("field")
+                            .field("type", "text")
+                        .endObject()
+                    .endObject()
+                .endObject()
+            .endObject();
+    }
+
+    public void testSimpleMultiTermPhrase() throws ExecutionException, InterruptedException {
+        // first search using regular synonym field using phrase
+        SearchResponse searchResponse = client().prepareSearch(INDEX)
+            .setQuery(QueryBuilders.matchPhraseQuery("field", "foo two three").analyzer("lower_syns")).get();
+
+        // because foo -> "bar baz" where "foo" and "bar" at position 0, "baz" and "two" at position 1.
+        // "bar two three", "bar baz three", "foo two three", "foo baz three"
+        assertHitCount(searchResponse, 1L);
+        assertSearchHits(searchResponse, "5"); // we should not match this but we do
+
+        // same query using graph should find correct result
+        searchResponse = client().prepareSearch(INDEX).setQuery(QueryBuilders.matchPhraseQuery("field", "foo two three")
+            .analyzer("lower_graphsyns")).get();
+
+        assertHitCount(searchResponse, 1L);
+        assertSearchHits(searchResponse, "6");
+    }
+
+    public void testSimpleMultiTermAnd() throws ExecutionException, InterruptedException {
+        // first search using regular synonym field using phrase
+        SearchResponse searchResponse = client().prepareSearch(INDEX).setQuery(QueryBuilders.matchQuery("field", "say what the fudge")
+            .operator(Operator.AND).analyzer("lower_syns")).get();
+
+        // 0 = say, 1 = OR(wtf, what), 2 = the, 3 = fudge
+        // "the" and "fudge" are required here, even though they were part of the synonym which is also expanded
+        assertNoSearchHits(searchResponse);
+
+        // same query using graph should find correct result
+        searchResponse = client().prepareSearch(INDEX).setQuery(QueryBuilders.matchQuery("field", "say what the fudge")
+            .operator(Operator.AND).analyzer("lower_graphsyns")).get();
+
+        assertHitCount(searchResponse, 1L);
+        assertSearchHits(searchResponse, "1");
+    }
+
+    public void testMinShouldMatch() throws ExecutionException, InterruptedException {
+        // no min should match
+        SearchResponse searchResponse = client().prepareSearch(INDEX).setQuery(QueryBuilders.matchQuery("field", "three what the fudge foo")
+            .operator(Operator.OR).analyzer("lower_graphsyns")).get();
+
+        assertHitCount(searchResponse, 6L);
+        assertSearchHits(searchResponse, "1", "2", "3", "4", "5", "6");
+
+        // same query, with min_should_match of 2
+        searchResponse = client().prepareSearch(INDEX).setQuery(QueryBuilders.matchQuery("field", "three what the fudge foo")
+            .operator(Operator.OR).analyzer("lower_graphsyns").minimumShouldMatch("80%")).get();
+
+        // three wtf foo = 2 terms, match #1
+        // three wtf bar baz = 3 terms, match #6
+        // three what the fudge foo = 4 terms, no match
+        // three what the fudge bar baz = 4 terms, match #2
+        assertHitCount(searchResponse, 3L);
+        assertSearchHits(searchResponse, "1", "2", "6");
+    }
+}

+ 1 - 0
docs/build.gradle

@@ -92,6 +92,7 @@ buildRestTests.expectedUnconvertedCandidates = [
   'reference/analysis/tokenfilters/stemmer-tokenfilter.asciidoc',
   'reference/analysis/tokenfilters/stop-tokenfilter.asciidoc',
   'reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc',
+  'reference/analysis/tokenfilters/synonym-graph-tokenfilter.asciidoc',
   'reference/analysis/tokenfilters/word-delimiter-tokenfilter.asciidoc',
   'reference/cat/recovery.asciidoc',
   'reference/cat/shards.asciidoc',

+ 2 - 0
docs/reference/analysis/tokenfilters.asciidoc

@@ -47,6 +47,8 @@ include::tokenfilters/phonetic-tokenfilter.asciidoc[]
 
 include::tokenfilters/synonym-tokenfilter.asciidoc[]
 
+include::tokenfilters/synonym-graph-tokenfilter.asciidoc[]
+
 include::tokenfilters/compound-word-tokenfilter.asciidoc[]
 
 include::tokenfilters/reverse-tokenfilter.asciidoc[]

+ 152 - 0
docs/reference/analysis/tokenfilters/synonym-graph-tokenfilter.asciidoc

@@ -0,0 +1,152 @@
+[[analysis-synonym-graph-tokenfilter]]
+=== Synonym Graph Token Filter
+
+experimental[]
+
+The `synonym_graph` token filter allows to easily handle synonyms, 
+including multi-word synonyms correctly during the analysis process.
+
+In order to properly handle multi-word synonyms this token filter
+creates a "graph token stream" during processing.  For more information
+on this topic and it's various complexities, please read
+http://blog.mikemccandless.com/2012/04/lucenes-tokenstreams-are-actually.html[Lucene's TokenStreams are actually graphs!]
+by Michael McCandless.
+
+["NOTE",id="synonym-graph-index-note"]
+===============================
+This token filter is designed to be used as part of a search analyzer 
+only.  If you want to apply synonyms during indexing please use the 
+standard <<analysis-synonym-tokenfilter,synonym token filter>>.
+===============================
+
+["NOTE",id="synonym-graph-query-note"]
+===============================
+The graph token stream created by this token filter requires special
+query handling. Currently only the <<query-dsl-match-query, Match>> and 
+<<query-dsl-multi-match-query, Multi Match>> queries can do this.  Using 
+it with any other type of analyzed query will potentially result in 
+incorrect search results.
+===============================
+
+Synonyms are configured using a configuration file.
+Here is an example:
+
+[source,js]
+--------------------------------------------------
+{
+    "index" : {
+        "analysis" : {
+            "analyzer" : {
+                "search_synonyms" : {
+                    "tokenizer" : "whitespace",
+                    "filter" : ["graph_synonyms"]
+                }
+            },
+            "filter" : {
+                "graph_synonyms" : {
+                    "type" : "synonym_graph",
+                    "synonyms_path" : "analysis/synonym.txt"
+                }
+            }
+        }
+    }
+}
+--------------------------------------------------
+
+The above configures a `search_synonyms` filter, with a path of
+`analysis/synonym.txt` (relative to the `config` location). The
+`search_synonyms` analyzer is then configured with the filter. 
+Additional settings are: `ignore_case` (defaults to `false`), and 
+`expand` (defaults to `true`).
+
+The `tokenizer` parameter controls the tokenizers that will be used to
+tokenize the synonym, and defaults to the `whitespace` tokenizer.
+
+Two synonym formats are supported: Solr, WordNet.
+
+[float]
+==== Solr synonyms
+
+The following is a sample format of the file:
+
+[source,js]
+--------------------------------------------------
+# Blank lines and lines starting with pound are comments.
+
+# Explicit mappings match any token sequence on the LHS of "=>"
+# and replace with all alternatives on the RHS.  These types of mappings
+# ignore the expand parameter in the schema.
+# Examples:
+i-pod, i pod => ipod,
+sea biscuit, sea biscit => seabiscuit
+
+# Equivalent synonyms may be separated with commas and give
+# no explicit mapping.  In this case the mapping behavior will
+# be taken from the expand parameter in the schema.  This allows
+# the same synonym file to be used in different synonym handling strategies.
+# Examples:
+ipod, i-pod, i pod
+foozball , foosball
+universe , cosmos
+lol, laughing out loud
+
+# If expand==true, "ipod, i-pod, i pod" is equivalent
+# to the explicit mapping:
+ipod, i-pod, i pod => ipod, i-pod, i pod
+# If expand==false, "ipod, i-pod, i pod" is equivalent
+# to the explicit mapping:
+ipod, i-pod, i pod => ipod
+
+# Multiple synonym mapping entries are merged.
+foo => foo bar
+foo => baz
+# is equivalent to
+foo => foo bar, baz
+--------------------------------------------------
+
+You can also define synonyms for the filter directly in the
+configuration file (note use of `synonyms` instead of `synonyms_path`):
+
+[source,js]
+--------------------------------------------------
+{
+    "filter" : {
+        "synonym" : {
+            "type" : "synonym_graph",
+            "synonyms" : [
+                "lol, laughing out loud",
+                "universe, cosmos"
+            ] 
+        }
+    }
+}
+--------------------------------------------------
+
+However, it is recommended to define large synonyms set in a file using
+`synonyms_path`, because specifying them inline increases cluster size unnecessarily.
+
+[float]
+==== WordNet synonyms
+
+Synonyms based on http://wordnet.princeton.edu/[WordNet] format can be
+declared using `format`:
+
+[source,js]
+--------------------------------------------------
+{
+    "filter" : {
+        "synonym" : {
+            "type" : "synonym_graph",
+            "format" : "wordnet",
+            "synonyms" : [
+                "s(100000001,1,'abstain',v,1,0).",
+                "s(100000001,2,'refrain',v,1,0).",
+                "s(100000001,3,'desist',v,1,0)."
+            ]
+        }
+    }
+}
+--------------------------------------------------
+
+Using `synonyms_path` to define WordNet synonyms in a file is supported
+as well.