You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2016/03/07 18:18:35 UTC

[3/4] lucene-solr git commit: SOLR-8736: schema GET operations on fields, dynamicFields, fieldTypes, copyField are reimplemented as a part of the bulk API with less details. The tests and write implementations are removed

SOLR-8736: schema GET operations on fields, dynamicFields, fieldTypes, copyField are
  reimplemented as a part of the bulk API with less details. The tests and write implementations are removed


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

Branch: refs/heads/master
Commit: f2c281abcbfc254d44c196ceb7c7f61311e7967c
Parents: 9082d5f
Author: Noble Paul <no...@apache.org>
Authored: Mon Mar 7 22:44:36 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Mon Mar 7 22:44:36 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   5 +
 .../org/apache/solr/handler/SchemaHandler.java  |  78 +++-
 .../org/apache/solr/rest/SolrSchemaRestApi.java |  56 +--
 .../solr/rest/schema/BaseFieldResource.java     | 146 --------
 .../solr/rest/schema/BaseFieldTypeResource.java |  98 -----
 .../schema/CopyFieldCollectionResource.java     | 198 ----------
 .../schema/DynamicFieldCollectionResource.java  | 207 -----------
 .../solr/rest/schema/DynamicFieldResource.java  | 197 ----------
 .../rest/schema/FieldCollectionResource.java    | 225 -----------
 .../apache/solr/rest/schema/FieldResource.java  | 201 ----------
 .../schema/FieldTypeCollectionResource.java     | 197 ----------
 .../solr/rest/schema/FieldTypeResource.java     | 203 ----------
 .../org/apache/solr/schema/IndexSchema.java     |  11 +-
 .../org/apache/solr/servlet/HttpSolrCall.java   |   3 +
 .../rest/schema/TestClassNameShortening.java    |   3 +-
 .../schema/TestCopyFieldCollectionResource.java |  96 +----
 .../TestDynamicFieldCollectionResource.java     |  29 --
 .../rest/schema/TestDynamicFieldResource.java   |   7 -
 .../schema/TestFieldCollectionResource.java     |  45 ---
 .../solr/rest/schema/TestFieldResource.java     |  23 +-
 .../schema/TestFieldTypeCollectionResource.java |   1 +
 .../solr/rest/schema/TestFieldTypeResource.java |  17 +-
 .../TestManagedSchemaDynamicFieldResource.java  | 366 ------------------
 .../schema/TestManagedSchemaFieldResource.java  | 369 -------------------
 .../TestManagedSchemaFieldTypeResource.java     | 350 ------------------
 .../schema/TestRemoveLastDynamicCopyField.java  |  80 ----
 .../schema/TestSchemaSimilarityResource.java    |   1 -
 .../analysis/TestManagedStopFilterFactory.java  |   2 +-
 .../TestManagedSynonymFilterFactory.java        |   6 +-
 .../TestCloudManagedSchemaConcurrent.java       |   3 +-
 .../solr/client/solrj/request/SchemaTest.java   |   4 -
 31 files changed, 125 insertions(+), 3102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 61fcd47..7893b88 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -105,6 +105,8 @@ Upgrading from Solr 5.x
 
 * When requesting stats in date fields, "sum" is now a double value instead of a date. See SOLR-8671
 
+* SOLR-8736: The deprecated GET methods for schema are now accessible and implemented differently
+
 Detailed Change List
 ----------------------
 
@@ -395,6 +397,9 @@ Other Changes
 * SOLR-8758: Add a new SolrCloudTestCase class, using MiniSolrCloudCluster (Alan
   Woodward)
 
+* SOLR-8736: schema GET operations on fields, dynamicFields, fieldTypes, copyField are
+  reimplemented as a part of the bulk API with less details (noble)
+
 ==================  5.5.1 ==================
 
 Bug Fixes

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
index 046de46..4279864 100644
--- a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
@@ -20,15 +20,20 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Locale;
+import java.util.Map;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.ContentStream;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
@@ -42,17 +47,29 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.common.params.CommonParams.JSON;
-import static org.apache.solr.core.ConfigSetProperties.IMMUTABLE_CONFIGSET_ARG;
 
 public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private boolean isImmutableConfigSet = false;
 
-  @Override
-  public void init(NamedList args) {
-    super.init(args);
+  private static final Map<String, String> level2;
+
+  static {
+    Set<String> s = ImmutableSet.of(
+        IndexSchema.FIELD_TYPES,
+        IndexSchema.FIELDS,
+        IndexSchema.DYNAMIC_FIELDS,
+        IndexSchema.COPY_FIELDS
+    );
+    Map<String, String> m = new HashMap<>();
+    for (String s1 : s) {
+      m.put(s1, s1);
+      m.put(s1.toLowerCase(Locale.ROOT), s1);
+    }
+    level2 = ImmutableMap.copyOf(m);
   }
 
+
   @Override
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
     SolrConfigHandler.setWt(req, JSON);
@@ -150,6 +167,33 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware {
           break;
         }
         default: {
+          List<String> parts = StrUtils.splitSmart(path, '/');
+          if (parts.get(0).isEmpty()) parts.remove(0);
+          if (parts.size() > 1 && level2.containsKey(parts.get(1))) {
+            String realName = level2.get(parts.get(1));
+            SimpleOrderedMap<Object> propertyValues = req.getSchema().getNamedPropertyValues(req.getParams());
+            Object o = propertyValues.get(realName);
+            if(parts.size()> 2) {
+              String name = parts.get(2);
+              if (o instanceof List) {
+                List list = (List) o;
+                for (Object obj : list) {
+                  if (obj instanceof SimpleOrderedMap) {
+                    SimpleOrderedMap simpleOrderedMap = (SimpleOrderedMap) obj;
+                    if(name.equals(simpleOrderedMap.get("name"))) {
+                      rsp.add(realName.substring(0, realName.length() - 1), simpleOrderedMap);
+                      return;
+                    }
+                  }
+                }
+              }
+              throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "No such path " + path);
+            } else {
+              rsp.add(realName, o);
+            }
+            return;
+          }
+
           throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "No such path " + path);
         }
       }
@@ -160,19 +204,25 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware {
   }
 
   private static Set<String> subPaths = new HashSet<>(Arrays.asList(
-      "/version",
-      "/uniquekey",
-      "/name",
-      "/similarity",
-      "/defaultsearchfield",
-      "/solrqueryparser",
-      "/zkversion",
-      "/solrqueryparser/defaultoperator"
+      "version",
+      "uniquekey",
+      "name",
+      "similarity",
+      "defaultsearchfield",
+      "solrqueryparser",
+      "zkversion"
   ));
