You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2018/05/31 20:39:36 UTC

[2/2] lucene-solr:LUCENE-8332: * ConcatenateGraphFilter now extends TokenStream and calls toAutomaton in incrementToken() * ConcatenateGraphFilter SEP_CHAR put back to SEP_LABEL (int) * ConcatenateGraphFilter end() like was before; no captureState * Adde

* ConcatenateGraphFilter now extends TokenStream and calls toAutomaton in incrementToken()
* ConcatenateGraphFilter SEP_CHAR put back to SEP_LABEL (int)
* ConcatenateGraphFilter end() like was before; no captureState
* Added a CompletionTokenStream wrapper to ConcatenateGraphFilter to
** provide the NRT suggester access to state it wants (now is private in CGF)
** setPayload (now removed from CGF).
* Fixed CGF Factory license header to be ASF


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

Branch: refs/heads/LUCENE-8332
Commit: d813eca1353d32c4187970141beefe7e784bd2a3
Parents: d60ba6a
Author: David Smiley <ds...@apache.org>
Authored: Thu May 31 16:39:09 2018 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Thu May 31 16:39:09 2018 -0400

----------------------------------------------------------------------
 .../miscellaneous/ConcatenateGraphFilter.java   | 108 ++++++++++---------
 .../ConcatenateGraphFilterFactory.java          |  55 ++++++----
 .../lucene/analysis/core/TestRandomChains.java  |   7 +-
 .../TestConcatenateGraphFilter.java             |  98 ++++++-----------
 .../TestConcatenateGraphFilterFactory.java      |  38 ++++++-
 .../suggest/document/CompletionAnalyzer.java    |   2 +-
 .../suggest/document/CompletionQuery.java       |   4 +-
 .../suggest/document/CompletionTokenStream.java |  82 ++++++++++++++
 .../search/suggest/document/ContextQuery.java   |  10 +-
 .../suggest/document/ContextSuggestField.java   |  17 ++-
 .../suggest/document/FuzzyCompletionQuery.java  |   3 +-
 .../suggest/document/NRTSuggesterBuilder.java   |   5 +-
 .../suggest/document/PrefixCompletionQuery.java |   3 +-
 .../search/suggest/document/SuggestField.java   |  12 +--
 .../document/TestContextSuggestField.java       |   4 +-
 .../suggest/document/TestSuggestField.java      |   2 +-
 16 files changed, 271 insertions(+), 179 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConcatenateGraphFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConcatenateGraphFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConcatenateGraphFilter.java
index e222d6b..b90ed45 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConcatenateGraphFilter.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConcatenateGraphFilter.java
@@ -18,11 +18,9 @@ package org.apache.lucene.analysis.miscellaneous;
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.TokenStreamToAutomaton;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.util.AttributeImpl;
@@ -47,7 +45,7 @@ import org.apache.lucene.util.fst.Util;
  *
  * @lucene.experimental
  */
