You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sa...@apache.org on 2013/04/01 18:16:04 UTC

svn commit: r1463182 [1/2] - in /lucene/dev/trunk/solr: ./ contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/ core/src/java/org/apache/solr/cloud/ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/response/ core/src...

Author: sarowe
Date: Mon Apr  1 16:16:03 2013
New Revision: 1463182

URL: http://svn.apache.org/r1463182
Log:
SOLR-4658: In preparation for REST API requests that can modify the schema, a "managed schema" is introduced.

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ClassicIndexSchemaFactory.java   (with props)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java   (with props)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java   (with props)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-managed-schema-named-schema.xml.xml   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-schema-mutable-but-not-managed.xml   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-unexpected-schema-attribute.xml   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog-managed-schema.xml   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchema.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestManagedSchema.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/SolrStopwordsCarrot2LexicalDataFactory.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/Config.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolr.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlBackCompat.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrProperties.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/SchemaXmlWriter.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/AbstractBadConfigTestBase.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/PrimitiveFieldTypeTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/SolrIndexConfigTest.java
    lucene/dev/trunk/solr/example/example-DIH/solr/db/conf/solrconfig.xml
    lucene/dev/trunk/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
    lucene/dev/trunk/solr/example/example-DIH/solr/rss/conf/solrconfig.xml
    lucene/dev/trunk/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
    lucene/dev/trunk/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
    lucene/dev/trunk/solr/example/multicore/core0/conf/solrconfig.xml
    lucene/dev/trunk/solr/example/multicore/core1/conf/solrconfig.xml
    lucene/dev/trunk/solr/example/solr/collection1/conf/solrconfig.xml
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Mon Apr  1 16:16:03 2013
@@ -97,6 +97,12 @@ New Features
   "dynamicFields", respectively, to align with all other REST API outputs, which
   use camelCase.
   (Steve Rowe)
+  
+* SOLR-4658: In preparation for REST API requests that can modify the schema,
+  a "managed schema" is introduced.  
+  Add '<schemaFactory class="ManagedSchemaFactory" mutable="true"/>' to solrconfig.xml
+  in order to use it, and to enable schema modifications via REST API requests.
+  (Steve Rowe, Robert Muir)
 
 Bug Fixes
 ----------------------

Modified: lucene/dev/trunk/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/SolrStopwordsCarrot2LexicalDataFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/SolrStopwordsCarrot2LexicalDataFactory.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/SolrStopwordsCarrot2LexicalDataFactory.java (original)
+++ lucene/dev/trunk/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/SolrStopwordsCarrot2LexicalDataFactory.java Mon Apr  1 16:16:03 2013
@@ -27,6 +27,7 @@ import org.apache.lucene.analysis.common
 import org.apache.lucene.analysis.core.StopFilterFactory;
 import org.apache.solr.analysis.TokenizerChain;
 import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.ManagedIndexSchema;
 import org.carrot2.core.LanguageCode;
 import org.carrot2.core.attribute.Init;
 import org.carrot2.core.attribute.Processing;
@@ -37,6 +38,7 @@ import org.carrot2.text.util.MutableChar
 import org.carrot2.util.attribute.Attribute;
 import org.carrot2.util.attribute.Bindable;
 import org.carrot2.util.attribute.Input;
+import org.carrot2.util.attribute.constraint.ImplementingClasses;
 import org.slf4j.Logger;
 
 import com.google.common.collect.HashMultimap;
@@ -60,6 +62,7 @@ public class SolrStopwordsCarrot2Lexical
   @Init
   @Input
   @Attribute(key = "solrIndexSchema")
+  @ImplementingClasses(classes = { IndexSchema.class, ManagedIndexSchema.class })
   private IndexSchema schema;
 
   @Processing

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java Mon Apr  1 16:16:03 2013
@@ -125,4 +125,7 @@ public class ZkSolrResourceLoader extend
     return collectionZkPath;
   }
   
+  public ZkController getZkController() {
+    return zkController;
+  }
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/Config.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/Config.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/Config.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/Config.java Mon Apr  1 16:16:03 2013
@@ -25,7 +25,10 @@ import org.apache.solr.common.util.XMLEr
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NamedNodeMap;
 import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
 import org.apache.commons.io.IOUtils;
@@ -41,6 +44,13 @@ import javax.xml.xpath.XPathExpressionEx
 import javax.xml.xpath.XPathFactory;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
@@ -235,6 +245,98 @@ public class Config {
     }
   }
 