+  static {
+    subPaths.addAll(level2.keySet());
+  }
 
   @Override
   public SolrRequestHandler getSubHandler(String subPath) {
-    if (subPaths.contains(subPath)) return this;
+    List<String> parts = StrUtils.splitSmart(subPath, '/');
+    if (parts.get(0).isEmpty()) parts.remove(0);
+    String prefix =  parts.get(0);
+    if(subPaths.contains(prefix)) return this;
+
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/java/org/apache/solr/rest/SolrSchemaRestApi.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/rest/SolrSchemaRestApi.java b/solr/core/src/java/org/apache/solr/rest/SolrSchemaRestApi.java
index 0e40f73..1310198 100644
--- a/solr/core/src/java/org/apache/solr/rest/SolrSchemaRestApi.java
+++ b/solr/core/src/java/org/apache/solr/rest/SolrSchemaRestApi.java
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 package org.apache.solr.rest;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.Set;
+
 import org.apache.solr.request.SolrRequestInfo;
-import org.apache.solr.rest.schema.CopyFieldCollectionResource;
-import org.apache.solr.rest.schema.DynamicFieldCollectionResource;
-import org.apache.solr.rest.schema.DynamicFieldResource;
-import org.apache.solr.rest.schema.FieldCollectionResource;
-import org.apache.solr.rest.schema.FieldResource;
-import org.apache.solr.rest.schema.FieldTypeCollectionResource;
-import org.apache.solr.rest.schema.FieldTypeResource;
 import org.apache.solr.schema.IndexSchema;
 import org.restlet.Application;
 import org.restlet.Restlet;
@@ -30,39 +30,18 @@ import org.restlet.routing.Router;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.lang.invoke.MethodHandles;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Locale;
-import java.util.Set;
-
 /**
  * Restlet servlet handling /&lt;context&gt;/&lt;collection&gt;/schema/* URL paths
  */
 public class SolrSchemaRestApi extends Application {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  public static final String FIELDS_PATH = "/" + IndexSchema.FIELDS;
-  
-  public static final String DYNAMIC_FIELDS = IndexSchema.DYNAMIC_FIELDS.toLowerCase(Locale.ROOT);
-  public static final String DYNAMIC_FIELDS_PATH = "/" + DYNAMIC_FIELDS;
-  
-  public static final String FIELDTYPES = IndexSchema.FIELD_TYPES.toLowerCase(Locale.ROOT);
-  public static final String FIELDTYPES_PATH = "/" + FIELDTYPES;
 
-  public static final String NAME_SEGMENT = "/{" + IndexSchema.NAME.toLowerCase(Locale.ROOT) + "}";
-  
-  public static final String COPY_FIELDS = IndexSchema.COPY_FIELDS.toLowerCase(Locale.ROOT);
-  public static final String COPY_FIELDS_PATH = "/" + COPY_FIELDS;
-  
+
   /**
    * Returns reserved endpoints under /schema
    */
   public static Set<String> getReservedEndpoints() {
     Set<String> reservedEndpoints = new HashSet<>();
-    reservedEndpoints.add(RestManager.SCHEMA_BASE_PATH + FIELDS_PATH);
-    reservedEndpoints.add(RestManager.SCHEMA_BASE_PATH + DYNAMIC_FIELDS_PATH);
-    reservedEndpoints.add(RestManager.SCHEMA_BASE_PATH + FIELDTYPES_PATH);
-    reservedEndpoints.add(RestManager.SCHEMA_BASE_PATH + COPY_FIELDS_PATH);
     return Collections.unmodifiableSet(reservedEndpoints);
   }
 
@@ -88,25 +67,6 @@ public class SolrSchemaRestApi extends Application {
     log.info("createInboundRoot started for /schema");
 
 
-    router.attach(FIELDS_PATH, FieldCollectionResource.class);
-    // Allow a trailing slash on collection requests
-    router.attach(FIELDS_PATH + "/", FieldCollectionResource.class);
-    router.attach(FIELDS_PATH + NAME_SEGMENT, FieldResource.class);
-
-    router.attach(DYNAMIC_FIELDS_PATH, DynamicFieldCollectionResource.class);
-    // Allow a trailing slash on collection requests
-    router.attach(DYNAMIC_FIELDS_PATH + "/", DynamicFieldCollectionResource.class);
-    router.attach(DYNAMIC_FIELDS_PATH + NAME_SEGMENT, DynamicFieldResource.class);
-
-    router.attach(FIELDTYPES_PATH, FieldTypeCollectionResource.class);
-    // Allow a trailing slash on collection requests
-    router.attach(FIELDTYPES_PATH + "/", FieldTypeCollectionResource.class);
-    router.attach(FIELDTYPES_PATH + NAME_SEGMENT, FieldTypeResource.class);
-
-    router.attach(COPY_FIELDS_PATH, CopyFieldCollectionResource.class);
-    // Allow a trailing slash on collection requests
-    router.attach(COPY_FIELDS_PATH + "/", CopyFieldCollectionResource.class);
-
     router.attachDefault(RestManager.ManagedEndpoint.class);
     
     // attach all the dynamically registered schema resources

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/java/org/apache/solr/rest/schema/BaseFieldResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/BaseFieldResource.java b/solr/core/src/java/org/apache/solr/rest/schema/BaseFieldResource.java
deleted file mode 100644
index 25f631d..0000000
--- a/solr/core/src/java/org/apache/solr/rest/schema/BaseFieldResource.java
+++ /dev/null
@@ -1,146 +0,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.
- */
-package org.apache.solr.rest.schema;
-import org.apache.solr.cloud.ZkSolrResourceLoader;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.core.CoreDescriptor;
-import org.apache.solr.rest.BaseSolrResource;
-import org.apache.solr.schema.IndexSchema;
-import org.apache.solr.schema.ManagedIndexSchema;
-import org.apache.solr.schema.SchemaField;
-import org.restlet.resource.ResourceException;
-
-import java.util.LinkedHashSet;
-import java.util.Map;
-
-
-/**
- * Base class for Schema Field and DynamicField requests.
- */
-abstract class BaseFieldResource extends BaseSolrResource {
-  protected static final String INCLUDE_DYNAMIC_PARAM = "includeDynamic";
-  private static final String DYNAMIC_BASE = "dynamicBase";
-
-  private LinkedHashSet<String> requestedFields;
-  private boolean showDefaults;
-
-  protected LinkedHashSet<String> getRequestedFields() {
-    return requestedFields; 
-  }
-  
-
-  protected BaseFieldResource() {
-    super();
-  }
-
-  /**
-   * Pulls the "fl" param from the request and splits it to get the
-   * requested list of fields.  The (Dynamic)FieldCollectionResource classes
-   * will then restrict the fields sent back in the response to those
-   * on this list.  The (Dynamic)FieldResource classes ignore this list, 
-   * since the (dynamic) field is specified in the URL path, rather than
-   * in a query parameter.
-   * <p>
-   * Also pulls the "showDefaults" param from the request, for use by all
-   * subclasses to include default values from the associated field type
-   * in the response.  By default this param is off.
-   */
-  @Override
-  public void doInit() throws ResourceException {
-    super.doInit();
-    if (isExisting()) {
-      String flParam = getSolrRequest().getParams().get(CommonParams.FL);
-      if (null != flParam) {
-        String[] fields = flParam.trim().split("[,\\s]+");
-        if (fields.length > 0) {
-          requestedFields = new LinkedHashSet<>();
-          for (String field : fields) {
-            if ( ! field.trim().isEmpty()) {
-              requestedFields.add(field.trim());
-            }
-          }
-        }
-      }
-      showDefaults = getSolrRequest().getParams().getBool(SHOW_DEFAULTS, false);
-    }
-  }
-
-  /** Get the properties for a given field.
-   *
-   * @param field not required to exist in the schema
-   */
-  protected SimpleOrderedMap<Object> getFieldProperties(SchemaField field) {
-    if (null == field) {
-      return null;
-    }
-    SimpleOrderedMap<Object> properties = field.getNamedPropertyValues(showDefaults);
-    if ( ! getSchema().getFields().containsKey(field.getName())) {
-      String dynamicBase = getSchema().getDynamicPattern(field.getName());
-      // Add dynamicBase property if it's different from the field name. 
-      if ( ! field.getName().equals(dynamicBase)) {
-        properties.add(DYNAMIC_BASE, dynamicBase);
-      }
-    }
-    if (field == getSchema().getUniqueKeyField()) {
-      properties.add(IndexSchema.UNIQUE_KEY, true);
-    }
-    return properties;
-  }
-
-  /**
-   * When running in cloud mode, waits for a schema update to be
-   * applied by all active replicas of the current collection.
-   */
-  protected void waitForSchemaUpdateToPropagate(IndexSchema newSchema) {
-    // If using ZooKeeper and the client application has requested an update timeout, then block until all
-    // active replicas for this collection process the updated schema
-    if (getUpdateTimeoutSecs() > 0 && newSchema != null &&
-        newSchema.getResourceLoader() instanceof ZkSolrResourceLoader)
-    {
-      CoreDescriptor cd = getSolrCore().getCoreDescriptor();
-      String collection = cd.getCollectionName();
-      if (collection != null) {
-        ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader) newSchema.getResourceLoader();
-        ManagedIndexSchema.waitForSchemaZkVersionAgreement(collection,
-            cd.getCloudDescriptor().getCoreNodeName(),
-            ((ManagedIndexSchema) newSchema).getSchemaZkVersion(),
-            zkLoader.getZkController(),
-            getUpdateTimeoutSecs());
-      }
-    }
-  }
-
-  // protected access on this class triggers a bug in javadoc generation caught by
-  // documentation-link: "BROKEN LINK" reported in javadoc for classes using
-  // NewFieldArguments because the link target file is BaseFieldResource.NewFieldArguments,
-  // but the actual file is BaseFieldResource$NewFieldArguments.
-  static class NewFieldArguments {
-    private String name;
-    private String type;
-    Map<String,Object> map;
-    NewFieldArguments(String name, String type, Map<String,Object> map) {
-      this.name = name;
-      this.type = type;
-      this.map = map;
-    }
-
-    public String getName() { return name; }
-    public String getType() { return type; }
-    public Map<String, Object> getMap() { return map; }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/java/org/apache/solr/rest/schema/BaseFieldTypeResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/BaseFieldTypeResource.java b/solr/core/src/java/org/apache/solr/rest/schema/BaseFieldTypeResource.java
deleted file mode 100644
index c475dd0..0000000
--- a/solr/core/src/java/org/apache/solr/rest/schema/BaseFieldTypeResource.java
+++ /dev/null
@@ -1,98 +0,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.
- */
-package org.apache.solr.rest.schema;
-
-import org.apache.solr.cloud.ZkSolrResourceLoader;
-import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.core.CoreDescriptor;
-import org.apache.solr.rest.BaseSolrResource;
-import org.apache.solr.schema.FieldType;
-import org.apache.solr.schema.IndexSchema;
-import org.apache.solr.schema.ManagedIndexSchema;
-import org.restlet.resource.ResourceException;
-
-import java.util.List;
-
-/**
- * Base class for the FieldType resource classes.
- */
-abstract class BaseFieldTypeResource extends BaseSolrResource {
-  private boolean showDefaults;
-
-  protected BaseFieldTypeResource() {
-    super();
-  }
-
-  @Override
-  public void doInit() throws ResourceException {
-    super.doInit();
-    if (isExisting()) {
-      showDefaults = getSolrRequest().getParams().getBool(SHOW_DEFAULTS, false);
-    }
-  }
-  
-  /** Used by subclasses to collect field type properties */
-  protected SimpleOrderedMap<Object> getFieldTypeProperties(FieldType fieldType) {
-    SimpleOrderedMap<Object> properties = fieldType.getNamedPropertyValues(showDefaults);
-    properties.add(IndexSchema.FIELDS, getFieldsWithFieldType(fieldType));
-    properties.add(IndexSchema.DYNAMIC_FIELDS, getDynamicFieldsWithFieldType(fieldType));
-    return properties;
-  }
-
-  
-  /** Return a list of names of Fields that have the given FieldType */
-  protected abstract List<String> getFieldsWithFieldType(FieldType fieldType);
-
-  /** Return a list of names of DynamicFields that have the given FieldType */
-  protected abstract List<String> getDynamicFieldsWithFieldType(FieldType fieldType);
-
-  /**
-   * Adds one or more new FieldType definitions to the managed schema for the given core.
-   */
-  protected void addNewFieldTypes(List<FieldType> newFieldTypes, ManagedIndexSchema oldSchema) {
-    IndexSchema newSchema = null;
-    boolean success = false;
-    while (!success) {
-      try {
-        synchronized (oldSchema.getSchemaUpdateLock()) {
-          newSchema = oldSchema.addFieldTypes(newFieldTypes, true);
-          getSolrCore().setLatestSchema(newSchema);
-          success = true;
-        }
-      } catch (ManagedIndexSchema.SchemaChangedInZkException e) {
-        oldSchema = (ManagedIndexSchema)getSolrCore().getLatestSchema();
-      }
-    }
-
-    // If using ZooKeeper and the client application has requested an update timeout, then block until all
-    // active replicas for this collection process the updated schema
-    if (getUpdateTimeoutSecs() > 0 && newSchema != null &&
-        newSchema.getResourceLoader() instanceof ZkSolrResourceLoader)
-    {
-      CoreDescriptor cd = getSolrCore().getCoreDescriptor();
-      String collection = cd.getCollectionName();
-      if (collection != null) {
-        ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader) newSchema.getResourceLoader();
-        ManagedIndexSchema.waitForSchemaZkVersionAgreement(collection,
-            cd.getCloudDescriptor().getCoreNodeName(),
-            ((ManagedIndexSchema) newSchema).getSchemaZkVersion(),
-            zkLoader.getZkController(),
-            getUpdateTimeoutSecs());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/java/org/apache/solr/rest/schema/CopyFieldCollectionResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/CopyFieldCollectionResource.java b/solr/core/src/java/org/apache/solr/rest/schema/CopyFieldCollectionResource.java
deleted file mode 100644
index 610c054..0000000
--- a/solr/core/src/java/org/apache/solr/rest/schema/CopyFieldCollectionResource.java
+++ /dev/null
@@ -1,198 +0,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.
- */
-package org.apache.solr.rest.schema;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.rest.GETable;
-import org.apache.solr.rest.POSTable;
-import org.apache.solr.schema.IndexSchema;
-import org.apache.solr.schema.ManagedIndexSchema;
-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.lang.invoke.MethodHandles;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import static org.apache.solr.common.SolrException.ErrorCode;
-
-/**
- * This class responds to requests at /solr/(corename)/schema/copyfields
- * <p>
- *
- * To restrict the set of copyFields in the response, specify one or both
- * of the following as query parameters, with values as space and/or comma
- * separated dynamic or explicit field names:
- *
- * <ul>
- *   <li>dest.fl: include copyFields that have one of these as a destination</li>
- *   <li>source.fl: include copyFields that have one of these as a source</li>
- * </ul>
- *
- * If both dest.fl and source.fl are given as query parameters, the copyfields
- * in the response will be restricted to those that match any of the destinations
- * in dest.fl and also match any of the sources in source.fl.
- */
-public class CopyFieldCollectionResource extends BaseFieldResource implements GETable, POSTable {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private static final String SOURCE_FIELD_LIST = IndexSchema.SOURCE + "." + CommonParams.FL;
-  private static final String DESTINATION_FIELD_LIST = IndexSchema.DESTINATION + "." + CommonParams.FL;
-
-  private Set<String> requestedSourceFields;
-  private Set<String> requestedDestinationFields;
-
-  public CopyFieldCollectionResource() {
-    super();
-  }
-
-  @Override
-  public void doInit() throws ResourceException {
-    super.doInit();
-    if (isExisting()) {
-      String sourceFieldListParam = getSolrRequest().getParams().get(SOURCE_FIELD_LIST);
-      if (null != sourceFieldListParam) {
-        String[] fields = sourceFieldListParam.trim().split("[,\\s]+");
-        if (fields.length > 0) {
-          requestedSourceFields = new HashSet<>(Arrays.asList(fields));
-          requestedSourceFields.remove(""); // Remove empty values, if any
-        }
-      }
-      String destinationFieldListParam = getSolrRequest().getParams().get(DESTINATION_FIELD_LIST);
-      if (null != destinationFieldListParam) {
-        String[] fields = destinationFieldListParam.trim().split("[,\\s]+");
-        if (fields.length > 0) {
-          requestedDestinationFields = new HashSet<>(Arrays.asList(fields));
-          requestedDestinationFields.remove(""); // Remove empty values, if any
-        }
-      }
-    }
-  }
-
-  @Override
-  public Representation get() {
-    try {
-      getSolrResponse().add(IndexSchema.COPY_FIELDS,
-          getSchema().getCopyFieldProperties(true, requestedSourceFields, requestedDestinationFields));
-    } catch (Exception e) {
-      getSolrResponse().setException(e);
-    }
-    handlePostExecution(log);
-
-    return new SolrOutputRepresentation();
-  }
-
-  @Override
-  public Representation post(Representation entity) throws ResourceException {
-    try {
-      if (!getSchema().isMutable()) {
-        final String message = "This IndexSchema is not mutable.";
-        throw new SolrException(ErrorCode.BAD_REQUEST, message);
-      } else {
-        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): [{\"source\":\"foo\",\"dest\":\"comma-separated list of targets\"}, {...}, ...]";
-            log.error(message);
-            throw new SolrException(ErrorCode.BAD_REQUEST, message);
-          } else {
-            List<Map<String, Object>> list = (List<Map<String, Object>>) object;
-            Map<String, Collection<String>> fieldsToCopy = new HashMap<>();
-            ManagedIndexSchema oldSchema = (ManagedIndexSchema) getSchema();
-            Set<String> malformed = new HashSet<>();
-            for (Map<String,Object> map : list) {
-              String fieldName = (String)map.get(IndexSchema.SOURCE);
-              if (null == fieldName) {
-                String message = "Missing '" + IndexSchema.SOURCE + "' mapping.";
-                log.error(message);
-                throw new SolrException(ErrorCode.BAD_REQUEST, message);
-              }
-              Object dest = map.get(IndexSchema.DESTINATION);
-              List<String> destinations = null;
-              if (dest != null) {
-                if (dest instanceof List){
-                  destinations = (List<String>)dest;
-                } else if (dest instanceof String){
-                  destinations = Collections.singletonList(dest.toString());
-                } else {
-                  String message = "Invalid '" + IndexSchema.DESTINATION + "' type.";
-                  log.error(message);
-                  throw new SolrException(ErrorCode.BAD_REQUEST, message);
-                }
-              }
-              if (destinations == null) {
-                malformed.add(fieldName);
-              } else {
-                fieldsToCopy.put(fieldName, destinations);
-              }
-            }
-            if (malformed.size() > 0){
-              StringBuilder message = new StringBuilder("Malformed destination(s) for: ");
-              for (String s : malformed) {
-                message.append(s).append(", ");
-              }
-              if (message.length() > 2) {
-                message.setLength(message.length() - 2);//drop the last ,
-              }
-              log.error(message.toString().trim());
-              throw new SolrException(ErrorCode.BAD_REQUEST, message.toString().trim());
-            }
-            IndexSchema newSchema = null;
-            boolean success = false;
-            while (!success) {
-              try {
-                synchronized (oldSchema.getSchemaUpdateLock()) {
-                  newSchema = oldSchema.addCopyFields(fieldsToCopy,true);
-                  if (null != newSchema) {
-                    getSolrCore().setLatestSchema(newSchema);
-                    success = true;
-                  } else {
-                    throw new SolrException(ErrorCode.SERVER_ERROR, "Failed to add fields.");
-                  }
-                }
-              } catch (ManagedIndexSchema.SchemaChangedInZkException e) {
-                  log.debug("Schema changed while processing request, retrying");
-                  oldSchema = (ManagedIndexSchema)getSolrCore().getLatestSchema();
-              }
-            }
-            waitForSchemaUpdateToPropagate(newSchema);
-          }
-        }
-      }
-    } catch (Exception e) {
-      getSolrResponse().setException(e);
-    }
-    handlePostExecution(log);
-    return new SolrOutputRepresentation();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/java/org/apache/solr/rest/schema/DynamicFieldCollectionResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/DynamicFieldCollectionResource.java b/solr/core/src/java/org/apache/solr/rest/schema/DynamicFieldCollectionResource.java
deleted file mode 100644
index bf94234..0000000
--- a/solr/core/src/java/org/apache/solr/rest/schema/DynamicFieldCollectionResource.java
+++ /dev/null
@@ -1,207 +0,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.
- */
-package org.apache.solr.rest.schema;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
-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.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.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * This class responds to requests at /solr/(corename)/schema/dynamicfields
- * <p>
- * To restrict the set of dynamic fields in the response, specify a comma
- * and/or space separated list of dynamic field patterns in the "fl" query
- * parameter. 
- */
-public class DynamicFieldCollectionResource extends BaseFieldResource implements GETable, POSTable {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  public DynamicFieldCollectionResource() {
-    super();
-  }
-
-  @Override
-  public void doInit() throws ResourceException {
-    super.doInit();
-  }
-
-  @Override
-  public Representation get() {
-    
-    try {
-      List<SimpleOrderedMap<Object>> props = new ArrayList<>();
-      if (null == getRequestedFields()) {
-        for (IndexSchema.DynamicField dynamicField : getSchema().getDynamicFields()) {
-          if ( ! dynamicField.getRegex().startsWith(IndexSchema.INTERNAL_POLY_FIELD_PREFIX)) { // omit internal polyfields
-            props.add(getFieldProperties(dynamicField.getPrototype()));
-          }
-        }
-      } else {
-        if (0 == getRequestedFields().size()) {
-          String message = "Empty " + CommonParams.FL + " parameter value";
-          throw new SolrException(ErrorCode.BAD_REQUEST, message);
-        }
-        Map<String,SchemaField> dynamicFieldsByName = new HashMap<>();
-        for (IndexSchema.DynamicField dynamicField : getSchema().getDynamicFields()) {
-          dynamicFieldsByName.put(dynamicField.getRegex(), dynamicField.getPrototype());
-        }
-        // Use the same order as the fl parameter
-        for (String dynamicFieldName : getRequestedFields()) {
-          final SchemaField dynamicSchemaField = dynamicFieldsByName.get(dynamicFieldName);
-          if (null == dynamicSchemaField) {
-            log.info("Requested dynamic field '" + dynamicFieldName + "' not found.");
-          } else {
-            props.add(getFieldProperties(dynamicSchemaField));
-          }
-        }
-      }
-      getSolrResponse().add(IndexSchema.DYNAMIC_FIELDS, props);
-    } catch (Exception e) {
-      getSolrResponse().setException(e);
-    }
-    handlePostExecution(log);
-
-    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> newDynamicFields = new ArrayList<>();
-            List<NewFieldArguments> newDynamicFieldArguments = new ArrayList<>();
-            ManagedIndexSchema oldSchema = (ManagedIndexSchema)getSchema();
-            Map<String,Collection<String>> copyFields = new HashMap<>();
-            for (Map<String,Object> map : list) {
-              String fieldNamePattern = (String)map.remove(IndexSchema.NAME);
-              if (null == fieldNamePattern) {
-                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);
-              }
-              // copyFields:"comma separated list of destination fields"
-              Object copies = map.get(IndexSchema.COPY_FIELDS);
-              List<String> copyTo = null;
-              if (copies != null) {
-                if (copies instanceof List){
-                  copyTo = (List<String>)copies;
-                } else if (copies instanceof String){
-                  copyTo = Collections.singletonList(copies.toString());
-                } else {
-                  String message = "Invalid '" + IndexSchema.COPY_FIELDS + "' type.";
-                  log.error(message);
-                  throw new SolrException(ErrorCode.BAD_REQUEST, message);
-                }
-              }
-              if (copyTo != null) {
-                map.remove(IndexSchema.COPY_FIELDS);
-                copyFields.put(fieldNamePattern, copyTo);
-              }
-              newDynamicFields.add(oldSchema.newDynamicField(fieldNamePattern, fieldType, map));
-              newDynamicFieldArguments.add(new NewFieldArguments(fieldNamePattern, fieldType, map));
-            }
-            IndexSchema newSchema = null;
-            boolean firstAttempt = true;
-            boolean success = false;
-            while ( ! success) {
-              try {
-                if ( ! firstAttempt) {
-                  // If this isn't the first attempt, we must have failed due to
-                  // the schema changing in Zk during optimistic concurrency control.
-                  // In that case, rerun creating the new fields, because they may
-                  // fail now due to changes in the schema.  This behavior is consistent
-                  // with what would happen if we locked the schema and the other schema
-                  // change went first.
-                  newDynamicFields.clear();
-                  for (NewFieldArguments args : newDynamicFieldArguments) {
-                    newDynamicFields.add(oldSchema.newDynamicField(args.getName(), args.getType(), args.getMap()));
-                  }
-                }
-                firstAttempt = false;
-                synchronized (oldSchema.getSchemaUpdateLock()) {
-                  newSchema = oldSchema.addDynamicFields(newDynamicFields, copyFields, true);
-                  if (null != newSchema) {
-                    getSolrCore().setLatestSchema(newSchema);
-                    success = true;
-                  } else {
-                    throw new SolrException(ErrorCode.SERVER_ERROR, "Failed to add dynamic fields.");
-                  }
-                }
-              } catch (ManagedIndexSchema.SchemaChangedInZkException e) {
-                log.debug("Schema changed while processing request, retrying");
-                oldSchema = (ManagedIndexSchema)getSolrCore().getLatestSchema();
-              }
-            }
-
-            waitForSchemaUpdateToPropagate(newSchema);
-
-          }
-        }
-      }
-    } catch (Exception e) {
-      getSolrResponse().setException(e);
-    }
-    handlePostExecution(log);
-
-    return new SolrOutputRepresentation();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/java/org/apache/solr/rest/schema/DynamicFieldResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/DynamicFieldResource.java b/solr/core/src/java/org/apache/solr/rest/schema/DynamicFieldResource.java
deleted file mode 100644
index bf67608..0000000
--- a/solr/core/src/java/org/apache/solr/rest/schema/DynamicFieldResource.java
+++ /dev/null
@@ -1,197 +0,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.
- */
-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.lang.invoke.MethodHandles;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import static java.util.Collections.singletonList;
-import static java.util.Collections.singletonMap;
-
-/**
- * This class responds to requests at /solr/(corename)/schema/dynamicfields/(pattern)
- * where pattern is a field name pattern (with an asterisk at the beginning or the end).
- */
-public class DynamicFieldResource extends BaseFieldResource implements GETable, PUTable {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  private String fieldNamePattern;
-
-  public DynamicFieldResource() {
-    super();
-  }
-
-  /**
-   * Gets the field name pattern from the request attribute where it's stored by Restlet. 
-   */
-  @Override
-  public void doInit() throws ResourceException {
-    super.doInit();
-    if (isExisting()) {
-      fieldNamePattern = (String)getRequestAttributes().get(IndexSchema.NAME);
-      try {
-        fieldNamePattern = null == fieldNamePattern ? "" : urlDecode(fieldNamePattern.trim()).trim();
-      } catch (UnsupportedEncodingException e) {
-        throw new ResourceException(e);
-      }
-    }
-  }
-
-  @Override
-  public Representation get() {
-    try {
-      if (fieldNamePattern.isEmpty()) {
-        final String message = "Dynamic field name is missing";
-        throw new SolrException(ErrorCode.BAD_REQUEST, message);
-      } else {
-        SchemaField field = null;
-        for (SchemaField prototype : getSchema().getDynamicFieldPrototypes()) {
-          if (prototype.getName().equals(fieldNamePattern)) {
-            field = prototype;
-            break;
-          }
-        }
-        if (null == field) {
-          final String message = "Dynamic field '" + fieldNamePattern + "' not found.";
-          throw new SolrException(ErrorCode.NOT_FOUND, message);
-        } else {
-          getSolrResponse().add(IndexSchema.DYNAMIC_FIELD, getFieldProperties(field));
-        }
-      }
-    } catch (Exception e) {
-      getSolrResponse().setException(e);
-    }
-    handlePostExecution(log);
-
-    return new SolrOutputRepresentation();
-  }
-
-  /**
-   * Accepts JSON add dynamic field request
-   */
-  @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/dynamicfields/(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.DYNAMIC_FIELD)) {
-              map = (Map<String,Object>)map.get(IndexSchema.DYNAMIC_FIELD);
-            }
-            String bodyFieldName;
-            if (null != (bodyFieldName = (String)map.remove(IndexSchema.NAME))
-                && ! fieldNamePattern.equals(bodyFieldName)) {
-              String message = "Dynamic field name in the request body '" + bodyFieldName
-                  + "' doesn't match dynamic field name in the request URL '" + fieldNamePattern + "'";
-              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();
-                Object copies = map.get(IndexSchema.COPY_FIELDS);
-                Collection<String> copyFieldNames = null;
-                if (copies != null) {
-                  if (copies instanceof List) {
-                    copyFieldNames = (List<String>)copies;
-                  } else if (copies instanceof String) {
-                    copyFieldNames = singletonList(copies.toString());
-                  } else {
-                    String message = "Invalid '" + IndexSchema.COPY_FIELDS + "' type.";
-                    log.error(message);
-                    throw new SolrException(ErrorCode.BAD_REQUEST, message);
-                  }
-                }
-                if (copyFieldNames != null) {
-                  map.remove(IndexSchema.COPY_FIELDS);
-                }
-                IndexSchema newSchema = null;
-                boolean success = false;
-                while ( ! success) {
-                  try {
-                    SchemaField newDynamicField = oldSchema.newDynamicField(fieldNamePattern, fieldType, map);
-                    synchronized (oldSchema.getSchemaUpdateLock()) {
-                      newSchema = oldSchema.addDynamicFields(singletonList(newDynamicField), singletonMap(newDynamicField.getName(), copyFieldNames), true);
-                      if (null != newSchema) {
-                        getSolrCore().setLatestSchema(newSchema);
-                        success = true;
-                      } else {
-                        throw new SolrException(ErrorCode.SERVER_ERROR, "Failed to add dynamic field.");
-                      }
-                    }
-                  } catch (ManagedIndexSchema.SchemaChangedInZkException e) {
-                    log.debug("Schema changed while processing request, retrying");
-                    oldSchema = (ManagedIndexSchema)getSolrCore().getLatestSchema();
-                  }
-                }
-                // if in cloud mode, wait for schema updates to propagate to all replicas
-                waitForSchemaUpdateToPropagate(newSchema);
-              }
-            }
-          }
-        }
-      }
-    } catch (Exception e) {
-      getSolrResponse().setException(e);
-    }
-    handlePostExecution(log);
-
-    return new SolrOutputRepresentation();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/java/org/apache/solr/rest/schema/FieldCollectionResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/FieldCollectionResource.java b/solr/core/src/java/org/apache/solr/rest/schema/FieldCollectionResource.java
deleted file mode 100644
index f1bf6a4..0000000
--- a/solr/core/src/java/org/apache/solr/rest/schema/FieldCollectionResource.java
+++ /dev/null
@@ -1,225 +0,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.
- */
-package org.apache.solr.rest.schema;
-import org.apache.solr.cloud.ZkSolrResourceLoader;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.core.CoreDescriptor;
-import org.apache.solr.rest.GETable;
-import org.apache.solr.rest.POSTable;
-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.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-/**
- * This class responds to requests at /solr/(corename)/schema/fields
- * <p>
- * Two query parameters are supported:
- * <ul>
- * <li>
- * "fl": a comma- and/or space-separated list of fields to send properties
- * for in the response, rather than the default: all of them.
- * </li>
- * <li>
- * "includeDynamic": if the "fl" parameter is specified, matching dynamic
- * fields are included in the response and identified with the "dynamicBase"
- * property.  If the "fl" parameter is not specified, the "includeDynamic"
- * query parameter is ignored.
- * </li>
- * </ul>
- */
-public class FieldCollectionResource extends BaseFieldResource implements GETable, POSTable {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private boolean includeDynamic;
-
-  public FieldCollectionResource() {
-    super();
-  }
-
-  @Override
-  public void doInit() throws ResourceException {
-    super.doInit();
-    if (isExisting()) {
-      includeDynamic = getSolrRequest().getParams().getBool(INCLUDE_DYNAMIC_PARAM, false);
-    }
-  }
-
-  @Override
-  public Representation get() {
-    try {
-      final List<SimpleOrderedMap<Object>> props = new ArrayList<>();
-      if (null == getRequestedFields()) {
-        SortedSet<String> fieldNames = new TreeSet<>(getSchema().getFields().keySet());
-        for (String fieldName : fieldNames) {
-          props.add(getFieldProperties(getSchema().getFields().get(fieldName)));
-        }
-      } else {
-        if (0 == getRequestedFields().size()) {
-          String message = "Empty " + CommonParams.FL + " parameter value";
-          throw new SolrException(ErrorCode.BAD_REQUEST, message);
-        }
-        // Use the same order as the fl parameter
-        for (String fieldName : getRequestedFields()) {
-          final SchemaField field;
-          if (includeDynamic) {
-            field = getSchema().getFieldOrNull(fieldName);
-          } else {
-            field = getSchema().getFields().get(fieldName);
-          }
-          if (null == field) {
-            log.info("Requested field '" + fieldName + "' not found.");
-          } else {
-            props.add(getFieldProperties(field));
-          }
-        }
-      }
-      getSolrResponse().add(IndexSchema.FIELDS, props);
-    } catch (Exception e) {
-      getSolrResponse().setException(e);
-    }
-    handlePostExecution(log);
-
-    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<>();
-            List<NewFieldArguments> newFieldArguments = new ArrayList<>();
-            IndexSchema oldSchema = getSchema();
-            Map<String, Collection<String>> copyFields = new HashMap<>();
-            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);
-              }
-              // copyFields:"comma separated list of destination fields"
-              Object copies = map.get(IndexSchema.COPY_FIELDS);
-              List<String> copyTo = null;
-              if (copies != null) {
-                if (copies instanceof List){
-                  copyTo = (List<String>) copies;
-                } else if (copies instanceof String){
-                  copyTo = Collections.singletonList(copies.toString());
-                } else {
-                  String message = "Invalid '" + IndexSchema.COPY_FIELDS + "' type.";
-                  log.error(message);
-                  throw new SolrException(ErrorCode.BAD_REQUEST, message);
-                }
-              }
-              if (copyTo != null) {
-                map.remove(IndexSchema.COPY_FIELDS);
-                copyFields.put(fieldName, copyTo);
-              }
-              newFields.add(oldSchema.newField(fieldName, fieldType, map));
-              newFieldArguments.add(new NewFieldArguments(fieldName, fieldType, map));
-            }
-            IndexSchema newSchema = null;
-            boolean firstAttempt = true;
-            boolean success = false;
-            while (!success) {
-              try {
-                if (!firstAttempt) {
-                  // If this isn't the first attempt, we must have failed due to
-                  // the schema changing in Zk during optimistic concurrency control.
-                  // In that case, rerun creating the new fields, because they may
-                  // fail now due to changes in the schema.  This behavior is consistent
-                  // with what would happen if we locked the schema and the other schema
-                  // change went first.
-                  newFields.clear();
-                  for (NewFieldArguments args : newFieldArguments) {
-                    newFields.add(oldSchema.newField(
-                      args.getName(), args.getType(), args.getMap()));
-                  }
-                }
-                firstAttempt = false;
-                synchronized (oldSchema.getSchemaUpdateLock()) {
-                  newSchema = oldSchema.addFields(newFields, copyFields, true);
-                  if (null != newSchema) {
-                    getSolrCore().setLatestSchema(newSchema);
-                    success = true;
-                  } else {
-                    throw new SolrException(ErrorCode.SERVER_ERROR, "Failed to add fields.");
-                  }
-                }
-              } catch (ManagedIndexSchema.SchemaChangedInZkException e) {
-                log.debug("Schema changed while processing request, retrying");
-                oldSchema = getSolrCore().getLatestSchema();
-              }
-            }
-            waitForSchemaUpdateToPropagate(newSchema);
-          }
-        }
-      }
-    } catch (Exception e) {
-      getSolrResponse().setException(e);
-    }
-    handlePostExecution(log);
-
-    return new SolrOutputRepresentation();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/java/org/apache/solr/rest/schema/FieldResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/FieldResource.java b/solr/core/src/java/org/apache/solr/rest/schema/FieldResource.java
deleted file mode 100644
index 2634bbd..0000000
--- a/solr/core/src/java/org/apache/solr/rest/schema/FieldResource.java
+++ /dev/null
@@ -1,201 +0,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.
- */
-package org.apache.solr.rest.schema;
-import org.apache.solr.cloud.ZkSolrResourceLoader;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.core.CoreDescriptor;
-import org.apache.solr.core.SolrResourceLoader;
-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.lang.invoke.MethodHandles;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-
-/**
- * This class responds to requests at /solr/(corename)/schema/fields/(fieldname)
- * where "fieldname" is the name of a field.
- * <p>
- * The GET method returns properties for the given fieldname.
- * 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, PUTable {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  private boolean includeDynamic;
-  private String fieldName;
-
-  public FieldResource() {
-    super();
-  }
-
-  @Override
-  public void doInit() throws ResourceException {
-    super.doInit();
-    if (isExisting()) {
-      includeDynamic = getSolrRequest().getParams().getBool(INCLUDE_DYNAMIC_PARAM, false);
-      fieldName = (String) getRequestAttributes().get(IndexSchema.NAME);
-      try {
-        fieldName = null == fieldName ? "" : urlDecode(fieldName.trim()).trim();
-      } catch (UnsupportedEncodingException e) {
-        throw new ResourceException(e);
-      }
-    }
-  }
-
-  @Override
-  public Representation get() {
-    try {
-      if (fieldName.isEmpty()) {
-        final String message = "Field name is missing";
-        throw new SolrException(ErrorCode.BAD_REQUEST, message);
-      } else {
-        final SchemaField field;
-        if (includeDynamic) {
-          field = getSchema().getFieldOrNull(fieldName);
-        } else {
-          field = getSchema().getFields().get(fieldName);
-        }
-        if (null == field) {
-          final String message = "Field '" + fieldName + "' not found.";
-          throw new SolrException(ErrorCode.NOT_FOUND, message);
-        } else {
-          getSolrResponse().add(IndexSchema.FIELD, getFieldProperties(field));
-        }
-      }
-    } catch (Exception e) {
-      getSolrResponse().setException(e);
-    }
-    handlePostExecution(log);
-
-    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();
-                Object copies = map.get(IndexSchema.COPY_FIELDS);
-                List<String> copyFieldNames = null;
-                if (copies != null) {
-                  if (copies instanceof List) {
-                    copyFieldNames = (List<String>) copies;
-                  } else if (copies instanceof String) {
-                    copyFieldNames = Collections.singletonList(copies.toString());
-                  } else {
-                    String message = "Invalid '" + IndexSchema.COPY_FIELDS + "' type.";
-                    log.error(message);
-                    throw new SolrException(ErrorCode.BAD_REQUEST, message);
-                  }
-                }
-                if (copyFieldNames != null) {
-                  map.remove(IndexSchema.COPY_FIELDS);
-                }
-
-                IndexSchema newSchema = null;
-                boolean success = false;
-                while (!success) {
-                  try {
-                    SchemaField newField = oldSchema.newField(fieldName, fieldType, map);
-                    synchronized (oldSchema.getSchemaUpdateLock()) {
-                      newSchema = oldSchema.addField(newField, copyFieldNames);
-                      if (null != newSchema) {
-                        getSolrCore().setLatestSchema(newSchema);
-                        success = true;
-                      } else {
-                        throw new SolrException(ErrorCode.SERVER_ERROR, "Failed to add field.");
-                      }
-                    }
-                  } catch (ManagedIndexSchema.SchemaChangedInZkException e) {
-                    log.debug("Schema changed while processing request, retrying");
-                    oldSchema = (ManagedIndexSchema)getSolrCore().getLatestSchema();
-                  }
-                }
-                waitForSchemaUpdateToPropagate(newSchema);
-              }
-            }
-          }
-        }
-      }
-    } catch (Exception e) {
-      getSolrResponse().setException(e);
-    }
-    handlePostExecution(log);
-
-    return new SolrOutputRepresentation();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeCollectionResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeCollectionResource.java b/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeCollectionResource.java
deleted file mode 100644
index d2eb1bd..0000000
--- a/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeCollectionResource.java
+++ /dev/null
@@ -1,197 +0,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.
- */
-package org.apache.solr.rest.schema;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.rest.GETable;
-import org.apache.solr.rest.POSTable;
-import org.apache.solr.schema.FieldType;
-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.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-/**
- * This class responds to requests at /solr/(corename)/schema/fieldtypes
- * 
- * The GET method returns properties for all field types defined in the schema.
- */
-public class FieldTypeCollectionResource extends BaseFieldTypeResource implements GETable, POSTable {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  
-  private Map<String,List<String>> fieldsByFieldType;
-  private Map<String,List<String>> dynamicFieldsByFieldType;
-
-  public FieldTypeCollectionResource() {
-    super();
-  }
-
-  @Override
-  public void doInit() throws ResourceException {
-    super.doInit();
-    if (isExisting()) {
-      fieldsByFieldType = getFieldsByFieldType();
-      dynamicFieldsByFieldType = getDynamicFieldsByFieldType();
-    }
-  }
-  
-  @Override
-  public Representation get() {
-    try {
-      List<SimpleOrderedMap<Object>> props = new ArrayList<>();
-      Map<String,FieldType> sortedFieldTypes = new TreeMap<>(getSchema().getFieldTypes());
-      for (FieldType fieldType : sortedFieldTypes.values()) {
-        props.add(getFieldTypeProperties(fieldType));
-      }
-      getSolrResponse().add(IndexSchema.FIELD_TYPES, props);
-    } catch (Exception e) {
-      getSolrResponse().setException(e);
-    }
-    handlePostExecution(log);
-
-    return new SolrOutputRepresentation();
-  }
-
-  /** Returns field lists from the map constructed in doInit() */
-  @Override
-  protected List<String> getFieldsWithFieldType(FieldType fieldType) {
-    List<String> fields = fieldsByFieldType.get(fieldType.getTypeName());
-    if (null == fields) {
-      fields = Collections.emptyList();
-    }
-    return fields;
-  }
-
-  /** Returns dynamic field lists from the map constructed in doInit() */
-  @Override
-  protected List<String> getDynamicFieldsWithFieldType(FieldType fieldType) {
-    List<String> dynamicFields = dynamicFieldsByFieldType.get(fieldType.getTypeName());
-    if (null == dynamicFields) {
-      dynamicFields = Collections.emptyList();
-    }
-    return dynamicFields;
-  }
-
-  /**
-   * Returns a map from field type names to a sorted list of fields that use the field type.
-   * The map only includes field types that are used by at least one field.  
-   */
-  private Map<String,List<String>> getFieldsByFieldType() {
-    Map<String,List<String>> fieldsByFieldType = new HashMap<>();
-    for (SchemaField schemaField : getSchema().getFields().values()) {
-      final String fieldType = schemaField.getType().getTypeName();
-      List<String> fields = fieldsByFieldType.get(fieldType);
-      if (null == fields) {
-        fields = new ArrayList<>();
-        fieldsByFieldType.put(fieldType, fields);
-      }
-      fields.add(schemaField.getName());
-    }
-    for (List<String> fields : fieldsByFieldType.values()) {
-      Collections.sort(fields);
-    }
-    return fieldsByFieldType;
-  }
-
-  /**
-   * Returns a map from field type names to a list of dynamic fields that use the field type.
-   * The map only includes field types that are used by at least one dynamic field.  
-   */
-  private Map<String,List<String>> getDynamicFieldsByFieldType() {
-    Map<String,List<String>> dynamicFieldsByFieldType = new HashMap<>();
-    for (SchemaField schemaField : getSchema().getDynamicFieldPrototypes()) {
-      final String fieldType = schemaField.getType().getTypeName();
-      List<String> dynamicFields = dynamicFieldsByFieldType.get(fieldType);
-      if (null == dynamicFields) {
-        dynamicFields = new ArrayList<>();
-        dynamicFieldsByFieldType.put(fieldType, dynamicFields);
-      }
-      dynamicFields.add(schemaField.getName());
-    }
-    return dynamicFieldsByFieldType;
-  }
-  
-  @SuppressWarnings("unchecked")
-  @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);
-      }
-      
-      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);
-      }
-      
-      Object object = ObjectBuilder.fromJSON(entity.getText());
-      if (!(object instanceof List)) {
-        String message = "Invalid JSON type " + object.getClass().getName() 
-            + ", expected List of field type definitions in the form of"
-            + " (ignore the backslashes): [{\"name\":\"text_general\",\"class\":\"solr.TextField\", ...}, {...}, ...]";
-        log.error(message);
-        throw new SolrException(ErrorCode.BAD_REQUEST, message);
-      }
-      
-      List<Map<String, Object>> fieldTypeList = (List<Map<String, Object>>) object;
-      if (fieldTypeList.size() > 0)
-        addOrUpdateFieldTypes(fieldTypeList);
-    } catch (Exception e) {
-      getSolrResponse().setException(e);
-    }
-    handlePostExecution(log);
-
-    return new SolrOutputRepresentation();
-  }  
-  
-  @SuppressWarnings("unchecked")
-  protected void addOrUpdateFieldTypes(List<Map<String, Object>> fieldTypeList) throws Exception {
-    List<FieldType> newFieldTypes = new ArrayList<>(fieldTypeList.size());
-    ManagedIndexSchema oldSchema = (ManagedIndexSchema) getSchema();
-    for (Map<String,Object> fieldTypeJson : fieldTypeList) {
-      if (1 == fieldTypeJson.size() && fieldTypeJson.containsKey(IndexSchema.FIELD_TYPE)) {
-        fieldTypeJson = (Map<String, Object>) fieldTypeJson.get(IndexSchema.FIELD_TYPE);
-      }
-      FieldType newFieldType = 
-          FieldTypeResource.buildFieldTypeFromJson(oldSchema, 
-              (String)fieldTypeJson.get(IndexSchema.NAME), fieldTypeJson);
-      newFieldTypes.add(newFieldType);
-    }
-    // now deploy the added types (all or nothing)
-    addNewFieldTypes(newFieldTypes, oldSchema);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeResource.java b/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeResource.java
deleted file mode 100644
index 361c8c2..0000000
--- a/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeResource.java
+++ /dev/null
@@ -1,203 +0,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.
- */
-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.FieldType;
-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.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-/**
- * This class responds to requests at /solr/(corename)/schema/fieldtype/(typename)
- * where "typename" is the name of a field type in the schema.
- * 
- * The GET method returns properties for the named field type.
- */
-public class FieldTypeResource extends BaseFieldTypeResource implements GETable, PUTable {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  private String typeName;
-
-  public FieldTypeResource() {
-    super();
-  }
-
-  @Override
-  public void doInit() throws ResourceException {
-    super.doInit();
-    if (isExisting()) {
-      typeName = (String)getRequestAttributes().get(IndexSchema.NAME);
-      try {
-        typeName = null == typeName ? "" : urlDecode(typeName.trim()).trim();
-      } catch (UnsupportedEncodingException e) {
-        throw new ResourceException(e);
-      }
-    }
-  }
-
-  @Override
-  public Representation get() {
-    try {
-      if (typeName.isEmpty()) {
-        final String message = "Field type name is missing";
-        throw new SolrException(ErrorCode.BAD_REQUEST, message);
-      } else {
-        FieldType fieldType = getSchema().getFieldTypes().get(typeName);
-        if (null == fieldType) {
-          final String message = "Field type '" + typeName + "' not found.";
-          throw new SolrException(ErrorCode.NOT_FOUND, message);
-        }
-        getSolrResponse().add(IndexSchema.FIELD_TYPE, getFieldTypeProperties(fieldType));
-      }
-    } catch (Exception e) {
-      getSolrResponse().setException(e);
-    }
-    handlePostExecution(log);
-
-    return new SolrOutputRepresentation();
-  }
-
-  /** 
-   * Returns a field list using the given field type by iterating over all fields
-   * defined in the schema.
-   */
-  @Override
-  protected List<String> getFieldsWithFieldType(FieldType fieldType) {
-    List<String> fields = new ArrayList<>();
-    for (SchemaField schemaField : getSchema().getFields().values()) {
-      if (schemaField.getType().getTypeName().equals(fieldType.getTypeName())) {
-        fields.add(schemaField.getName());
-      }
-    }
-    Collections.sort(fields);
-    return fields;
-  }
-
-  /**
-   * Returns a dynamic field list using the given field type by iterating over all
-   * dynamic fields defined in the schema. 
-   */
-  @Override
-  protected List<String> getDynamicFieldsWithFieldType(FieldType fieldType) {
-    List<String> dynamicFields = new ArrayList<>();
-    for (SchemaField prototype : getSchema().getDynamicFieldPrototypes()) {
-      if (prototype.getType().getTypeName().equals(fieldType.getTypeName())) {
-        dynamicFields.add(prototype.getName());
-      }
-    }
-    return dynamicFields; // Don't sort these - they're matched in order
-  }
-  
-  /**
-   * Accepts JSON add fieldtype request, to URL
-   */
-  @SuppressWarnings("unchecked")
-  @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);
-      }
-      
-      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);
-      }
-      
-      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): {\"name\":\"text_general\", \"class\":\"solr.TextField\" ...},"
-            + " either with or without a \"name\" mapping.  If the \"name\" is specified, it must match the"
-            + " name given in the request URL: /schema/fieldtypes/(name)";
-        log.error(message);
-        throw new SolrException(ErrorCode.BAD_REQUEST, message);
-      }
-      
-      // basic validation passed, let's try to create it!
-      addOrUpdateFieldType((Map<String, Object>)object);
-      
-    } catch (Exception e) {
-      getSolrResponse().setException(e);
-    }
-    handlePostExecution(log);
-
-    return new SolrOutputRepresentation();
-  }
-  
-  protected void addOrUpdateFieldType(Map<String,Object> fieldTypeJson) {
-    ManagedIndexSchema oldSchema = (ManagedIndexSchema) getSchema();
-    FieldType newFieldType = buildFieldTypeFromJson(oldSchema, typeName, fieldTypeJson);
-    addNewFieldTypes(Collections.singletonList(newFieldType), oldSchema);
-  }
-
-  /**
-   * Builds a FieldType definition from a JSON object.
-   */
-  @SuppressWarnings("unchecked")
-  static FieldType buildFieldTypeFromJson(ManagedIndexSchema oldSchema, String fieldTypeName, Map<String,Object> fieldTypeJson) {
-    if (1 == fieldTypeJson.size() && fieldTypeJson.containsKey(IndexSchema.FIELD_TYPE)) {
-      fieldTypeJson = (Map<String, Object>)fieldTypeJson.get(IndexSchema.FIELD_TYPE);
-    }
-
-    String bodyTypeName = (String) fieldTypeJson.get(IndexSchema.NAME);
-    if (bodyTypeName == null) {
-      // must provide the name in the JSON for converting to the XML format needed
-      // to create FieldType objects using the FieldTypePluginLoader
-      fieldTypeJson.put(IndexSchema.NAME, fieldTypeName);
-    } else {
-      // if they provide it in the JSON, then it must match the value from the path
-      if (!fieldTypeName.equals(bodyTypeName)) {
-        String message = "Field type name in the request body '" + bodyTypeName
-            + "' doesn't match field type name in the request URL '" + fieldTypeName + "'";
-        log.error(message);
-        throw new SolrException(ErrorCode.BAD_REQUEST, message);
-      }
-    }
-
-    String className = (String)fieldTypeJson.get(FieldType.CLASS_NAME);
-    if (className == null) {
-      String message = "Missing required '" + FieldType.CLASS_NAME + "' property!";
-      log.error(message);
-      throw new SolrException(ErrorCode.BAD_REQUEST, message);
-    }
-
-    return oldSchema.newFieldType(fieldTypeName, className, fieldTypeJson);
-  }
-}