-public final class ConcatenateGraphFilter extends TokenFilter {
+public final class ConcatenateGraphFilter extends TokenStream {
 
   /*
    * Token stream which converts a provided token stream to an automaton.
@@ -61,25 +59,22 @@ public final class ConcatenateGraphFilter extends TokenFilter {
    * Represents the separation between tokens, if
    * <code>preserveSep</code> is <code>true</code>.
    */
-  public final static char SEP_CHAR = '\u001F';
+  public final static int SEP_LABEL = TokenStreamToAutomaton.POS_SEP;
   public final static int DEFAULT_MAX_GRAPH_EXPANSIONS = Operations.DEFAULT_MAX_DETERMINIZED_STATES;
   public final static boolean DEFAULT_PRESERVE_SEP = true;
   public final static boolean DEFAULT_PRESERVE_POSITION_INCREMENTS = true;
 
-  private final PayloadAttribute payloadAttr = addAttribute(PayloadAttribute.class);
   private final BytesRefBuilderTermAttribute bytesAtt = addAttribute(BytesRefBuilderTermAttribute.class);
   private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
 
-  //nocommit add getters for these instead?  Accessor: org.apache.lucene.search.suggest.document.ContextSuggestField.wrapTokenStream
-  public final boolean preserveSep;
-  public final boolean preservePositionIncrements;
-  public final int maxGraphExpansions;
-  public TokenStream getInput() { return super.input; } // nocommit remove? see ContextSuggestField.wrapTokenStream
+  private final TokenStream inputTokenStream;
+  private final boolean preserveSep;
+  private final boolean preservePositionIncrements;
+  private final int maxGraphExpansions;
 
   private LimitedFiniteStringsIterator finiteStrings;
-  private BytesRef payload;
   private CharTermAttribute charTermAttribute;
-  private State endState;
+  private boolean wasReset = false;
 
   /**
    * Creates a token stream to convert <code>input</code> to a token stream
@@ -96,8 +91,11 @@ public final class ConcatenateGraphFilter extends TokenFilter {
    * of accepted strings by its token stream graph.
    *
    * @param inputTokenStream The input/incoming TokenStream
-   * @param preserveSep
-   * @param preservePositionIncrements Whether to generate holes in the automaton for missing positions
+   * @param preserveSep Whether {@link #SEP_LABEL} should separate the input tokens in the concatenated token
+   * @param preservePositionIncrements Whether to add an empty token for missing positions.
+   *                                   The effect is a consecutive {@link #SEP_LABEL}.
+   *                                   When false, it's as if there were no missing positions
+   *                                     (we pretend the surrounding tokens were adjacent).
    * @param maxGraphExpansions If the tokenStream graph has more than this many possible paths through, then we'll throw
    *                           {@link TooComplexToDeterminizeException} to preserve the stability and memory of the
    *                           machine.
@@ -106,22 +104,34 @@ public final class ConcatenateGraphFilter extends TokenFilter {
    *
    */
   public ConcatenateGraphFilter(TokenStream inputTokenStream, boolean preserveSep, boolean preservePositionIncrements, int maxGraphExpansions) {
-    super(inputTokenStream);
+    // Don't call the super(input) ctor - this is a true delegate and has a new attribute source since we consume
+    // the input stream entirely in the first call to incrementToken
+    this.inputTokenStream = inputTokenStream;
     this.preserveSep = preserveSep;
     this.preservePositionIncrements = preservePositionIncrements;
     this.maxGraphExpansions = maxGraphExpansions;
   }
 
-  /**
-   * Sets a payload available throughout successive token stream enumeration
-   * @lucene.internal
-   */
-  public void setPayload(BytesRef payload) {
-    this.payload = payload;
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    // we only capture this if we really need it to save the UTF-8 to UTF-16 conversion
+    charTermAttribute = getAttribute(CharTermAttribute.class); // may return null
+    finiteStrings = null;
+    wasReset = true;
   }
 
   @Override
   public boolean incrementToken() throws IOException {
+    if (finiteStrings == null) {
+      if (wasReset == false) {
+        throw new IllegalStateException("reset() missing before incrementToken");
+      }
+      // lazy init/consume
+      Automaton automaton = toAutomaton(); // calls reset(), incrementToken() repeatedly, end(), and close() on inputTokenStream
+      finiteStrings = new LimitedFiniteStringsIterator(automaton, maxGraphExpansions);
+    }
+
     IntsRef string = finiteStrings.next();
     if (string == null) {
       return false;
@@ -138,52 +148,53 @@ public final class ConcatenateGraphFilter extends TokenFilter {
       charTermAttribute.setLength(0);
       charTermAttribute.append(bytesAtt.toUTF16());
     }
-    if (payload != null) {
-      payloadAttr.setPayload(this.payload);
-    }
 
     return true;
   }
 
   @Override
   public void end() throws IOException {
-    restoreState(endState);
+    super.end();
+    //nocommit I think we don't need to delegate this under any condition
+    if (finiteStrings == null) {
+      //toAutomaton not called yet, so delegate lifecycle
+      inputTokenStream.end();
+    }
   }
 
-  //nocommit move method to before incrementToken
   @Override
-  public void reset() throws IOException {
-    Automaton automaton = toAutomaton(); // calls reset(), incrementToken() repeatedly, then end()
-    endState = captureState();
-    finiteStrings = new LimitedFiniteStringsIterator(automaton, maxGraphExpansions);
-    if (charTermAttribute == null) {
-      // we only capture this if we really need it to save the UTF-8 to UTF-16 conversion
-      charTermAttribute = getAttribute(CharTermAttribute.class); // null if none; it's okay
+  public void close() throws IOException {
+    super.close();
+    if (finiteStrings == null) {
+      //toAutomaton not called yet, so delegate lifecycle
+      inputTokenStream.close();
+    } else {
+      finiteStrings = null;
     }
+    wasReset = false;//reset
   }
 
   /**
-   * Converts the token stream to an automaton,
-   * treating the transition labels as utf-8
+   * Converts the tokenStream to an automaton, treating the transition labels as utf-8.  Closes it.
    */
   public Automaton toAutomaton() throws IOException {
     return toAutomaton(false);
   }
 
   /**
-   * Converts the tokenStream to an automaton
+   * Converts the tokenStream to an automaton.  Closes it.
    */
   public Automaton toAutomaton(boolean unicodeAware) throws IOException {
     // TODO refactor this
     // maybe we could hook up a modified automaton from TermAutomatonQuery here?
-    Automaton automaton = null;
+    Automaton automaton;
     try {
       // Create corresponding automaton: labels are bytes
       // from each analyzed token, with byte 0 used as
       // separator between tokens:
       final TokenStreamToAutomaton tsta;
       if (preserveSep) {
-        tsta = new EscapingTokenStreamToAutomaton(SEP_CHAR);
+        tsta = new EscapingTokenStreamToAutomaton(SEP_LABEL);
       } else {
         // When we're not preserving sep, we don't steal 0xff
         // byte, so we don't need to do any escaping:
@@ -192,28 +203,29 @@ public final class ConcatenateGraphFilter extends TokenFilter {
       tsta.setPreservePositionIncrements(preservePositionIncrements);
       tsta.setUnicodeArcs(unicodeAware);
 
-      automaton = tsta.toAutomaton(input);
+      automaton = tsta.toAutomaton(inputTokenStream);
     } finally {
-      IOUtils.closeWhileHandlingException(input);
+      IOUtils.closeWhileHandlingException(inputTokenStream);
     }
 
     // TODO: we can optimize this somewhat by determinizing
     // while we convert
-    automaton = replaceSep(automaton, preserveSep, SEP_CHAR);
+    automaton = replaceSep(automaton, preserveSep, SEP_LABEL);
     // This automaton should not blow up during determinize:
     return Operations.determinize(automaton, maxGraphExpansions);
   }
 
   /**
-   * Just escapes the 0xff byte (which we still for SEP).
+   * Just escapes the {@link #SEP_LABEL} byte with an extra.
    */
   private static final class EscapingTokenStreamToAutomaton extends TokenStreamToAutomaton {
 
     final BytesRefBuilder spare = new BytesRefBuilder();
-    private char sepLabel;
+    final byte sepLabel;
 
-    public EscapingTokenStreamToAutomaton(char sepLabel) {
-      this.sepLabel = sepLabel;
+    public EscapingTokenStreamToAutomaton(int sepLabel) {
+      assert sepLabel <= Byte.MAX_VALUE;
+      this.sepLabel = (byte) sepLabel;
     }
 
     @Override
@@ -221,9 +233,9 @@ public final class ConcatenateGraphFilter extends TokenFilter {
       int upto = 0;
       for (int i = 0; i < in.length; i++) {
         byte b = in.bytes[in.offset + i];
-        if (b == (byte) sepLabel) {
+        if (b == sepLabel) {
           spare.grow(upto + 2);
-          spare.setByteAt(upto++, (byte) sepLabel);
+          spare.setByteAt(upto++, sepLabel);
           spare.setByteAt(upto++, b);
         } else {
           spare.grow(upto + 1);
@@ -260,7 +272,7 @@ public final class ConcatenateGraphFilter extends TokenFilter {
         if (t.min == TokenStreamToAutomaton.POS_SEP) {
           assert t.max == TokenStreamToAutomaton.POS_SEP;
           if (preserveSep) {
-            // Remap to SEP_CHAR:
+            // Remap to SEP_LABEL:
             result.addTransition(state, t.dest, sepLabel);
           } else {
             result.addEpsilon(state, t.dest);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConcatenateGraphFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConcatenateGraphFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConcatenateGraphFilterFactory.java
index d8d1039..5d8ccba 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConcatenateGraphFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ConcatenateGraphFilterFactory.java
@@ -1,38 +1,47 @@
 /*
- This software was produced for the U. S. Government
- under Contract No. W15P7T-11-C-F600, and is
- subject to the Rights in Noncommercial Computer Software
- and Noncommercial Computer Software Documentation
- Clause 252.227-7014 (JUN 1995)
-
- Copyright 2013 The MITRE Corporation. All Rights Reserved.
-
- Licensed 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.
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
-
 package org.apache.lucene.analysis.miscellaneous;
 
 import java.util.Map;
 
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.util.TokenFilterFactory;
+import org.apache.lucene.util.automaton.TooComplexToDeterminizeException;
 
 /**
  * Factory for {@link ConcatenateGraphFilter}.
  *
- * <pre class="prettyprint">
- * nocommit update
- * </pre>
+ * <ul>
+ *   <li><tt>preserveSep</tt>:
+ *                            Whether {@link ConcatenateGraphFilter#SEP_LABEL}
+ *                            should separate the input tokens in the concatenated token
+ *                            </li>
+ *   <li><tt>preservePositionIncrements</tt>:
+ *                                       Whether to add an empty token for missing positions.
+ *                                       The effect is a consecutive {@link ConcatenateGraphFilter#SEP_LABEL}.
+ *                                       When false, it's as if there were no missing positions
+ *                                         (we pretend the surrounding tokens were adjacent).
+ *                                       </li>
+ *   <li><tt>maxGraphExpansions</tt>:
+ *                            If the tokenStream graph has more than this many possible paths through, then we'll throw
+ *                            {@link TooComplexToDeterminizeException} to preserve the stability and memory of the
+ *                            machine.
+ *                            </li>
+ * </ul>
  * @see ConcatenateGraphFilter
  * @since 7.4.0
  */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
index 8cb1591..d94b396 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
@@ -72,6 +72,7 @@ import org.apache.lucene.analysis.compound.hyphenation.HyphenationTree;
 import org.apache.lucene.analysis.hunspell.Dictionary;
 import org.apache.lucene.analysis.hunspell.TestHunspellStemFilter;
 import org.apache.lucene.analysis.minhash.MinHashFilter;
+import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
 import org.apache.lucene.analysis.miscellaneous.ConditionalTokenFilter;
 import org.apache.lucene.analysis.miscellaneous.DelimitedTermFrequencyTokenFilter;
 import org.apache.lucene.analysis.miscellaneous.FingerprintFilter;
@@ -119,10 +120,10 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
 
   private static final Set<Class<?>> avoidConditionals = new HashSet<>();
   static {
-    // Fingerprint filter needs to consume the whole tokenstream, so conditionals don't make sense here
+    // These filters needs to consume the whole tokenstream, so conditionals don't make sense here
     avoidConditionals.add(FingerprintFilter.class);
-    // Ditto MinHashFilter
     avoidConditionals.add(MinHashFilter.class);
+    avoidConditionals.add(ConcatenateGraphFilter.class);
   }
 
   private static final Map<Constructor<?>,Predicate<Object[]>> brokenConstructors = new HashMap<>();
@@ -156,7 +157,7 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
               return !((Boolean) args[2]); // args are broken if consumeAllTokens is false
           });
       for (Class<?> c : Arrays.<Class<?>>asList(
-          // doesn't actual reset itself!
+          // doesn't actual reset itself!  TODO this statement is probably obsolete as of LUCENE-6121 ?
           CachingTokenFilter.class,
           // LUCENE-8092: doesn't handle graph inputs
           CJKBigramFilter.class,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConcatenateGraphFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConcatenateGraphFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConcatenateGraphFilter.java
index c4cb0d5..925d540 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConcatenateGraphFilter.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConcatenateGraphFilter.java
@@ -23,12 +23,9 @@ import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.StopFilter;
 import org.apache.lucene.analysis.TokenFilter;
-import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.synonym.SynonymFilter;
 import org.apache.lucene.analysis.synonym.SynonymMap;
-import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
-import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.CharsRefBuilder;
@@ -36,28 +33,24 @@ import org.junit.Test;
 
 public class TestConcatenateGraphFilter extends BaseTokenStreamTestCase {
 
+  private static final char SEP_LABEL = (char) ConcatenateGraphFilter.SEP_LABEL;
+  
   @Test
   public void testBasic() throws Exception {
     Tokenizer tokenStream = new MockTokenizer(MockTokenizer.WHITESPACE, true);
     String input = "mykeyword";
-    BytesRef payload = new BytesRef("payload");
     tokenStream.setReader(new StringReader(input));
-    ConcatenateGraphFilter concatStream = new ConcatenateGraphFilter(tokenStream);
-    concatStream.setPayload(payload);
-    PayloadAttrToTypeAttrFilter stream = new PayloadAttrToTypeAttrFilter(concatStream);
-    assertTokenStreamContents(stream, new String[] {input}, null, null, new String[] {payload.utf8ToString()}, new int[] { 1 }, null, null);
+    ConcatenateGraphFilter stream = new ConcatenateGraphFilter(tokenStream);
+    assertTokenStreamContents(stream, new String[] {input}, null, null, new int[] { 1 });
   }
 
   @Test
   public void testWithNoPreserveSep() throws Exception {
     Tokenizer tokenStream = new MockTokenizer(MockTokenizer.WHITESPACE, true);
     String input = "mykeyword another keyword";
-    BytesRef payload = new BytesRef("payload");
     tokenStream.setReader(new StringReader(input));
-    ConcatenateGraphFilter concatStream = new ConcatenateGraphFilter(tokenStream, false, false, 100);
-    concatStream.setPayload(payload);
-    PayloadAttrToTypeAttrFilter stream = new PayloadAttrToTypeAttrFilter(concatStream);
-    assertTokenStreamContents(stream, new String[] {"mykeywordanotherkeyword"}, null, null, new String[] {payload.utf8ToString()}, new int[] { 1 }, null, null);
+    ConcatenateGraphFilter stream = new ConcatenateGraphFilter(tokenStream, false, false, 100);
+    assertTokenStreamContents(stream, new String[] {"mykeywordanotherkeyword"}, null, null, new int[] { 1 });
   }
 
   @Test
@@ -65,17 +58,14 @@ public class TestConcatenateGraphFilter extends BaseTokenStreamTestCase {
     Tokenizer tokenStream = new MockTokenizer(MockTokenizer.WHITESPACE, true);
     String input = "mykeyword another keyword";
     tokenStream.setReader(new StringReader(input));
-    BytesRef payload = new BytesRef("payload");
-    ConcatenateGraphFilter concatStream = new ConcatenateGraphFilter(tokenStream);
-    concatStream.setPayload(payload);
-    PayloadAttrToTypeAttrFilter stream = new PayloadAttrToTypeAttrFilter(concatStream);
+    ConcatenateGraphFilter stream = new ConcatenateGraphFilter(tokenStream);
     CharsRefBuilder builder = new CharsRefBuilder();
     builder.append("mykeyword");
-    builder.append((ConcatenateGraphFilter.SEP_CHAR));
+    builder.append(SEP_LABEL);
     builder.append("another");
-    builder.append((ConcatenateGraphFilter.SEP_CHAR));
+    builder.append(SEP_LABEL);
     builder.append("keyword");
-    assertTokenStreamContents(stream, new String[]{builder.toCharsRef().toString()}, null, null, new String[]{payload.utf8ToString()}, new int[]{1}, null, null);
+    assertTokenStreamContents(stream, new String[]{builder.toCharsRef().toString()}, null, null, new int[]{1});
   }
 
   @Test
@@ -85,11 +75,8 @@ public class TestConcatenateGraphFilter extends BaseTokenStreamTestCase {
     Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, true);
     tokenizer.setReader(new StringReader("mykeyword"));
     SynonymFilter filter = new SynonymFilter(tokenizer, builder.build(), true);
-    ConcatenateGraphFilter concatStream = new ConcatenateGraphFilter(filter);
-    BytesRef payload = new BytesRef("payload");
-    concatStream.setPayload(payload);
-    PayloadAttrToTypeAttrFilter stream = new PayloadAttrToTypeAttrFilter(concatStream);
-    assertTokenStreamContents(stream, new String[] {"mykeyword", "mysynonym"}, null, null, new String[] {payload.utf8ToString(), payload.utf8ToString()}, new int[] { 1, 0 }, null, null);
+    ConcatenateGraphFilter stream = new ConcatenateGraphFilter(filter);
+    assertTokenStreamContents(stream, new String[] {"mykeyword", "mysynonym"}, null, null, new int[] { 1, 0 });
   }
 
   @Test
@@ -101,25 +88,23 @@ public class TestConcatenateGraphFilter extends BaseTokenStreamTestCase {
     tokenStream.setReader(new StringReader(input));
     SynonymFilter filter = new SynonymFilter(tokenStream, builder.build(), true);
     BytesRef payload = new BytesRef("payload");
-    ConcatenateGraphFilter concatStream = new ConcatenateGraphFilter(filter, true, false, 100);
-    concatStream.setPayload(payload);
-    PayloadAttrToTypeAttrFilter stream = new PayloadAttrToTypeAttrFilter(concatStream);
+    ConcatenateGraphFilter stream = new ConcatenateGraphFilter(filter, true, false, 100);
     String[] expectedOutputs = new String[2];
     CharsRefBuilder expectedOutput = new CharsRefBuilder();
     expectedOutput.append("mykeyword");
-    expectedOutput.append(ConcatenateGraphFilter.SEP_CHAR);
+    expectedOutput.append(SEP_LABEL);
     expectedOutput.append("another");
-    expectedOutput.append(ConcatenateGraphFilter.SEP_CHAR);
+    expectedOutput.append(SEP_LABEL);
     expectedOutput.append("keyword");
     expectedOutputs[0] = expectedOutput.toCharsRef().toString();
     expectedOutput.clear();
     expectedOutput.append("mysynonym");
-    expectedOutput.append(ConcatenateGraphFilter.SEP_CHAR);
+    expectedOutput.append(SEP_LABEL);
     expectedOutput.append("another");
-    expectedOutput.append(ConcatenateGraphFilter.SEP_CHAR);
+    expectedOutput.append(SEP_LABEL);
     expectedOutput.append("keyword");
     expectedOutputs[1] = expectedOutput.toCharsRef().toString();
-    assertTokenStreamContents(stream, expectedOutputs, null, null, new String[]{payload.utf8ToString(), payload.utf8ToString()}, new int[]{1, 0}, null, null);
+    assertTokenStreamContents(stream, expectedOutputs, null, null, new int[]{1, 0});
   }
 
   @Test
@@ -132,16 +117,16 @@ public class TestConcatenateGraphFilter extends BaseTokenStreamTestCase {
       ConcatenateGraphFilter concatStream = new ConcatenateGraphFilter(tokenFilter, true, preservePosInc, 10);
       CharsRefBuilder builder = new CharsRefBuilder();
       if (preservePosInc) {
-        builder.append((ConcatenateGraphFilter.SEP_CHAR));
+        builder.append(SEP_LABEL);
       }
       builder.append("mykeyword");
-      builder.append((ConcatenateGraphFilter.SEP_CHAR));
+      builder.append(SEP_LABEL);
       if (preservePosInc) {
-        builder.append((ConcatenateGraphFilter.SEP_CHAR));
+        builder.append(SEP_LABEL);
       }
       builder.append("keyword");
       if (preservePosInc) {
-        builder.append((ConcatenateGraphFilter.SEP_CHAR));
+        builder.append(SEP_LABEL);
       }
       assertTokenStreamContents(concatStream, new String[]{builder.toCharsRef().toString()});
     }
@@ -162,18 +147,17 @@ public class TestConcatenateGraphFilter extends BaseTokenStreamTestCase {
     tokenizer.setReader(new StringReader(valueBuilder.toString()));
     SynonymFilter filter = new SynonymFilter(tokenizer, builder.build(), true);
 
-    ConcatenateGraphFilter concatStream = new ConcatenateGraphFilter(filter);
-    concatStream.setPayload(new BytesRef());
-    PayloadAttrToTypeAttrFilter stream = new PayloadAttrToTypeAttrFilter(concatStream);
-    stream.reset();
-    ConcatenateGraphFilter.BytesRefBuilderTermAttribute attr = stream.addAttribute(ConcatenateGraphFilter.BytesRefBuilderTermAttribute.class);
-    int count = 0;
-    while(stream.incrementToken()) {
-      count++;
-      assertNotNull(attr.getBytesRef());
-      assertTrue(attr.getBytesRef().length > 0);
+    int count;
+    try (ConcatenateGraphFilter stream = new ConcatenateGraphFilter(filter)) {
+      stream.reset();
+      ConcatenateGraphFilter.BytesRefBuilderTermAttribute attr = stream.addAttribute(ConcatenateGraphFilter.BytesRefBuilderTermAttribute.class);
+      count = 0;
+      while (stream.incrementToken()) {
+        count++;
+        assertNotNull(attr.getBytesRef());
+        assertTrue(attr.getBytesRef().length > 0);
+      }
     }
-    stream.close();
     assertEquals(count, 256);
   }
 
@@ -183,22 +167,4 @@ public class TestConcatenateGraphFilter extends BaseTokenStreamTestCase {
     assertTokenStreamContents(filter, new String[0]);
   }
 
-  public final static class PayloadAttrToTypeAttrFilter extends TokenFilter {
-    private PayloadAttribute payload = addAttribute(PayloadAttribute.class);
-    private TypeAttribute type = addAttribute(TypeAttribute.class);
-
-    protected PayloadAttrToTypeAttrFilter(TokenStream input) {
-      super(input);
-    }
-
-    @Override
-    public boolean incrementToken() throws IOException {
-      if (input.incrementToken()) {
-        // we move them over so we can assert them more easily in the tests
-        type.setType(payload.getPayload().utf8ToString());
-        return true;
-      }
-      return false;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConcatenateGraphFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConcatenateGraphFilterFactory.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConcatenateGraphFilterFactory.java
index 5b3ebac..1e149f0 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConcatenateGraphFilterFactory.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestConcatenateGraphFilterFactory.java
@@ -21,6 +21,7 @@ import java.io.Reader;
 import java.io.StringReader;
 
 import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.analysis.StopFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.util.BaseTokenStreamFactoryTestCase;
 
@@ -33,15 +34,42 @@ public class TestConcatenateGraphFilterFactory extends BaseTokenStreamFactoryTes
       tokenizer.setReader(reader);
       tokenizer.setEnableChecks(consumeAll);
       TokenStream stream = tokenizer;
-      stream = tokenFilterFactory("Concatenate"//nocommit test an arg
-      ).create(stream);
-      assertTokenStreamContents(stream, new String[]{input.replace(' ', ConcatenateGraphFilter.SEP_CHAR)});
+      stream = tokenFilterFactory("ConcatenateGraph").create(stream);
+      assertTokenStreamContents(stream, new String[]{input.replace(' ', (char) ConcatenateGraphFilter.SEP_LABEL)});
     }
   }
 
+  public void testPreserveSep() throws Exception {
+    final String input = "A1 B2 A1 D4 C3";
+    final String output = "A1A1D4C3";
+    Reader reader = new StringReader(input);
+    MockTokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+    tokenizer.setReader(reader);
+    TokenStream stream = tokenizer;
+    stream = new StopFilter(stream, StopFilter.makeStopSet("B2"));
+    stream = tokenFilterFactory("ConcatenateGraph",
+        "preserveSep", "false"
+    ).create(stream);
+    assertTokenStreamContents(stream, new String[]{output});
+  }
+
+  public void testPreservePositionIncrements() throws Exception {
+    final String input = "A1 B2 A1 D4 C3";
+    final String output = "A1 A1 D4 C3";
+    Reader reader = new StringReader(input);
+    MockTokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+    tokenizer.setReader(reader);
+    TokenStream stream = tokenizer;
+    stream = new StopFilter(stream, StopFilter.makeStopSet("B2"));
+    stream = tokenFilterFactory("ConcatenateGraph",
+        "preservePositionIncrements", "false"
+        ).create(stream);
+    assertTokenStreamContents(stream, new String[]{output.replace(' ', (char) ConcatenateGraphFilter.SEP_LABEL)});
+  }
+
   public void testRequired() throws Exception {
     // no params are required
-    tokenFilterFactory("Concatenate");
+    tokenFilterFactory("ConcatenateGraph");
   }
 
   /**
@@ -49,7 +77,7 @@ public class TestConcatenateGraphFilterFactory extends BaseTokenStreamFactoryTes
    */
   public void testBogusArguments() throws Exception {
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () ->
-        tokenFilterFactory("Concatenate", "bogusArg", "bogusValue"));
+        tokenFilterFactory("ConcatenateGraph", "bogusArg", "bogusValue"));
     assertTrue(expected.getMessage().contains("Unknown parameters"));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java
index cdb49de..8888382 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java
@@ -130,7 +130,7 @@ public final class CompletionAnalyzer extends AnalyzerWrapper {
 
   @Override
   protected TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components) {
-    ConcatenateGraphFilter tokenStream = new ConcatenateGraphFilter(components.getTokenStream(),
+    CompletionTokenStream tokenStream = new CompletionTokenStream(components.getTokenStream(),
         preserveSep, preservePositionIncrements, maxGraphExpansions);
     return new TokenStreamComponents(components.getTokenizer(), tokenStream);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java
index 1ed1373..6be0c91 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java
@@ -27,7 +27,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.suggest.BitsProducer;
 
 import static org.apache.lucene.search.suggest.document.CompletionAnalyzer.HOLE_CHARACTER;
-import static org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter.SEP_CHAR;
+import static org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter.SEP_LABEL;
 
 /**
  * Abstract {@link Query} that match documents containing terms with a specified prefix
@@ -158,7 +158,7 @@ public abstract class CompletionQuery extends Query {
         case HOLE_CHARACTER:
           throw new IllegalArgumentException(
               "Term text cannot contain HOLE character U+001E; this character is reserved");
-        case SEP_CHAR:
+        case SEP_LABEL:
           throw new IllegalArgumentException(
               "Term text cannot contain unit separator character U+001F; this character is reserved");
         default:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java
new file mode 100644
index 0000000..0713f5b
--- /dev/null
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java
@@ -0,0 +1,82 @@
+/*
+ * 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.suggest.document;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
+import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.Automaton;
+
+/**
+ * A {@link ConcatenateGraphFilter} but we can set the payload and provide access to config options.
+ * @lucene.internal
+ */
+public final class CompletionTokenStream extends TokenFilter {
+  // package accessible on purpose
+  final TokenStream inputTokenStream;
+  final boolean preserveSep;
+  final boolean preservePositionIncrements;
+  final int maxGraphExpansions;
+
+  private final PayloadAttribute payloadAtt = addAttribute(PayloadAttribute.class);
+
+  private BytesRef payload; // note doesn't participate in TokenStream lifecycle; it's effectively constant
+
+  CompletionTokenStream(TokenStream inputTokenStream) {
+    this(inputTokenStream,
+        ConcatenateGraphFilter.DEFAULT_PRESERVE_SEP,
+        ConcatenateGraphFilter.DEFAULT_PRESERVE_POSITION_INCREMENTS,
+        ConcatenateGraphFilter.DEFAULT_MAX_GRAPH_EXPANSIONS);
+  }
+
+  CompletionTokenStream(TokenStream inputTokenStream, boolean preserveSep, boolean preservePositionIncrements, int maxGraphExpansions) {
+    super(new ConcatenateGraphFilter(inputTokenStream, preserveSep, preservePositionIncrements, maxGraphExpansions));
+    this.inputTokenStream = inputTokenStream;
+    this.preserveSep = preserveSep;
+    this.preservePositionIncrements = preservePositionIncrements;
+    this.maxGraphExpansions = maxGraphExpansions;
+  }
+
+  public void setPayload(BytesRef payload) {
+    this.payload = payload;
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+    if (input.incrementToken()) {
+      payloadAtt.setPayload(payload);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /** @see ConcatenateGraphFilter#toAutomaton()  */
+  public Automaton toAutomaton() throws IOException {
+    return ((ConcatenateGraphFilter)input).toAutomaton();
+  }
+
+  /** @see ConcatenateGraphFilter#toAutomaton(boolean) */
+  public Automaton toAutomaton(boolean unicodeAware) throws IOException {
+    return ((ConcatenateGraphFilter)input).toAutomaton(unicodeAware);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java
index 5d56795..1a2680c 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java
@@ -176,10 +176,10 @@ public class ContextQuery extends CompletionQuery {
       return new CompletionWeight(this, innerAutomaton);
     }
 
-    // if separators are preserved the fst contains a SEP_CHAR
+    // if separators are preserved the fst contains a SEP_LABEL
     // behind each gap. To have a matching automaton, we need to
-    // include the SEP_CHAR in the query as well
-    Automaton optionalSepLabel = Operations.optional(Automata.makeChar(ConcatenateGraphFilter.SEP_CHAR));
+    // include the SEP_LABEL in the query as well
+    Automaton optionalSepLabel = Operations.optional(Automata.makeChar(ConcatenateGraphFilter.SEP_LABEL));
     Automaton prefixAutomaton = Operations.concatenate(optionalSepLabel, innerAutomaton);
     Automaton contextsAutomaton = Operations.concatenate(toContextAutomaton(contexts, matchAllContexts), prefixAutomaton);
     contextsAutomaton = Operations.determinize(contextsAutomaton, Operations.DEFAULT_MAX_DETERMINIZED_STATES);
@@ -303,9 +303,9 @@ public class ContextQuery extends CompletionQuery {
           }
           ref.offset = ++i;
           assert ref.offset < ref.length : "input should not end with the context separator";
-          if (ref.ints[i] == ConcatenateGraphFilter.SEP_CHAR) {
+          if (ref.ints[i] == ConcatenateGraphFilter.SEP_LABEL) {
             ref.offset++;
-            assert ref.offset < ref.length : "input should not end with a context separator followed by SEP_CHAR";
+            assert ref.offset < ref.length : "input should not end with a context separator followed by SEP_LABEL";
           }
           ref.length = ref.length - ref.offset;
           refBuilder.copyInts(ref.ints, ref.offset, ref.length);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java
index 84a99c0..d608339 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java
@@ -24,7 +24,6 @@ import java.util.Set;
 
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 
@@ -84,22 +83,22 @@ public class ContextSuggestField extends SuggestField {
   }
 
   @Override
-  protected ConcatenateGraphFilter wrapTokenStream(TokenStream stream) {
+  protected CompletionTokenStream wrapTokenStream(TokenStream stream) {
     final Iterable<CharSequence> contexts = contexts();
     for (CharSequence context : contexts) {
       validate(context);
     }
-    ConcatenateGraphFilter concatStream;
-    if (stream instanceof ConcatenateGraphFilter) {
-      //nocommit this is awkward; is there a better way avoiding re-creating the chain?
-      concatStream = (ConcatenateGraphFilter) stream;
-      PrefixTokenFilter prefixTokenFilter = new PrefixTokenFilter(concatStream.getInput(), (char) CONTEXT_SEPARATOR, contexts);
-      concatStream = new ConcatenateGraphFilter(prefixTokenFilter,
+    CompletionTokenStream concatStream;
+    if (stream instanceof CompletionTokenStream) {
+      //TODO this is awkward; is there a better way avoiding re-creating the chain?
+      concatStream = (CompletionTokenStream) stream;
+      PrefixTokenFilter prefixTokenFilter = new PrefixTokenFilter(concatStream.inputTokenStream, (char) CONTEXT_SEPARATOR, contexts);
+      concatStream = new CompletionTokenStream(prefixTokenFilter,
           concatStream.preserveSep,
           concatStream.preservePositionIncrements,
           concatStream.maxGraphExpansions);
     } else {
-      concatStream = new ConcatenateGraphFilter(new PrefixTokenFilter(stream, (char) CONTEXT_SEPARATOR, contexts));
+      concatStream = new CompletionTokenStream(new PrefixTokenFilter(stream, (char) CONTEXT_SEPARATOR, contexts));
     }
     return concatStream;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java
index b5754e8..b243f4e 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java
@@ -23,7 +23,6 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.ScoreMode;
@@ -145,7 +144,7 @@ public class FuzzyCompletionQuery extends PrefixCompletionQuery {
 
   @Override
   public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
-    ConcatenateGraphFilter stream = (ConcatenateGraphFilter) analyzer.tokenStream(getField(), getTerm().text());
+    CompletionTokenStream stream = (CompletionTokenStream) analyzer.tokenStream(getField(), getTerm().text());
     Set<IntsRef> refs = new HashSet<>();
     Automaton automaton = toLevenshteinAutomata(stream.toAutomaton(unicodeAware), refs);
     if (unicodeAware) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java
index 1dd2b1f..5ca4993 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java
@@ -43,10 +43,7 @@ final class NRTSuggesterBuilder {
    * Label used to separate surface form and docID
    * in the output
    */
-  public static final int PAYLOAD_SEP = '\u001F';
-  static {
-    assert PAYLOAD_SEP == ConcatenateGraphFilter.SEP_CHAR;
-  }
+  public static final int PAYLOAD_SEP = ConcatenateGraphFilter.SEP_LABEL;
 
   /**
    * Marks end of the analyzed input and start of dedup

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/PrefixCompletionQuery.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/PrefixCompletionQuery.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/PrefixCompletionQuery.java
index 2cffc76..7bb75e9 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/PrefixCompletionQuery.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/PrefixCompletionQuery.java
@@ -19,7 +19,6 @@ package org.apache.lucene.search.suggest.document;
 import java.io.IOException;
 
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.ScoreMode;
@@ -69,7 +68,7 @@ public class PrefixCompletionQuery extends CompletionQuery {
 
   @Override
   public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
-    ConcatenateGraphFilter stream = (ConcatenateGraphFilter) analyzer.tokenStream(getField(), getTerm().text());
+    CompletionTokenStream stream = (CompletionTokenStream) analyzer.tokenStream(getField(), getTerm().text());
     return new CompletionWeight(this, stream.toAutomaton());
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java
index 0ee338f..ce8c2e9 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java
@@ -101,7 +101,7 @@ public class SuggestField extends Field {
 
   @Override
   public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
-    ConcatenateGraphFilter completionStream = wrapTokenStream(super.tokenStream(analyzer, reuse));
+    CompletionTokenStream completionStream = wrapTokenStream(super.tokenStream(analyzer, reuse));
     completionStream.setPayload(buildSuggestPayload());
     return completionStream;
   }
@@ -111,11 +111,11 @@ public class SuggestField extends Field {
    *
    * Subclasses can override this method to change the indexing pipeline.
    */
-  protected ConcatenateGraphFilter wrapTokenStream(TokenStream stream) {
-    if (stream instanceof ConcatenateGraphFilter) {
-      return (ConcatenateGraphFilter) stream;
+  protected CompletionTokenStream wrapTokenStream(TokenStream stream) {
+    if (stream instanceof CompletionTokenStream) {
+      return (CompletionTokenStream) stream;
     } else {
-      return new ConcatenateGraphFilter(stream);
+      return new CompletionTokenStream(stream);
     }
   }
 
@@ -141,7 +141,7 @@ public class SuggestField extends Field {
 
   private boolean isReserved(char c) {
     switch (c) {
-      case ConcatenateGraphFilter.SEP_CHAR:
+      case ConcatenateGraphFilter.SEP_LABEL:
       case CompletionAnalyzer.HOLE_CHARACTER:
       case NRTSuggesterBuilder.END_BYTE:
         return true;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java
index 23b1267..8beea12 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java
@@ -110,13 +110,13 @@ public class TestContextSuggestField extends LuceneTestCase {
     CharsRefBuilder builder = new CharsRefBuilder();
     builder.append("context1");
     builder.append(((char) ContextSuggestField.CONTEXT_SEPARATOR));
-    builder.append(ConcatenateGraphFilter.SEP_CHAR);
+    builder.append((char) ConcatenateGraphFilter.SEP_LABEL);
     builder.append("input");
     expectedOutputs[0] = builder.toCharsRef().toString();
     builder.clear();
     builder.append("context2");
     builder.append(((char) ContextSuggestField.CONTEXT_SEPARATOR));
-    builder.append(ConcatenateGraphFilter.SEP_CHAR);
+    builder.append((char) ConcatenateGraphFilter.SEP_LABEL);
     builder.append("input");
     expectedOutputs[1] = builder.toCharsRef().toString();
     TokenStream stream = new TestSuggestField.PayloadAttrToTypeAttrFilter(field.tokenStream(analyzer, null));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d813eca1/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
index a083535..e6d7062 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
@@ -103,7 +103,7 @@ public class TestSuggestField extends LuceneTestCase {
   public void testReservedChars() throws Exception {
     CharsRefBuilder charsRefBuilder = new CharsRefBuilder();
     charsRefBuilder.append("sugg");
-    charsRefBuilder.setCharAt(2, ConcatenateGraphFilter.SEP_CHAR);
+    charsRefBuilder.setCharAt(2, (char) ConcatenateGraphFilter.SEP_LABEL);
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
       new SuggestField("name", charsRefBuilder.toString(), 1);
     });