+  public NodeList getNodeList(String path, boolean errIfMissing) {
+    XPath xpath = xpathFactory.newXPath();
+    String xstr = normalize(path);
+
+    try {
+      NodeList nodeList = (NodeList)xpath.evaluate(xstr, doc, XPathConstants.NODESET);
+
+      if (null == nodeList) {
+        if (errIfMissing) {
+          throw new RuntimeException(name + " missing "+path);
+        } else {
+          log.debug(name + " missing optional " + path);
+          return null;
+        }
+      }
+
+      log.trace(name + ":" + path + "=" + nodeList);
+      return nodeList;
+
+    } catch (XPathExpressionException e) {
+      SolrException.log(log,"Error in xpath",e);
+      throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"Error in xpath:" + xstr + " for " + name,e);
+    } catch (SolrException e) {
+      throw(e);
+    } catch (Throwable e) {
+      SolrException.log(log,"Error in xpath",e);
+      throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"Error in xpath:" + xstr+ " for " + name,e);
+    }
+  }
+
+  /**
+   * Returns the set of attributes on the given element that are not among the given knownAttributes,
+   * or null if all attributes are known.
+   */
+  public Set<String> getUnknownAttributes(Element element, String... knownAttributes) {
+    Set<String> knownAttributeSet = new HashSet<String>(Arrays.asList(knownAttributes));
+    Set<String> unknownAttributeSet = null;
+    NamedNodeMap attributes = element.getAttributes();
+    for (int i = 0 ; i < attributes.getLength() ; ++i) {
+      final String attributeName = attributes.item(i).getNodeName();
+      if ( ! knownAttributeSet.contains(attributeName)) {
+        if (null == unknownAttributeSet) {
+          unknownAttributeSet = new HashSet<String>();
+        }
+        unknownAttributeSet.add(attributeName);
+      }
+    }
+    return unknownAttributeSet;
+  }
+
+  /**
+   * Logs an error and throws an exception if any of the element(s) at the given elementXpath
+   * contains an attribute name that is not among knownAttributes. 
+   */
+  public void complainAboutUnknownAttributes(String elementXpath, String... knownAttributes) {
+    SortedMap<String,SortedSet<String>> problems = new TreeMap<String,SortedSet<String>>(); 
+    NodeList nodeList = getNodeList(elementXpath, false);
+    for (int i = 0 ; i < nodeList.getLength() ; ++i) {
+      Element element = (Element)nodeList.item(i);
+      Set<String> unknownAttributes = getUnknownAttributes(element, knownAttributes);
+      if (null != unknownAttributes) {
+        String elementName = element.getNodeName();
+        SortedSet<String> allUnknownAttributes = problems.get(elementName);
+        if (null == allUnknownAttributes) {
+          allUnknownAttributes = new TreeSet<String>();
+          problems.put(elementName, allUnknownAttributes);
+        }
+        allUnknownAttributes.addAll(unknownAttributes);
+      }
+    }
+    if (problems.size() > 0) {
+      StringBuilder message = new StringBuilder();
+      for (Map.Entry<String,SortedSet<String>> entry : problems.entrySet()) {
+        if (message.length() > 0) {
+          message.append(", ");
+        }
+        message.append('<');
+        message.append(entry.getKey());
+        for (String attributeName : entry.getValue()) {
+          message.append(' ');
+          message.append(attributeName);
+          message.append("=\"...\"");
+        }
+        message.append('>');
+      }
+      message.insert(0, "Unknown attribute(s) on element(s): ");
+      String msg = message.toString();
+      SolrException.log(log, msg);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
+    }
+  }
+
   public String getVal(String path, boolean errIfMissing) {
     Node nd = getNode(path,errIfMissing);
     if (nd==null) return null;

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolr.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolr.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolr.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolr.java Mon Apr  1 16:16:03 2013
@@ -19,8 +19,6 @@ package org.apache.solr.core;
 
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.handler.component.ShardHandlerFactory;
-import org.apache.solr.schema.IndexSchema;
-import org.apache.zookeeper.KeeperException;
 
 import java.io.File;
 import java.util.List;
@@ -58,9 +56,6 @@ public interface ConfigSolr {
 
   public Properties getSolrProperties(ConfigSolr cfg, String context);
 
-  public IndexSchema getSchemaFromZk(ZkController zkController, String zkConfigName, String schemaName,
-                                     SolrConfig config) throws KeeperException, InterruptedException;
-
   public SolrConfig getSolrConfigFromZk(ZkController zkController, String zkConfigName, String solrConfigFileName,
                                         SolrResourceLoader resourceLoader);
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlBackCompat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlBackCompat.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlBackCompat.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlBackCompat.java Mon Apr  1 16:16:03 2013
@@ -21,12 +21,10 @@ import org.apache.solr.cloud.ZkControlle
 import org.apache.solr.common.SolrException;
 import org.apache.solr.handler.component.HttpShardHandlerFactory;
 import org.apache.solr.handler.component.ShardHandlerFactory;
-import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.util.DOMUtil;
 import org.apache.solr.util.PropertiesUtil;
 import org.apache.solr.util.SystemIdResolver;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
-import org.apache.zookeeper.KeeperException;
 import org.w3c.dom.Document;
 import org.w3c.dom.NamedNodeMap;
 import org.w3c.dom.Node;
@@ -197,16 +195,6 @@ public class ConfigSolrXmlBackCompat ext
     return attrs;
   }
 
-  public IndexSchema getSchemaFromZk(ZkController zkController, String zkConfigName, String schemaName,
-                                     SolrConfig config)
-      throws KeeperException, InterruptedException {
-    byte[] configBytes = zkController.getConfigFileData(zkConfigName, schemaName);
-    InputSource is = new InputSource(new ByteArrayInputStream(configBytes));
-    is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(schemaName));
-    IndexSchema schema = new IndexSchema(config, schemaName, is);
-    return schema;
-  }
-
   @Override
   public SolrConfig getSolrConfigFromZk(ZkController zkController, String zkConfigName, String solrConfigFileName,
                                         SolrResourceLoader resourceLoader) {

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java Mon Apr  1 16:16:03 2013
@@ -71,6 +71,7 @@ import org.apache.solr.logging.LogWatche
 import org.apache.solr.logging.jul.JulWatcher;
 import org.apache.solr.logging.log4j.Log4jWatcher;
 import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.IndexSchemaFactory;
 import org.apache.solr.update.SolrCoreState;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.FileUtils;
@@ -868,7 +869,7 @@ public class CoreContainer
       }
       solrLoader = new ZkSolrResourceLoader(instanceDir, zkConfigName, libLoader, SolrProperties.getCoreProperties(instanceDir, dcore), zkController);
       config = getSolrConfigFromZk(zkConfigName, dcore.getConfigName(), solrLoader);
-      schema = getSchemaFromZk(zkConfigName, dcore.getSchemaName(), config);
+      schema = IndexSchemaFactory.buildIndexSchema(dcore.getSchemaName(), config);
       return new SolrCore(dcore.getName(), null, config, schema, dcore);
 
     } catch (KeeperException e) {
@@ -912,7 +913,7 @@ public class CoreContainer
         schema = indexSchemaCache.get(key);
         if (schema == null) {
           log.info("creating new schema object for core: " + dcore.getProperty(CoreDescriptor.CORE_NAME));
-          schema = new IndexSchema(config, dcore.getSchemaName(), null);
+          schema = IndexSchemaFactory.buildIndexSchema(dcore.getSchemaName(), config);
           indexSchemaCache.put(key, schema);
         } else {
           log.info("re-using schema object for core: " + dcore.getProperty(CoreDescriptor.CORE_NAME));
@@ -921,7 +922,7 @@ public class CoreContainer
     }
 
     if (schema == null) {
-      schema = new IndexSchema(config, dcore.getSchemaName(), null);
+      schema = IndexSchemaFactory.buildIndexSchema(dcore.getSchemaName(), config);
     }
 
     SolrCore core = new SolrCore(dcore.getName(), null, config, schema, dcore);
@@ -1361,12 +1362,6 @@ public class CoreContainer
     log.error(msg, ex);
     return new SolrException(ErrorCode.SERVER_ERROR, msg, ex);
   }
-
-  private IndexSchema getSchemaFromZk(String zkConfigName, String schemaName,
-      SolrConfig config)
-      throws KeeperException, InterruptedException {
-    return cfg.getSchemaFromZk(zkController, zkConfigName, schemaName, config);
-  }
 }
 
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java Mon Apr  1 16:16:03 2013
@@ -19,6 +19,8 @@ package org.apache.solr.core;
 
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.IndexSchemaFactory;
 import org.apache.solr.util.DOMUtil;
 import org.apache.solr.util.RegexFileFilter;
 import org.apache.solr.handler.component.SearchComponent;
