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/22 16:26:58 UTC

svn commit: r1470539 [2/5] - in /lucene/dev/trunk/solr: ./ contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/ contrib/dataimporthandler-extras/src/test/org/apache/solr/handler/dataimport/ contrib/dataimporthandler/src/java/org/apac...

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java Mon Apr 22 14:26:55 2013
@@ -43,7 +43,6 @@ import org.apache.solr.common.cloud.Slic
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.Hash;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.core.SolrCore;
@@ -121,11 +120,12 @@ public class RealTimeGetComponent extend
       allIds = lst.toArray(new String[lst.size()]);
     }
 
-    SchemaField idField = req.getSchema().getUniqueKeyField();
+    SolrCore core = req.getCore();
+    SchemaField idField = core.getLatestSchema().getUniqueKeyField();
     FieldType fieldType = idField.getType();
 
     SolrDocumentList docList = new SolrDocumentList();
-    UpdateLog ulog = req.getCore().getUpdateHandler().getUpdateLog();
+    UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
 
     RefCounted<SolrIndexSearcher> searcherHolder = null;
 
@@ -150,7 +150,7 @@ public class RealTimeGetComponent extend
            int oper = (Integer)entry.get(0) & UpdateLog.OPERATION_MASK;
            switch (oper) {
              case UpdateLog.ADD:
-               SolrDocument doc = toSolrDoc((SolrInputDocument)entry.get(entry.size()-1), req.getSchema());
+               SolrDocument doc = toSolrDoc((SolrInputDocument)entry.get(entry.size()-1), core.getLatestSchema());
                if(transformer!=null) {
                  transformer.transform(doc, -1); // unknown docID
                }
@@ -167,7 +167,7 @@ public class RealTimeGetComponent extend
 
        // didn't find it in the update log, so it should be in the newest searcher opened
        if (searcher == null) {
-         searcherHolder = req.getCore().getRealtimeSearcher();
+         searcherHolder = core.getRealtimeSearcher();
          searcher = searcherHolder.get();
        }
 
@@ -176,7 +176,7 @@ public class RealTimeGetComponent extend
        int docid = searcher.getFirstMatch(new Term(idField.getName(), idBytes));
        if (docid < 0) continue;
        StoredDocument luceneDocument = searcher.doc(docid);
-       SolrDocument doc = toSolrDoc(luceneDocument,  req.getSchema());
+       SolrDocument doc = toSolrDoc(luceneDocument,  core.getLatestSchema());
        if( transformer != null ) {
          transformer.transform(doc, docid);
        }
@@ -238,12 +238,12 @@ public class RealTimeGetComponent extend
         }
 
         // SolrCore.verbose("RealTimeGet using searcher ", searcher);
-        SchemaField idField = core.getSchema().getUniqueKeyField();
+        SchemaField idField = core.getLatestSchema().getUniqueKeyField();
 
         int docid = searcher.getFirstMatch(new Term(idField.getName(), idBytes));
         if (docid < 0) return null;
         StoredDocument luceneDocument = searcher.doc(docid);
