You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by to...@apache.org on 2019/08/11 07:36:14 UTC

[lucene-solr] branch master updated: SOLR-13593: Allow to look up analyzer components by their SPI names in field type configuration.

This is an automated email from the ASF dual-hosted git repository.

tomoko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 9b986d2  SOLR-13593: Allow to look up analyzer components by their SPI names in field type configuration.
9b986d2 is described below

commit 9b986d268f3618d2137bbc8bd068a3db0d772049
Author: Tomoko Uchida <to...@apache.org>
AuthorDate: Sun Aug 11 16:35:25 2019 +0900

    SOLR-13593: Allow to look up analyzer components by their SPI names in field type configuration.
---
 .../analysis/util/AbstractAnalysisFactory.java     |   3 +
 solr/CHANGES.txt                                   |   2 +
 .../solr/collection1/conf/schema-folding-extra.xml |   2 +-
 .../solr/rest/schema/FieldTypeXmlAdapter.java      |   9 +-
 .../src/java/org/apache/solr/schema/FieldType.java |  12 +-
 .../apache/solr/schema/FieldTypePluginLoader.java  |  52 +++++++-
 .../solr/util/plugin/AbstractPluginLoader.java     |  11 +-
 ....apache.lucene.analysis.util.TokenFilterFactory |  18 +++
 .../conf/bad-schema-analyzer-by-name.xml           |  32 +++++
 .../conf/bad-schema-analyzer-class-and-name-cf.xml |  33 +++++
 .../conf/bad-schema-analyzer-class-and-name-tf.xml |  33 +++++
 .../bad-schema-analyzer-class-and-name-tok.xml     |  32 +++++
 .../collection1/conf/schema-analyzer-by-name.xml   |  78 ++++++++++++
 .../apache/solr/rest/schema/TestBulkSchemaAPI.java |  56 +++++++++
 .../solr/schema/ResolveAnalyzerByNameTest.java     | 135 +++++++++++++++++++++
 solr/solr-ref-guide/src/charfilterfactories.adoc   |   2 +-
 solr/solr-ref-guide/src/filter-descriptions.adoc   |   6 +-
 .../solr/analysis/MockCharFilterFactory.java       |   4 +
 .../solr/analysis/MockTokenFilterFactory.java      |   4 +
 .../apache/solr/analysis/MockTokenizerFactory.java |   4 +
 20 files changed, 508 insertions(+), 20 deletions(-)

diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AbstractAnalysisFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AbstractAnalysisFactory.java
index 8a4f849..fec784e 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AbstractAnalysisFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/AbstractAnalysisFactory.java
@@ -83,6 +83,7 @@ public abstract class AbstractAnalysisFactory {
       }
     }
     args.remove(CLASS_NAME);  // consume the class arg