@@ -214,9 +216,9 @@ public class SolrConfig extends Config {
      loadPluginInfo(QueryConverter.class,"queryConverter",true, true);
 
      // this is hackish, since it picks up all SolrEventListeners,
-     // regardless of when/how/why thye are used (or even if they are 
+     // regardless of when/how/why they are used (or even if they are 
      // declared outside of the appropriate context) but there's no nice 
-     // way arround that in the PluginInfo framework
+     // way around that in the PluginInfo framework
      loadPluginInfo(SolrEventListener.class, "//listener",false, true);
 
      loadPluginInfo(DirectoryFactory.class,"directoryFactory",false, true);
@@ -225,6 +227,7 @@ public class SolrConfig extends Config {
      loadPluginInfo(IndexReaderFactory.class,"indexReaderFactory",false, true);
      loadPluginInfo(UpdateRequestProcessorChain.class,"updateRequestProcessorChain",false, false);
      loadPluginInfo(UpdateLog.class,"updateHandler/updateLog",false, false);
+     loadPluginInfo(IndexSchemaFactory.class,"schemaFactory",false, true);
 
      updateHandlerInfo = loadUpdatehandlerInfo();
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java Mon Apr  1 16:16:03 2013
@@ -96,6 +96,7 @@ import org.apache.solr.response.XMLRespo
 import org.apache.solr.response.transform.TransformerFactory;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.IndexSchemaFactory;
 import org.apache.solr.schema.SchemaAware;
 import org.apache.solr.search.QParserPlugin;
 import org.apache.solr.search.SolrFieldCacheMBean;
@@ -393,11 +394,9 @@ public final class SolrCore implements S
   public SolrCore reload(SolrResourceLoader resourceLoader, SolrCore prev) throws IOException,
       ParserConfigurationException, SAXException {
     
-    SolrConfig config = new SolrConfig(resourceLoader,
-        getSolrConfig().getName(), null);
+    SolrConfig config = new SolrConfig(resourceLoader, getSolrConfig().getName(), null);
     
-    IndexSchema schema = new IndexSchema(config,
-        getSchema().getResourceName(), null);
+    IndexSchema schema = IndexSchemaFactory.buildIndexSchema(getSchema().getResourceName(), config);
     
     solrCoreState.increfSolrCoreState();
     
@@ -681,7 +680,7 @@ public final class SolrCore implements S
     log.info(logid+"Opening new SolrCore at " + resourceLoader.getInstanceDir() + ", dataDir="+dataDir);
 
     if (schema==null) {
-      schema = new IndexSchema(config, IndexSchema.DEFAULT_SCHEMA_FILE, null);
+      schema = IndexSchemaFactory.buildIndexSchema(IndexSchema.DEFAULT_SCHEMA_FILE, config);
     }
 
     if (null != cd && null != cd.getCloudDescriptor()) {
@@ -689,7 +688,7 @@ public final class SolrCore implements S
       //
       // In cloud mode, version field is required for correct consistency
       // ideally this check would be more fine grained, and individual features
-      // would assert it when they initialize, but DistribuedUpdateProcessor
+      // would assert it when they initialize, but DistributedUpdateProcessor
       // is currently a big ball of wax that does more then just distributing
       // updates (ie: partial document updates), so it needs to work in no cloud
       // mode as well, and can't assert version field support on init.

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrProperties.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrProperties.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrProperties.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrProperties.java Mon Apr  1 16:16:03 2013
@@ -24,11 +24,9 @@ import org.apache.solr.common.SolrExcept
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.handler.component.HttpShardHandlerFactory;
 import org.apache.solr.handler.component.ShardHandlerFactory;
-import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.util.PropertiesUtil;
 import org.apache.solr.util.SystemIdResolver;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
-import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.InputSource;
@@ -412,18 +410,6 @@ public class SolrProperties implements C
 
   // Copied verbatim from the old code, presumably this will be tested when we eliminate solr.xml
   @Override
-  public IndexSchema getSchemaFromZk(ZkController zkController, String zkConfigName, String schemaName,
-                                     SolrConfig config)
-      throws KeeperException, InterruptedException {
-    byte[] configBytes = zkController.getConfigFileData(zkConfigName, schemaName);
-    InputSource is = new InputSource(new ByteArrayInputStream(configBytes));
-    is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(schemaName));
-    IndexSchema schema = new IndexSchema(config, schemaName, is);
-    return schema;
-  }
-
-  // Copied verbatim from the old code, presumably this will be tested when we eliminate solr.xml
-  @Override
   public SolrConfig getSolrConfigFromZk(ZkController zkController, String zkConfigName, String solrConfigFileName,
                                         SolrResourceLoader resourceLoader) {
     SolrConfig cfg = null;

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/SchemaXmlWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/SchemaXmlWriter.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/SchemaXmlWriter.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/SchemaXmlWriter.java Mon Apr  1 16:16:03 2013
@@ -40,8 +40,15 @@ import org.slf4j.LoggerFactory;
  * @lucene.internal
  */
 public class SchemaXmlWriter extends TextResponseWriter {
-  final static Logger log = LoggerFactory.getLogger(SchemaXmlWriter.class);
+  private static final Logger log = LoggerFactory.getLogger(SchemaXmlWriter.class);
   private static final char[] XML_DECLARATION = "<?xml version=\"1.0\" encoding=\"UTF-8\"?>".toCharArray();
+  private static final char[] MANAGED_SCHEMA_DO_NOT_EDIT_WARNING 
+      = "<!-- Solr managed schema - automatically generated - DO NOT EDIT -->".toCharArray();
+  
+  private boolean emitManagedSchemaDoNotEditWarning = false;
+  public void setEmitManagedSchemaDoNotEditWarning(boolean emitManagedSchemaDoNotEditWarning) { 
+    this.emitManagedSchemaDoNotEditWarning = emitManagedSchemaDoNotEditWarning; 
+  }
 
   public static void writeResponse(Writer writer, SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
     SchemaXmlWriter schemaXmlWriter = null;
@@ -62,6 +69,13 @@ public class SchemaXmlWriter extends Tex
   public void writeResponse() throws IOException {
     
     writer.write(XML_DECLARATION);
+    if (emitManagedSchemaDoNotEditWarning) {
+      if (doIndent) {
+        writer.write('\n');
+      }
+      writer.write(MANAGED_SCHEMA_DO_NOT_EDIT_WARNING);
+    }
+
     @SuppressWarnings("unchecked") SimpleOrderedMap<Object> schemaProperties
         = (SimpleOrderedMap<Object>)rsp.getValues().get(IndexSchema.SCHEMA);
 
@@ -139,7 +153,6 @@ public class SchemaXmlWriter extends Tex
     }
     decLevel();
     endTag(IndexSchema.SCHEMA);
-    
   }
 
   private void writeFieldTypes(List<SimpleOrderedMap<Object>> fieldTypePropertiesList) throws IOException {

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ClassicIndexSchemaFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ClassicIndexSchemaFactory.java?rev=1463182&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ClassicIndexSchemaFactory.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ClassicIndexSchemaFactory.java Mon Apr  1 16:16:03 2013
@@ -0,0 +1,65 @@
+package org.apache.solr.schema;
+/*
+ * 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.
+ */
+
+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.SolrConfig;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.util.SystemIdResolver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.xml.sax.InputSource;
+
+import java.io.InputStream;
+
+public class ClassicIndexSchemaFactory extends IndexSchemaFactory {
+  private static final Logger log = LoggerFactory.getLogger(ClassicIndexSchemaFactory.class);
+
+  @Override
+  public void init(NamedList args) {
+    // no arguments expected
+    if (args.size() > 0) {
+      String msg = "Unexpected arg(s): " + args;
+      log.error(msg);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+    }
+  }
+
+  @Override
+  public IndexSchema create(String resourceName, SolrConfig config) {
+    SolrResourceLoader loader = config.getResourceLoader();
+    InputStream schemaInputStream = null;
+
+    if (null == resourceName) {
+      resourceName = IndexSchema.DEFAULT_SCHEMA_FILE;
+    }
+
+    try {
+      schemaInputStream = loader.openSchema(resourceName);
+    } catch (Exception e) {
+      final String msg = "Error loading schema resource " + resourceName;
+      log.error(msg, e);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg, e);
+    }
+    InputSource inputSource = new InputSource(schemaInputStream);
+    inputSource.setSystemId(SystemIdResolver.createSystemIdFromResourceName(resourceName));
+    IndexSchema schema = new IndexSchema(config, resourceName, inputSource);
+    return schema;
+  }
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchema.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchema.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchema.java Mon Apr  1 16:16:03 2013
@@ -17,6 +17,7 @@
 
 package org.apache.solr.schema;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.AnalyzerWrapper;
 import org.apache.lucene.index.IndexableField;
@@ -24,17 +25,27 @@ import org.apache.lucene.index.StorableF
 import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.Version;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.ZkCmdExecutor;
+import org.apache.solr.common.cloud.ZooKeeperException;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.apache.solr.response.SchemaXmlWriter;
+import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.util.DOMUtil;
+import org.apache.solr.util.FileUtils;
 import org.apache.solr.util.SystemIdResolver;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.Config;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.search.similarities.DefaultSimilarityFactory;
 import org.apache.solr.util.plugin.SolrCoreAware;
+import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.w3c.dom.Document;
@@ -47,7 +58,13 @@ import org.xml.sax.InputSource;
 import javax.xml.xpath.XPath;
 import javax.xml.xpath.XPathConstants;
 
+import java.io.File;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.io.StringWriter;
+import java.io.Writer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -71,7 +88,7 @@ import java.util.regex.Pattern;
  *
  *
  */
-public final class IndexSchema {
+public class IndexSchema {
   public static final String COPY_FIELD = "copyField";
   public static final String COPY_FIELDS = COPY_FIELD + "s";
   public static final String DEFAULT_OPERATOR = "defaultOperator";
@@ -108,7 +125,7 @@ public final class IndexSchema {
 
   final static Logger log = LoggerFactory.getLogger(IndexSchema.class);
   private final SolrConfig solrConfig;
-  private final String resourceName;
+  private String resourceName;
   private String name;
   private float version;
   private final SolrResourceLoader loader;
@@ -145,24 +162,21 @@ public final class IndexSchema {
 
     /**
    * Constructs a schema using the specified resource name and stream.
-   * If the is stream is null, the resource loader will load the schema resource by name.
    * @see SolrResourceLoader#openSchema
    * By default, this follows the normal config path directory searching rules.
    * @see SolrResourceLoader#openResource
    */
   public IndexSchema(SolrConfig solrConfig, String name, InputSource is) {
+    assert null != solrConfig : "SolrConfig should never be null";
+    assert null != name : "schema resource name should never be null";
+    assert null != is : "schema InputSource should never be null";
+
     this.solrConfig = solrConfig;
-    if (name == null)
-      name = DEFAULT_SCHEMA_FILE;
     this.resourceName = name;
     loader = solrConfig.getResourceLoader();
     try {
-      if (is == null) {
-        is = new InputSource(loader.openSchema(name));
-        is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(name));
-      }
       readSchema(is);
-      loader.inform( loader );
+      loader.inform(loader);
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
@@ -179,7 +193,12 @@ public final class IndexSchema {
   public String getResourceName() {
     return resourceName;
   }
-  
+
+  /** Sets the name of the resource used to instantiate this schema. */
+  public void setResourceName(String resourceName) {
+    this.resourceName = resourceName;
+  }
+
   /** Gets the name of the schema as specified in the schema resource. */
   public String getSchemaName() {
     return name;
@@ -345,6 +364,24 @@ public final class IndexSchema {
     queryAnalyzer = new SolrQueryAnalyzer();
   }
 
+  /**
+   * Writes the schema in schema.xml format to the given writer 
+   */
+  void persist(Writer writer) throws IOException {
+    final SolrQueryResponse response = new SolrQueryResponse();
+    response.add(IndexSchema.SCHEMA, getNamedPropertyValues());
+    final NamedList args = new NamedList(Arrays.<Object>asList("indent", "on"));
+    final LocalSolrQueryRequest req = new LocalSolrQueryRequest(null, args);
+    final SchemaXmlWriter schemaXmlWriter = new SchemaXmlWriter(writer, req, response);
+    schemaXmlWriter.setEmitManagedSchemaDoNotEditWarning(true);
+    schemaXmlWriter.writeResponse();
+    schemaXmlWriter.close();
+  }
+
+  public boolean isMutable() {
+    return false;
+  }
+
   private class SolrIndexAnalyzer extends AnalyzerWrapper {
     protected final HashMap<String, Analyzer> analyzers;
 
@@ -392,7 +429,7 @@ public final class IndexSchema {
   }
 
   private void readSchema(InputSource is) {
-    log.info("Reading Solr Schema");
+    log.info("Reading Solr Schema from " + resourceName);
 
     try {
       // pass the config resource loader to avoid building an empty one for no reason:
@@ -1239,7 +1276,7 @@ public final class IndexSchema {
   }
 
   /**
-   * Get a map of property name -> value for this field.
+   * Get a map of property name -> value for the whole schema.
    */
   public SimpleOrderedMap<Object> getNamedPropertyValues() {
     SimpleOrderedMap<Object> topLevel = new SimpleOrderedMap<Object>();

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java?rev=1463182&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java Mon Apr  1 16:16:03 2013
@@ -0,0 +1,39 @@
+package org.apache.solr.schema;
+/*
+ * 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.
+ */
+
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrConfig;
+import org.apache.solr.util.plugin.NamedListInitializedPlugin;
+
+public abstract class IndexSchemaFactory implements NamedListInitializedPlugin {
+  
+  public abstract IndexSchema create(String resourceName, SolrConfig config);
+
+  public static IndexSchema buildIndexSchema(String resourceName, SolrConfig config) {
+    PluginInfo info = config.getPluginInfo(IndexSchemaFactory.class.getName());
+    IndexSchemaFactory factory;
+    if (null != info) {
+      factory = config.getResourceLoader().newInstance(info.className, IndexSchemaFactory.class);
+      factory.init(info.initArgs);
+    } else {
+      factory = new ClassicIndexSchemaFactory();
+    }
+    IndexSchema schema = factory.create(resourceName, config);
+    return schema;
+  }
+}

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java?rev=1463182&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java Mon Apr  1 16:16:03 2013
@@ -0,0 +1,43 @@
+package org.apache.solr.schema;
+/*
+ * 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.
+ */
+
+import org.apache.solr.core.SolrConfig;
+import org.xml.sax.InputSource;
+
+/** Solr-managed schema - non-user-editable, but can be mutable via internal and external REST API requests. */
+public final class ManagedIndexSchema extends IndexSchema {
+
+  private boolean isMutable = false;
+
+  @Override
+  public boolean isMutable() {
+    return isMutable;
+  }
+  
+  /**
+   * Constructs a schema using the specified resource name and stream.
+   *
+   * @see org.apache.solr.core.SolrResourceLoader#openSchema
+   *      By default, this follows the normal config path directory searching rules.
+   * @see org.apache.solr.core.SolrResourceLoader#openResource
+   */
+  ManagedIndexSchema(SolrConfig solrConfig, String name, InputSource is, boolean isMutable) {
+    super(solrConfig, name, is);
+    this.isMutable = isMutable;
+  }
+}

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java?rev=1463182&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java Mon Apr  1 16:16:03 2013
@@ -0,0 +1,359 @@
+package org.apache.solr.schema;
+/*
+ * 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.
+ */
+
+import org.apache.commons.io.IOUtils;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.cloud.ZkSolrResourceLoader;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.ZkCmdExecutor;
+import org.apache.solr.common.cloud.ZooKeeperException;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrConfig;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.util.FileUtils;
+import org.apache.solr.util.SystemIdResolver;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.xml.sax.InputSource;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.io.StringWriter;
+
+public class ManagedIndexSchemaFactory extends IndexSchemaFactory {
+  private static final Logger log = LoggerFactory.getLogger(ManagedIndexSchemaFactory.class);
+  private static final String UPGRADED_SCHEMA_EXTENSION = ".bak";
+
+  private boolean isMutable;
+  private String managedSchemaResourceName;
+  private SolrConfig config;
+  private SolrResourceLoader loader;
+  private String resourceName;
+  private IndexSchema schema;
+
+  @Override
+  public void init(NamedList args) {
+    SolrParams params = SolrParams.toSolrParams(args);
+    isMutable = params.getBool("mutable", false);
+    args.remove("mutable");
+    managedSchemaResourceName = params.get("managedSchemaResourceName", "managed-schema");
+    args.remove("managedSchemaResourceName");
+    if ("schema.xml".equals(managedSchemaResourceName)) {
+      String msg = "managedSchemaResourceName can't be 'schema.xml'";
+      log.error(msg);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+    }
+    
+    if (args.size() > 0) {
+      String msg = "Unexpected arg(s): " + args;
+      log.error(msg);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+    }
+  }
+
+  /**
+   * First, try to locate the managed schema file named in the managedSchemaResourceName
+   * param. If the managed schema file exists and is accessible, it is used to instantiate
+   * an IndexSchema.
+   *
+   * If the managed schema file can't be found, the resource named by the resourceName
+   * parameter is used to instantiate an IndexSchema.
+   *
+   * Once the IndexSchema is instantiated, if the managed schema file does not exist,
+   * the instantiated IndexSchema is persisted to the managed schema file named in the
+   * managedSchemaResourceName param, in the directory given by 
+   * {@link org.apache.solr.core.SolrResourceLoader#getConfigDir()}, or if configs are
+   * in ZooKeeper, under {@link org.apache.solr.cloud.ZkSolrResourceLoader#collectionZkPath}.
+   *
+   * After the managed schema file is persisted, the original schema file is
+   * renamed by appending the extension named in {@link #UPGRADED_SCHEMA_EXTENSION}.
+   */
+  public IndexSchema create(String resourceName, SolrConfig config) {
+    this.resourceName = resourceName;
+    this.config = config;
+    SolrResourceLoader loader = config.getResourceLoader();
+    this.loader = loader;
+    InputStream schemaInputStream = null;
+    boolean shouldUpgrade = false;
+    String loadedResource = null;
+
+    if (null == resourceName) {
+      resourceName = IndexSchema.DEFAULT_SCHEMA_FILE;
+    }
+
+    try {
+      // Attempt to load the managed schema
+      schemaInputStream = loader.openSchema(managedSchemaResourceName);
+      loadedResource = managedSchemaResourceName;
+
+      // Check if the non-managed schema is also present
+      if ( ! resourceName.equals(managedSchemaResourceName)) {
+        if (nonManagedSchemaExists()) {
+          // Warn if the non-managed schema is present
+          log.warn("The schema has been upgraded to managed, but the non-managed schema " + resourceName
+              + " is still loadable.  PLEASE REMOVE THIS FILE.");
+        }
+      }
+    } catch (IOException e) {
+      log.info("SolrConfig.isManagedSchema = true, but managed schema resource " + managedSchemaResourceName
+          + " not found - loading non-managed schema " + resourceName + " instead");
+    }
+    if (null == schemaInputStream) {
+      // The managed schema file could not be found - load the non-managed schema
+      try {
+        schemaInputStream = loader.openSchema(resourceName);
+        loadedResource = resourceName;
+        shouldUpgrade = true;
+      } catch (Exception e) {
+        try {
+          // Retry to load the managed schema, in case it was created since the first attempt
+          schemaInputStream = loader.openSchema(managedSchemaResourceName);
+          loadedResource = managedSchemaResourceName;
+        } catch (IOException e1) {
+          final String msg = "Error loading both non-managed schema '" + resourceName + "' and managed schema '"
+                           + managedSchemaResourceName + "'";
+          log.error(msg, e);
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg, e);
+        }
+      }
+    }
+    InputSource inputSource = new InputSource(schemaInputStream);
+    inputSource.setSystemId(SystemIdResolver.createSystemIdFromResourceName(loadedResource));
+    schema = new ManagedIndexSchema(config, loadedResource, inputSource, isMutable);
+
+    if (shouldUpgrade) {
+      // Persist the managed schema if it doesn't already exist
+      upgradeToManagedSchema();
+    }
+    return schema;
+  }
+
+  /**
+   * Return whether a non-managed schema exists, either in local storage or on ZooKeeper. 
+   */
+  private boolean nonManagedSchemaExists() {
+    boolean exists = false;
+    SolrResourceLoader loader = config.getResourceLoader();
+    if (loader instanceof ZkSolrResourceLoader) {
+      ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
+      String nonManagedSchemaPath = zkLoader.getCollectionZkPath() + "/" + resourceName;
+      try {
+        exists = zkLoader.getZkController().pathExists(nonManagedSchemaPath);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt(); // Restore the interrupted status
+        log.warn("", e); // Log as warning and suppress the exception 
+      } catch (KeeperException e) {
+        // log as warning and suppress the exception
+        log.warn("Error checking for the existence of the non-managed schema " + resourceName, e);
+      }
+    } else { // Config is not in ZooKeeper
+      InputStream nonManagedSchemaInputStream = null;
+      try {
+        nonManagedSchemaInputStream = loader.openSchema(resourceName);
+        if (null != nonManagedSchemaInputStream) {
+          exists = true;
+        }
+      } catch (IOException e) {
+        // This is expected when the non-managed schema does not exist
+      } finally {
+        IOUtils.closeQuietly(nonManagedSchemaInputStream);
+      }
+    }
+    return exists;
+  }
+
+  /**
+   * Persist the managed schema and rename the non-managed schema 
+   * by appending {@link #UPGRADED_SCHEMA_EXTENSION}.
+   *
+   * Failure to rename the non-managed schema will be logged as a warning,
+   * and no exception will be thrown.
+   */
+  private void upgradeToManagedSchema() {
+    SolrResourceLoader loader = config.getResourceLoader();
+    if (loader instanceof ZkSolrResourceLoader) {
+      zkUgradeToManagedSchema();
+    } else {
+      // Configs are not on ZooKeeper
+      File managedSchemaFile = new File(loader.getConfigDir(), managedSchemaResourceName);
+      OutputStreamWriter writer = null;
+      try {
+        File parentDir = managedSchemaFile.getParentFile();
+        if (!parentDir.isDirectory()) {
+          if (!parentDir.mkdirs()) {
+            final String msg = "Can't create managed schema directory " + parentDir.getAbsolutePath();
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
+          }
+        }
+        final FileOutputStream out = new FileOutputStream(managedSchemaFile);
+        writer = new OutputStreamWriter(out, "UTF-8");
+        schema.persist(writer);
+        log.info("Upgraded to managed schema at " + managedSchemaFile.getPath());
+      } catch (IOException e) {
+        final String msg = "Error persisting managed schema " + managedSchemaFile;
+        log.error(msg, e);
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg, e);
+      } finally {
+        IOUtils.closeQuietly(writer);
+        try {
+          FileUtils.sync(managedSchemaFile);
+        } catch (IOException e) {
+          final String msg = "Error syncing the managed schema file " + managedSchemaFile;
+          log.error(msg, e);
+        }
+      }
+
+      // After successfully persisting the managed schema, rename the non-managed
+      // schema file by appending UPGRADED_SCHEMA_EXTENSION to its name.
+
+      if (resourceName.equals(managedSchemaResourceName)) {
+        log.info("On upgrading to managed schema, did not rename non-managed schema '"
+            + resourceName + "' because it's the same as the managed schema's name.");
+      } else {
+        final File nonManagedSchemaFile = locateConfigFile(resourceName);
+        if (null == nonManagedSchemaFile) {
+          // Don't throw an exception for failure to rename the non-managed schema
+          log.warn("On upgrading to managed schema, did not rename non-managed schema "
+              + resourceName + " because it's neither an absolute file "
+              + "nor under SolrConfig.getConfigDir() or the current directory."
+              + "  PLEASE REMOVE THIS FILE.");
+        } else {
+          File upgradedSchemaFile = new File(nonManagedSchemaFile.getPath() + UPGRADED_SCHEMA_EXTENSION);
+          if (nonManagedSchemaFile.renameTo(upgradedSchemaFile)) {
+            // Set the resource name to the managed schema so that the CoreAdminHandler returns a findable filename 
+            schema.setResourceName(managedSchemaResourceName);
+
+            log.info("After upgrading to managed schema, renamed the non-managed schema "
+                + nonManagedSchemaFile + " to " + upgradedSchemaFile);
+          } else {
+            // Don't throw an exception for failure to rename the non-managed schema
+            log.warn("Can't rename " + nonManagedSchemaFile.toString() + " to "
+                + upgradedSchemaFile.toString() + " - PLEASE REMOVE THIS FILE.");
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Finds any resource by its name on the filesystem.  The classpath is not consulted.
+   *
+   * If the resource is not absolute, the resource is sought in $configDir and then in the current directory.
+   *
+   *@return the File for the named resource, or null if it can't be found
+   */
+  private File locateConfigFile(String resource) {
+    File located = null;
+    File file = new File(resource);
+    if (file.isAbsolute()) {
+      if (file.isFile() && file.canRead()) {
+        located = file;
+      }
+    } else {
+      // try $configDir/$resource
+      File fileUnderConfigDir = new File(config.getResourceLoader().getConfigDir() + resource);
+      if (fileUnderConfigDir.isFile() && fileUnderConfigDir.canRead()) {
+        located = fileUnderConfigDir;
+      } else {
+        // no success with $configDir/$resource - try $CWD/$resource
+        if (file.isFile() && file.canRead()) {
+          located = file;
+        }
+      }
+    }
+    return located;
+  }
+
+  /**
+   * Persist the managed schema to ZooKeeper and rename the non-managed schema 
+   * by appending {@link #UPGRADED_SCHEMA_EXTENSION}.
+   *
+   * Failure to rename the non-managed schema will be logged as a warning,
+   * and no exception will be thrown.
+   */
+  private void zkUgradeToManagedSchema() {
+    ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)config.getResourceLoader();
+    ZkCmdExecutor zkCmdExecutor = new ZkCmdExecutor(30);
+    ZkController zkController = zkLoader.getZkController();
+    final String managedSchemaPath = zkLoader.getCollectionZkPath() + "/" + managedSchemaResourceName;
+    try {
+      // Create the managed schema znode
+      zkCmdExecutor.ensureExists(managedSchemaPath, zkController.getZkClient());
+      // Persist the managed schema
+      StringWriter writer = new StringWriter();
+      schema.persist(writer);
+      zkController.getZkClient().setData(managedSchemaPath, writer.toString().getBytes("UTF-8"), true);
+      log.info("Upgraded to managed schema at " + managedSchemaPath + "");
+    } catch (Exception e) {
+      if (e instanceof InterruptedException) {
+        Thread.currentThread().interrupt(); // Restore the interrupted status
+        log.error("", e);
+        throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
+      } else {
+        final String msg = "Error persisting managed schema resource " + managedSchemaResourceName;
+        log.error(msg, e);
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg, e);
+      }
+    }
+
+    // After successfully persisting the managed schema, rename the non-managed
+    // schema znode by appending UPGRADED_SCHEMA_EXTENSION to its name.
+
+    if (resourceName.equals(managedSchemaResourceName)) {
+      log.info("On upgrading to managed schema, did not rename non-managed schema "
+          + resourceName + " because it's the same as the managed schema's name.");
+    } else {
+      // Rename the non-managed schema znode in ZooKeeper
+      final String nonManagedSchemaPath = zkLoader.getCollectionZkPath() + "/" + resourceName;
+      try {
+        if (zkController.pathExists(nonManagedSchemaPath)) {
+          // First, copy the non-managed schema znode content to the upgraded schema znode
+          byte[] bytes = zkController.getZkClient().getData(nonManagedSchemaPath, null, null, true);
+          final String upgradedSchemaPath = nonManagedSchemaPath + UPGRADED_SCHEMA_EXTENSION;
+          zkCmdExecutor.ensureExists(upgradedSchemaPath, zkController.getZkClient());
+          zkController.getZkClient().setData(upgradedSchemaPath, bytes, true);
+          // Then delete the non-managed schema znode
+          zkController.getZkClient().delete(nonManagedSchemaPath, -1, true);
+
+          // Set the resource name to the managed schema so that the CoreAdminHandler returns a findable filename 
+          schema.setResourceName(managedSchemaResourceName);
+
+          log.info("After upgrading to managed schema in ZooKeeper, renamed the non-managed schema "
+              + nonManagedSchemaPath + " to " + upgradedSchemaPath);
+        } else {
+          log.info("After upgrading to managed schema in ZooKeeper, the non-managed schema "
+              + nonManagedSchemaPath + " no longer exists.");
+        }
+      } catch (Exception e) {
+        if (e instanceof InterruptedException) {
+          Thread.currentThread().interrupt(); // Restore the interrupted status
+          log.warn("", e); // Log as warning and suppress the exception 
+        } else {
+          final String msg = "Error persisting managed schema resource " + managedSchemaResourceName;
+          log.warn(msg, e); // Log as warning and suppress the exception
+        }
+      }
+    }
+  }
+}

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-managed-schema-named-schema.xml.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-managed-schema-named-schema.xml.xml?rev=1463182&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-managed-schema-named-schema.xml.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-managed-schema-named-schema.xml.xml Mon Apr  1 16:16:03 2013
@@ -0,0 +1,30 @@
+<?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.
+-->
+
+<config>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+
+  <!-- managed schema can't be named schema.xml -->
+  <schemaFactory class="ManagedIndexSchemaFactory">
+    <bool name="mutable">false</bool>
+    <str name="managedSchemaResourceName">schema.xml</str>
+  </schemaFactory>
+
+</config>

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-schema-mutable-but-not-managed.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-schema-mutable-but-not-managed.xml?rev=1463182&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-schema-mutable-but-not-managed.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-schema-mutable-but-not-managed.xml Mon Apr  1 16:16:03 2013
@@ -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.
+-->
+
+<config>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+
+  <!-- schema must be managed in order to be mutable -->
+  <!-- managed schema can't be named schema.xml -->
+  <schemaFactory class="ClassicIndexSchemaFactory">
+    <bool name="mutable">false</bool>
+    <str name="managedSchemaResourceName">schema.xml</str>
+  </schemaFactory>
+
+
+</config>

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-unexpected-schema-attribute.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-unexpected-schema-attribute.xml?rev=1463182&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-unexpected-schema-attribute.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-unexpected-schema-attribute.xml Mon Apr  1 16:16:03 2013
@@ -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.
+-->
+
+<config>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+
+  <!-- The presence of bogusParam should trigger failure -->
+  <schemaFactory class="ManagedIndexSchemaFactory">
+    <bool name="mutable">false</bool>
+    <str name="managedSchemaResourceName">managed-schema</str>
+    <str name="bogusParam">bogusValue</str>
+  </schemaFactory>
+
+
+</config>

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml?rev=1463182&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml Mon Apr  1 16:16:03 2013
@@ -0,0 +1,28 @@
+<?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.
+-->
+                                                           
+<config>
+  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+
+  <schemaFactory class="ManagedIndexSchemaFactory">
+    <bool name="mutable">false</bool>
+    <str name="managedSchemaResourceName">managed-schema</str>
+  </schemaFactory>
+
+</config>

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog-managed-schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog-managed-schema.xml?rev=1463182&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog-managed-schema.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog-managed-schema.xml Mon Apr  1 16:16:03 2013
@@ -0,0 +1,123 @@
+<?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.
+-->
+
+<config>
+  <jmx />
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+
+  <schemaFactory class="ManagedIndexSchemaFactory">
+    <bool name="mutable">false</bool>
+    <str name="managedSchemaResourceName">managed-schema</str>
+  </schemaFactory>
+
+  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}">
+    <!-- used to keep RAM reqs down for HdfsDirectoryFactory -->
+    <int name="solr.hdfs.blockcache.blocksperbank">${solr.hdfs.blockcache.blocksperbank:1024}</int>
+  </directoryFactory>
+
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <indexConfig>
+    <lockType>${solr.lock.type:native}</lockType>
+  </indexConfig>
+
+  <!-- an update processor the explicitly excludes distrib to test
+       clean errors when people attempt atomic updates w/o it
+  -->
+  <updateRequestProcessorChain name="nodistrib" >
+    <processor class="solr.NoOpDistributingUpdateProcessorFactory" />
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+  <requestHandler name="standard" class="solr.StandardRequestHandler">
+  </requestHandler>
+
+  <requestHandler name="/get" class="solr.RealTimeGetHandler">
+    <lst name="defaults">
+      <str name="omitHeader">true</str>
+    </lst>
+  </requestHandler>
+
+  <requestHandler name="/replication" class="solr.ReplicationHandler" startup="lazy" />
+
+  <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+    <updateLog>
+      <str name="dir">${solr.ulog.dir:}</str>
+    </updateLog>
+  </updateHandler>
+
+  <updateRequestProcessorChain name="dedupe">
+    <processor class="org.apache.solr.update.processor.SignatureUpdateProcessorFactory">
+      <bool name="enabled">true</bool>
+      <bool name="overwriteDupes">true</bool>
+      <str name="fields">v_t,t_field</str>
+      <str name="signatureClass">org.apache.solr.update.processor.TextProfileSignature</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+  <updateRequestProcessorChain name="stored_sig">
+    <!-- this chain is valid even though the signature field is not
+         indexed, because we are not asking for dups to be overwritten
+      -->
+    <processor class="org.apache.solr.update.processor.SignatureUpdateProcessorFactory">
+      <bool name="enabled">true</bool>
+      <str name="signatureField">non_indexed_signature_sS</str>
+      <bool name="overwriteDupes">false</bool>
+      <str name="fields">v_t,t_field</str>
+      <str name="signatureClass">org.apache.solr.update.processor.TextProfileSignature</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+  <requestHandler name="/admin/" class="org.apache.solr.handler.admin.AdminHandlers" />
+
+  <updateRequestProcessorChain name="distrib-dup-test-chain-explicit">
+    <!-- explicit test using processors before and after distrib -->
+    <processor class="solr.RegexReplaceProcessorFactory">
+      <str name="fieldName">regex_dup_A_s</str>
+      <str name="pattern">x</str>
+      <str name="replacement">x_x</str>
+    </processor>
+    <processor class="solr.DistributedUpdateProcessorFactory" />
+    <processor class="solr.RegexReplaceProcessorFactory">
+      <str name="fieldName">regex_dup_B_s</str>
+      <str name="pattern">x</str>
+      <str name="replacement">x_x</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="distrib-dup-test-chain-implicit">
+    <!-- implicit test w/o distrib declared-->
+    <processor class="solr.RegexReplaceProcessorFactory">
+      <str name="fieldName">regex_dup_A_s</str>
+      <str name="pattern">x</str>
+      <str name="replacement">x_x</str>
+    </processor>
+    <processor class="solr.RegexReplaceProcessorFactory">
+      <str name="fieldName">regex_dup_B_s</str>
+      <str name="pattern">x</str>
+      <str name="replacement">x_x</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+</config>

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java Mon Apr  1 16:16:03 2013
@@ -53,6 +53,7 @@ import org.apache.solr.response.ResultCo
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.response.XMLWriter;
 import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.IndexSchemaFactory;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.DocIterator;
 import org.apache.solr.search.DocList;
@@ -530,7 +531,7 @@ public class BasicFunctionalityTest exte
   @Test
   public void testTermVectorFields() {
     
-    IndexSchema ischema = new IndexSchema(solrConfig, getSchemaFile(), null);
+    IndexSchema ischema = IndexSchemaFactory.buildIndexSchema(getSchemaFile(), solrConfig);
     SchemaField f; // Solr field type
     StorableField luf; // Lucene field
 

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java Mon Apr  1 16:16:03 2013
@@ -34,6 +34,7 @@ import org.apache.lucene.util.automaton.
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.IndexSchemaFactory;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.SolrQueryParser;
 import org.junit.Before;
@@ -56,7 +57,7 @@ public class TestReversedWildcardFilterF
   @Before
   public void setUp() throws Exception {
     super.setUp();
-    schema = new IndexSchema(solrConfig, getSchemaFile(), null);
+    schema = IndexSchemaFactory.buildIndexSchema(getSchemaFile(), solrConfig);
     clearIndex();
     assertU(commit());
   }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/AbstractBadConfigTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/AbstractBadConfigTestBase.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/AbstractBadConfigTestBase.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/AbstractBadConfigTestBase.java Mon Apr  1 16:16:03 2013
@@ -23,10 +23,6 @@ import org.apache.solr.common.SolrExcept
 
 import java.util.regex.Pattern;
 
-import javax.script.ScriptEngineManager;
-
-import org.junit.Assume;
-
 public abstract class AbstractBadConfigTestBase extends SolrTestCaseJ4 {
 
   /**
@@ -35,13 +31,30 @@ public abstract class AbstractBadConfigT
    * files causes an error matching the specified errString ot be thrown.
    */
   protected final void assertConfigs(final String solrconfigFile,
-                                     final String schemaFile, 
+                                     final String schemaFile,
+                                     final String errString)
+      throws Exception {
+    assertConfigs(solrconfigFile, schemaFile, null, errString);
+  }
+
+    /**
+     * Given a solrconfig.xml file name, a schema file name, a solr home directory, 
+     * and an expected errString, asserts that initializing a core with these 
+     * files causes an error matching the specified errString ot be thrown.
+     */
+  protected final void assertConfigs(final String solrconfigFile,
+                                     final String schemaFile,
+                                     final String solrHome,
                                      final String errString) 
     throws Exception {
 
     ignoreException(Pattern.quote(errString));
     try {
-      initCore( solrconfigFile, schemaFile );
+      if (null == solrHome) {
+        initCore( solrconfigFile, schemaFile );
+      } else {
+        initCore( solrconfigFile, schemaFile, solrHome );
+      }
     } catch (Exception e) {
       // short circuit out if we found what we expected
       if (-1 != e.getMessage().indexOf(errString)) return;

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestBadConfig.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestBadConfig.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestBadConfig.java Mon Apr  1 16:16:03 2013
@@ -64,4 +64,18 @@ public class TestBadConfig extends Abstr
                   "DummyMergePolicy");
   }
 
+  public void testSchemaMutableButNotManaged() throws Exception {
+    assertConfigs("bad-solrconfig-schema-mutable-but-not-managed.xml",
+                  "schema-minimal.xml", "Unexpected arg(s): {mutable=false,managedSchemaResourceName=schema.xml}");
+  }
+
+  public void testManagedSchemaCannotBeNamedSchemaDotXml() throws Exception {
+    assertConfigs("bad-solrconfig-managed-schema-named-schema.xml.xml",
+                  "schema-minimal.xml", "managedSchemaResourceName can't be 'schema.xml'");
+  }
+  
+  public void testUnknownSchemaAttribute() throws Exception {
+    assertConfigs("bad-solrconfig-unexpected-schema-attribute.xml", "schema-minimal.xml",
+                  "Unexpected arg(s): {bogusParam=bogusValue}");
+  }
 }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/PrimitiveFieldTypeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/PrimitiveFieldTypeTest.java?rev=1463182&r1=1463181&r2=1463182&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/PrimitiveFieldTypeTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/PrimitiveFieldTypeTest.java Mon Apr  1 16:16:03 2013
@@ -68,7 +68,7 @@ public class PrimitiveFieldTypeTest exte
     // ***********************
     // With schema version 1.4:
     // ***********************
-    schema = new IndexSchema(config, testConfHome + "schema12.xml", null);
+    schema = IndexSchemaFactory.buildIndexSchema(testConfHome + "schema12.xml", config);
     
     dt = new DateField();
     dt.init(schema, initMap);
@@ -130,7 +130,7 @@ public class PrimitiveFieldTypeTest exte
     // ***********************
     // With schema version 1.5
     // ***********************
-    schema = new IndexSchema(config, testConfHome + "schema15.xml", null);
+    schema = IndexSchemaFactory.buildIndexSchema(testConfHome + "schema15.xml", config);
 
     dt = new DateField();
     dt.init(schema, initMap);

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchema.java?rev=1463182&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchema.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchema.java Mon Apr  1 16:16:03 2013
@@ -0,0 +1,133 @@
+package org.apache.solr.schema;
+/*
+ * 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.
+ */
+
+import org.apache.commons.io.IOUtils;
+import org.apache.solr.client.solrj.ResponseParser;
+import org.apache.solr.client.solrj.SolrServer;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.HttpSolrServer;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.NamedList;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Reader;
+import java.util.regex.Pattern;
+
+public class TestCloudManagedSchema extends AbstractFullDistribZkTestBase {
+
+  public TestCloudManagedSchema() {
+    super();
+  }
+
+  @Override
+  protected String getCloudSolrConfig() {
+    return "solrconfig-tlog-managed-schema.xml";
+  }
+      
+  @Override
+  public void doTest() throws Exception {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.STATUS.toString());
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/cores");
+    int which = r.nextInt(clients.size());
+    HttpSolrServer client = (HttpSolrServer)clients.get(which);
+    String previousBaseURL = client.getBaseURL();
+    // Strip /collection1 step from baseURL - requests fail otherwise
+    client.setBaseURL(previousBaseURL.substring(0, previousBaseURL.lastIndexOf("/")));
+    NamedList namedListResponse = client.request(request);
+    client.setBaseURL(previousBaseURL); // Restore baseURL 
+    NamedList status = (NamedList)namedListResponse.get("status");
+    NamedList collectionStatus = (NamedList)status.get("collection1");
+    String collectionSchema = (String)collectionStatus.get(CoreAdminParams.SCHEMA);
+    // Make sure the upgrade to managed schema happened
+    assertEquals("Schema resource name differs from expected name", "managed-schema", collectionSchema);
+    
+    // Make sure "DO NOT EDIT" is in the content of the managed schema
+    String fileContent = getFileContentFromZooKeeper("managed-schema");
+    assertTrue("Managed schema is missing", fileContent.contains("DO NOT EDIT"));
+    
+    // Make sure the original non-managed schema is no longer in ZooKeeper
+    assertFileNotInZooKeeper("schema.xml");
+
+    // Make sure the renamed non-managed schema is present in ZooKeeper
+    fileContent = getFileContentFromZooKeeper("schema.xml.bak");
+    assertTrue("schema file doesn't contain '<schema'", fileContent.contains("<schema"));
+  }
+  
+  private String getFileContentFromZooKeeper(String fileName) throws IOException, SolrServerException {
+    QueryRequest request = new QueryRequest(params("file", fileName));
+    request.setPath("/admin/file");
+    RawResponseParser responseParser = new RawResponseParser();
+    request.setResponseParser(responseParser);
+    int which = r.nextInt(clients.size());
+    // For some reason, /admin/file requests work without stripping the /collection1 step from the URL
+    // (unlike /admin/cores requests - see above)
+    SolrServer client = clients.get(which);
+    client.request(request);
+    return responseParser.getRawFileContent();   
+  }
+  
+  private class RawResponseParser extends ResponseParser {
+    // Stolen from ShowFileRequestHandlerTest
+    private String rawFileContent = null;
+    String getRawFileContent() { return rawFileContent; }
+    @Override
+    public String getWriterType() {
+      return "mock";//unfortunately this gets put onto params wt=mock but it apparently has no effect
+    }
+    @Override
+    public NamedList<Object> processResponse(InputStream body, String encoding) {
+      try {
+        rawFileContent = IOUtils.toString(body, encoding);
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+      return null;
+    }
+    @Override
+    public NamedList<Object> processResponse(Reader reader) {
+      throw new UnsupportedOperationException("TODO unimplemented");//TODO
+    }
+  }
+
+  protected final void assertFileNotInZooKeeper(String fileName) throws Exception {
+    // Stolen from AbstractBadConfigTestBase
+    String errString = "returned non ok status:404, message:Not Found";
+    ignoreException(Pattern.quote(errString));
+    String rawContent = null;
+    try {
+      rawContent = getFileContentFromZooKeeper(fileName);
+    } catch (Exception e) {
+      // short circuit out if we found what we expected
+      if (-1 != e.getMessage().indexOf(errString)) return;
+      // otherwise, rethrow it, possibly completely unrelated
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
+                              "Unexpected error, expected error matching: " + errString, e);
+    } finally {
+      resetExceptionIgnores();
+    }
+    fail("File '" + fileName + "' was unexpectedly found in ZooKeeper.  Content starts with '" 
+        + rawContent.substring(0, 100) + " [...]'");
+  }
+}