You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cp...@apache.org on 2017/05/26 09:42:22 UTC

[5/8] lucene-solr:jira/solr-8668: SOLR-10379: Add ManagedSynonymGraphFilterFactory, deprecate ManagedSynonymFilterFactory

SOLR-10379: Add ManagedSynonymGraphFilterFactory, deprecate ManagedSynonymFilterFactory


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

Branch: refs/heads/jira/solr-8668
Commit: 78e7e1c3072b315c92cbb2934c1874b7978cb99b
Parents: 077c9ec
Author: Steve Rowe <sa...@gmail.com>
Authored: Thu May 25 17:49:20 2017 -0400
Committer: Steve Rowe <sa...@gmail.com>
Committed: Thu May 25 17:49:20 2017 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   4 +
 .../analysis/ManagedSynonymFilterFactory.java   |   5 +
 .../ManagedSynonymGraphFilterFactory.java       | 437 +++++++++++++++++++
 .../solr/collection1/conf/schema-rest.xml       |  30 ++
 .../org/apache/solr/rest/TestRestManager.java   |   4 +-
 .../TestManagedSynonymGraphFilterFactory.java   | 297 +++++++++++++
 .../conf/managed-schema                         |  10 +-
 .../solr-ref-guide/src/filter-descriptions.adoc |  28 +-
 solr/solr-ref-guide/src/managed-resources.adoc  |  20 +-
 9 files changed, 819 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/78e7e1c3/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index f6f5481..1da3fe0 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -219,6 +219,8 @@ Upgrade Notes
   passwords via the env variables SOLR_SSL_KEY_STORE_PASSWORD and SOLR_SSL_TRUST_STORE_PASSWORD rather
   than system properties.
 
+* SOLR-10379: ManagedSynonymFilterFactory has been deprecated in favor of ManagedSynonymGraphFilterFactory.
+
 New Features
 ----------------------
 
@@ -226,6 +228,8 @@ New Features
 
 * SOLR-10721: Provide a way to know when Core Discovery is finished and when all async cores are done loading
   (Erick Erickson)
+  
+* SOLR-10379: Add ManagedSynonymGraphFilterFactory, deprecate ManagedSynonymFilterFactory. (Steve Rowe)
 
 * SOLR-10479: Adds support for HttpShardHandlerFactory.loadBalancerRequests(MinimumAbsolute|MaximumFraction)
   configuration. (Ramsey Haddad, Daniel Collins, Christine Poerschke)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/78e7e1c3/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymFilterFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymFilterFactory.java b/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymFilterFactory.java
index ffbbb36..63b4747 100644
--- a/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymFilterFactory.java
+++ b/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymFilterFactory.java
@@ -28,6 +28,7 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.FlattenGraphFilterFactory;  // javadocs
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.synonym.SynonymFilterFactory;
 import org.apache.lucene.analysis.synonym.SynonymMap;
@@ -50,7 +51,11 @@ import org.slf4j.LoggerFactory;
 /**
  * TokenFilterFactory and ManagedResource implementation for 
  * doing CRUD on synonyms using the REST API.
+ * 
+ * @deprecated Use {@link ManagedSynonymGraphFilterFactory} instead, but be sure to also
+ * use {@link FlattenGraphFilterFactory} at index time (not at search time) as well.
  */