+    args.remove(SPI_NAME);    // consume the spi arg
   }
   
   public final Map<String,String> getOriginalArgs() {
@@ -316,6 +317,8 @@ public abstract class AbstractAnalysisFactory {
   }
 
   private static final String CLASS_NAME = "class";
+
+  private static final String SPI_NAME = "name";
   
   /**
    * @return the string used to specify the concrete class name in a serialized representation: the class arg.  
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 094951a..6db4714 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -60,6 +60,8 @@ Upgrade Notes
 * SOLR-11266: default Content-Type override for JSONResponseWriter from _default configSet is removed. Example has been
   provided in sample_techproducts_configs to override content-type. (Ishan Chattopadhyaya, Munendra S N, Gus Heck)
 
+* SOLR-13593: Allow to look up analyzer components by their SPI names in field type configuration. (Tomoko Uchida)
+
 Other Changes
 ----------------------
 
diff --git a/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/schema-folding-extra.xml b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/schema-folding-extra.xml
index 573ca53..2f7474f 100644
--- a/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/schema-folding-extra.xml
+++ b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/schema-folding-extra.xml
@@ -29,7 +29,7 @@
   <fieldType name="text_icunormalizer2" class="solr.TextField">
     <analyzer>
       <tokenizer class="solr.WhitespaceTokenizerFactory"/>
-      <filter class="solr.ICUNormalizer2FilterFactory" name="nfkc_cf" mode="compose"/>
+      <filter class="solr.ICUNormalizer2FilterFactory" form="nfkc_cf" mode="compose"/>
     </analyzer>
   </fieldType>
 
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java b/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java
index d25ab89..4abeedd 100644
--- a/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java
+++ b/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java
@@ -140,8 +140,8 @@ public class FieldTypeXmlAdapter {
       if (tokenizer == null)
         throw new SolrException(ErrorCode.BAD_REQUEST, "Analyzer must define a tokenizer!");
 
-      if (tokenizer.get("class") == null)
-        throw new SolrException(ErrorCode.BAD_REQUEST, "Every tokenizer must define a class property!");
+      if (tokenizer.get("class") == null && tokenizer.get("name") == null)
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Every tokenizer must define a class or name property!");
 
       analyzerElem.appendChild(appendAttrs(doc.createElement("tokenizer"), tokenizer));
 
@@ -168,9 +168,10 @@ public class FieldTypeXmlAdapter {
   protected static void appendFilterElements(Document doc, Element analyzer, String filterName, List<Map<String,?>> filters) {
     for (Map<String,?> next : filters) {
       String filterClass = (String)next.get("class");
-      if (filterClass == null)
+      String filterSPIName = (String)next.get("name");
+      if (filterClass == null && filterSPIName == null)
         throw new SolrException(ErrorCode.BAD_REQUEST, 
-            "Every "+filterName+" must define a class property!");      
+            "Every "+filterName+" must define a class or name property!");
       analyzer.appendChild(appendAttrs(doc.createElement(filterName), next));
     }    
   }
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java
index 69ef981..3bada27 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -1109,8 +1109,10 @@ public abstract class FieldType extends FieldProperties {
         List<SimpleOrderedMap<Object>> charFilterProps = new ArrayList<>();
         for (CharFilterFactory charFilterFactory : charFilterFactories) {
           SimpleOrderedMap<Object> props = new SimpleOrderedMap<>();
-          props.add(CLASS_NAME, charFilterFactory.getClassArg());
           factoryArgs = charFilterFactory.getOriginalArgs();
+          if (!factoryArgs.containsKey(TYPE_NAME)) {
+            props.add(CLASS_NAME, charFilterFactory.getClassArg());
+          }
           if (null != factoryArgs) {
             for (String key : factoryArgs.keySet()) {
               if ( ! CLASS_NAME.equals(key)) {
@@ -1131,8 +1133,10 @@ public abstract class FieldType extends FieldProperties {
 
       SimpleOrderedMap<Object> tokenizerProps = new SimpleOrderedMap<>();
       TokenizerFactory tokenizerFactory = tokenizerChain.getTokenizerFactory();
-      tokenizerProps.add(CLASS_NAME, tokenizerFactory.getClassArg());
       factoryArgs = tokenizerFactory.getOriginalArgs();
+      if (!factoryArgs.containsKey(TYPE_NAME)) {
+        tokenizerProps.add(CLASS_NAME, tokenizerFactory.getClassArg());
+      }
       if (null != factoryArgs) {
         for (String key : factoryArgs.keySet()) {
           if ( ! CLASS_NAME.equals(key)) {
@@ -1153,8 +1157,10 @@ public abstract class FieldType extends FieldProperties {
         List<SimpleOrderedMap<Object>> filterProps = new ArrayList<>();
         for (TokenFilterFactory filterFactory : filterFactories) {
           SimpleOrderedMap<Object> props = new SimpleOrderedMap<>();
-          props.add(CLASS_NAME, filterFactory.getClassArg());
           factoryArgs = filterFactory.getOriginalArgs();
+          if (!factoryArgs.containsKey(TYPE_NAME)) {
+            props.add(CLASS_NAME, filterFactory.getClassArg());
+          }
           if (null != factoryArgs) {
             for (String key : factoryArgs.keySet()) {
               if ( ! CLASS_NAME.equals(key)) {
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java b/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
index 36be446..c8da0bb 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
@@ -24,6 +24,7 @@ import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Map;
+import java.util.Objects;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
@@ -79,7 +80,7 @@ public final class FieldTypePluginLoader
   @Override
   protected FieldType create( SolrResourceLoader loader, 
                               String name, 
-                              String className, 
+                              String className,
                               Node node ) throws Exception {
 
     FieldType ft = loader.newInstance(className, FieldType.class);
@@ -260,7 +261,21 @@ public final class FieldTypePluginLoader
         final Map<String,String> params = DOMUtil.toMap(node.getAttributes());
         String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
         params.put(LUCENE_MATCH_VERSION_PARAM, parseConfiguredVersion(configuredVersion, CharFilterFactory.class.getSimpleName()).toString());
-        CharFilterFactory factory = loader.newInstance(className, CharFilterFactory.class, getDefaultPackages(), new Class[] { Map.class }, new Object[] { params });
+        CharFilterFactory factory;
+        if (Objects.nonNull(name)) {
+          factory = CharFilterFactory.forName(name, params);
+          if (Objects.nonNull(className)) {
+            log.error("Both of name: " + name + " and className: " + className + " are specified for charFilter.");
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+                "Cannot create charFilter: Both of name and className are specified.");
+          }
+        } else if (Objects.nonNull(className)) {
+          factory = loader.newInstance(className, CharFilterFactory.class, getDefaultPackages(), new Class[]{Map.class}, new Object[]{params});
+        } else {
+          log.error("Neither of name or className is specified for charFilter.");
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+              "Cannot create charFilter: Neither of name or className is specified.");
+        }
         factory.setExplicitLuceneMatchVersion(null != configuredVersion);
         return factory;
       }
@@ -296,7 +311,21 @@ public final class FieldTypePluginLoader
         final Map<String,String> params = DOMUtil.toMap(node.getAttributes());
         String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
         params.put(LUCENE_MATCH_VERSION_PARAM, parseConfiguredVersion(configuredVersion, TokenizerFactory.class.getSimpleName()).toString());
-        TokenizerFactory factory = loader.newInstance(className, TokenizerFactory.class, getDefaultPackages(), new Class[] { Map.class }, new Object[] { params });
+        TokenizerFactory factory;
+        if (Objects.nonNull(name)) {
+          factory = TokenizerFactory.forName(name, params);
+          if (Objects.nonNull(className)) {
+            log.error("Both of name: " + name + " and className: " + className + " are specified for tokenizer.");
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+                "Cannot create tokenizer: Both of name and className are specified.");
+          }
+        } else if (Objects.nonNull(className)) {
+          factory = loader.newInstance(className, TokenizerFactory.class, getDefaultPackages(), new Class[]{Map.class}, new Object[]{params});
+        } else {
+          log.error("Neither of name or className is specified for tokenizer.");
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+              "Cannot create tokenizer: Neither of name or className is specified.");
+        }
         factory.setExplicitLuceneMatchVersion(null != configuredVersion);
         return factory;
       }
@@ -336,8 +365,21 @@ public final class FieldTypePluginLoader
         final Map<String,String> params = DOMUtil.toMap(node.getAttributes());
         String configuredVersion = params.remove(LUCENE_MATCH_VERSION_PARAM);
         params.put(LUCENE_MATCH_VERSION_PARAM, parseConfiguredVersion(configuredVersion, TokenFilterFactory.class.getSimpleName()).toString());
-        TokenFilterFactory factory = loader.newInstance
-            (className, TokenFilterFactory.class, getDefaultPackages(), new Class[] { Map.class }, new Object[] { params });
+        TokenFilterFactory factory;
+        if (Objects.nonNull(name)) {
+          factory = TokenFilterFactory.forName(name, params);
+          if (Objects.nonNull(className)) {
+            log.error("Both of name: " + name + " and className: " + className + " are specified for tokenFilter.");
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+                "Cannot create tokenFilter: Both of name and className are specified.");
+          }
+        } else if (Objects.nonNull(className)) {
+          factory = loader.newInstance(className, TokenFilterFactory.class, getDefaultPackages(), new Class[]{Map.class}, new Object[]{params});
+        } else {
+          log.error("Neither of name or className is specified for tokenFilter.");
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+              "Cannot create tokenFilter: Neither of name or className is specified.");
+        }
         factory.setExplicitLuceneMatchVersion(null != configuredVersion);
         return factory;
       }
diff --git a/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java b/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java
index d0b8785..1243017 100644
--- a/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java
+++ b/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java
@@ -19,6 +19,7 @@ package org.apache.solr.util.plugin;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -146,10 +147,14 @@ public abstract class AbstractPluginLoader<T>
         String name = null;
         try {
           name = DOMUtil.getAttr(node, NAME, requireName ? type : null);
-          String className  = DOMUtil.getAttr(node,"class", type);
+          String className  = DOMUtil.getAttr(node,"class", null);
           String defaultStr = DOMUtil.getAttr(node,"default", null );
-            
-          T plugin = create(loader, name, className, node );
+
+          if (Objects.isNull(className) && Objects.isNull(name)) {
+            throw new RuntimeException(type + ": missing mandatory attribute 'class' or 'name'");
+          }
+
+          T plugin = create(loader, name, className, node);
           log.debug("created " + ((name != null) ? name : "") + ": " + plugin.getClass().getName());
           
           // Either initialize now or wait till everything has been registered
diff --git a/solr/core/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory b/solr/core/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
new file mode 100644
index 0000000..6163bd3
--- /dev/null
+++ b/solr/core/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
@@ -0,0 +1,18 @@
+#  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.
+
+org.apache.solr.rest.schema.analysis.ManagedStopFilterFactory
+org.apache.solr.rest.schema.analysis.ManagedSynonymFilterFactory
+org.apache.solr.rest.schema.analysis.ManagedSynonymGraphFilterFactory
\ No newline at end of file
diff --git a/solr/core/src/test-files/solr/collection1/conf/bad-schema-analyzer-by-name.xml b/solr/core/src/test-files/solr/collection1/conf/bad-schema-analyzer-by-name.xml
new file mode 100644
index 0000000..ec3e2ce
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/bad-schema-analyzer-by-name.xml
@@ -0,0 +1,32 @@
+<?xml version="1.0" ?>
+<!--
+ 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.
+-->
+
+<schema name="test" version="1.0">
+
+  <!--
+   Resolve analysis factories by name. See: SOLR-13593
+   -->
+
+  <!-- A text field with fake tokenizer factory. An exception should be thrown when loading this schema. -->
+  <fieldType name="text_ws" class="solr.TextField" positionIncrementGap="100">
+    <analyzer>
+      <tokenizer name="bogus"/>
+    </analyzer>
+  </fieldType>
+
+</schema>
diff --git a/solr/core/src/test-files/solr/collection1/conf/bad-schema-analyzer-class-and-name-cf.xml b/solr/core/src/test-files/solr/collection1/conf/bad-schema-analyzer-class-and-name-cf.xml
new file mode 100644
index 0000000..4c0252a
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/bad-schema-analyzer-class-and-name-cf.xml
@@ -0,0 +1,33 @@
+<?xml version="1.0" ?>
+<!--
+ 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.
+-->
+
+<schema name="test" version="1.0">
+
+  <!--
+   Resolve analysis factories by name. See: SOLR-13593
+   -->
+
+  <!-- A text field with charfilter that have 'name' and 'class' attributes. This should not be allowed. -->
+  <fieldType name="text_redundant" class="solr.TextField" positionIncrementGap="100">
+    <analyzer>
+      <charFilter name="htmlStrip" class="solr.MappingCharFilterFactory"/>
+      <tokenizer name="whitespace"/>
+    </analyzer>
+  </fieldType>
+
+</schema>
diff --git a/solr/core/src/test-files/solr/collection1/conf/bad-schema-analyzer-class-and-name-tf.xml b/solr/core/src/test-files/solr/collection1/conf/bad-schema-analyzer-class-and-name-tf.xml
new file mode 100644
index 0000000..d60aa06
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/bad-schema-analyzer-class-and-name-tf.xml
@@ -0,0 +1,33 @@
+<?xml version="1.0" ?>
+<!--
+ 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.
+-->
+
+<schema name="test" version="1.0">
+
+  <!--
+   Resolve analysis factories by name. See: SOLR-13593
+   -->
+
+  <!-- A text field with tokenfilter that have 'name' and 'class' attributes. This should not be allowed. -->
+  <fieldType name="text_redundant" class="solr.TextField" positionIncrementGap="100">
+    <analyzer>
+      <tokenizer name="whitespace"/>
+      <filter name="lowercase" class="solr.StopFilterFactory"/>
+    </analyzer>
+  </fieldType>
+
+</schema>
diff --git a/solr/core/src/test-files/solr/collection1/conf/bad-schema-analyzer-class-and-name-tok.xml b/solr/core/src/test-files/solr/collection1/conf/bad-schema-analyzer-class-and-name-tok.xml
new file mode 100644
index 0000000..3f89f72
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/bad-schema-analyzer-class-and-name-tok.xml
@@ -0,0 +1,32 @@
+<?xml version="1.0" ?>
+<!--
+ 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.
+-->
+
+<schema name="test" version="1.0">
+
+  <!--
+   Resolve analysis factories by name. See: SOLR-13593
+   -->
+
+  <!-- A text field with tokenizer that have 'name' and 'class' attributes. This should not be allowed. -->
+  <fieldType name="text_redundant" class="solr.TextField" positionIncrementGap="100">
+    <analyzer>
+      <tokenizer name="whitespace" class="solr.StandardTokenizerFactory"/>
+    </analyzer>
+  </fieldType>
+
+</schema>
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-analyzer-by-name.xml b/solr/core/src/test-files/solr/collection1/conf/schema-analyzer-by-name.xml
new file mode 100644
index 0000000..b570425
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-analyzer-by-name.xml
@@ -0,0 +1,78 @@
+<?xml version="1.0" ?>
+<!--
+ 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.
+-->
+
+<schema name="test" version="1.0">
+
+  <!--
+   Resolve analysis factories by name. See: SOLR-13593
+   -->
+
+  <!-- A text field that only splits on whitespace for exact matching of words -->
+  <fieldType name="text_ws" class="solr.TextField" positionIncrementGap="100">
+    <analyzer>
+      <tokenizer name="whitespace"/>
+    </analyzer>
+  </fieldType>
+
+  <!-- A text field that uses WordDelimiterGraphFilter to enable splitting and matching of
+      words on case-change, alpha numeric boundaries, and non-alphanumeric chars,
+      so that a query of "wifi" or "wi fi" could match a document containing "Wi-Fi".
+      Synonyms and stopwords are customized by external files, and stemming is enabled.
+      Duplicate tokens at the same position (which may result from Stemmed Synonyms or
+      WordDelim parts) are removed.
+      -->
+  <fieldType name="text" class="solr.TextField" positionIncrementGap="100">
+    <analyzer type="index">
+      <tokenizer name="whitespace"/>
+      <!-- in this example, we will only use synonyms at query time
+      <filter name="synonymGraph" synonyms="index_synonyms.txt" ignoreCase="true" expand="false"/>
+      -->
+      <filter name="stop" ignoreCase="true" words="stopwords.txt"/>
+      <filter name="wordDelimiterGraph" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0" splitOnCaseChange="1"/>
+      <filter name="lowercase"/>
+      <filter name="keywordMarker" protected="protwords.txt"/>
+      <filter name="porterStem"/>
+      <filter name="removeDuplicates"/>
+      <filter name="flattenGraph"/>
+    </analyzer>
+    <analyzer type="query">
+      <tokenizer name="whitespace"/>
+      <filter name="synonymGraph" synonyms="synonyms.txt" ignoreCase="true" expand="true"/>
+      <filter name="stop" ignoreCase="true" words="stopwords.txt"/>
+      <filter name="wordDelimiterGraph" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="1"/>
+      <filter name="lowercase"/>
+      <filter name="keywordMarker" protected="protwords.txt"/>
+      <filter name="porterStem"/>
+      <filter name="removeDuplicates"/>
+    </analyzer>
+  </fieldType>
+
+
+  <fieldType name="charfilthtmlmap" class="solr.TextField">
+    <analyzer>
+      <charFilter name="htmlStrip"/>
+      <charFilter name="mapping" mapping="mapping-ISOLatin1Accent.txt"/>
+      <tokenizer name="whitespace"/>
+    </analyzer>
+  </fieldType>
+
+  <field name="text1" type="text_ws" indexed="true" stored="false" />
+  <field name="text2" type="text" indexed="true" stored="false" />
+  <field name="text3" type="charfilthtmlmap" indexed="true" stored="false" />
+
+</schema>
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java b/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
index ab7bf3a..e40db98 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
@@ -191,6 +191,62 @@ public class TestBulkSchemaAPI extends RestTestBase {
     assertEquals("5.0.0", String.valueOf(analyzer.get("luceneMatchVersion")));
   }
 
+  public void testAnalyzerByName() throws Exception {
+
+    String addFieldTypeAnalyzer = "{\n" +
+        "'add-field-type' : {" +
+        "    'name' : 'myNewTextField',\n" +
+        "    'class':'solr.TextField',\n" +
+        "    'analyzer' : {\n" +
+        "        'charFilters' : [{\n" +
+        "                'name':'patternReplace',\n" +
+        "                'replacement':'$1$1',\n" +
+        "                'pattern':'([a-zA-Z])\\\\\\\\1+'\n" +
+        "            }],\n" +
+        "        'tokenizer' : { 'name':'whitespace' },\n" +
+        "        'filters' : [{ 'name':'asciiFolding' }]\n" +
+        "    }\n"+
+        "}}";
+
+    String response = restTestHarness.post("/schema", json(addFieldTypeAnalyzer));
+    Map map = (Map) fromJSONString(response);
+    assertNull(response, map.get("error"));
+
+    map = getObj(restTestHarness, "myNewTextField", "fieldTypes");
+    assertNotNull(map);
+    Map analyzer = (Map)map.get("analyzer");
+    Map tokenizer = (Map)analyzer.get("tokenizer");
+    List charFilters = (List)analyzer.get("charFilters");
+    List tokenFilters = (List)analyzer.get("filters");
+    assertEquals("whitespace", String.valueOf(tokenizer.get("name")));
+    assertEquals("patternReplace", String.valueOf(((Map)charFilters.get(0)).get("name")));
+    assertEquals("asciiFolding", String.valueOf(((Map)tokenFilters.get(0)).get("name")));
+  }
+
+  public void testAnalyzerByBogusName() throws Exception {
+
+    String addFieldTypeAnalyzer = "{\n" +
+        "'add-field-type' : {" +
+        "    'name' : 'myNewTextField',\n" +
+        "    'class':'solr.TextField',\n" +
+        "    'analyzer' : {\n" +
+        "        'tokenizer' : { 'name':'bogus' }\n" +
+        "    }\n"+
+        "}}";
+
+    String response = restTestHarness.post("/schema", json(addFieldTypeAnalyzer));
+    Map map = (Map) fromJSONString(response);
+    Map error = (Map)map.get("error");
+    assertNotNull("No errors", error);
+    List details = (List)error.get("details");
+    assertNotNull("No details", details);
+    assertEquals("Wrong number of details", 1, details.size());
+    List errorList = (List)((Map)details.get(0)).get("errorMessages");
+    assertEquals(1, errorList.size());
+    assertTrue (((String)errorList.get(0)).contains
+        ("A SPI class of type org.apache.lucene.analysis.util.TokenizerFactory with name 'bogus' does not exist."));
+  }
+
   public void testAddFieldMatchingExistingDynamicField() throws Exception {
     RestTestHarness harness = restTestHarness;
 
diff --git a/solr/core/src/test/org/apache/solr/schema/ResolveAnalyzerByNameTest.java b/solr/core/src/test/org/apache/solr/schema/ResolveAnalyzerByNameTest.java
new file mode 100644
index 0000000..366dbaa
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/schema/ResolveAnalyzerByNameTest.java
@@ -0,0 +1,135 @@
+/*
+ * 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.schema;
+
+import java.util.List;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.core.SolrCore;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * This is a simple test to make sure the schema loads when
+ * provided analyzers resolve tokenizer/tokenfilter/charfilter factories by (SPI) name.
+ *
+ */
+
+public class ResolveAnalyzerByNameTest extends SolrTestCaseJ4 {
+
+  @BeforeClass
+  public static void beforeTests() throws Exception {
+    initCore("solrconfig-basic.xml", "schema-analyzer-by-name.xml");
+  }
+
+  @Test
+  public void testSchemaLoadingSimpleAnalyzer() {
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    assertTrue( schema.getFieldTypes().containsKey("text_ws") );
+    SimpleOrderedMap<Object> analyzerProps =
+        (SimpleOrderedMap<Object>)schema.getFieldTypeByName("text_ws")
+        .getNamedPropertyValues(true).get("analyzer");
+    checkTokenizerName(analyzerProps, "whitespace");
+
+    assertNotNull(schema.getFieldTypeByName("text_ws").getIndexAnalyzer());
+    assertNotNull(schema.getFieldTypeByName("text_ws").getQueryAnalyzer());
+  }
+
+  @Test
+  public void testSchemaLoadingComplexAnalyzer() {
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    assertTrue( schema.getFieldTypes().containsKey("text") );
+
+    SimpleOrderedMap<Object> indexAnalyzerProps =
+        (SimpleOrderedMap<Object>)schema.getFieldTypeByName("text")
+            .getNamedPropertyValues(true).get("indexAnalyzer");
+    checkTokenizerName(indexAnalyzerProps, "whitespace");
+    checkTokenFilterNames(indexAnalyzerProps, new String[]{"stop", "wordDelimiterGraph", "lowercase", "keywordMarker", "porterStem", "removeDuplicates", "flattenGraph"});
+
+    SimpleOrderedMap<Object> queryAnalyzerProps =
+        (SimpleOrderedMap<Object>)schema.getFieldTypeByName("text")
+            .getNamedPropertyValues(true).get("queryAnalyzer");
+    checkTokenizerName(queryAnalyzerProps, "whitespace");
+    checkTokenFilterNames(queryAnalyzerProps, new String[]{"synonymGraph", "stop", "wordDelimiterGraph", "lowercase", "keywordMarker", "porterStem", "removeDuplicates"});
+
+    assertNotNull(schema.getFieldTypeByName("text").getIndexAnalyzer());
+    assertNotNull(schema.getFieldTypeByName("text").getQueryAnalyzer());
+  }
+
+  @Test
+  public void testSchemaLoadingAnalyzerWithCharFilters() {
+    SolrCore core = h.getCore();
+    IndexSchema schema = core.getLatestSchema();
+    assertTrue( schema.getFieldTypes().containsKey("charfilthtmlmap") );
+    SimpleOrderedMap<Object> analyzerProps =
+        (SimpleOrderedMap<Object>)schema.getFieldTypeByName("charfilthtmlmap")
+            .getNamedPropertyValues(true).get("analyzer");
+    checkTokenizerName(analyzerProps, "whitespace");
+    checkCharFilterNames(analyzerProps, new String[]{"htmlStrip", "mapping"});
+
+    assertNotNull(schema.getFieldTypeByName("charfilthtmlmap").getIndexAnalyzer());
+    assertNotNull(schema.getFieldTypeByName("charfilthtmlmap").getQueryAnalyzer());
+  }
+
+  @Test(expected = SolrException.class)
+  public void testSchemaLoadingBogus() throws Exception {
+    initCore("solrconfig-basic.xml", "bad-schema-analyzer-by-name.xml");
+  }
+
+  @Test(expected = SolrException.class)
+  public void testSchemaLoadingClassAndNameTokenizer() throws Exception {
+    initCore("solrconfig-basic.xml", "bad-schema-analyzer-class-and-name-tok.xml");
+  }
+
+  @Test(expected = SolrException.class)
+  public void testSchemaLoadingClassAndNameCharFilter() throws Exception {
+    initCore("solrconfig-basic.xml", "bad-schema-analyzer-class-and-name-cf.xml");
+  }
+
+  @Test(expected = SolrException.class)
+  public void testSchemaLoadingClassAndNameTokenFilter() throws Exception {
+    initCore("solrconfig-basic.xml", "bad-schema-analyzer-class-and-name-tf.xml");
+  }
+
+  private void checkTokenizerName(SimpleOrderedMap<Object> analyzerProps, String name) {
+    SimpleOrderedMap<Object> tokenizerProps = (SimpleOrderedMap<Object>)analyzerProps.get("tokenizer");
+    assertNull(tokenizerProps.get("class"));
+    assertEquals(name, tokenizerProps.get("name"));
+  }
+
+  private void checkTokenFilterNames(SimpleOrderedMap<Object> analyzerProps, String[] names) {
+    List<SimpleOrderedMap<Object>> tokenFilterProps = (List<SimpleOrderedMap<Object>>)analyzerProps.get("filters");
+    assertEquals(names.length, tokenFilterProps.size());
+    for (int i = 0; i < names.length; i++) {
+      assertNull(tokenFilterProps.get(i).get("class"));
+      assertEquals(names[i], tokenFilterProps.get(i).get("name"));
+    }
+  }
+
+  private void checkCharFilterNames(SimpleOrderedMap<Object> analyzerProps, String[] names) {
+    List<SimpleOrderedMap<Object>> charFilterProps = (List<SimpleOrderedMap<Object>>)analyzerProps.get("charFilters");
+    assertEquals(names.length, charFilterProps.size());
+    for (int i = 0; i < names.length; i++) {
+      assertNull(charFilterProps.get(i).get("class"));
+      assertEquals(names[i], charFilterProps.get(i).get("name"));
+    }
+  }
+}
diff --git a/solr/solr-ref-guide/src/charfilterfactories.adoc b/solr/solr-ref-guide/src/charfilterfactories.adoc
index c210c38..cd81b3e 100644
--- a/solr/solr-ref-guide/src/charfilterfactories.adoc
+++ b/solr/solr-ref-guide/src/charfilterfactories.adoc
@@ -116,7 +116,7 @@ This filter performs pre-tokenization Unicode normalization using http://site.ic
 
 Arguments:
 
-`name`:: A http://unicode.org/reports/tr15/[Unicode Normalization Form], one of `nfc`, `nfkc`, `nfkc_cf`. Default is `nfkc_cf`.
+`form`:: A http://unicode.org/reports/tr15/[Unicode Normalization Form], one of `nfc`, `nfkc`, `nfkc_cf`. Default is `nfkc_cf`.
 
 `mode`:: Either `compose` or `decompose`. Default is `compose`. Use `decompose` with `name="nfc"` or `name="nfkc"` to get NFD or NFKD, respectively.
 
diff --git a/solr/solr-ref-guide/src/filter-descriptions.adoc b/solr/solr-ref-guide/src/filter-descriptions.adoc
index 7fabb75..eedfbe9 100644
--- a/solr/solr-ref-guide/src/filter-descriptions.adoc
+++ b/solr/solr-ref-guide/src/filter-descriptions.adoc
@@ -515,7 +515,7 @@ This filter factory normalizes text according to one of five Unicode Normalizati
 
 *Arguments:*
 
-`name`:: The name of the normalization form. Valid options are `nfc`, `nfd`, `nfkc`, `nfkd`, or `nfkc_cf` (the default). Required.
+`form`:: The name of the normalization form. Valid options are `nfc`, `nfd`, `nfkc`, `nfkd`, or `nfkc_cf` (the default). Required.
 
 `mode`:: The mode of Unicode character composition and decomposition. Valid options are: `compose` (the default) or `decompose`. Required.
 
@@ -527,7 +527,7 @@ This filter factory normalizes text according to one of five Unicode Normalizati
 ----
 <analyzer>
   <tokenizer class="solr.StandardTokenizerFactory"/>
-  <filter class="solr.ICUNormalizer2FilterFactory" name="nfkc_cf" mode="compose"/>
+  <filter class="solr.ICUNormalizer2FilterFactory" form="nfkc_cf" mode="compose"/>
 </analyzer>
 ----
 
@@ -537,7 +537,7 @@ This filter factory normalizes text according to one of five Unicode Normalizati
 ----
 <analyzer>
   <tokenizer class="solr.StandardTokenizerFactory"/>
-  <filter class="solr.ICUNormalizer2FilterFactory" name="nfkc_cf" mode="compose" filter="[^åäöÅÄÖ]"/>
+  <filter class="solr.ICUNormalizer2FilterFactory" form="nfkc_cf" mode="compose" filter="[^åäöÅÄÖ]"/>
 </analyzer>
 ----
 
diff --git a/solr/test-framework/src/java/org/apache/solr/analysis/MockCharFilterFactory.java b/solr/test-framework/src/java/org/apache/solr/analysis/MockCharFilterFactory.java
index 48a5728..1dfe97d 100644
--- a/solr/test-framework/src/java/org/apache/solr/analysis/MockCharFilterFactory.java
+++ b/solr/test-framework/src/java/org/apache/solr/analysis/MockCharFilterFactory.java
@@ -26,6 +26,10 @@ import org.apache.lucene.analysis.util.CharFilterFactory;
  * Factory for {@link MockCharFilter} for testing purposes.
  */
 public class MockCharFilterFactory extends CharFilterFactory {
+
+  /** SPI name */
+  public static final String NAME = "mock";
+
   final int remainder;
 
   /** Creates a new MockCharFilterFactory */
diff --git a/solr/test-framework/src/java/org/apache/solr/analysis/MockTokenFilterFactory.java b/solr/test-framework/src/java/org/apache/solr/analysis/MockTokenFilterFactory.java
index db329f2..bdc1837 100644
--- a/solr/test-framework/src/java/org/apache/solr/analysis/MockTokenFilterFactory.java
+++ b/solr/test-framework/src/java/org/apache/solr/analysis/MockTokenFilterFactory.java
@@ -29,6 +29,10 @@ import org.apache.lucene.util.automaton.RegExp;
  * Factory for {@link MockTokenFilter} for testing purposes.
  */
 public class MockTokenFilterFactory extends TokenFilterFactory {
+
+  /** SPI name */
+  public static final String NAME = "mock";
+
   final CharacterRunAutomaton filter;
 
   /** Creates a new MockTokenizerFactory */
diff --git a/solr/test-framework/src/java/org/apache/solr/analysis/MockTokenizerFactory.java b/solr/test-framework/src/java/org/apache/solr/analysis/MockTokenizerFactory.java
index d6b2050..bccc76d 100644
--- a/solr/test-framework/src/java/org/apache/solr/analysis/MockTokenizerFactory.java
+++ b/solr/test-framework/src/java/org/apache/solr/analysis/MockTokenizerFactory.java
@@ -28,6 +28,10 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton;
  * Factory for {@link MockTokenizer} for testing purposes.
  */
 public class MockTokenizerFactory extends TokenizerFactory {
+
+  /** SPI name */
+  public static final String NAME = "mock";
+
   final CharacterRunAutomaton pattern;
   final boolean enableChecks;