-        sid = toSolrInputDocument(luceneDocument, core.getSchema());
+        sid = toSolrInputDocument(luceneDocument, core.getLatestSchema());
       }
     } finally {
       if (searcherHolder != null) {

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ResponseLogComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ResponseLogComponent.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ResponseLogComponent.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ResponseLogComponent.java Mon Apr 22 14:26:55 2013
@@ -20,7 +20,6 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.Set;
 
-import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.response.ResultContext;
 import org.apache.solr.schema.IndexSchema;
@@ -67,11 +66,11 @@ public class ResponseLogComponent extend
     SolrParams params = rb.req.getParams();
     if (!params.getBool(COMPONENT_NAME, false)) return;
     
-    IndexSchema schema = rb.req.getSchema();
+    SolrIndexSearcher searcher = rb.req.getSearcher();
+    IndexSchema schema = searcher.getSchema();
     if (schema.getUniqueKeyField() == null) return;
 
     ResultContext rc = (ResultContext) rb.rsp.getValues().get("response");
-    SolrIndexSearcher searcher = rb.req.getSearcher();    
     
     if (rc.docs.hasScores()) {
       processScores(rb, rc.docs, schema, searcher);

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java Mon Apr 22 14:26:55 2013
@@ -19,29 +19,35 @@ package org.apache.solr.handler.componen
 
 import java.io.IOException;
 import java.io.StringReader;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.lucene.search.spell.SuggestMode;
-import org.apache.lucene.search.spell.SuggestWord;
-import org.apache.solr.client.solrj.response.SpellCheckResponse;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.spell.SuggestMode;
+import org.apache.lucene.search.spell.SuggestWord;
+import org.apache.solr.client.solrj.response.SpellCheckResponse;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.SpellingParams;
@@ -53,8 +59,19 @@ import org.apache.solr.core.SolrResource
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.spelling.*;
+import org.apache.solr.spelling.AbstractLuceneSpellChecker;
+import org.apache.solr.spelling.ConjunctionSolrSpellChecker;
+import org.apache.solr.spelling.IndexBasedSpellChecker;
+import org.apache.solr.spelling.QueryConverter;
+import org.apache.solr.spelling.SolrSpellChecker;
+import org.apache.solr.spelling.SpellCheckCollation;
+import org.apache.solr.spelling.SpellCheckCollator;
+import org.apache.solr.spelling.SpellingOptions;
+import org.apache.solr.spelling.SpellingQueryConverter;
+import org.apache.solr.spelling.SpellingResult;
 import org.apache.solr.util.plugin.SolrCoreAware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A SearchComponent implementation which provides support for spell checking
@@ -637,7 +654,7 @@ public class SpellCheckComponent extends
       //there should only be one
       if (queryConverters.size() == 1) {
         queryConverter = queryConverters.values().iterator().next();
-        IndexSchema schema = core.getSchema();
+        IndexSchema schema = core.getLatestSchema();
         String fieldTypeName = (String) initParams.get("queryAnalyzerFieldType");
         FieldType fieldType = schema.getFieldTypes().get(fieldTypeName);
         Analyzer analyzer = fieldType == null ? new WhitespaceAnalyzer(core.getSolrConfig().luceneMatchVersion)

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/StatsComponent.java Mon Apr 22 14:26:55 2013
@@ -34,6 +34,7 @@ import org.apache.solr.common.util.Simpl
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.UnInvertedField;
 import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.DocIterator;
 import org.apache.solr.search.DocSet;
@@ -204,12 +205,13 @@ class SimpleStats {
     String[] statsFs = params.getParams(StatsParams.STATS_FIELD);
     boolean isShard = params.getBool(ShardParams.IS_SHARD, false);
     if (null != statsFs) {
+      final IndexSchema schema = searcher.getSchema();
       for (String f : statsFs) {
         String[] facets = params.getFieldParams(f, StatsParams.STATS_FACET);
         if (facets == null) {
           facets = new String[0]; // make sure it is something...
         }
-        SchemaField sf = searcher.getSchema().getField(f);
+        SchemaField sf = schema.getField(f);
         FieldType ft = sf.getType();
         NamedList<?> stv;
 
@@ -231,13 +233,14 @@ class SimpleStats {
   }
 
   public NamedList<?> getFieldCacheStats(String fieldName, String[] facet) throws IOException {
-    final SchemaField sf = searcher.getSchema().getField(fieldName);
+    IndexSchema schema = searcher.getSchema();
+    final SchemaField sf = schema.getField(fieldName);
 
     final StatsValues allstats = StatsValuesFactory.createStatsValues(sf);
 
     List<FieldFacetStats> facetStats = new ArrayList<FieldFacetStats>();
     for( String facetField : facet ) {
-      SchemaField fsf = searcher.getSchema().getField(facetField);
+      SchemaField fsf = schema.getField(facetField);
 
       if ( fsf.multiValued()) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java Mon Apr 22 14:26:55 2013
@@ -71,7 +71,7 @@ public interface SolrQueryRequest {
   /** The solr core (coordinator, etc) associated with this request */
   public SolrCore getCore();
 
-  /** The index schema associated with this request */
+  /** The schema snapshot from core.getLatestSchema() at request creation. */
   public IndexSchema getSchema();
 
   /**

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java Mon Apr 22 14:26:55 2013
@@ -42,6 +42,7 @@ import java.util.HashMap;
  */
 public abstract class SolrQueryRequestBase implements SolrQueryRequest {
   protected final SolrCore core;
+  protected final IndexSchema schema;
   protected final SolrParams origParams;
   protected SolrParams params;
   protected Map<Object,Object> context;
@@ -49,6 +50,7 @@ public abstract class SolrQueryRequestBa
 
   public SolrQueryRequestBase(SolrCore core, SolrParams params) {
     this.core = core;
+    this.schema = null == core ? null : core.getLatestSchema();
     this.params = this.origParams = params;
   }
 
@@ -85,7 +87,7 @@ public abstract class SolrQueryRequestBa
   protected RefCounted<SolrIndexSearcher> searcherHolder;
   @Override
   public SolrIndexSearcher getSearcher() {
-    if(core == null) return null;//a request for a core admin will no have a core
+    if(core == null) return null;//a request for a core admin will not have a core
     // should this reach out and get a searcher from the core singleton, or
     // should the core populate one in a factory method to create requests?
     // or there could be a setSearcher() method that Solr calls
@@ -107,7 +109,7 @@ public abstract class SolrQueryRequestBa
   @Override
   public IndexSchema getSchema() {
     //a request for a core admin will no have a core
-    return core == null? null: core.getSchema();
+    return schema;
   }
 
   /**

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/POSTable.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/POSTable.java?rev=1470539&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/POSTable.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/POSTable.java Mon Apr 22 14:26:55 2013
@@ -0,0 +1,27 @@
+package org.apache.solr.rest;
+
+/*
+ * 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.restlet.representation.Representation;
+import org.restlet.resource.Post;
+
+/** Marker interface for resource classes that handle POST requests. */
+public interface POSTable {
+  @Post
+  public Representation post(Representation representation);
+}

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/PUTable.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/PUTable.java?rev=1470539&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/PUTable.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/PUTable.java Mon Apr 22 14:26:55 2013
@@ -0,0 +1,27 @@
+package org.apache.solr.rest;
+
+/*
+ * 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.restlet.representation.Representation;
+import org.restlet.resource.Put;
+
+/** Marker interface for resource classes that handle PUT requests. */
+public interface PUTable {
+  @Put
+  public Representation put(Representation entity);
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/schema/BaseSchemaResource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/schema/BaseSchemaResource.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/schema/BaseSchemaResource.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/schema/BaseSchemaResource.java Mon Apr 22 14:26:55 2013
@@ -107,7 +107,7 @@ abstract class BaseSchemaResource extend
           } else {
             solrResponse = solrRequestInfo.getRsp();
             solrCore = solrRequest.getCore();
-            schema = solrCore.getSchema();
+            schema = solrRequest.getSchema();
             String responseWriterName = solrRequest.getParams().get(CommonParams.WT);
             if (null == responseWriterName) {
               responseWriterName = "json"; // Default to json writer
@@ -124,8 +124,12 @@ abstract class BaseSchemaResource extend
             responseWriter = solrCore.getQueryResponseWriter(responseWriterName);
             contentType = responseWriter.getContentType(solrRequest, solrResponse);
             final String path = getRequest().getRootRef().getPath();
-            final String firstPathElement = path.substring(0, path.indexOf("/", 1));
-            solrRequest.getContext().put("webapp", firstPathElement); // Context path
+            if ( ! "/schema".equals(path)) { 
+              // don't set webapp property on the request when context and core/collection are excluded 
+              final int cutoffPoint = path.indexOf("/", 1);
+              final String firstPathElement = -1 == cutoffPoint ? path : path.substring(0, cutoffPoint);
+              solrRequest.getContext().put("webapp", firstPathElement); // Context path
+            }
             SolrCore.preDecorateResponse(solrRequest, solrResponse);
           }
         }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/schema/FieldCollectionResource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/schema/FieldCollectionResource.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/schema/FieldCollectionResource.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/schema/FieldCollectionResource.java Mon Apr 22 14:26:55 2013
@@ -22,8 +22,11 @@ import org.apache.solr.common.SolrExcept
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.rest.GETable;
+import org.apache.solr.rest.POSTable;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
+import org.noggit.ObjectBuilder;
+import org.restlet.data.MediaType;
 import org.restlet.representation.Representation;
 import org.restlet.resource.ResourceException;
 import org.slf4j.Logger;
@@ -31,6 +34,7 @@ import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
@@ -51,7 +55,7 @@ import java.util.TreeSet;
  *   </li>
  * </ul>
  */
-public class FieldCollectionResource extends BaseFieldResource implements GETable {
+public class FieldCollectionResource extends BaseFieldResource implements GETable,POSTable {
   private static final Logger log = LoggerFactory.getLogger(FieldCollectionResource.class);
   private boolean includeDynamic;
   
@@ -104,4 +108,58 @@ public class FieldCollectionResource ext
 
     return new SolrOutputRepresentation();
   }
+  
+  @Override
+  public Representation post(Representation entity) {
+    try {
+      if ( ! getSchema().isMutable()) {
+        final String message = "This IndexSchema is not mutable.";
+        throw new SolrException(ErrorCode.BAD_REQUEST, message);
+      } else {
+        if (null == entity.getMediaType()) {
+          entity.setMediaType(MediaType.APPLICATION_JSON);
+        }
+        if ( ! entity.getMediaType().equals(MediaType.APPLICATION_JSON, true)) {
+          String message = "Only media type " + MediaType.APPLICATION_JSON.toString() + " is accepted."
+              + "  Request has media type " + entity.getMediaType().toString() + ".";
+          log.error(message);
+          throw new SolrException(ErrorCode.BAD_REQUEST, message);
+        } else {
+          Object object = ObjectBuilder.fromJSON(entity.getText());
+          if ( ! (object instanceof List)) {
+            String message = "Invalid JSON type " + object.getClass().getName() + ", expected List of the form"
+                + " (ignore the backslashes): [{\"name\":\"foo\",\"type\":\"text_general\", ...}, {...}, ...]";
+            log.error(message);
+            throw new SolrException(ErrorCode.BAD_REQUEST, message);
+          } else {
+            List<Map<String,Object>> list = (List<Map<String,Object>>)object;
+            List<SchemaField> newFields = new ArrayList<SchemaField>();
+            IndexSchema oldSchema = getSchema();
+            for (Map<String,Object> map : list) {
+              String fieldName = (String)map.remove(IndexSchema.NAME);
+              if (null == fieldName) {
+                String message = "Missing '" + IndexSchema.NAME + "' mapping.";
+                log.error(message);
+                throw new SolrException(ErrorCode.BAD_REQUEST, message);
+              }
+              String fieldType = (String)map.remove(IndexSchema.TYPE);
+              if (null == fieldType) {
+                String message = "Missing '" + IndexSchema.TYPE + "' mapping.";
+                log.error(message);
+                throw new SolrException(ErrorCode.BAD_REQUEST, message);
+              }
+              newFields.add(oldSchema.newField(fieldName, fieldType, map));
+            }
+            IndexSchema newSchema = oldSchema.addFields(newFields);
+            getSolrCore().setLatestSchema(newSchema);
+          }
+        }
+      }
+    } catch (Exception e) {
+      getSolrResponse().setException(e);
+    }
+    handlePostExecution(log);
+
+    return new SolrOutputRepresentation();
+  }
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/schema/FieldResource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/schema/FieldResource.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/schema/FieldResource.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/schema/FieldResource.java Mon Apr 22 14:26:55 2013
@@ -19,14 +19,19 @@ package org.apache.solr.rest.schema;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.rest.GETable;
+import org.apache.solr.rest.PUTable;
 import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.ManagedIndexSchema;
 import org.apache.solr.schema.SchemaField;
+import org.noggit.ObjectBuilder;
+import org.restlet.data.MediaType;
 import org.restlet.representation.Representation;
 import org.restlet.resource.ResourceException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.UnsupportedEncodingException;
+import java.util.Map;
 
 /**
  * This class responds to requests at /solr/(corename)/schema/fields/(fieldname)
@@ -36,8 +41,10 @@ import java.io.UnsupportedEncodingExcept
  * The "includeDynamic" query parameter, if specified, will cause the
  * dynamic field matching the given fieldname to be returned if fieldname
  * is not explicitly declared in the schema.
+ * <p/>
+ * The PUT method accepts field addition requests in JSON format.
  */
-public class FieldResource extends BaseFieldResource implements GETable {
+public class FieldResource extends BaseFieldResource implements GETable,PUTable {
   private static final Logger log = LoggerFactory.getLogger(FieldResource.class);
   
   private boolean includeDynamic;
@@ -88,4 +95,66 @@ public class FieldResource extends BaseF
 
     return new SolrOutputRepresentation();
   }
+
+  /**
+   * Accepts JSON add field request, to URL  
+   */
+  @Override
+  public Representation put(Representation entity) {
+    try {
+      if ( ! getSchema().isMutable()) {
+        final String message = "This IndexSchema is not mutable.";
+        throw new SolrException(ErrorCode.BAD_REQUEST, message);
+      } else {
+        if (null == entity.getMediaType()) {
+          entity.setMediaType(MediaType.APPLICATION_JSON);
+        }
+        if ( ! entity.getMediaType().equals(MediaType.APPLICATION_JSON, true)) {
+          String message = "Only media type " + MediaType.APPLICATION_JSON.toString() + " is accepted."
+                         + "  Request has media type " + entity.getMediaType().toString() + ".";
+          log.error(message);
+          throw new SolrException(ErrorCode.BAD_REQUEST, message);
+        } else {
+          Object object = ObjectBuilder.fromJSON(entity.getText());
+          if ( ! (object instanceof Map)) {
+            String message = "Invalid JSON type " + object.getClass().getName() + ", expected Map of the form"
+                           + " (ignore the backslashes): {\"type\":\"text_general\", ...}, either with or"
+                           + " without a \"name\" mapping.  If the \"name\" is specified, it must match the"
+                           + " name given in the request URL: /schema/fields/(name)";
+            log.error(message);
+            throw new SolrException(ErrorCode.BAD_REQUEST, message);
+          } else {
+            Map<String,Object> map = (Map<String,Object>)object;
+            if (1 == map.size() && map.containsKey(IndexSchema.FIELD)) {
+              map = (Map<String,Object>)map.get(IndexSchema.FIELD);
+            }
+            String bodyFieldName;
+            if (null != (bodyFieldName = (String)map.remove(IndexSchema.NAME)) && ! fieldName.equals(bodyFieldName)) {
+              String message = "Field name in the request body '" + bodyFieldName 
+                             + "' doesn't match field name in the request URL '" + fieldName + "'";
+              log.error(message);
+              throw new SolrException(ErrorCode.BAD_REQUEST, message);
+            } else {
+              String fieldType;
+              if (null == (fieldType = (String)map.remove(IndexSchema.TYPE))) {
+                String message = "Missing '" + IndexSchema.TYPE + "' mapping.";
+                log.error(message);
+                throw new SolrException(ErrorCode.BAD_REQUEST, message);
+              } else {
+                ManagedIndexSchema oldSchema = (ManagedIndexSchema)getSchema();
+                SchemaField newField = oldSchema.newField(fieldName, fieldType, map);
+                ManagedIndexSchema newSchema = oldSchema.addField(newField);
+                getSolrCore().setLatestSchema(newSchema);
+              }
+            }
+          }
+        }
+      }
+    } catch (Exception e) {
+      getSolrResponse().setException(e);
+    }
+    handlePostExecution(log);
+
+    return new SolrOutputRepresentation();
+  }
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/AbstractSubTypeFieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/AbstractSubTypeFieldType.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/AbstractSubTypeFieldType.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/AbstractSubTypeFieldType.java Mon Apr 22 14:26:55 2013
@@ -41,6 +41,8 @@ public abstract class AbstractSubTypeFie
   protected String suffix;
   protected int dynFieldProps;
   protected String[] suffixes;
+  protected String subFieldType = null;
+  protected String subSuffix = null;
   protected IndexSchema schema;   // needed for retrieving SchemaFields
 
   public FieldType getSubType() {
@@ -53,11 +55,11 @@ public abstract class AbstractSubTypeFie
     this.schema = schema;
     //it's not a first class citizen for the IndexSchema
     SolrParams p = new MapSolrParams(args);
-    String subFT = p.get(SUB_FIELD_TYPE);
-    String subSuffix = p.get(SUB_FIELD_SUFFIX);
-    if (subFT != null) {
+    subFieldType = p.get(SUB_FIELD_TYPE);
+    subSuffix = p.get(SUB_FIELD_SUFFIX);
+    if (subFieldType != null) {
       args.remove(SUB_FIELD_TYPE);
-      subType = schema.getFieldTypeByName(subFT.trim());
+      subType = schema.getFieldTypeByName(subFieldType.trim());
       suffix = POLY_FIELD_SEPARATOR + subType.typeName;
     } else if (subSuffix != null) {
       args.remove(SUB_FIELD_SUFFIX);
@@ -67,7 +69,6 @@ public abstract class AbstractSubTypeFie
               + " must specify the " +
               SUB_FIELD_TYPE + " attribute or the " + SUB_FIELD_SUFFIX + " attribute.");
     }
-
   }
 
   /**
@@ -94,8 +95,17 @@ public abstract class AbstractSubTypeFie
     return proto;
   }
 
+  /**
+   * Registers the polyfield dynamic prototype for this field type: : "*___(field type name)" 
+   * 
+   * {@inheritDoc}
+   *  
+   * @param schema {@inheritDoc}
+   *
+   */
   @Override
   public void inform(IndexSchema schema) {
+    this.schema = schema;
     //Can't do this until here b/c the Dynamic Fields are not initialized until here.
     if (subType != null) {
       SchemaField proto = registerPolyFieldDynamicPrototype(schema, subType);
@@ -118,7 +128,7 @@ public abstract class AbstractSubTypeFie
     }
   }
 
-  protected SchemaField subField(SchemaField base, int i) {
+  protected SchemaField subField(SchemaField base, int i, IndexSchema schema) {
     return schema.getField(base.getName() + suffixes[i]);
   }
 }

Modified: 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=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ClassicIndexSchemaFactory.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ClassicIndexSchemaFactory.java Mon Apr 22 14:26:55 2013
@@ -19,14 +19,8 @@ package org.apache.solr.schema;
 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);
@@ -40,26 +34,4 @@ public class ClassicIndexSchemaFactory e
       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/CurrencyField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CurrencyField.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CurrencyField.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CurrencyField.java Mon Apr 22 14:26:55 2013
@@ -209,12 +209,15 @@ public class CurrencyField extends Field
   }
 
   /**
-   * When index schema is informed, add dynamic fields.
-   *
-   * @param indexSchema The index schema.
+   * When index schema is informed, add dynamic fields "*____currency" and "*____amount_raw". 
+   * 
+   * {@inheritDoc}
+   * 
+   * @param schema {@inheritDoc}
    */
   @Override
-  public void inform(IndexSchema indexSchema) {
+  public void inform(IndexSchema schema) {
+    this.schema = schema;
     createDynamicCurrencyField(FIELD_SUFFIX_CURRENCY,   fieldTypeCurrency);
     createDynamicCurrencyField(FIELD_SUFFIX_AMOUNT_RAW, fieldTypeAmountRaw);
   }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ExternalFileField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ExternalFileField.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ExternalFileField.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ExternalFileField.java Mon Apr 22 14:26:55 2013
@@ -55,7 +55,7 @@ import java.util.Map;
  *
  * @see ExternalFileFieldReloader
  */
-public class ExternalFileField extends FieldType {
+public class ExternalFileField extends FieldType implements SchemaAware {
   private FieldType ftype;
   private String keyFieldName;
   private IndexSchema schema;
@@ -127,4 +127,8 @@ public class ExternalFileField extends F
         schema.getField(keyFieldName);
   }
 
+  @Override
+  public void inform(IndexSchema schema) {
+    this.schema = schema;
+  }
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ExternalFileFieldReloader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ExternalFileFieldReloader.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ExternalFileFieldReloader.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ExternalFileFieldReloader.java Mon Apr 22 14:26:55 2013
@@ -50,7 +50,6 @@ import java.util.List;
  */
 public class ExternalFileFieldReloader extends AbstractSolrEventListener {
 
-  private IndexSchema schema;
   private String datadir;
   private List<FileFloatSource> fieldSources = new ArrayList<FileFloatSource>();
 
@@ -58,29 +57,36 @@ public class ExternalFileFieldReloader e
 
   public ExternalFileFieldReloader(SolrCore core) {
     super(core);
-    schema = core.getSchema();
     datadir = core.getDataDir();
   }
 
   @Override
   public void init(NamedList args) {
-    for (SchemaField field : schema.getFields().values()) {
-      FieldType type = field.getType();
-      if (type instanceof ExternalFileField) {
-        ExternalFileField eff = (ExternalFileField) type;
-        fieldSources.add(eff.getFileFloatSource(field, datadir));
-        log.info("Adding ExternalFileFieldReloader listener for field {}", field.getName());
-      }
-    }
+    cacheFieldSources(getCore().getLatestSchema());
   }
 
   @Override
   public void newSearcher(SolrIndexSearcher newSearcher, SolrIndexSearcher currentSearcher) {
     // We need to reload the caches for the new searcher
+    if (null == currentSearcher || newSearcher.getSchema() != currentSearcher.getSchema()) {
+      cacheFieldSources(newSearcher.getSchema());
+    }
     IndexReader reader = newSearcher.getIndexReader();
     for (FileFloatSource fieldSource : fieldSources) {
       fieldSource.refreshCache(reader);
     }
   }
-}
 
+  /** Caches FileFloatSource's from all ExternalFileField instances in the schema */
+  public void cacheFieldSources(IndexSchema schema) {
+    fieldSources.clear();
+    for (SchemaField field : schema.getFields().values()) {
+      FieldType type = field.getType();
+      if (type instanceof ExternalFileField) {
+        ExternalFileField eff = (ExternalFileField)type;
+        fieldSources.add(eff.getFileFloatSource(field, datadir));
+        log.info("Adding ExternalFileFieldReloader listener for field {}", field.getName());
+      }
+    }
+  }
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/FieldProperties.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/FieldProperties.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/FieldProperties.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/FieldProperties.java Mon Apr 22 14:26:55 2013
@@ -109,16 +109,16 @@ public abstract class FieldProperties {
     return (bitfield & props) == 0;
   }
 
-  static int parseProperties(Map<String,String> properties, boolean which, boolean failOnError) {
+  static int parseProperties(Map<String,?> properties, boolean which, boolean failOnError) {
     int props = 0;
-    for (Map.Entry<String, String> entry : properties.entrySet()) {
-      String val = entry.getValue();
+    for (Map.Entry<String,?> entry : properties.entrySet()) {
+      Object val = entry.getValue();
       if(val == null) continue;
-      if (Boolean.parseBoolean(val) == which) {
+      boolean boolVal = val instanceof Boolean ? (Boolean)val : Boolean.parseBoolean(val.toString());
+      if (boolVal == which) {
         props |= propertyNameToInt(entry.getKey(), failOnError);
       }
     }
     return props;
   }
-
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/FieldType.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/FieldType.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/FieldType.java Mon Apr 22 14:26:55 2013
@@ -649,6 +649,7 @@ public abstract class FieldType extends 
    * Sub-classes should override this method to provide their own range query implementation. They should strive to
    * handle nulls in part1 and/or part2 as well as unequal minInclusive and maxInclusive parameters gracefully.
    *
+   * @param parser       the {@link org.apache.solr.search.QParser} calling the method
    * @param field        the schema field
    * @param part1        the lower boundary of the range, nulls are allowed.
    * @param part2        the upper boundary of the range, nulls are allowed

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=1470539&r1=1470538&r2=1470539&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 22 14:26:55 2013
@@ -17,7 +17,6 @@
 
 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;
@@ -25,12 +24,8 @@ 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;
@@ -38,14 +33,11 @@ import org.apache.solr.request.LocalSolr
 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;
@@ -57,13 +49,9 @@ import org.xml.sax.InputSource;
 
 import javax.xml.xpath.XPath;
 import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpressionException;
 
-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;
@@ -107,8 +95,10 @@ public class IndexSchema {
   public static final String REQUIRED = "required";
   public static final String SCHEMA = "schema";
   public static final String SIMILARITY = "similarity";
+  public static final String SLASH = "/";
   public static final String SOLR_QUERY_PARSER = "solrQueryParser";
   public static final String SOURCE = "source";
+  public static final String TYPE = "type";
   public static final String TYPES = "types";
   public static final String UNIQUE_KEY = "uniqueKey";
   public static final String VERSION = "version";
@@ -116,49 +106,47 @@ public class IndexSchema {
   private static final String AT = "@";
   private static final String DESTINATION_DYNAMIC_BASE = "destDynamicBase";
   private static final String MAX_CHARS = "maxChars";
-  private static final String SLASH = "/";
   private static final String SOURCE_DYNAMIC_BASE = "sourceDynamicBase";
   private static final String SOURCE_EXPLICIT_FIELDS = "sourceExplicitFields";
   private static final String TEXT_FUNCTION = "text()";
-  private static final String TYPE = "type";
   private static final String XPATH_OR = " | ";
 
   final static Logger log = LoggerFactory.getLogger(IndexSchema.class);
-  private final SolrConfig solrConfig;
-  private String resourceName;
-  private String name;
-  private float version;
-  private final SolrResourceLoader loader;
-
-  private final HashMap<String, SchemaField> fields = new HashMap<String,SchemaField>();
-
-
-  private final HashMap<String, FieldType> fieldTypes = new HashMap<String,FieldType>();
-
-  private final List<SchemaField> fieldsWithDefaultValue = new ArrayList<SchemaField>();
-  private final Collection<SchemaField> requiredFields = new HashSet<SchemaField>();
-  private DynamicField[] dynamicFields;
+  protected final SolrConfig solrConfig;
+  protected String resourceName;
+  protected String name;
+  protected float version;
+  protected final SolrResourceLoader loader;
+
+  protected Map<String,SchemaField> fields = new HashMap<String,SchemaField>();
+  protected Map<String,FieldType> fieldTypes = new HashMap<String,FieldType>();
+
+  protected List<SchemaField> fieldsWithDefaultValue = new ArrayList<SchemaField>();
+  protected Collection<SchemaField> requiredFields = new HashSet<SchemaField>();
+  protected volatile DynamicField[] dynamicFields;
   public DynamicField[] getDynamicFields() { return dynamicFields; }
 
   private Analyzer analyzer;
   private Analyzer queryAnalyzer;
 
-  private String defaultSearchFieldName=null;
-  private String queryParserDefaultOperator = "OR";
-  private boolean isExplicitQueryParserDefaultOperator = false;
+  protected List<SchemaAware> schemaAware = new ArrayList<SchemaAware>();
+
+  protected String defaultSearchFieldName=null;
+  protected String queryParserDefaultOperator = "OR";
+  protected boolean isExplicitQueryParserDefaultOperator = false;
 
 
-  private final Map<String, List<CopyField>> copyFieldsMap = new HashMap<String, List<CopyField>>();
+  protected Map<String, List<CopyField>> copyFieldsMap = new HashMap<String, List<CopyField>>();
   public Map<String,List<CopyField>> getCopyFieldsMap() { return Collections.unmodifiableMap(copyFieldsMap); }
   
-  private DynamicCopy[] dynamicCopyFields;
+  protected DynamicCopy[] dynamicCopyFields;
   public DynamicCopy[] getDynamicCopyFields() { return dynamicCopyFields; }
 
   /**
    * keys are all fields copied to, count is num of copyField
    * directives that target them.
    */
-  private Map<SchemaField, Integer> copyFieldTargetCounts = new HashMap<SchemaField, Integer>();
+  protected Map<SchemaField, Integer> copyFieldTargetCounts = new HashMap<SchemaField, Integer>();
 
     /**
    * Constructs a schema using the specified resource name and stream.
@@ -259,15 +247,20 @@ public class IndexSchema {
    */
   public Collection<SchemaField> getRequiredFields() { return requiredFields; }
 
-  private Similarity similarity;
+  protected Similarity similarity;
 
   /**
    * Returns the Similarity used for this index
    */
-  public Similarity getSimilarity() { return similarity; }
+  public Similarity getSimilarity() {
+    if (null == similarity) {
+      similarity = similarityFactory.getSimilarity();
+    }
+    return similarity; 
+  }
 
-  private SimilarityFactory similarityFactory;
-  private boolean isExplicitSimilarity = false;
+  protected SimilarityFactory similarityFactory;
+  protected boolean isExplicitSimilarity = false;
 
 
   /** Returns the SimilarityFactory that constructed the Similarity for this index */
@@ -310,7 +303,7 @@ public class IndexSchema {
     return queryParserDefaultOperator;
   }
 
-  private SchemaField uniqueKeyField;
+  protected SchemaField uniqueKeyField;
 
   /**
    * Unique Key field specified in the schema file
@@ -428,7 +421,7 @@ public class IndexSchema {
     }
   }
 
-  private void readSchema(InputSource is) {
+  protected void readSchema(InputSource is) {
     log.info("Reading Solr Schema from " + resourceName);
 
     try {
@@ -437,7 +430,6 @@ public class IndexSchema {
       Config schemaConf = new Config(loader, SCHEMA, is, SLASH+SCHEMA+SLASH);
       Document document = schemaConf.getDocument();
       final XPath xpath = schemaConf.getXPath();
-      final List<SchemaAware> schemaAware = new ArrayList<SchemaAware>();
       String expression = stepsToPath(SCHEMA, AT + NAME);
       Node nd = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
       if (nd==null) {
@@ -454,94 +446,16 @@ public class IndexSchema {
 
       // load the Field Types
 
-      final FieldTypePluginLoader typeLoader 
-        = new FieldTypePluginLoader(this, fieldTypes, schemaAware);
+      final FieldTypePluginLoader typeLoader = new FieldTypePluginLoader(this, fieldTypes, schemaAware);
 
       //               /schema/types/fieldtype | /schema/types/fieldType 
       expression =     stepsToPath(SCHEMA, TYPES, FIELD_TYPE.toLowerCase(Locale.ROOT)) // backcompat(?) 
           + XPATH_OR + stepsToPath(SCHEMA, TYPES, FIELD_TYPE);
       NodeList nodes = (NodeList) xpath.evaluate(expression, document, XPathConstants.NODESET);
-      typeLoader.load( loader, nodes );
-
-      // load the Fields
-
-      // Hang on to the fields that say if they are required -- this lets us set a reasonable default for the unique key
-      Map<String,Boolean> explicitRequiredProp = new HashMap<String, Boolean>();
-      ArrayList<DynamicField> dFields = new ArrayList<DynamicField>();
-
-      //               /schema/fields/field | /schema/fields/dynamicField
-      expression =     stepsToPath(SCHEMA, FIELDS, FIELD)
-          + XPATH_OR + stepsToPath(SCHEMA, FIELDS, DYNAMIC_FIELD);
-      nodes = (NodeList) xpath.evaluate(expression, document, XPathConstants.NODESET);
-
-      for (int i=0; i<nodes.getLength(); i++) {
-        Node node = nodes.item(i);
-
-        NamedNodeMap attrs = node.getAttributes();
-
-        String name = DOMUtil.getAttr(attrs, NAME, "field definition");
-        log.trace("reading field def "+name);
-        String type = DOMUtil.getAttr(attrs, TYPE, "field " + name);
-
-        FieldType ft = fieldTypes.get(type);
-        if (ft==null) {
-          throw new SolrException
-              (ErrorCode.BAD_REQUEST, "Unknown " + FIELD_TYPE + " '" + type + "' specified on field " + name);
-        }
-
-        Map<String,String> args = DOMUtil.toMapExcept(attrs, NAME, TYPE);
-        if (null != args.get(REQUIRED)) {
-          explicitRequiredProp.put(name, Boolean.valueOf(args.get(REQUIRED)));
-        }
+      typeLoader.load(loader, nodes);
 
-        SchemaField f = SchemaField.create(name,ft,args);
-
-        if (node.getNodeName().equals(FIELD)) {
-          SchemaField old = fields.put(f.getName(),f);
-          if( old != null ) {
-            String msg = "[schema.xml] Duplicate field definition for '"
-              + f.getName() + "' [[["+old.toString()+"]]] and [[["+f.toString()+"]]]";
-            throw new SolrException(ErrorCode.SERVER_ERROR, msg );
-          }
-          log.debug("field defined: " + f);
-          if( f.getDefaultValue() != null ) {
-            log.debug(name+" contains default value: " + f.getDefaultValue());
-            fieldsWithDefaultValue.add( f );
-          }
-          if (f.isRequired()) {
-            log.debug(name+" is required in this schema");
-            requiredFields.add(f);
-          }
-        } else if (node.getNodeName().equals(DYNAMIC_FIELD)) {
-          if (isValidFieldGlob(name)) {
-            // make sure nothing else has the same path
-            addDynamicField(dFields, f);
-          } else {
-            String msg = "Dynamic field name '" + name 
-                + "' should have either a leading or a trailing asterisk, and no others.";
-            throw new SolrException(ErrorCode.SERVER_ERROR, msg);
-          }
-        } else {
-          // we should never get here
-          throw new RuntimeException("Unknown field type");
-        }
-      }
-      
-      //fields with default values are by definition required
-      //add them to required fields, and we only have to loop once
-      // in DocumentBuilder.getDoc()
-      requiredFields.addAll(getFieldsWithDefaultValue());
-
-
-      // OK, now sort the dynamic fields largest to smallest size so we don't get
-      // any false matches.  We want to act like a compiler tool and try and match
-      // the largest string possible.
-      Collections.sort(dFields);
-
-      log.trace("Dynamic Field Ordering:" + dFields);
-
-      // stuff it in a normal array for faster access
-      dynamicFields = dFields.toArray(new DynamicField[dFields.size()]);
+      // load the fields
+      Map<String,Boolean> explicitRequiredProp = loadFields(document, xpath);
 
       expression = stepsToPath(SCHEMA, SIMILARITY); //   /schema/similarity
       Node node = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
@@ -551,11 +465,9 @@ public class IndexSchema {
       } else {
         isExplicitSimilarity = true;
       }
-      if (similarityFactory instanceof SchemaAware) {
-        ((SchemaAware)similarityFactory).inform(this);
-      } else {
-        // if the sim factory isn't schema aware, then we are responsible for
-        // erroring if a field type is trying to specify a sim.
+      if ( ! (similarityFactory instanceof SolrCoreAware)) {
+        // if the sim factory isn't SolrCoreAware (and hence schema aware), 
+        // then we are responsible for erroring if a field type is trying to specify a sim.
         for (FieldType ft : fieldTypes.values()) {
           if (null != ft.getSimilarity()) {
             String msg = "FieldType '" + ft.getTypeName()
@@ -566,7 +478,6 @@ public class IndexSchema {
           }
         }
       }
-      similarity = similarityFactory.getSimilarity();
 
       //                      /schema/defaultSearchField/@text()
       expression = stepsToPath(SCHEMA, DEFAULT_SEARCH_FIELD, TEXT_FUNCTION);
@@ -691,6 +602,93 @@ public class IndexSchema {
     refreshAnalyzers();
   }
 
+  /** 
+   * Loads fields and dynamic fields.
+   * 
+   * @return a map from field name to explicit required value  
+   */ 
+  protected synchronized Map<String,Boolean> loadFields(Document document, XPath xpath) throws XPathExpressionException {
+    // Hang on to the fields that say if they are required -- this lets us set a reasonable default for the unique key
+    Map<String,Boolean> explicitRequiredProp = new HashMap<String,Boolean>();
+    
+    ArrayList<DynamicField> dFields = new ArrayList<DynamicField>();
+
+    //                  /schema/fields/field | /schema/fields/dynamicField
+    String expression = stepsToPath(SCHEMA, FIELDS, FIELD)
+           + XPATH_OR + stepsToPath(SCHEMA, FIELDS, DYNAMIC_FIELD);
+    NodeList nodes = (NodeList)xpath.evaluate(expression, document, XPathConstants.NODESET);
+
+    for (int i=0; i<nodes.getLength(); i++) {
+      Node node = nodes.item(i);
+
+      NamedNodeMap attrs = node.getAttributes();
+
+      String name = DOMUtil.getAttr(attrs, NAME, "field definition");
+      log.trace("reading field def "+name);
+      String type = DOMUtil.getAttr(attrs, TYPE, "field " + name);
+
+      FieldType ft = fieldTypes.get(type);
+      if (ft==null) {
+        throw new SolrException
+            (ErrorCode.BAD_REQUEST, "Unknown " + FIELD_TYPE + " '" + type + "' specified on field " + name);
+      }
+
+      Map<String,String> args = DOMUtil.toMapExcept(attrs, NAME, TYPE);
+      if (null != args.get(REQUIRED)) {
+        explicitRequiredProp.put(name, Boolean.valueOf(args.get(REQUIRED)));
+      }
+
+      SchemaField f = SchemaField.create(name,ft,args);
+
+      if (node.getNodeName().equals(FIELD)) {
+        SchemaField old = fields.put(f.getName(),f);
+        if( old != null ) {
+          String msg = "[schema.xml] Duplicate field definition for '"
+            + f.getName() + "' [[["+old.toString()+"]]] and [[["+f.toString()+"]]]";
+          throw new SolrException(ErrorCode.SERVER_ERROR, msg );
+        }
+        log.debug("field defined: " + f);
+        if( f.getDefaultValue() != null ) {
+          log.debug(name+" contains default value: " + f.getDefaultValue());
+          fieldsWithDefaultValue.add( f );
+        }
+        if (f.isRequired()) {
+          log.debug(name+" is required in this schema");
+          requiredFields.add(f);
+        }
+      } else if (node.getNodeName().equals(DYNAMIC_FIELD)) {
+        if (isValidFieldGlob(name)) {
+          // make sure nothing else has the same path
+          addDynamicField(dFields, f);
+        } else {
+          String msg = "Dynamic field name '" + name 
+              + "' should have either a leading or a trailing asterisk, and no others.";
+          throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+        }
+      } else {
+        // we should never get here
+        throw new RuntimeException("Unknown field type");
+      }
+    }
+
+    //fields with default values are by definition required
+    //add them to required fields, and we only have to loop once
+    // in DocumentBuilder.getDoc()
+    requiredFields.addAll(fieldsWithDefaultValue);
+
+    // OK, now sort the dynamic fields largest to smallest size so we don't get
+    // any false matches.  We want to act like a compiler tool and try and match
+    // the largest string possible.
+    Collections.sort(dFields);
+
+    log.trace("Dynamic Field Ordering:" + dFields);
+
+    // stuff it in a normal array for faster access
+    dynamicFields = dFields.toArray(new DynamicField[dFields.size()]);
+
+    return explicitRequiredProp;
+  }
+
   /**
    * Converts a sequence of path steps into a rooted path, by inserting slashes in front of each step.
    * @param steps The steps to join with slashes to form a path
@@ -1405,4 +1403,53 @@ public class IndexSchema {
     }
     return copyFieldProperties;
   }
+
+  protected IndexSchema(final SolrConfig solrConfig, final SolrResourceLoader loader) {
+    this.solrConfig = solrConfig;
+    this.loader = loader;
+  }
+
+  /**
+   * Copies this schema, adds the given field to the copy, then persists the new schema.
+   *
+   * @param newField the SchemaField to add 
+   * @return a new IndexSchema based on this schema with newField added
+   * @see #newField(String, String, Map)
+   */
+  public IndexSchema addField(SchemaField newField) {
+    String msg = "This IndexSchema is not mutable.";
+    log.error(msg);
+    throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+  }
+
+  /**
+   * Copies this schema, adds the given fields to the copy, then persists the new schema.
+   *
+   * @param newFields the SchemaFields to add 
+   * @return a new IndexSchema based on this schema with newFields added
+   * @see #newField(String, String, Map)
+   */
+  public IndexSchema addFields(Collection<SchemaField> newFields) {
+    String msg = "This IndexSchema is not mutable.";
+    log.error(msg);
+    throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+  }
+
+  /**
+   * Returns a SchemaField if the given fieldName does not already 
+   * exist in this schema, and does not match any dynamic fields 
+   * in this schema.  The resulting SchemaField can be used in a call
+   * to {@link #addField(SchemaField)}.
+   *
+   * @param fieldName the name of the field to add
+   * @param fieldType the field type for the new field
+   * @param options the options to use when creating the SchemaField
+   * @return The created SchemaField
+   * @see #addField(SchemaField)
+   */
+  public SchemaField newField(String fieldName, String fieldType, Map<String,?> options) {
+    String msg = "This IndexSchema is not mutable.";
+    log.error(msg);
+    throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+  }
 }

Modified: 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=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java Mon Apr 22 14:26:55 2013
@@ -16,14 +16,47 @@ package org.apache.solr.schema;
  * limitations under the License.
  */
 
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.core.PluginInfo;
-import org.apache.solr.core.SolrConfig;
+import org.apache.solr.core.SolrConfig;         
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.util.SystemIdResolver;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.xml.sax.InputSource;
 
+import java.io.File;
+import java.io.InputStream;
+
+/** Base class for factories for IndexSchema implementations */
 public abstract class IndexSchemaFactory implements NamedListInitializedPlugin {
+  private static final Logger log = LoggerFactory.getLogger(IndexSchemaFactory.class);
   
-  public abstract IndexSchema create(String resourceName, SolrConfig config);
+  /** Returns an index schema created from a local resource */
+  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;
+  }
 
+  /** Instantiates the configured schema factory, then calls create on it. */
   public static IndexSchema buildIndexSchema(String resourceName, SolrConfig config) {
     PluginInfo info = config.getPluginInfo(IndexSchemaFactory.class.getName());
     IndexSchemaFactory factory;
@@ -36,4 +69,31 @@ public abstract class IndexSchemaFactory
     IndexSchema schema = factory.create(resourceName, config);
     return schema;
   }
+
+  /** 
+   * Returns the resource name that will be used: if the schema is managed, the resource
+   * name will be drawn from the schema factory configuration in the given SolrConfig.
+   * Otherwise, the given resourceName will be returned.
+   * 
+   * @param resourceName The name to use if the schema is not managed
+   * @param config The SolrConfig from which to get the schema factory config
+   * @return If the schema is managed, the resource name from the given SolrConfig,
+   *         otherwise the given resourceName. 
+   */
+  public static String getResourceNameToBeUsed(String resourceName, SolrConfig config) {
+    PluginInfo info = config.getPluginInfo(IndexSchemaFactory.class.getName());
+    final String nonManagedResourceName = null == resourceName ? IndexSchema.DEFAULT_SCHEMA_FILE : resourceName;
+    if (null == info) {
+      return nonManagedResourceName;
+    }
+    String managedSchemaResourceName
+        = (String)info.initArgs.get(ManagedIndexSchemaFactory.MANAGED_SCHEMA_RESOURCE_NAME);
+    if (null == managedSchemaResourceName) {
+      managedSchemaResourceName = ManagedIndexSchemaFactory.DEFAULT_MANAGED_SCHEMA_RESOURCE_NAME;
+    }
+    if ((new File(config.getResourceLoader().getConfigDir(), managedSchemaResourceName)).exists()) {
+      return managedSchemaResourceName;
+    }
+    return nonManagedResourceName;
+  }
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/LatLonType.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/LatLonType.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/LatLonType.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/LatLonType.java Mon Apr 22 14:26:55 2013
@@ -82,11 +82,11 @@ public class LatLonType extends Abstract
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
       }
       //latitude
-      SchemaField lat = subField(field, i);
+      SchemaField lat = subField(field, i, schema);
       f.add(lat.createField(String.valueOf(latLon[LAT]), lat.indexed() && !lat.omitNorms() ? boost : 1f));
       i++;
       //longitude
-      SchemaField lon = subField(field, i);
+      SchemaField lon = subField(field, i, schema);
       f.add(lon.createField(String.valueOf(latLon[LON]), lon.indexed() && !lon.omitNorms() ? boost : 1f));
 
     }
@@ -114,7 +114,7 @@ public class LatLonType extends Abstract
     }
     BooleanQuery result = new BooleanQuery(true);
     for (int i = 0; i < dimension; i++) {
-      SchemaField subSF = subField(field, i);
+      SchemaField subSF = subField(field, i, parser.getReq().getSchema());
       // points must currently be ordered... should we support specifying any two opposite corner points?
       result.add(subSF.getType().getRangeQuery(parser, subSF, p1[i], p2[i], minInclusive, maxInclusive), BooleanClause.Occur.MUST);
     }
@@ -134,7 +134,7 @@ public class LatLonType extends Abstract
     }
     BooleanQuery bq = new BooleanQuery(true);
     for (int i = 0; i < dimension; i++) {
-      SchemaField sf = subField(field, i);
+      SchemaField sf = subField(field, i, parser.getReq().getSchema());
       Query tq = sf.getType().getFieldQuery(parser, sf, p1[i]);
       bq.add(tq, BooleanClause.Occur.MUST);
     }
@@ -173,9 +173,11 @@ public class LatLonType extends Abstract
        lon2Max = 180;
     }
     
+    IndexSchema schema = parser.getReq().getSchema();
+    
     // Now that we've figured out the ranges, build them!
-    SchemaField latField = subField(options.field, LAT);
-    SchemaField lonField = subField(options.field, LON);
+    SchemaField latField = subField(options.field, LAT, schema);
+    SchemaField lonField = subField(options.field, LON, schema);
 
     SpatialDistanceQuery spatial = new SpatialDistanceQuery();
 
@@ -240,7 +242,7 @@ public class LatLonType extends Abstract
   public ValueSource getValueSource(SchemaField field, QParser parser) {
     ArrayList<ValueSource> vs = new ArrayList<ValueSource>(2);
     for (int i = 0; i < 2; i++) {
-      SchemaField sub = subField(field, i);
+      SchemaField sub = subField(field, i, parser.getReq().getSchema());
       vs.add(sub.getType().getValueSource(sub, parser));
     }
     return new LatLonValueSource(field, vs);

Modified: 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=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java Mon Apr 22 14:26:55 2013
@@ -16,18 +16,44 @@ package org.apache.solr.schema;
  * 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.SolrZkClient;
+import org.apache.solr.core.Config;
 import org.apache.solr.core.SolrConfig;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.util.FileUtils;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.w3c.dom.Document;
 import org.xml.sax.InputSource;
 
+import javax.xml.xpath.XPath;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.StringWriter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+
 /** 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;
-  }
+  @Override public boolean isMutable() { return isMutable; }
+
+  final String managedSchemaResourceName;
+  
+  int schemaZkVersion;
+  
+  final Object schemaUpdateLock;
   
   /**
    * Constructs a schema using the specified resource name and stream.
@@ -36,8 +62,292 @@ public final class ManagedIndexSchema ex
    *      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) {
+  ManagedIndexSchema(SolrConfig solrConfig, String name, InputSource is, boolean isMutable, 
+                     String managedSchemaResourceName, int schemaZkVersion, Object schemaUpdateLock) 
+      throws KeeperException, InterruptedException {
     super(solrConfig, name, is);
     this.isMutable = isMutable;
+    this.managedSchemaResourceName = managedSchemaResourceName;
+    this.schemaZkVersion = schemaZkVersion;
+    this.schemaUpdateLock = schemaUpdateLock;
+  }
+  
+  
+  /** Persist the schema to local storage or to ZooKeeper */
+  boolean persistManagedSchema(boolean createOnly) {
+    if (loader instanceof ZkSolrResourceLoader) {
+      return persistManagedSchemaToZooKeeper(createOnly);
+    }
+    // Persist locally
+    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();
+          log.error(msg);
+          throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+        }
+      }
+      final FileOutputStream out = new FileOutputStream(managedSchemaFile);
+      writer = new OutputStreamWriter(out, "UTF-8");
+      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(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);
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Persists the managed schema to ZooKeeper using optimistic concurrency.
+   * <p/>
+   * If createOnly is true, success is when the schema is created or if it previously existed.
+   * <p/>
+   * If createOnly is false, success is when the schema is persisted - this will only happen
+   * if schemaZkVersion matches the version in ZooKeeper.
+   * 
+   * @return true on success 
+   */
+  boolean persistManagedSchemaToZooKeeper(boolean createOnly) {
+    final ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
+    final ZkController zkController = zkLoader.getZkController();
+    final SolrZkClient zkClient = zkController.getZkClient();
+    final String managedSchemaPath = zkLoader.getCollectionZkPath() + "/" + managedSchemaResourceName;
+    boolean success = true;
+    try {
+      // Persist the managed schema
+      StringWriter writer = new StringWriter();
+      persist(writer);
+
+      final byte[] data = writer.toString().getBytes("UTF-8");
+      if (createOnly) {
+        try {
+          zkClient.create(managedSchemaPath, data, CreateMode.PERSISTENT, true);
+          schemaZkVersion = 0;
+          log.info("Created and persisted managed schema znode at " + managedSchemaPath);
+        } catch (KeeperException.NodeExistsException e) {
+          // This is okay - do nothing and fall through
+          log.info("Managed schema znode at " + managedSchemaPath + " already exists - no need to create it");
+        }
+      } else {
+        try {
+          // Assumption: the path exists
+          Stat stat = zkClient.setData(managedSchemaPath, data, schemaZkVersion, true);
+          schemaZkVersion = stat.getVersion();
+          log.info("Persisted managed schema at " + managedSchemaPath);
+        } catch (KeeperException.BadVersionException e) {
+          log.info("Failed to persist managed schema at " + managedSchemaPath 
+                  + " - version mismatch");
+          success = false;
+        }
+      }
+    } catch (Exception e) {
+      if (e instanceof InterruptedException) {
+        Thread.currentThread().interrupt(); // Restore the interrupted status
+      }
+      final String msg = "Error persisting managed schema at " + managedSchemaPath;
+      log.error(msg, e);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg, e);
+    }
+    return success; 
+  }
+
+  @Override
+  public ManagedIndexSchema addField(SchemaField newField) {
+    return addFields(Arrays.asList(newField));
+  }
+
+  @Override
+  public ManagedIndexSchema addFields(Collection<SchemaField> newFields) {
+    ManagedIndexSchema newSchema = null;
+    if (isMutable) {
+      boolean success = false;
+      while ( ! success) { // optimistic concurrency
+        // even though fields is volatile, we need to synchronize to avoid two addFields
+        // happening concurrently (and ending up missing one of them)
+        synchronized (getSchemaUpdateLock()) {
+          newSchema = shallowCopy(true);
+          
+          for (SchemaField newField : newFields) {
+            if (null != newSchema.getFieldOrNull(newField.getName())) {
+              String msg = "Field '" + newField.getName() + "' already exists.";
+              throw new SolrException(ErrorCode.BAD_REQUEST, msg);
+            }
+            newSchema.fields.put(newField.getName(), newField);
+
+            if (null != newField.getDefaultValue()) {
+              log.debug(newField.getName() + " contains default value: " + newField.getDefaultValue());
+              newSchema.fieldsWithDefaultValue.add(newField);
+            }
+            if (newField.isRequired()) {
+              log.debug("{} is required in this schema", newField.getName());
+              newSchema.requiredFields.add(newField);
+            }
+          }
+          // Run the callbacks on SchemaAware now that everything else is done
+          for (SchemaAware aware : newSchema.schemaAware) {
+            aware.inform(newSchema);
+          }
+          newSchema.refreshAnalyzers();
+          success = newSchema.persistManagedSchema(false); // don't just create - update it if it already exists
+          if (success) {
+            log.debug("Added field(s): {}", newFields);
+          }
+        }
+        // release the lock between tries to allow the schema reader to update the schema & schemaZkVersion
+      }
+    } else {
+      String msg = "This ManagedIndexSchema is not mutable.";
+      log.error(msg);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+    }
+    return newSchema;
+  }
+
+  @Override
+  public SchemaField newField(String fieldName, String fieldType, Map<String,?> options) {
+    SchemaField sf; 
+    if (isMutable) {
+      try {
+        if (-1 != fieldName.indexOf('*')) {
+          String msg = "Can't add dynamic field '" + fieldName + "'.";
+          throw new SolrException(ErrorCode.BAD_REQUEST, msg);
+        }
+        SchemaField existingFieldWithTheSameName = getFieldOrNull(fieldName);
+        if (null != existingFieldWithTheSameName) {
+          String msg = "Field '" + fieldName + "' already exists.";
+          throw new SolrException(ErrorCode.BAD_REQUEST, msg);
+        }
+        FieldType type = getFieldTypeByName(fieldType);
+        if (null == type) {
+          String msg = "Field '" + fieldName + "': Field type '" + fieldType + "' not found.";
+          log.error(msg);
+          throw new SolrException(ErrorCode.BAD_REQUEST, msg);
+        }
+        sf = SchemaField.create(fieldName, type, options);
+      } catch (SolrException e) {
+        throw e;
+      } catch (Exception e) {
+        throw new SolrException(ErrorCode.BAD_REQUEST, e);
+      }
+    } else {
+      String msg = "This ManagedIndexSchema is not mutable.";
+      log.error(msg);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+    }
+    return sf;
+  }
+
+  /** 
+   * Called from ZkIndexSchemaReader to merge the fields from the serialized managed schema
+   * on ZooKeeper with the local managed schema.
+   * 
+   * @param inputSource The serialized content of the managed schema from ZooKeeper
+   * @param schemaZkVersion The ZK version of the managed schema on ZooKeeper
+   * @return The new merged schema
+   */
+  ManagedIndexSchema reloadFields(InputSource inputSource, int schemaZkVersion) {
+    ManagedIndexSchema newSchema;
+    try {
+      newSchema = shallowCopy(false);
+      Config schemaConf = new Config(loader, SCHEMA, inputSource, SLASH+SCHEMA+SLASH);
+      Document document = schemaConf.getDocument();
+      final XPath xpath = schemaConf.getXPath();
+      newSchema.loadFields(document, xpath);
+      if (null != uniqueKeyField) {
+        newSchema.requiredFields.add(uniqueKeyField);
+      }
+      //Run the callbacks on SchemaAware now that everything else is done
+      for (SchemaAware aware : newSchema.schemaAware) {
+        aware.inform(newSchema);
+      }
+      newSchema.refreshAnalyzers();
+      newSchema.schemaZkVersion = schemaZkVersion;
+    } catch (SolrException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Schema Parsing Failed: " + e.getMessage(), e);
+    }
+    return newSchema;
+  }
+  
+  private ManagedIndexSchema(final SolrConfig solrConfig, final SolrResourceLoader loader, boolean isMutable,
+                             String managedSchemaResourceName, int schemaZkVersion, Object schemaUpdateLock) 
+      throws KeeperException, InterruptedException {
+    super(solrConfig, loader);
+    this.isMutable = isMutable;
+    this.managedSchemaResourceName = managedSchemaResourceName;
+    this.schemaZkVersion = schemaZkVersion;
+    this.schemaUpdateLock = schemaUpdateLock;
+  }
+
+  /**
+   * Makes a shallow copy of this schema.
+   * 
+   * Not copied: analyzers 
+   * 
+   * @param includeFieldDataStructures if true, fields, fieldsWithDefaultValue, and requiredFields
+   *                                   are copied; otherwise, they are not.
+   * @return A shallow copy of this schema
+   */
+  private ManagedIndexSchema shallowCopy(boolean includeFieldDataStructures) {
+    ManagedIndexSchema newSchema = null;
+    try {
+      newSchema = new ManagedIndexSchema
+          (solrConfig, loader, isMutable, managedSchemaResourceName, schemaZkVersion, getSchemaUpdateLock());
+    } catch (KeeperException e) {
+      final String msg = "Error instantiating ManagedIndexSchema";
+      log.error(msg, e);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg, e);
+    } catch (InterruptedException e) {
+      // Restore the interrupted status
+      Thread.currentThread().interrupt();
+      log.warn("", e);
+    }
+
+    assert newSchema != null;
+    
+    newSchema.name = name;
+    newSchema.version = version;
+    newSchema.defaultSearchFieldName = defaultSearchFieldName;
+    newSchema.queryParserDefaultOperator = queryParserDefaultOperator;
+    newSchema.isExplicitQueryParserDefaultOperator = isExplicitQueryParserDefaultOperator;
+    newSchema.similarity = similarity;
+    newSchema.similarityFactory = similarityFactory;
+    newSchema.isExplicitSimilarity = isExplicitSimilarity;
+    newSchema.uniqueKeyField = uniqueKeyField;
+
+    if (includeFieldDataStructures) {
+      // These need new collections, since addFields() can add members to them
+      newSchema.fields.putAll(fields);
+      newSchema.fieldsWithDefaultValue.addAll(fieldsWithDefaultValue);
+      newSchema.requiredFields.addAll(requiredFields);
+    }
+
+    // These don't need new collections - addFields() won't add members to them 
+    newSchema.fieldTypes = fieldTypes;
+    newSchema.dynamicFields = dynamicFields;
+    newSchema.dynamicCopyFields = dynamicCopyFields;
+    newSchema.copyFieldsMap = copyFieldsMap;
+    newSchema.copyFieldTargetCounts = copyFieldTargetCounts;
+    newSchema.schemaAware = schemaAware;
+
+    return newSchema;
+  }
+  
+  public Object getSchemaUpdateLock() {
+    return schemaUpdateLock;
   }
 }