+@Deprecated
 public class ManagedSynonymFilterFactory extends BaseManagedTokenFilterFactory {
   
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/78e7e1c3/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymGraphFilterFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymGraphFilterFactory.java b/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymGraphFilterFactory.java
new file mode 100644
index 0000000..1f4a970
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymGraphFilterFactory.java
@@ -0,0 +1,437 @@
+/*
+ * 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.solr.rest.schema.analysis;
+import java.io.IOException;
+import java.io.Reader;
+import java.lang.invoke.MethodHandles;
+import java.text.ParseException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.synonym.SynonymGraphFilterFactory;
+import org.apache.lucene.analysis.synonym.SynonymMap;
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.rest.BaseSolrResource;
+import org.apache.solr.rest.ManagedResource;
+import org.apache.solr.rest.ManagedResourceStorage.StorageIO;
+import org.restlet.data.Status;
+import org.restlet.resource.ResourceException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * TokenFilterFactory and ManagedResource implementation for 
+ * doing CRUD on synonyms using the REST API.
+ */
+public class ManagedSynonymGraphFilterFactory extends BaseManagedTokenFilterFactory {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final String SYNONYM_MAPPINGS = "synonymMappings";
+  public static final String IGNORE_CASE_INIT_ARG = "ignoreCase";
+
+  /**
+   * Used internally to preserve the case of synonym mappings regardless
+   * of the ignoreCase setting.
+   */
+  private static class CasePreservedSynonymMappings {
+    Map<String,Set<String>> mappings = new TreeMap<>();
+
+    /**
+     * Provides a view of the mappings for a given term; specifically, if
+     * ignoreCase is true, then the returned "view" contains the mappings
+     * for all known cases of the term, if it is false, then only the
+     * mappings for the specific case is returned. 
+     */
+    Set<String> getMappings(boolean ignoreCase, String key) {
+      Set<String> synMappings = null;
+      if (ignoreCase) {
+        // TODO: should we return the mapped values in all lower-case here?
+        if (mappings.size() == 1) {
+          // if only one in the map (which is common) just return it directly
+          return mappings.values().iterator().next();
+        }
+
+        synMappings = new TreeSet<>();
+        for (Set<String> next : mappings.values())
+          synMappings.addAll(next);
+      } else {
+        synMappings = mappings.get(key);
+      }
+      return synMappings;
+    }
+
+    public String toString() {
+      return mappings.toString();
+    }
+  }
+
+  /**
+   * ManagedResource implementation for synonyms, which are so specialized that
+   * it makes sense to implement this class as an inner class as it has little 
+   * application outside the SynonymFilterFactory use cases.
+   */
+  public static class SynonymManager extends ManagedResource
+      implements ManagedResource.ChildResourceSupport
+  {
+    protected Map<String,CasePreservedSynonymMappings> synonymMappings;
+
+    public SynonymManager(String resourceId, SolrResourceLoader loader, StorageIO storageIO)
+        throws SolrException {
+      super(resourceId, loader, storageIO);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected void onManagedDataLoadedFromStorage(NamedList<?> managedInitArgs, Object managedData)
+        throws SolrException
+    {
+      NamedList<Object> initArgs = (NamedList<Object>)managedInitArgs;
+
+      String format = (String)initArgs.get("format");
+      if (format != null && !"solr".equals(format)) {
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid format "+
+            format+"! Only 'solr' is supported.");
+      }
+
+      // the default behavior is to not ignore case, 
+      // so if not supplied, then install the default
+      if (initArgs.get(IGNORE_CASE_INIT_ARG) == null) {
+        initArgs.add(IGNORE_CASE_INIT_ARG, Boolean.FALSE);
+      }
+
+      boolean ignoreCase = getIgnoreCase(managedInitArgs);
+      synonymMappings = new TreeMap<>();
+      if (managedData != null) {
+        Map<String,Object> storedSyns = (Map<String,Object>)managedData;
+        for (String key : storedSyns.keySet()) {
+
+          String caseKey = applyCaseSetting(ignoreCase, key);
+          CasePreservedSynonymMappings cpsm = synonymMappings.get(caseKey);
+          if (cpsm == null) {
+            cpsm = new CasePreservedSynonymMappings();
+            synonymMappings.put(caseKey, cpsm);
+          }
+
+          // give the nature of our JSON parsing solution, we really have
+          // no guarantees on what is in the file
+          Object mapping = storedSyns.get(key);
+          if (!(mapping instanceof List)) {
+            throw new SolrException(ErrorCode.SERVER_ERROR,
+                "Invalid synonym file format! Expected a list of synonyms for "+key+
+                    " but got "+mapping.getClass().getName());
+          }
+
+          Set<String> sortedVals = new TreeSet<>();
+          sortedVals.addAll((List<String>)storedSyns.get(key));
+          cpsm.mappings.put(key, sortedVals);
+        }
+      }
+      log.info("Loaded {} synonym mappings for {}", synonymMappings.size(), getResourceId());
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected Object applyUpdatesToManagedData(Object updates) {
+      boolean ignoreCase = getIgnoreCase();
+      boolean madeChanges = false;
+      if (updates instanceof List) {
+        madeChanges = applyListUpdates((List<String>)updates, ignoreCase);
+      } else if (updates instanceof Map) {
+        madeChanges = applyMapUpdates((Map<String,Object>)updates, ignoreCase);
+      } else {
+        throw new ResourceException(Status.CLIENT_ERROR_BAD_REQUEST,
+            "Unsupported data format (" + updates.getClass().getName() + "); expected a JSON object (Map or List)!");
+      }
+      return madeChanges ? getStoredView() : null;
+    }
+
+    protected boolean applyListUpdates(List<String> jsonList, boolean ignoreCase) {
+      boolean madeChanges = false;
+      for (String term : jsonList) {
+        // find the mappings using the case aware key
+        String origTerm = term;
+        term = applyCaseSetting(ignoreCase, term);
+        CasePreservedSynonymMappings cpsm = synonymMappings.get(term);
+        if (cpsm == null)
+          cpsm = new CasePreservedSynonymMappings();
+
+        Set<String> treeTerms = new TreeSet<>();
+        treeTerms.addAll(jsonList);
+        cpsm.mappings.put(origTerm, treeTerms);
+        madeChanges = true;
+        // only add the cpsm to the synonymMappings if it has valid data
+        if (!synonymMappings.containsKey(term) && cpsm.mappings.get(origTerm) != null) {
+          synonymMappings.put(term, cpsm);
+        }
+      }
+      return madeChanges;
+    }
+
+    protected boolean applyMapUpdates(Map<String,Object> jsonMap, boolean ignoreCase) {
+      boolean madeChanges = false;
+
+      for (String term : jsonMap.keySet()) {
+
+        String origTerm = term;
+        term = applyCaseSetting(ignoreCase, term);
+
+        // find the mappings using the case aware key
+        CasePreservedSynonymMappings cpsm = synonymMappings.get(term);
+        if (cpsm == null)
+          cpsm = new CasePreservedSynonymMappings();
+
+        Set<String> output = cpsm.mappings.get(origTerm);
+
+        Object val = jsonMap.get(origTerm); // IMPORTANT: use the original
+        if (val instanceof String) {
+          String strVal = (String)val;
+
+          if (output == null) {
+            output = new TreeSet<>();
+            cpsm.mappings.put(origTerm, output);
+          }
+
+          if (output.add(strVal)) {
+            madeChanges = true;
+          }
+        } else if (val instanceof List) {
+          List<String> vals = (List<String>)val;
+
+          if (output == null) {
+            output = new TreeSet<>();
+            cpsm.mappings.put(origTerm, output);
+          }
+
+          for (String nextVal : vals) {
+            if (output.add(nextVal)) {
+              madeChanges = true;
+            }
+          }
+
+        } else {
+          throw new ResourceException(Status.CLIENT_ERROR_BAD_REQUEST, "Unsupported value "+val+
+              " for "+term+"; expected single value or a JSON array!");
+        }
+
+        // only add the cpsm to the synonymMappings if it has valid data
+        if (!synonymMappings.containsKey(term) && cpsm.mappings.get(origTerm) != null) {
+          synonymMappings.put(term, cpsm);
+        }
+      }
+
+      return madeChanges;
+    }
+
+    /**
+     * Returns a Map of how we store and load data managed by this resource,
+     * which is different than how it is managed at runtime in order to support
+     * the ignoreCase setting. 
+     */
+    protected Map<String,Set<String>> getStoredView() {
+      Map<String,Set<String>> storedView = new TreeMap<>();
+      for (CasePreservedSynonymMappings cpsm : synonymMappings.values()) {
+        for (String key : cpsm.mappings.keySet()) {
+          storedView.put(key, cpsm.mappings.get(key));
+        }
+      }
+      return storedView;
+    }
+
+    protected String applyCaseSetting(boolean ignoreCase, String str) {
+      return (ignoreCase && str != null) ? str.toLowerCase(Locale.ROOT) : str;
+    }
+
+    public boolean getIgnoreCase() {
+      return getIgnoreCase(managedInitArgs);
+    }
+
+    public boolean getIgnoreCase(NamedList<?> initArgs) {
+      Boolean ignoreCase = initArgs.getBooleanArg(IGNORE_CASE_INIT_ARG);
+      // ignoreCase = false by default
+      return null == ignoreCase ? false : ignoreCase;
+    }
+
+    @Override
+    public void doGet(BaseSolrResource endpoint, String childId) {
+      SolrQueryResponse response = endpoint.getSolrResponse();
+      if (childId != null) {
+        boolean ignoreCase = getIgnoreCase();
+        String key = applyCaseSetting(ignoreCase, childId);
+
+        // if ignoreCase==true, then we get the mappings using the lower-cased key
+        // and then return a union of all case-sensitive keys, if false, then
+        // we only return the mappings for the exact case requested
+        CasePreservedSynonymMappings cpsm = synonymMappings.get(key);
+        Set<String> mappings = (cpsm != null) ? cpsm.getMappings(ignoreCase, childId) : null;
+        if (mappings == null)
+          throw new SolrException(ErrorCode.NOT_FOUND,
+              String.format(Locale.ROOT, "%s not found in %s", childId, getResourceId()));
+
+        response.add(childId, mappings);
+      } else {
+        response.add(SYNONYM_MAPPINGS, buildMapToStore(getStoredView()));
+      }
+    }
+
+    @Override
+    public synchronized void doDeleteChild(BaseSolrResource endpoint, String childId) {
+      boolean ignoreCase = getIgnoreCase();
+      String key = applyCaseSetting(ignoreCase, childId);
+
+      CasePreservedSynonymMappings cpsm = synonymMappings.get(key);
+      if (cpsm == null)
+        throw new SolrException(ErrorCode.NOT_FOUND,
+            String.format(Locale.ROOT, "%s not found in %s", childId, getResourceId()));
+
+      if (ignoreCase) {
+        // delete all mappings regardless of case
+        synonymMappings.remove(key);
+      } else {
+        // just delete the mappings for the specific case-sensitive key
+        if (cpsm.mappings.containsKey(childId)) {
+          cpsm.mappings.remove(childId);
+
+          if (cpsm.mappings.isEmpty())
+            synonymMappings.remove(key);
+        } else {
+          throw new SolrException(ErrorCode.NOT_FOUND,
+              String.format(Locale.ROOT, "%s not found in %s", childId, getResourceId()));
+        }
+      }
+
+      // store the updated data (using the stored view)
+      storeManagedData(getStoredView());
+
+      log.info("Removed synonym mappings for: {}", childId);
+    }
+  }
+
+  /**
+   * Custom SynonymMap.Parser implementation that provides synonym
+   * mappings from the managed JSON in this class during SynonymMap
+   * building.
+   */
+  private class ManagedSynonymParser extends SynonymMap.Parser {
+
+    SynonymManager synonymManager;
+
+    public ManagedSynonymParser(SynonymManager synonymManager, boolean dedup, Analyzer analyzer) {
+      super(dedup, analyzer);
+      this.synonymManager = synonymManager;
+    }
+
+    /**
+     * Add the managed synonyms and their mappings into the SynonymMap builder.
+     */
+    @Override
+    public void parse(Reader in) throws IOException, ParseException {
+      boolean ignoreCase = synonymManager.getIgnoreCase();
+      for (CasePreservedSynonymMappings cpsm : synonymManager.synonymMappings.values()) {
+        for (String term : cpsm.mappings.keySet()) {
+          for (String mapping : cpsm.mappings.get(term)) {
+            // apply the case setting to match the behavior of the SynonymMap builder
+            CharsRef casedTerm = analyze(synonymManager.applyCaseSetting(ignoreCase, term), new CharsRefBuilder());
+            CharsRef casedMapping = analyze(synonymManager.applyCaseSetting(ignoreCase, mapping), new CharsRefBuilder());
+            add(casedTerm, casedMapping, false);
+          }
+        }
+      }
+    }
+  }
+
+  protected SynonymGraphFilterFactory delegate;
+
+  public ManagedSynonymGraphFilterFactory(Map<String,String> args) {
+    super(args);
+  }
+
+  @Override
+  public String getResourceId() {
+    return "/schema/analysis/synonyms/"+handle;
+  }
+
+  protected Class<? extends ManagedResource> getManagedResourceImplClass() {
+    return SynonymManager.class;
+  }
+
+  /**
+   * Called once, during core initialization, to initialize any analysis components
+   * that depend on the data managed by this resource. It is important that the
+   * analysis component is only initialized once during core initialization so that
+   * text analysis is consistent, especially in a distributed environment, as we
+   * don't want one server applying a different set of stop words than other servers.
+   */
+  @SuppressWarnings("unchecked")
+  @Override
+  public void onManagedResourceInitialized(NamedList<?> initArgs, final ManagedResource res)
+      throws SolrException
+  {
+    NamedList<Object> args = (NamedList<Object>)initArgs;
+    args.add("synonyms", getResourceId());
+    args.add("expand", "false");
+    args.add("format", "solr");
+
+    Map<String,String> filtArgs = new HashMap<>();
+    for (Map.Entry<String,?> entry : args) {
+      filtArgs.put(entry.getKey(), entry.getValue().toString());
+    }
+    // create the actual filter factory that pulls the synonym mappings
+    // from synonymMappings using a custom parser implementation
+    delegate = new SynonymGraphFilterFactory(filtArgs) {
+      @Override
+      protected SynonymMap loadSynonyms
+          (ResourceLoader loader, String cname, boolean dedup, Analyzer analyzer)
+          throws IOException, ParseException {
+
+        ManagedSynonymParser parser =
+            new ManagedSynonymParser((SynonymManager)res, dedup, analyzer);
+        // null is safe here because there's no actual parsing done against a input Reader
+        parser.parse(null);
+        return parser.build();
+      }
+    };
+    try {
+      delegate.inform(res.getResourceLoader());
+    } catch (IOException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, e);
+    }
+  }
+
+  @Override
+  public TokenStream create(TokenStream input) {
+    if (delegate == null)
+      throw new IllegalStateException(this.getClass().getName()+
+          " not initialized correctly! The SynonymFilterFactory delegate was not initialized.");
+
+    return delegate.create(input);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/78e7e1c3/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml b/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml
index 3233258..58b05eb 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml
@@ -511,6 +511,36 @@
     </analyzer>
   </fieldType>
 
+  <!-- Field type where english stopwords are managed by the REST API -->
+  <fieldType name="managed_graph_en" class="solr.TextField" autoGeneratePhraseQueries="false">
+    <analyzer type="index">
+      <tokenizer class="solr.StandardTokenizerFactory"/>
+      <filter class="solr.ManagedStopFilterFactory" managed="english"/>
+      <filter class="solr.ManagedSynonymGraphFilterFactory" managed="englishgraph"/>
+      <filter class="solr.FlattenGraphFilterFactory"/>
+    </analyzer>
+    <analyzer type="query">
+      <tokenizer class="solr.StandardTokenizerFactory"/>
+      <filter class="solr.ManagedStopFilterFactory" managed="english"/>
+      <filter class="solr.ManagedSynonymGraphFilterFactory" managed="englishgraph"/>
+    </analyzer>
+  </fieldType>
+
+  <!-- Field type where german stopwords are managed by the REST API -->
+  <fieldType name="managed_graph_de" class="solr.TextField">
+    <analyzer type="index">
+      <tokenizer class="solr.StandardTokenizerFactory"/>
+      <filter class="solr.ManagedStopFilterFactory" managed="german"/>
+      <filter class="solr.ManagedSynonymGraphFilterFactory" managed="germangraph"/>
+      <filter class="solr.FlattenGraphFilterFactory"/>
+    </analyzer>
+    <analyzer type="query">
+      <tokenizer class="solr.StandardTokenizerFactory"/>
+      <filter class="solr.ManagedStopFilterFactory" managed="german"/>
+      <filter class="solr.ManagedSynonymGraphFilterFactory" managed="germangraph"/>
+    </analyzer>
+  </fieldType>
+  
   <field name="id" type="string" indexed="true" stored="true" multiValued="false" required="true"/>
   <field name="signatureField" type="string" indexed="true" stored="false"/>
   <field name="uuid" type="uuid" stored="true"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/78e7e1c3/solr/core/src/test/org/apache/solr/rest/TestRestManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/TestRestManager.java b/solr/core/src/test/org/apache/solr/rest/TestRestManager.java
index cc6c58a..a39e774 100644
--- a/solr/core/src/test/org/apache/solr/rest/TestRestManager.java
+++ b/solr/core/src/test/org/apache/solr/rest/TestRestManager.java
@@ -174,8 +174,8 @@ public class TestRestManager extends SolrRestletTestBase {
      * 
              "/managedResources/[0]/class=='org.apache.solr.rest.schema.analysis.ManagedWordSetResource'",
              "/managedResources/[0]/resourceId=='/schema/analysis/stopwords/english'",
-             "/managedResources/[1]/class=='org.apache.solr.rest.schema.analysis.ManagedSynonymFilterFactory$SynonymManager'",
-             "/managedResources/[1]/resourceId=='/schema/analysis/synonyms/english'");
+             "/managedResources/[1]/class=='org.apache.solr.rest.schema.analysis.ManagedSynonymGraphFilterFactory$SynonymManager'",
+             "/managedResources/[1]/resourceId=='/schema/analysis/synonyms/englishgraph'");
     */
     
     // no pre-existing managed config components

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/78e7e1c3/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymGraphFilterFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymGraphFilterFactory.java b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymGraphFilterFactory.java
new file mode 100644
index 0000000..9b442a8
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymGraphFilterFactory.java
@@ -0,0 +1,297 @@
+/*
+ * 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.solr.rest.schema.analysis;
+
+import java.io.File;
+import java.net.URLEncoder;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.solr.util.RestTestBase;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.noggit.JSONUtil;
+import org.restlet.ext.servlet.ServerServlet;
+
+public class TestManagedSynonymGraphFilterFactory extends RestTestBase {
+
+  private static File tmpSolrHome;
+
+  /**
+   * Setup to make the schema mutable
+   */
+  @Before
+  public void before() throws Exception {
+    tmpSolrHome = createTempDir().toFile();
+    FileUtils.copyDirectory(new File(TEST_HOME()), tmpSolrHome.getAbsoluteFile());
+
+    final SortedMap<ServletHolder,String> extraServlets = new TreeMap<>();
+    final ServletHolder solrRestApi = new ServletHolder("SolrSchemaRestApi", ServerServlet.class);
+    solrRestApi.setInitParameter("org.restlet.application", "org.apache.solr.rest.SolrSchemaRestApi");
+    extraServlets.put(solrRestApi, "/schema/*");
+
+    System.setProperty("managed.schema.mutable", "true");
+    System.setProperty("enable.update.log", "false");
+    createJettyAndHarness(tmpSolrHome.getAbsolutePath(), "solrconfig-managed-schema.xml", "schema-rest.xml",
+        "/solr", true, extraServlets);
+  }
+
+  @After
+  private void after() throws Exception {
+    jetty.stop();
+    jetty = null;
+    FileUtils.deleteDirectory(tmpSolrHome);
+    System.clearProperty("managed.schema.mutable");
+    System.clearProperty("enable.update.log");
+
+    if (restTestHarness != null) {
+      restTestHarness.close();
+    }
+    restTestHarness = null;
+  }
+
+  @Test
+  public void testManagedSynonyms() throws Exception {
+    // this endpoint depends on at least one field type containing the following
+    // declaration in the schema-rest.xml:
+    // 
+    //   <filter class="solr.ManagedSynonymGraphFilterFactory" managed="englishgraph" />
+    //      
+    String endpoint = "/schema/analysis/synonyms/englishgraph";
+
+    assertJQ(endpoint,
+        "/synonymMappings/initArgs/ignoreCase==false",
+        "/synonymMappings/managedMap=={}");
+
+    // put a new mapping into the synonyms
+    Map<String,List<String>> syns = new HashMap<>();
+    syns.put("happy", Arrays.asList("glad","cheerful","joyful"));
+    assertJPut(endpoint,
+        JSONUtil.toJSON(syns),
+        "/responseHeader/status==0");
+
+    assertJQ(endpoint,
+        "/synonymMappings/managedMap/happy==['cheerful','glad','joyful']");
+
+    // request to a specific mapping
+    assertJQ(endpoint+"/happy",
+        "/happy==['cheerful','glad','joyful']");
+
+    // does not exist
+    assertJQ(endpoint+"/sad",
+        "/error/code==404");
+
+    // verify the user can update the ignoreCase initArg
+    assertJPut(endpoint,
+        json("{ 'initArgs':{ 'ignoreCase':true } }"),
+        "responseHeader/status==0");
+
+    assertJQ(endpoint,
+        "/synonymMappings/initArgs/ignoreCase==true");
+
+    syns = new HashMap<>();
+    syns.put("sad", Arrays.asList("unhappy"));
+    syns.put("SAD", Arrays.asList("bummed"));
+    assertJPut(endpoint,
+        JSONUtil.toJSON(syns),
+        "/responseHeader/status==0");
+
+    assertJQ(endpoint,
+        "/synonymMappings/managedMap/sad==['unhappy']");
+    assertJQ(endpoint,
+        "/synonymMappings/managedMap/SAD==['bummed']");
+
+    // expect a union of values when requesting the "sad" child
+    assertJQ(endpoint+"/sad",
+        "/sad==['bummed','unhappy']");
+
+    // verify delete works
+    assertJDelete(endpoint+"/sad",
+        "/responseHeader/status==0");
+
+    assertJQ(endpoint,
+        "/synonymMappings/managedMap=={'happy':['cheerful','glad','joyful']}");
+
+    // should fail with 404 as foo doesn't exist
+    assertJDelete(endpoint+"/foo",
+        "/error/code==404");
+
+    // verify that a newly added synonym gets expanded on the query side after core reload
+
+    String newFieldName = "managed_graph_en_field";
+    // make sure the new field doesn't already exist
+    assertQ("/schema/fields/" + newFieldName + "?indent=on&wt=xml",
+        "count(/response/lst[@name='field']) = 0",
+        "/response/lst[@name='responseHeader']/int[@name='status'] = '404'",
+        "/response/lst[@name='error']/int[@name='code'] = '404'");
+
+    // add the new field
+    assertJPost("/schema", "{ add-field :  { name: managed_graph_en_field, type : managed_graph_en}}",
+        "/responseHeader/status==0");
+
+    // make sure the new field exists now
+    assertQ("/schema/fields/" + newFieldName + "?indent=on&wt=xml",
+        "count(/response/lst[@name='field']) = 1",
+        "/response/lst[@name='responseHeader']/int[@name='status'] = '0'");
+
+    // multi-term synonym logic - SOLR-10264
+    final String multiTermOrigin;
+    final String multiTermSynonym;
+    if (random().nextBoolean()) {
+      multiTermOrigin  = "hansestadt hamburg";
+      multiTermSynonym = "hh";
+    } else {
+      multiTermOrigin  = "hh";
+      multiTermSynonym = "hansestadt hamburg";
+    }
+    // multi-term logic similar to the angry/mad logic (angry ~ origin, mad ~ synonym)
+
+    assertU(adoc(newFieldName, "I am a happy test today but yesterday I was angry", "id", "5150"));
+    assertU(adoc(newFieldName, multiTermOrigin+" is in North Germany.", "id", "040"));
+    assertU(commit());
+
+    assertQ("/select?q=" + newFieldName + ":angry",
+        "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
+        "/response/result[@name='response'][@numFound='1']",
+        "/response/result[@name='response']/doc/str[@name='id'][.='5150']");
+    assertQ("/select?q=" + newFieldName + ":"+URLEncoder.encode(multiTermOrigin, "UTF-8"),
+        "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
+        "/response/result[@name='response'][@numFound='1']",
+        "/response/result[@name='response']/doc/str[@name='id'][.='040']");
+
+    // add a mapping that will expand a query for "mad" to match docs with "angry"
+    syns = new HashMap<>();
+    syns.put("mad", Arrays.asList("angry"));
+    assertJPut(endpoint,
+        JSONUtil.toJSON(syns),
+        "/responseHeader/status==0");
+
+    assertJQ(endpoint,
+        "/synonymMappings/managedMap/mad==['angry']");
+
+    // add a mapping that will expand a query for "multi-term synonym" to match docs with "acronym"
+    syns = new HashMap<>();
+    syns.put(multiTermSynonym, Arrays.asList(multiTermOrigin));
+    assertJPut(endpoint,
+        JSONUtil.toJSON(syns),
+        "/responseHeader/status==0");
+
+    assertJQ(endpoint+"/"+URLEncoder.encode(multiTermSynonym, "UTF-8"),
+        "/"+multiTermSynonym+"==['"+multiTermOrigin+"']");
+
+    // should not match as the synonym mapping between mad and angry does not    
+    // get applied until core reload
+    assertQ("/select?q=" + newFieldName + ":mad",
+        "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
+        "/response/result[@name='response'][@numFound='0']");
+
+    // should not match as the synonym mapping between "origin" and "synonym"
+    // was not added before the document was indexed
+    assertQ("/select?q=" + newFieldName + ":("+URLEncoder.encode(multiTermSynonym, "UTF-8") + ")&sow=false",
+        "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
+        "/response/result[@name='response'][@numFound='0']");
+
+    restTestHarness.reload();
+
+    // now query for mad and we should see our test doc
+    assertQ("/select?q=" + newFieldName + ":mad",
+        "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
+        "/response/result[@name='response'][@numFound='1']",
+        "/response/result[@name='response']/doc/str[@name='id'][.='5150']");
+
+    // now query for "synonym" and we should see our test doc with "origin"
+    assertQ("/select?q=" + newFieldName + ":("+URLEncoder.encode(multiTermSynonym, "UTF-8") + ")&sow=false",
+        "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
+        "/response/result[@name='response'][@numFound='1']",
+        "/response/result[@name='response']/doc/str[@name='id'][.='040']");
+
+    // test for SOLR-6015
+    syns = new HashMap<>();
+    syns.put("mb", Arrays.asList("megabyte"));
+    assertJPut(endpoint,
+        JSONUtil.toJSON(syns),
+        "/responseHeader/status==0");
+
+    syns.put("MB", Arrays.asList("MiB", "Megabyte"));
+    assertJPut(endpoint,
+        JSONUtil.toJSON(syns),
+        "/responseHeader/status==0");
+
+    assertJQ(endpoint + "/MB",
+        "/MB==['Megabyte','MiB','megabyte']");
+
+    // test for SOLR-6878 - by default, expand is true, but only applies when sending in a list
+    List<String> m2mSyns = new ArrayList<>();
+    m2mSyns.addAll(Arrays.asList("funny", "entertaining", "whimiscal", "jocular"));
+    assertJPut(endpoint, JSONUtil.toJSON(m2mSyns), "/responseHeader/status==0");
+
+    assertJQ(endpoint + "/funny",
+        "/funny==['entertaining','funny','jocular','whimiscal']");
+    assertJQ(endpoint + "/entertaining",
+        "/entertaining==['entertaining','funny','jocular','whimiscal']");
+    assertJQ(endpoint + "/jocular",
+        "/jocular==['entertaining','funny','jocular','whimiscal']");
+    assertJQ(endpoint + "/whimiscal",
+        "/whimiscal==['entertaining','funny','jocular','whimiscal']");
+  }
+
+  /**
+   * Can we add and remove stopwords with umlauts
+   */
+  @Test
+  public void testCanHandleDecodingAndEncodingForSynonyms() throws Exception  {
+    String endpoint = "/schema/analysis/synonyms/germangraph";
+
+    assertJQ(endpoint,
+        "/synonymMappings/initArgs/ignoreCase==false",
+        "/synonymMappings/managedMap=={}");
+
+    // does not exist
+    assertJQ(endpoint+"/fröhlich",
+        "/error/code==404");
+
+    Map<String,List<String>> syns = new HashMap<>();
+
+    // now put a synonym
+    syns.put("fröhlich", Arrays.asList("glücklick"));
+    assertJPut(endpoint,
+        JSONUtil.toJSON(syns),
+        "/responseHeader/status==0");
+
+    // and check if it exists
+    assertJQ(endpoint,
+        "/synonymMappings/managedMap/fröhlich==['glücklick']");
+
+    // verify delete works
+    assertJDelete(endpoint+"/fröhlich",
+        "/responseHeader/status==0");
+
+
+    // was it really deleted?
+    assertJDelete(endpoint+"/fröhlich",
+        "/error/code==404");
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/78e7e1c3/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema b/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema
index 6bda748..1c4f1fe 100644
--- a/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema
+++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema
@@ -455,10 +455,16 @@
 
     <!-- A text type for English text where stopwords and synonyms are managed using the REST API -->
     <fieldType name="managed_en" class="solr.TextField" positionIncrementGap="100">
-      <analyzer>
+      <analyzer type="index">
+        <tokenizer class="solr.StandardTokenizerFactory"/>
+        <filter class="solr.ManagedStopFilterFactory" managed="english" />
+        <filter class="solr.ManagedSynonymGraphFilterFactory" managed="english" />
+        <filter class="solr.FlattenGraphFilterFactory"/>
+      </analyzer>
+      <analyzer type="query">
         <tokenizer class="solr.StandardTokenizerFactory"/>
         <filter class="solr.ManagedStopFilterFactory" managed="english" />
-        <filter class="solr.ManagedSynonymFilterFactory" managed="english" />
+        <filter class="solr.ManagedSynonymGraphFilterFactory" managed="english" />
       </analyzer>
     </fieldType>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/78e7e1c3/solr/solr-ref-guide/src/filter-descriptions.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/filter-descriptions.adoc b/solr/solr-ref-guide/src/filter-descriptions.adoc
index 4550685..06e0986 100644
--- a/solr/solr-ref-guide/src/filter-descriptions.adoc
+++ b/solr/solr-ref-guide/src/filter-descriptions.adoc
@@ -843,6 +843,23 @@ See <<FilterDescriptions-StopFilter,Stop Filter>> for example input/output.
 
 This is specialized version of the <<FilterDescriptions-SynonymFilter,Synonym Filter Factory>> that uses a mapping on synonyms that is <<managed-resources.adoc#managed-resources,managed from a REST API.>>
 
+.Managed Synonym Filter has been Deprecated
+[WARNING]
+====
+Managed Synonym Filter has been deprecated in favor of Managed Synonym Graph Filter, which is required for multi-term synonym support.
+====
+
+*Factory class:* `solr.ManagedSynonymFilterFactory`
+
+For arguments and examples, see the Managed Synonym Graph Filter below.
+
+[[FilterDescriptions-ManagedSynonymGraphFilter]]
+== Managed Synonym Graph Filter
+
+This is specialized version of the <<FilterDescriptions-SynonymGraphFilter,Synonym Graph Filter Factory>> that uses a mapping on synonyms that is <<managed-resources.adoc#managed-resources,managed from a REST API.>>
+
+This filter maps single- or multi-token synonyms, producing a fully correct graph output. This filter is a replacement for the Managed Synonym Filter, which produces incorrect graphs for multi-token synonyms.
+
 *Arguments:*
 
 `managed`:: The name that should be used for this mapping on synonyms in the managed REST API.
@@ -853,13 +870,18 @@ With this configuration the set of mappings is named "english" and can be manage
 
 [source,xml]
 ----
-<analyzer>
+<analyzer type="index">
+  <tokenizer class="solr.StandardTokenizerFactory"/>
+  <filter class="solr.ManagedSynonymGraphFilterFactory" managed="english"/>
+  <filter class="solr.FlattenGraphFilterFactory"/> <!-- required on index analyzers after graph filters -->
+</analyzer>
+<analyzer type="query">
   <tokenizer class="solr.StandardTokenizerFactory"/>
-  <filter class="solr.ManagedSynonymFilterFactory" managed="english"/>
+  <filter class="solr.ManagedSynonymGraphFilterFactory" managed="english"/>
 </analyzer>
 ----
 
-See <<FilterDescriptions-SynonymFilter,Synonym Filter>> for example input/output.
+See <<FilterDescriptions-ManagedSynonymFilter,Managed Synonym Filter>> for example input/output.
 
 [[FilterDescriptions-N-GramFilter]]
 == N-Gram Filter

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/78e7e1c3/solr/solr-ref-guide/src/managed-resources.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/managed-resources.adoc b/solr/solr-ref-guide/src/managed-resources.adoc
index f7d062d..72b879a 100644
--- a/solr/solr-ref-guide/src/managed-resources.adoc
+++ b/solr/solr-ref-guide/src/managed-resources.adoc
@@ -22,7 +22,7 @@ Managed resources expose a REST API endpoint for performing Create-Read-Update-D
 
 Any long-lived Solr object that has configuration settings and/or data is a good candidate to be a managed resource. Managed resources complement other programmatically manageable components in Solr, such as the RESTful schema API to add fields to a managed schema.
 
-Consider a Web-based UI that offers Solr-as-a-Service where users need to configure a set of stop words and synonym mappings as part of an initial setup process for their search application. This type of use case can easily be supported using the Managed Stop Filter & Managed Synonym Filter Factories provided by Solr, via the Managed resources REST API.
+Consider a Web-based UI that offers Solr-as-a-Service where users need to configure a set of stop words and synonym mappings as part of an initial setup process for their search application. This type of use case can easily be supported using the Managed Stop Filter & Managed Synonym Graph Filter Factories provided by Solr, via the Managed resources REST API.
 
 Users can also write their own custom plugins, that leverage the same internal hooks to make additional resources REST managed.
 
@@ -142,14 +142,16 @@ For the most part, the API for managing synonyms behaves similar to the API for
 [source,xml]
 ----
 <fieldType name="managed_en" positionIncrementGap="100">
-  <analyzer>
+  <analyzer type="index">
     <tokenizer class="solr.StandardTokenizerFactory"/>
-    <filter class="solr.ManagedStopFilterFactory"
-            managed="english" />
-
-    <filter class="solr.ManagedSynonymFilterFactory"
-            managed="english" />
-
+    <filter class="solr.ManagedStopFilterFactory" managed="english" />
+    <filter class="solr.ManagedSynonymGraphFilterFactory" managed="english" />
+    <filter class="solr.FlattenGraphFilterFactory"/> <!-- required on index analyzers after graph filters -->
+  </analyzer>
+  <analyzer type="query">
+    <tokenizer class="solr.StandardTokenizerFactory"/>
+    <filter class="solr.ManagedStopFilterFactory" managed="english" />
+    <filter class="solr.ManagedSynonymGraphFilterFactory" managed="english" />
   </analyzer>
 </fieldType>
 ----
@@ -254,7 +256,7 @@ The response body is a JSON document containing metadata about managed resources
     },
     {
       "resourceId":"/schema/analysis/synonyms/english",
-      "class":"org.apache.solr.rest.schema.analysis.ManagedSynonymFilterFactory$SynonymManager",
+      "class":"org.apache.solr.rest.schema.analysis.ManagedSynonymGraphFilterFactory$SynonymManager",
       "numObservers":"1"
     }
   ]