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:33 UTC

[1/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

Repository: lucene-solr
Updated Branches:
  refs/heads/master 9617d3d40 -> b87e39209


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/TestRemoveLastDynamicCopyField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestRemoveLastDynamicCopyField.java b/solr/core/src/test/org/apache/solr/rest/schema/TestRemoveLastDynamicCopyField.java
deleted file mode 100644
index 4b4ddd3..0000000
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestRemoveLastDynamicCopyField.java
+++ /dev/null
@@ -1,80 +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 java.io.File;
-import java.io.StringReader;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.solr.util.RestTestBase;
-import org.junit.After;
-import org.junit.Before;
-import org.noggit.JSONParser;
-import org.noggit.ObjectBuilder;
-
-public class TestRemoveLastDynamicCopyField extends RestTestBase {
-  private static File tmpSolrHome;
-
-  @Before
-  public void before() throws Exception {
-    tmpSolrHome = createTempDir().toFile();
-    FileUtils.copyDirectory(new File(TEST_HOME()), tmpSolrHome.getAbsoluteFile());
-
-    System.setProperty("managed.schema.mutable", "true");
-    System.setProperty("enable.update.log", "false");
-
-    createJettyAndHarness(tmpSolrHome.getAbsolutePath(), "solrconfig-managed-schema.xml", "schema-single-dynamic-copy-field.xml",
-        "/solr", true, null);
-  }
-
-  @After
-  public void after() throws Exception {
-    if (jetty != null) {
-      jetty.stop();
-      jetty = null;
-    }
-    client = null;
-    if (restTestHarness != null) {
-      restTestHarness.close();
-    }
-    restTestHarness = null;
-  }
-
-  public void test() throws Exception {
-    List copyFields = getCopyFields();
-    assertEquals("There is more than one copyField directive", 1, copyFields.size());
-    assertEquals("The copyField source is not '*'", "*", ((Map)copyFields.get(0)).get("source"));
-    assertEquals("The copyField dest is not 'text'", "text", ((Map)copyFields.get(0)).get("dest"));
-
-    String payload = "{ 'delete-copy-field': { 'source': '*', 'dest': 'text' } }";
-
-    String response = restTestHarness.post("/schema?wt=json", json(payload));
-    Map map = (Map)ObjectBuilder.getVal(new JSONParser(new StringReader(response)));
-    assertNull(response, map.get("errors"));
-    
-    assertEquals(0, getCopyFields().size());
-  }
-  
-  private List getCopyFields() throws Exception {
-    String response = restTestHarness.query("/schema?wt=json");
-    System.err.println(response);
-    Map map = (Map)ObjectBuilder.getVal(new JSONParser(new StringReader(response)));
-    return (List)((Map)map.get("schema")).get("copyFields");
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/TestSchemaSimilarityResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestSchemaSimilarityResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestSchemaSimilarityResource.java
index 6b7be71..2016677 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestSchemaSimilarityResource.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestSchemaSimilarityResource.java
@@ -24,7 +24,6 @@ public class TestSchemaSimilarityResource extends SolrRestletTestBase {
    * NOTE: schema used by parent class doesn't define a global sim, so we get the implicit default
    * which causes the FQN of the class to be returned
    * 
-   * @see TestClassNameShortening#testShortenedGlobalSimilarityStaysShortened
    */
   @Test
   public void testGetSchemaSimilarity() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedStopFilterFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedStopFilterFactory.java b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedStopFilterFactory.java
index 242e5b5..be8c394 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedStopFilterFactory.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedStopFilterFactory.java
@@ -140,7 +140,7 @@ public class TestManagedStopFilterFactory extends RestTestBase {
             "/response/lst[@name='error']/int[@name='code'] = '404'");
 
     // add the new field
-    assertJPut("/schema/fields/" + newFieldName, json("{'type':'managed_en'}"),
+    assertJPost("/schema/fields", "{add-field : { name :managed_en_field, type : managed_en}}",
                "/responseHeader/status==0");
 
     // make sure the new field exists now

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymFilterFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymFilterFactory.java b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymFilterFactory.java
index 9afaf6f..26fcde1 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymFilterFactory.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymFilterFactory.java
@@ -90,8 +90,8 @@ public class TestManagedSynonymFilterFactory extends RestTestBase {
                JSONUtil.toJSON(syns),
                "/responseHeader/status==0");
     
-    assertJQ(endpoint, 
-             "/synonymMappings/managedMap/happy==['cheerful','glad','joyful']");
+    assertJQ(endpoint,
+        "/synonymMappings/managedMap/happy==['cheerful','glad','joyful']");
 
     // request to a specific mapping
     assertJQ(endpoint+"/happy", 
@@ -146,7 +146,7 @@ public class TestManagedSynonymFilterFactory extends RestTestBase {
             "/response/lst[@name='error']/int[@name='code'] = '404'");
 
     // add the new field
-    assertJPut("/schema/fields/" + newFieldName, json("{'type':'managed_en'}"),
+    assertJPost("/schema", "{ add-field :  { name: managed_en_field, type : managed_en}}",
                "/responseHeader/status==0");
 
     // make sure the new field exists now

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java b/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java
index 3d0d120..0993ba1 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java
@@ -37,11 +37,12 @@ import org.apache.solr.util.RestTestHarness;
 import org.apache.zookeeper.data.Stat;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.restlet.ext.servlet.ServerServlet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
+@Ignore
 public class TestCloudManagedSchemaConcurrent extends AbstractFullDistribZkTestBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final String SUCCESS_XPATH = "/response/lst[@name='responseHeader']/int[@name='status'][.='0']";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java
index b723abf..72051b1 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java
@@ -575,10 +575,6 @@ public class SchemaTest extends RestTestBase {
     assertThat("solr.TextField", is(equalTo(newFieldTypeRepresentation.getAttributes().get("class"))));
     assertThat(analyzerDefinition.getTokenizer().get("class"),
         is(equalTo(newFieldTypeRepresentation.getAnalyzer().getTokenizer().get("class"))));
-    assertTrue(newFieldTypeRepresentation.getFields().size() == 1);
-    assertTrue(newFieldTypeRepresentation.getFields().contains(fieldName));
-    assertTrue(newFieldTypeRepresentation.getDynamicFields().size() == 1);
-    assertTrue(newFieldTypeRepresentation.getDynamicFields().contains(dynamicFieldName));
   }
 
   @Test


[4/4] lucene-solr git commit: Merge remote-tracking branch 'origin/master'

Posted by no...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/master
Commit: b87e3920976b3c7f7c765bb6d6ad1eeabd539c30
Parents: f2c281a 9617d3d
Author: Noble Paul <no...@apache.org>
Authored: Mon Mar 7 22:48:25 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Mon Mar 7 22:48:25 2016 +0530

----------------------------------------------------------------------
 dev-tools/scripts/smokeTestRelease.py | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------



[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

Posted by no...@apache.org.
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);
-  }
-}


[2/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

Posted by no...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
index acc8c13..4319c3e 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
@@ -39,6 +39,7 @@ import org.apache.lucene.uninverting.UninvertingReader;
 import org.apache.lucene.util.Version;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
@@ -1352,6 +1353,10 @@ public class IndexSchema {
    * Get a map of property name -&gt; value for the whole schema.
    */
   public SimpleOrderedMap<Object> getNamedPropertyValues() {
+    return getNamedPropertyValues(new MapSolrParams(Collections.EMPTY_MAP));
+
+  }
+  public SimpleOrderedMap<Object> getNamedPropertyValues(SolrParams params) {
     SimpleOrderedMap<Object> topLevel = new SimpleOrderedMap<>();
     topLevel.add(NAME, getSchemaName());
     topLevel.add(VERSION, getVersion());
@@ -1372,19 +1377,19 @@ public class IndexSchema {
     List<SimpleOrderedMap<Object>> fieldTypeProperties = new ArrayList<>();
     SortedMap<String,FieldType> sortedFieldTypes = new TreeMap<>(fieldTypes);
     for (FieldType fieldType : sortedFieldTypes.values()) {
-      fieldTypeProperties.add(fieldType.getNamedPropertyValues(false));
+      fieldTypeProperties.add(fieldType.getNamedPropertyValues(params.getBool("showDefaults", false)));
     }
     topLevel.add(FIELD_TYPES, fieldTypeProperties);  
     List<SimpleOrderedMap<Object>> fieldProperties = new ArrayList<>();
     SortedSet<String> fieldNames = new TreeSet<>(fields.keySet());
     for (String fieldName : fieldNames) {
-      fieldProperties.add(fields.get(fieldName).getNamedPropertyValues(false));
+      fieldProperties.add(fields.get(fieldName).getNamedPropertyValues(params.getBool("showDefaults", false)));
     }
     topLevel.add(FIELDS, fieldProperties);
     List<SimpleOrderedMap<Object>> dynamicFieldProperties = new ArrayList<>();
     for (IndexSchema.DynamicField dynamicField : dynamicFields) {
       if ( ! dynamicField.getRegex().startsWith(INTERNAL_POLY_FIELD_PREFIX)) { // omit internal polyfields
-        dynamicFieldProperties.add(dynamicField.getPrototype().getNamedPropertyValues(false));
+        dynamicFieldProperties.add(dynamicField.getPrototype().getNamedPropertyValues(params.getBool("showDefaults", false)));
       }
     }
     topLevel.add(DYNAMIC_FIELDS, dynamicFieldProperties);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index d87eb69..f291b2f 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -35,10 +35,12 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableSet;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.http.Header;
@@ -84,6 +86,7 @@ import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.response.QueryResponseWriter;
 import org.apache.solr.response.QueryResponseWriterUtil;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.AuthorizationContext;
 import org.apache.solr.security.AuthorizationContext.CollectionRequest;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/TestClassNameShortening.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestClassNameShortening.java b/solr/core/src/test/org/apache/solr/rest/schema/TestClassNameShortening.java
index ddbd331..52ba10b 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestClassNameShortening.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestClassNameShortening.java
@@ -18,13 +18,14 @@ package org.apache.solr.rest.schema;
 import org.apache.solr.util.RestTestBase;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.restlet.ext.servlet.ServerServlet;
 
 import java.util.SortedMap;
 import java.util.TreeMap;
 
-
+@Ignore
 public class TestClassNameShortening extends RestTestBase {
 
   @BeforeClass

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/TestCopyFieldCollectionResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestCopyFieldCollectionResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestCopyFieldCollectionResource.java
index 5eeee8c..c0f936d 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestCopyFieldCollectionResource.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestCopyFieldCollectionResource.java
@@ -30,73 +30,51 @@ public class TestCopyFieldCollectionResource extends SolrRestletTestBase {
            +"                                      and int[@name='maxChars'][.='200']]",
 
             "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='title']"
-           +"                                      and str[@name='dest'][.='dest_sub_no_ast_s']"
-           +"                                      and str[@name='destDynamicBase'][.='*_s']]",
+           +"                                      and str[@name='dest'][.='dest_sub_no_ast_s']]",
 
-            "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_i']"
+        "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_i']"
            +"                                      and str[@name='dest'][.='title']]",
 
             "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_i']"
            +"                                      and str[@name='dest'][.='*_s']]",
 
             "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_i']"
-           +"                                      and str[@name='dest'][.='*_dest_sub_s']"
-           +"                                      and str[@name='destDynamicBase'][.='*_s']]",
+           +"                                      and str[@name='dest'][.='*_dest_sub_s']]",
 
-            "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_i']"
-           +"                                      and str[@name='dest'][.='dest_sub_no_ast_s']"
-           +"                                      and str[@name='destDynamicBase'][.='*_s']]",
+        "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_i']"
+           +"                                      and str[@name='dest'][.='dest_sub_no_ast_s']]",
 
             "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_src_sub_i']"
-           +"                                      and str[@name='sourceDynamicBase'][.='*_i']"
            +"                                      and str[@name='dest'][.='title']]",
 
             "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_src_sub_i']"
-           +"                                      and str[@name='sourceDynamicBase'][.='*_i']"
            +"                                      and str[@name='dest'][.='*_s']]",
 
             "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_src_sub_i']"
-           +"                                      and str[@name='sourceDynamicBase'][.='*_i']"
-           +"                                      and str[@name='dest'][.='*_dest_sub_s']"
-           +"                                      and str[@name='destDynamicBase'][.='*_s']]",
+           +"                                      and str[@name='dest'][.='*_dest_sub_s']]",
 
-            "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_src_sub_i']"
-           +"                                      and str[@name='sourceDynamicBase'][.='*_i']"
-           +"                                      and str[@name='dest'][.='dest_sub_no_ast_s']"
-           +"                                      and str[@name='destDynamicBase'][.='*_s']]",
+        "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_src_sub_i']"
+           +"                                      and str[@name='dest'][.='dest_sub_no_ast_s']]",
 
             "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='src_sub_no_ast_i']"
-           +"                                      and str[@name='sourceDynamicBase'][.='*_i']"
            +"                                      and str[@name='dest'][.='*_s']]",
 
             "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='src_sub_no_ast_i']"
-           +"                                      and str[@name='sourceDynamicBase'][.='*_i']"
-           +"                                      and str[@name='dest'][.='*_dest_sub_s']"
-           +"                                      and str[@name='destDynamicBase'][.='*_s']]",
+           +"                                      and str[@name='dest'][.='*_dest_sub_s']]",
 
             "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='src_sub_no_ast_i']"
-           +"                                      and str[@name='sourceDynamicBase'][.='*_i']"
-           +"                                      and str[@name='dest'][.='dest_sub_no_ast_s']"
-           +"                                      and str[@name='destDynamicBase'][.='*_s']]",
+           +"                                      and str[@name='dest'][.='dest_sub_no_ast_s']]",
 
             "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='title_*']"
-           +"                                      and arr[@name='sourceExplicitFields']/str[.='title_stemmed']"
-           +"                                      and arr[@name='sourceExplicitFields']/str[.='title_lettertok']"
            +"                                      and str[@name='dest'][.='text']]",
 
             "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='title_*']"
-           +"                                      and arr[@name='sourceExplicitFields']/str[.='title_stemmed']"
-           +"                                      and arr[@name='sourceExplicitFields']/str[.='title_lettertok']"
            +"                                      and str[@name='dest'][.='*_s']]",
 
             "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='title_*']"
-           +"                                      and arr[@name='sourceExplicitFields']/str[.='title_stemmed']"
-           +"                                      and arr[@name='sourceExplicitFields']/str[.='title_lettertok']"
            +"                                      and str[@name='dest'][.='*_dest_sub_s']]",
 
             "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='title_*']"
-           +"                                      and arr[@name='sourceExplicitFields']/str[.='title_stemmed']"
-           +"                                      and arr[@name='sourceExplicitFields']/str[.='title_lettertok']"
            +"                                      and str[@name='dest'][.='dest_sub_no_ast_s']]");
   }
 
@@ -104,56 +82,22 @@ public class TestCopyFieldCollectionResource extends SolrRestletTestBase {
   public void testJsonGetAllCopyFields() throws Exception {
     assertJQ("/schema/copyfields?indent=on&wt=json",
              "/copyFields/[1]=={'source':'src_sub_no_ast_i','dest':'title'}",
-             "/copyFields/[7]=={'source':'title','dest':'dest_sub_no_ast_s','destDynamicBase':'*_s'}",
+             "/copyFields/[7]=={'source':'title','dest':'dest_sub_no_ast_s'}",
 
              "/copyFields/[8]=={'source':'*_i','dest':'title'}",
              "/copyFields/[9]=={'source':'*_i','dest':'*_s'}",
-             "/copyFields/[10]=={'source':'*_i','dest':'*_dest_sub_s','destDynamicBase':'*_s'}",
-             "/copyFields/[11]=={'source':'*_i','dest':'dest_sub_no_ast_s','destDynamicBase':'*_s'}",
+             "/copyFields/[10]=={'source':'*_i','dest':'*_dest_sub_s'}",
+             "/copyFields/[11]=={'source':'*_i','dest':'dest_sub_no_ast_s'}",
 
-             "/copyFields/[12]=={'source':'*_src_sub_i','sourceDynamicBase':'*_i','dest':'title'}",
-             "/copyFields/[13]=={'source':'*_src_sub_i','sourceDynamicBase':'*_i','dest':'*_s'}",
-             "/copyFields/[14]=={'source':'*_src_sub_i','sourceDynamicBase':'*_i','dest':'*_dest_sub_s','destDynamicBase':'*_s'}",
-             "/copyFields/[15]=={'source':'*_src_sub_i','sourceDynamicBase':'*_i','dest':'dest_sub_no_ast_s','destDynamicBase':'*_s'}",
+             "/copyFields/[12]=={'source':'*_src_sub_i','dest':'title'}",
+             "/copyFields/[13]=={'source':'*_src_sub_i','dest':'*_s'}",
+             "/copyFields/[14]=={'source':'*_src_sub_i','dest':'*_dest_sub_s'}",
+             "/copyFields/[15]=={'source':'*_src_sub_i','dest':'dest_sub_no_ast_s'}",
 
-             "/copyFields/[16]=={'source':'src_sub_no_ast_i','sourceDynamicBase':'*_i','dest':'*_s'}",
-             "/copyFields/[17]=={'source':'src_sub_no_ast_i','sourceDynamicBase':'*_i','dest':'*_dest_sub_s','destDynamicBase':'*_s'}",
-             "/copyFields/[18]=={'source':'src_sub_no_ast_i','sourceDynamicBase':'*_i','dest':'dest_sub_no_ast_s','destDynamicBase':'*_s'}");
+             "/copyFields/[16]=={'source':'src_sub_no_ast_i','dest':'*_s'}",
+             "/copyFields/[17]=={'source':'src_sub_no_ast_i','dest':'*_dest_sub_s'}",
+             "/copyFields/[18]=={'source':'src_sub_no_ast_i','dest':'dest_sub_no_ast_s'}");
 
   }
 
-  @Test
-  public void testRestrictSource() throws Exception {
-    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=title,*_i,*_src_sub_i,src_sub_no_ast_i",
-            "count(/response/arr[@name='copyFields']/lst) = 16", // 4 + 4 + 4 + 4
-            "count(/response/arr[@name='copyFields']/lst/str[@name='source'][.='title']) = 4",
-            "count(/response/arr[@name='copyFields']/lst/str[@name='source'][.='*_i']) = 4",
-            "count(/response/arr[@name='copyFields']/lst/str[@name='source'][.='*_src_sub_i']) = 4",
-            "count(/response/arr[@name='copyFields']/lst/str[@name='source'][.='src_sub_no_ast_i']) = 4");
-  }
-
-  @Test
-  public void testRestrictDest() throws Exception {
-    assertQ("/schema/copyfields/?indent=on&wt=xml&dest.fl=title,*_s,*_dest_sub_s,dest_sub_no_ast_s",
-            "count(/response/arr[@name='copyFields']/lst) = 16", // 3 + 4 + 4 + 5
-            "count(/response/arr[@name='copyFields']/lst/str[@name='dest'][.='title']) = 3",
-            "count(/response/arr[@name='copyFields']/lst/str[@name='dest'][.='*_s']) = 4",
-            "count(/response/arr[@name='copyFields']/lst/str[@name='dest'][.='*_dest_sub_s']) = 4",
-            "count(/response/arr[@name='copyFields']/lst/str[@name='dest'][.='dest_sub_no_ast_s']) = 5");
-  }
-
-  @Test
-  public void testRestrictSourceAndDest() throws Exception {
-    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=title,*_i&dest.fl=title,dest_sub_no_ast_s",
-            "count(/response/arr[@name='copyFields']/lst) = 3",
-
-            "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='title']"
-           +"                                      and str[@name='dest'][.='dest_sub_no_ast_s']]",
-
-            "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_i']"
-           +"                                      and str[@name='dest'][.='title']]",
-
-            "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_i']"
-           +"                                      and str[@name='dest'][.='dest_sub_no_ast_s']]");
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldCollectionResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldCollectionResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldCollectionResource.java
index 318b28a..032bbad 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldCollectionResource.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldCollectionResource.java
@@ -30,39 +30,10 @@ public class TestDynamicFieldCollectionResource extends SolrRestletTestBase {
   }
 
   @Test
-  public void testGetTwoDynamicFields() throws IOException {
-    assertQ("/schema/dynamicfields?indent=on&wt=xml&fl=*_i,*_s",
-            "count(/response/arr[@name='dynamicFields']/lst/str[@name='name']) = 2",
-            "(/response/arr[@name='dynamicFields']/lst/str[@name='name'])[1] = '*_i'",
-            "(/response/arr[@name='dynamicFields']/lst/str[@name='name'])[2] = '*_s'");
-  }
-
-  @Test
-  public void testNotFoundDynamicFields() throws IOException {
-    assertQ("/schema/dynamicfields?indent=on&wt=xml&fl=*_not_in_there,this_one_isnt_either_*",
-            "count(/response/arr[@name='dynamicFields']) = 1",
-            "count(/response/arr[@name='dynamicfields']/lst/str[@name='name']) = 0");
-  }
-
-  @Test
   public void testJsonGetAllDynamicFields() throws Exception {
     assertJQ("/schema/dynamicfields?indent=on",
              "/dynamicFields/[0]/name=='*_coordinate'",
              "/dynamicFields/[1]/name=='ignored_*'",
              "/dynamicFields/[2]/name=='*_mfacet'");
   }
-  
-  @Test
-  public void testJsonGetTwoDynamicFields() throws Exception {
-    assertJQ("/schema/dynamicfields?indent=on&fl=*_i,*_s&wt=xml", // assertJQ will fix the wt param to be json
-             "/dynamicFields/[0]/name=='*_i'",
-             "/dynamicFields/[1]/name=='*_s'");
-  }
-
-  @Test
-  public void testJsonPostFieldsToNonMutableIndexSchema() throws Exception {
-    assertJPost("/schema/dynamicfields",
-        "[{\"name\":\"foobarbaz\", \"type\":\"text_general\", \"stored\":\"false\"}]",
-        "/error/msg=='This IndexSchema is not mutable.'");
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldResource.java
index 54b17fc..7ca7953 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldResource.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldResource.java
@@ -67,11 +67,4 @@ public class TestDynamicFieldResource extends SolrRestletTestBase {
              "/dynamicField/required==false",
              "/dynamicField/tokenized==false");
   }
-
-  @Test
-  public void testJsonPutFieldToNonMutableIndexSchema() throws Exception {
-    assertJPut("/schema/dynamicfields/newfield_*",
-        "{\"type\":\"text_general\", \"stored\":\"false\"}",
-        "/error/msg=='This IndexSchema is not mutable.'");
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/TestFieldCollectionResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldCollectionResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldCollectionResource.java
index 571acc5..dd55415 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldCollectionResource.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldCollectionResource.java
@@ -30,45 +30,6 @@ public class TestFieldCollectionResource extends SolrRestletTestBase {
             "(/response/arr[@name='fields']/lst/str[@name='name'])[3] = '_version_'");
   }
 
-  @Test
-  public void testGetTwoFields() throws IOException {
-    assertQ("/schema/fields?indent=on&wt=xml&fl=id,_version_",
-            "count(/response/arr[@name='fields']/lst/str[@name='name']) = 2",
-            "(/response/arr[@name='fields']/lst/str[@name='name'])[1] = 'id'",
-            "(/response/arr[@name='fields']/lst/str[@name='name'])[2] = '_version_'");
-  }
-  
-  @Test
-  public void testGetThreeFieldsDontIncludeDynamic() throws IOException {
-    // 
-    assertQ("/schema/fields?indent=on&wt=xml&fl=id,_version_,price_i",
-            "count(/response/arr[@name='fields']/lst/str[@name='name']) = 2",
-            "(/response/arr[@name='fields']/lst/str[@name='name'])[1] = 'id'",
-            "(/response/arr[@name='fields']/lst/str[@name='name'])[2] = '_version_'");
-  }
-
-  @Test
-  public void testGetThreeFieldsIncludeDynamic() throws IOException {
-    assertQ("/schema/fields?indent=on&wt=xml&fl=id,_version_,price_i&includeDynamic=on",
-
-            "count(/response/arr[@name='fields']/lst/str[@name='name']) = 3",
-
-            "(/response/arr[@name='fields']/lst/str[@name='name'])[1] = 'id'",
-
-            "(/response/arr[@name='fields']/lst/str[@name='name'])[2] = '_version_'",
-
-            "(/response/arr[@name='fields']/lst/str[@name='name'])[3] = 'price_i'",
-
-            "/response/arr[@name='fields']/lst[    str[@name='name']='price_i'    "
-           +"                                  and str[@name='dynamicBase']='*_i']");
-  }
-
-  @Test
-  public void testNotFoundFields() throws IOException {
-    assertQ("/schema/fields?indent=on&wt=xml&fl=not_in_there,this_one_either",
-            "count(/response/arr[@name='fields']) = 1",
-            "count(/response/arr[@name='fields']/lst/str[@name='name']) = 0");
-  }
 
   @Test
   public void testJsonGetAllFields() throws Exception {
@@ -78,10 +39,4 @@ public class TestFieldCollectionResource extends SolrRestletTestBase {
              "/fields/[2]/name=='_version_'");
   }
 
-  @Test
-  public void testJsonGetTwoFields() throws Exception {
-    assertJQ("/schema/fields?indent=on&fl=id,_version_&wt=xml", // assertJQ should fix the wt param to be json
-             "/fields/[0]/name=='id'",
-             "/fields/[1]/name=='_version_'");
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java
index 711e3c0..627aee0 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java
@@ -72,18 +72,12 @@ public class TestFieldResource extends SolrRestletTestBase {
              "/field/tokenized==true");
   }
   
-  @Test
-  public void testGetFieldIncludeDynamic() throws Exception {
-    assertQ("/schema/fields/some_crazy_name_i?indent=on&wt=xml&includeDynamic=true",
-            "/response/lst[@name='field']/str[@name='name'] = 'some_crazy_name_i'",
-            "/response/lst[@name='field']/str[@name='dynamicBase'] = '*_i'");    
-  }
-  
+
   @Test
   public void testGetFieldDontShowDefaults() throws Exception {
     String[] tests = { 
         "count(/response/lst[@name='field']) = 1",
-        "count(/response/lst[@name='field']/*) = 7",
+        "count(/response/lst[@name='field']/*) = 6",
         "/response/lst[@name='field']/str[@name='name'] = 'id'",
         "/response/lst[@name='field']/str[@name='type'] = 'string'",
         "/response/lst[@name='field']/bool[@name='indexed'] = 'true'",
@@ -95,17 +89,4 @@ public class TestFieldResource extends SolrRestletTestBase {
     assertQ("/schema/fields/id?indent=on&wt=xml&showDefaults=false", tests);
   }
   
-  @Test
-  public void testJsonPutFieldToNonMutableIndexSchema() throws Exception {
-    assertJPut("/schema/fields/newfield",
-        "{\"type\":\"text_general\", \"stored\":\"false\"}",
-        "/error/msg=='This IndexSchema is not mutable.'");
-  }
-
-  @Test
-  public void testJsonPostFieldsToNonMutableIndexSchema() throws Exception {
-    assertJPost("/schema/fields",
-        "[{\"name\":\"foobarbaz\", \"type\":\"text_general\", \"stored\":\"false\"}]",
-        "/error/msg=='This IndexSchema is not mutable.'");
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeCollectionResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeCollectionResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeCollectionResource.java
index a61f644..53cd1c0 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeCollectionResource.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeCollectionResource.java
@@ -19,6 +19,7 @@ import org.apache.solr.rest.SolrRestletTestBase;
 import org.junit.Test;
 
 public class TestFieldTypeCollectionResource extends SolrRestletTestBase {
+
   @Test
   public void testGetAllFieldTypes() throws Exception {
     assertQ("/schema/fieldtypes?indent=on&wt=xml",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeResource.java
index 2f0c241..eb72aed 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeResource.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldTypeResource.java
@@ -24,7 +24,7 @@ public class TestFieldTypeResource extends SolrRestletTestBase {
   public void testGetFieldType() throws Exception {
     assertQ("/schema/fieldtypes/float?indent=on&wt=xml&showDefaults=true",
             "count(/response/lst[@name='fieldType']) = 1",
-            "count(/response/lst[@name='fieldType']/*) = 18",
+            "count(/response/lst[@name='fieldType']/*) = 16",
             "/response/lst[@name='fieldType']/str[@name='name'] = 'float'",
             "/response/lst[@name='fieldType']/str[@name='class'] = 'solr.TrieFloatField'",
             "/response/lst[@name='fieldType']/str[@name='precisionStep'] ='0'",
@@ -39,9 +39,7 @@ public class TestFieldTypeResource extends SolrRestletTestBase {
             "/response/lst[@name='fieldType']/bool[@name='omitPositions'] = 'false'",
             "/response/lst[@name='fieldType']/bool[@name='storeOffsetsWithPositions'] = 'false'",
             "/response/lst[@name='fieldType']/bool[@name='multiValued'] = 'false'",
-            "/response/lst[@name='fieldType']/bool[@name='tokenized'] = 'false'",
-            "/response/lst[@name='fieldType']/arr[@name='fields']/str = 'weight'",
-            "/response/lst[@name='fieldType']/arr[@name='dynamicFields']/str = '*_f'");
+            "/response/lst[@name='fieldType']/bool[@name='tokenized'] = 'false'");
   }
 
   @Test
@@ -69,22 +67,19 @@ public class TestFieldTypeResource extends SolrRestletTestBase {
              "/fieldType/omitPositions==false",
              "/fieldType/storeOffsetsWithPositions==false",
              "/fieldType/multiValued==false",
-             "/fieldType/tokenized==false",
-             "/fieldType/fields==['weight']",
-             "/fieldType/dynamicFields==['*_f']");
+             "/fieldType/tokenized==false");
   }
   
   @Test
   public void testGetFieldTypeDontShowDefaults() throws Exception {
     assertQ("/schema/fieldtypes/teststop?wt=xml&indent=on",
-            "count(/response/lst[@name='fieldType']/*) = 5",
+            "count(/response/lst[@name='fieldType']/*) = 3",
             "/response/lst[@name='fieldType']/str[@name='name'] = 'teststop'",
             "/response/lst[@name='fieldType']/str[@name='class'] = 'solr.TextField'",
             "/response/lst[@name='fieldType']/lst[@name='analyzer']/lst[@name='tokenizer']/str[@name='class'] = 'solr.LowerCaseTokenizerFactory'",
             "/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst/str[@name='class'][.='solr.StandardFilterFactory']",
             "/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst/str[@name='class'][.='solr.StopFilterFactory']",
-            "/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst/str[@name='words'][.='stopwords.txt']",
-            "/response/lst[@name='fieldType']/arr[@name='fields']/str[.='teststop']",
-            "/response/lst[@name='fieldType']/arr[@name='dynamicFields']");
+            "/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst/str[@name='words'][.='stopwords.txt']"
+            );
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/TestManagedSchemaDynamicFieldResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestManagedSchemaDynamicFieldResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestManagedSchemaDynamicFieldResource.java
deleted file mode 100644
index 6572709..0000000
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestManagedSchemaDynamicFieldResource.java
+++ /dev/null
@@ -1,366 +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.commons.io.FileUtils;
-import org.apache.solr.util.RestTestBase;
-import org.eclipse.jetty.servlet.ServletHolder;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.restlet.ext.servlet.ServerServlet;
-
-import java.io.File;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.regex.Pattern;
-
-public class TestManagedSchemaDynamicFieldResource extends RestTestBase {
-
-  private static File tmpSolrHome;
-  private static File tmpConfDir;
-
-  private static final String collection = "collection1";
-  private static final String confDir = collection + "/conf";
-
-
-  @Before
-  public void before() throws Exception {
-    tmpSolrHome = createTempDir().toFile();
-    tmpConfDir = new File(tmpSolrHome, confDir);
-    FileUtils.copyDirectory(new File(TEST_HOME()), tmpSolrHome.getAbsoluteFile());
-
-    final SortedMap<ServletHolder,String> extraServlets = new TreeMap<>();
-    final ServletHolder solrRestApi = new ServletHolder("SolrSchemaRestApi", ServerServlet.class);
-    solrRestApi.setInitParameter("org.restlet.application", "org.apache.solr.rest.SolrSchemaRestApi");
-    extraServlets.put(solrRestApi, "/schema/*");  // '/schema/*' matches '/schema', '/schema/', and '/schema/whatever...'
-
-    System.setProperty("managed.schema.mutable", "true");
-    System.setProperty("enable.update.log", "false");
-
-    createJettyAndHarness(tmpSolrHome.getAbsolutePath(), "solrconfig-managed-schema.xml", "schema-rest.xml",
-                          "/solr", true, extraServlets);
-  }
-
-  @After
-  public void after() throws Exception {
-    if (jetty != null) {
-      jetty.stop();
-      jetty = null;
-    }
-    client = null;
-    if (restTestHarness != null) {
-      restTestHarness.close();
-    }
-    restTestHarness = null;
-  }
-
-  @Test
-  public void testAddDynamicFieldBadFieldType() throws Exception {
-    assertJPut("/schema/dynamicfields/*_newdynamicfield",
-               json( "{'type':'not_in_there_at_all','stored':false}" ),
-               "/error/msg==\"Dynamic field \\'*_newdynamicfield\\': Field type \\'not_in_there_at_all\\' not found.\"");
-  }
-
-  @Test
-  public void testAddDynamicFieldMismatchedName() throws Exception {
-    assertJPut("/schema/dynamicfields/*_newdynamicfield",
-               json( "{'name':'*_something_else','type':'text','stored':false}" ),
-               "/error/msg=='///regex:\\\\*_newdynamicfield///'");
-  }
-
-  @Test
-  public void testAddDynamicFieldBadProperty() throws Exception {
-    assertJPut("/schema/dynamicfields/*_newdynamicfield",
-               json( "{'type':'text','no_property_with_this_name':false}" ),
-               "/error/msg==\"java.lang.IllegalArgumentException: Invalid field property: no_property_with_this_name\"");
-  }
-
-  @Test
-  public void testAddDynamicField() throws Exception {
-    assertQ("/schema/dynamicfields/newdynamicfield_*?indent=on&wt=xml",
-            "count(/response/lst[@name='newdynamicfield_*']) = 0",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '404'",
-            "/response/lst[@name='error']/int[@name='code'] = '404'");
-
-    assertJPut("/schema/dynamicfields/newdynamicfield_*",
-               json("{'type':'text','stored':false}"),
-               "/responseHeader/status==0");
-
-    assertQ("/schema/dynamicfields/newdynamicfield_*?indent=on&wt=xml",
-            "count(/response/lst[@name='dynamicField']) = 1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'");
-
-    assertU(adoc("newdynamicfield_A", "value1 value2", "id", "123"));
-    assertU(commit());
-
-    assertQ("/select?q=newdynamicfield_A:value1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
-            "/response/result[@name='response'][@numFound='1']",
-            "count(/response/result[@name='response']/doc/*) = 1",
-            "/response/result[@name='response']/doc/str[@name='id'][.='123']");
-  }
-
-  @Test
-  public void testAddDynamicFieldWithMulipleOptions() throws Exception {
-    assertQ("/schema/dynamicfields/newdynamicfield_*?indent=on&wt=xml",
-            "count(/response/lst[@name='dynamicField']) = 0",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '404'",
-            "/response/lst[@name='error']/int[@name='code'] = '404'");
-
-    assertJPut("/schema/dynamicfields/newdynamicfield_*",
-               json("{'type':'text_en','stored':true,'indexed':false}"),
-               "/responseHeader/status==0");
-
-    File managedSchemaFile = new File(tmpConfDir, "managed-schema");
-    assertTrue(managedSchemaFile.exists());
-    String managedSchemaContents = FileUtils.readFileToString(managedSchemaFile, "UTF-8");
-    Pattern newdynamicfieldStoredTrueIndexedFalsePattern
-        = Pattern.compile( "<dynamicField name=\"newdynamicfield_\\*\" type=\"text_en\""
-                         + "(?=.*stored=\"true\")(?=.*indexed=\"false\").*/>");
-    assertTrue(newdynamicfieldStoredTrueIndexedFalsePattern.matcher(managedSchemaContents).find());
-
-    assertQ("/schema/dynamicfields/newdynamicfield_*?indent=on&wt=xml",
-            "count(/response/lst[@name='dynamicField']) = 1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
-            "/response/lst[@name='dynamicField']/str[@name='name'] = 'newdynamicfield_*'",
-            "/response/lst[@name='dynamicField']/str[@name='type'] = 'text_en'",
-            "/response/lst[@name='dynamicField']/bool[@name='indexed'] = 'false'",
-            "/response/lst[@name='dynamicField']/bool[@name='stored'] = 'true'");
-
-    assertU(adoc("newdynamicfield_A", "value1 value2", "id", "1234"));
-    assertU(commit());
-
-    assertQ("/schema/dynamicfields/newdynamicfield2_*?indent=on&wt=xml",
-            "count(/response/lst[@name='dynamicField']) = 0",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '404'",
-            "/response/lst[@name='error']/int[@name='code'] = '404'");
-
-    assertJPut("/schema/dynamicfields/newdynamicfield2_*",
-               json("{'type':'text_en','stored':true,'indexed':true,'multiValued':true}"),
-               "/responseHeader/status==0");
-
-    managedSchemaContents = FileUtils.readFileToString(managedSchemaFile, "UTF-8");
-    Pattern newdynamicfield2StoredTrueIndexedTrueMultiValuedTruePattern
-        = Pattern.compile( "<dynamicField name=\"newdynamicfield2_\\*\" type=\"text_en\" "
-                         + "(?=.*stored=\"true\")(?=.*indexed=\"true\")(?=.*multiValued=\"true\").*/>");
-    assertTrue(newdynamicfield2StoredTrueIndexedTrueMultiValuedTruePattern.matcher(managedSchemaContents).find());
-
-    assertQ("/schema/dynamicfields/newdynamicfield2_*?indent=on&wt=xml",
-            "count(/response/lst[@name='dynamicField']) = 1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
-            "/response/lst[@name='dynamicField']/str[@name='name'] = 'newdynamicfield2_*'",
-            "/response/lst[@name='dynamicField']/str[@name='type'] = 'text_en'",
-            "/response/lst[@name='dynamicField']/bool[@name='indexed'] = 'true'",
-            "/response/lst[@name='dynamicField']/bool[@name='stored'] = 'true'",
-            "/response/lst[@name='dynamicField']/bool[@name='multiValued'] = 'true'");
-
-    assertU(adoc("newdynamicfield2_A", "value1 value2", "newdynamicfield2_A", "value3 value4", "id", "5678"));
-    assertU(commit());
-
-    assertQ("/select?q=newdynamicfield2_A:value3",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
-            "/response/result[@name='response'][@numFound='1']",
-            "count(/response/result[@name='response']/doc) = 1",
-            "/response/result[@name='response']/doc/str[@name='id'][.='5678']");
-  }
-
-  @Test
-  public void testAddDynamicFieldCollectionWithMultipleOptions() throws Exception {
-    assertQ("/schema/dynamicfields?indent=on&wt=xml",
-            "count(/response/arr[@name='dynamicFields']/lst/str[@name]) > 0",                          // there are fields
-            "count(/response/arr[@name='dynamicFields']/lst/str[starts-with(@name,'newfield')]) = 0"); // but none named newfield*
-
-    assertJPost("/schema/dynamicfields",
-                json("[{'name':'newdynamicfield_*','type':'text_en','stored':true,'indexed':false}]"),
-                "/responseHeader/status==0");
-
-    File managedSchemaFile = new File(tmpConfDir, "managed-schema");
-    assertTrue(managedSchemaFile.exists());
-    String managedSchemaContents = FileUtils.readFileToString(managedSchemaFile, "UTF-8");
-    Pattern newfieldStoredTrueIndexedFalsePattern
-        = Pattern.compile( "<dynamicField name=\"newdynamicfield_\\*\" type=\"text_en\""
-                         + "(?=.*stored=\"true\")(?=.*indexed=\"false\").*/>");
-    assertTrue(newfieldStoredTrueIndexedFalsePattern.matcher(managedSchemaContents).find());
-
-    assertQ("/schema/dynamicfields?indent=on&wt=xml",
-             "/response/arr[@name='dynamicFields']/lst"
-           + "[str[@name='name']='newdynamicfield_*' and str[@name='type']='text_en'"
-           + " and bool[@name='stored']='true' and bool[@name='indexed']='false']");
-
-    assertU(adoc("newdynamicfield_A", "value1 value2", "id", "789"));
-    assertU(commit());
-
-    assertJPost("/schema/dynamicfields",
-                json("[{'name':'newdynamicfield2_*','type':'text_en','stored':true,'indexed':true,'multiValued':true}]"),
-                "/responseHeader/status==0");
-
-    managedSchemaContents = FileUtils.readFileToString(managedSchemaFile, "UTF-8");
-    Pattern newdynamicfield2StoredTrueIndexedTrueMultiValuedTruePattern
-        = Pattern.compile( "<dynamicField name=\"newdynamicfield2_\\*\" type=\"text_en\" "
-                         + "(?=.*stored=\"true\")(?=.*indexed=\"true\")(?=.*multiValued=\"true\").*/>");
-    assertTrue(newdynamicfield2StoredTrueIndexedTrueMultiValuedTruePattern.matcher(managedSchemaContents).find());
-
-    assertQ("/schema/dynamicfields?indent=on&wt=xml",
-             "/response/arr[@name='dynamicFields']/lst"
-           + "[str[@name='name']='newdynamicfield2_*' and str[@name='type']='text_en'"
-           + " and bool[@name='stored']='true' and bool[@name='indexed']='true' and bool[@name='multiValued']='true']");
-
-    assertU(adoc("newdynamicfield2_A", "value1 value2", "newdynamicfield2_A", "value3 value4", "id", "790"));
-    assertU(commit());
-
-    assertQ("/select?q=newdynamicfield2_A:value3",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
-            "/response/result[@name='response'][@numFound='1']",
-            "count(/response/result[@name='response']/doc) = 1",
-            "/response/result[@name='response']/doc/str[@name='id'][.='790']");
-  }
-
-
-  @Test
-  public void testAddCopyField() throws Exception {
-    assertQ("/schema/dynamicfields/newdynamicfield2_*?indent=on&wt=xml",
-            "count(/response/lst[@name='dynamicField']) = 0",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '404'",
-            "/response/lst[@name='error']/int[@name='code'] = '404'");
-
-    assertJPut("/schema/dynamicfields/dynamicfieldA_*",
-               json("{'type':'text','stored':false}"),
-               "/responseHeader/status==0");
-    assertJPut("/schema/dynamicfields/dynamicfieldB_*",
-               json("{'type':'text','stored':false, 'copyFields':['dynamicfieldA_*']}"),
-               "/responseHeader/status==0");
-    assertJPut("/schema/dynamicfields/dynamicfieldC_*",
-               json("{'type':'text','stored':false, 'copyFields':'dynamicfieldA_*'}"),
-               "/responseHeader/status==0");
-
-    assertQ("/schema/dynamicfields/dynamicfieldB_*?indent=on&wt=xml",
-            "count(/response/lst[@name='dynamicField']) = 1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'");
-    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=dynamicfieldB_*",
-            "count(/response/arr[@name='copyFields']/lst) = 1");
-    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=dynamicfieldC_*",
-            "count(/response/arr[@name='copyFields']/lst) = 1");
-    //fine to pass in empty list, just won't do anything
-    assertJPut("/schema/dynamicfields/dynamicfieldD_*",
-               json("{'type':'text','stored':false, 'copyFields':[]}"),
-               "/responseHeader/status==0");
-    //some bad usages
-    assertJPut("/schema/dynamicfields/dynamicfieldF_*",
-               json("{'type':'text','stored':false, 'copyFields':['some_nonexistent_dynamicfield_ignore_exception_*']}"),
-               "/error/msg==\"copyField dest :\\'some_nonexistent_dynamicfield_ignore_exception_*\\' is not an explicit field and doesn\\'t match a dynamicField.\"");
-  }
-
-  @Test
-  public void testPostMultipleDynamicFields() throws Exception {
-    assertQ("/schema/dynamicfields/newdynamicfield1_*?indent=on&wt=xml",
-            "count(/response/lst[@name='dynamicField']) = 0",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '404'",
-            "/response/lst[@name='error']/int[@name='code'] = '404'");
-
-    assertQ("/schema/dynamicfields/newdynamicfield2_*?indent=on&wt=xml",
-            "count(/response/lst[@name='dynamicField']) = 0",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '404'",
-            "/response/lst[@name='error']/int[@name='code'] = '404'");
-
-    assertJPost("/schema/dynamicfields",
-                json( "[{'name':'newdynamicfield1_*','type':'text','stored':false},"
-                    + " {'name':'newdynamicfield2_*','type':'text','stored':false}]"),
-                "/responseHeader/status==0");
-
-    assertQ("/schema/dynamicfields/newdynamicfield1_*?indent=on&wt=xml",
-            "count(/response/lst[@name='dynamicField']) = 1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'");
-
-    assertQ("/schema/dynamicfields/newdynamicfield2_*?indent=on&wt=xml",
-            "count(/response/lst[@name='dynamicField']) = 1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'");
-
-    assertU(adoc("newdynamicfield1_A", "value1 value2", "id", "123"));
-    assertU(adoc("newdynamicfield2_A", "value3 value4", "id", "456"));
-    assertU(commit());
-
-    assertQ("/select?q=newdynamicfield1_A:value1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
-            "/response/result[@name='response'][@numFound='1']",
-            "count(/response/result[@name='response']/doc/*) = 1",
-            "/response/result[@name='response']/doc/str[@name='id'][.='123']");
-    assertQ("/select?q=newdynamicfield2_A:value3",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
-            "/response/result[@name='response'][@numFound='1']",
-            "count(/response/result[@name='response']/doc/*) = 1",
-            "/response/result[@name='response']/doc/str[@name='id'][.='456']");
-  }
-
-  @Test
-  public void testPostCopy() throws Exception {
-    assertJPost("/schema/dynamicfields",
-                json( "[{'name':'dynamicfieldA_*','type':'text','stored':false},"
-                    + " {'name':'dynamicfieldB_*','type':'text','stored':false},"
-                    + " {'name':'dynamicfieldC_*','type':'text','stored':false, 'copyFields':['dynamicfieldB_*']}]"),
-                "/responseHeader/status==0");
-    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=dynamicfieldC_*",
-            "count(/response/arr[@name='copyFields']/lst) = 1");
-    assertJPost("/schema/dynamicfields",
-                json( "[{'name':'dynamicfieldD_*','type':'text','stored':false},"
-                    + " {'name':'dynamicfieldE_*','type':'text','stored':false},"
-                    + " {'name':'dynamicfieldF_*','type':'text','stored':false, 'copyFields':['dynamicfieldD_*','dynamicfieldE_*']},"
-                    + " {'name':'dynamicfieldG_*','type':'text','stored':false, 'copyFields':'dynamicfieldD_*'}]"),//single
-                "/responseHeader/status==0");
-    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=dynamicfieldF_*",
-            "count(/response/arr[@name='copyFields']/lst) = 2");
-    //passing in an empty list is perfectly acceptable, it just won't do anything
-    assertJPost("/schema/dynamicfields",
-                json( "[{'name':'dynamicfieldX_*','type':'text','stored':false},"
-                    + " {'name':'dynamicfieldY_*','type':'text','stored':false},"
-                    + " {'name':'dynamicfieldZ_*','type':'text','stored':false, 'copyFields':[]}]"),
-                "/responseHeader/status==0");
-    //some bad usages
-
-    assertJPost("/schema/dynamicfields",
-                json( "[{'name':'dynamicfieldH_*','type':'text','stored':false},"
-                    + " {'name':'dynamicfieldI_*','type':'text','stored':false},"
-                    + " {'name':'dynamicfieldJ_*','type':'text','stored':false, 'copyFields':['some_nonexistent_dynamicfield_ignore_exception_*']}]"),
-                "/error/msg=='copyField dest :\\'some_nonexistent_dynamicfield_ignore_exception_*\\' is not an explicit field and doesn\\'t match a dynamicField.'");
-  }
-
-  @Test
-  public void testPostCopyFields() throws Exception {
-    assertJPost("/schema/dynamicfields",
-                json( "[{'name':'dynamicfieldA_*','type':'text','stored':false},"
-                    + " {'name':'dynamicfieldB_*','type':'text','stored':false},"
-                    + " {'name':'dynamicfieldC_*','type':'text','stored':false},"
-                    + " {'name':'dynamicfieldD_*','type':'text','stored':false},"
-                    + " {'name':'dynamicfieldE_*','type':'text','stored':false}]"),
-                "/responseHeader/status==0");
-    assertJPost("/schema/copyfields",
-                json( "[{'source':'dynamicfieldA_*', 'dest':'dynamicfieldB_*'},"
-                    + " {'source':'dynamicfieldD_*', 'dest':['dynamicfieldC_*', 'dynamicfieldE_*']}]"),
-                "/responseHeader/status==0");
-    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=dynamicfieldA_*",
-            "count(/response/arr[@name='copyFields']/lst) = 1");
-    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=dynamicfieldD_*",
-            "count(/response/arr[@name='copyFields']/lst) = 2");
-    assertJPost("/schema/copyfields", // copyField glob sources are not required to match a dynamic field
-                json("[{'source':'some_glob_not_necessarily_matching_any_dynamicfield_*', 'dest':['dynamicfieldA_*']},"
-                    +" {'source':'*', 'dest':['dynamicfieldD_*']}]"),
-                "/responseHeader/status==0");
-    assertJPost("/schema/copyfields",
-                json("[{'source':'dynamicfieldD_*', 'dest':['some_nonexistent_dynamicfield_ignore_exception_*']}]"),
-                "/error/msg=='copyField dest :\\'some_nonexistent_dynamicfield_ignore_exception_*\\' is not an explicit field and doesn\\'t match a dynamicField.'");
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/TestManagedSchemaFieldResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestManagedSchemaFieldResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestManagedSchemaFieldResource.java
deleted file mode 100644
index b39d266..0000000
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestManagedSchemaFieldResource.java
+++ /dev/null
@@ -1,369 +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.commons.io.FileUtils;
-import org.apache.solr.util.RestTestBase;
-import org.eclipse.jetty.servlet.ServletHolder;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.restlet.ext.servlet.ServerServlet;
-
-import java.io.File;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.regex.Pattern;
-
-public class TestManagedSchemaFieldResource extends RestTestBase {
-
-  private static File tmpSolrHome;
-  private static File tmpConfDir;
-
-  private static final String collection = "collection1";
-  private static final String confDir = collection + "/conf";
-
-
-  @Before
-  public void before() throws Exception {
-    tmpSolrHome = createTempDir().toFile();
-    tmpConfDir = new File(tmpSolrHome, confDir);
-    FileUtils.copyDirectory(new File(TEST_HOME()), tmpSolrHome.getAbsoluteFile());
-
-    final SortedMap<ServletHolder,String> extraServlets = new TreeMap<>();
-    final ServletHolder solrRestApi = new ServletHolder("SolrSchemaRestApi", ServerServlet.class);
-    solrRestApi.setInitParameter("org.restlet.application", "org.apache.solr.rest.SolrSchemaRestApi");
-    extraServlets.put(solrRestApi, "/schema/*");  // '/schema/*' matches '/schema', '/schema/', and '/schema/whatever...'
-
-    System.setProperty("managed.schema.mutable", "true");
-    System.setProperty("enable.update.log", "false");
-
-    createJettyAndHarness(tmpSolrHome.getAbsolutePath(), "solrconfig-managed-schema.xml", "schema-rest.xml",
-                          "/solr", true, extraServlets);
-  }
-
-  @After
-  public void after() throws Exception {
-    if (jetty != null) {
-      jetty.stop();
-      jetty = null;
-    }
-    client = null;
-    if (restTestHarness != null) {
-      restTestHarness.close();
-    }
-    restTestHarness = null;
-  }
-  
-  @Test
-  public void testAddFieldBadFieldType() throws Exception {
-    assertJPut("/schema/fields/newfield",
-        json( "{'type':'not_in_there_at_all','stored':false}" ),
-        "/error/msg==\"Field \\'newfield\\': Field type \\'not_in_there_at_all\\' not found.\"");
-  }
-
-  @Test
-  public void testAddFieldMismatchedName() throws Exception {
-    assertJPut("/schema/fields/newfield",
-        json( "{'name':'something_else','type':'text','stored':false}" ),
-        "/error/msg=='///regex:newfield///'");
-  }
-  
-  @Test
-  public void testAddFieldBadProperty() throws Exception {
-    assertJPut("/schema/fields/newfield",
-               json( "{'type':'text','no_property_with_this_name':false}" ),
-               "/error/msg==\"java.lang.IllegalArgumentException: Invalid field property: no_property_with_this_name\"");
-  }
-  
-  @Test
-  public void testAddField() throws Exception {
-    assertQ("/schema/fields/newfield?indent=on&wt=xml",
-            "count(/response/lst[@name='field']) = 0",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '404'",
-            "/response/lst[@name='error']/int[@name='code'] = '404'");
-    
-    assertJPut("/schema/fields/newfield",
-               json("{'type':'text','stored':false}"),
-               "/responseHeader/status==0");
-    
-    assertQ("/schema/fields/newfield?indent=on&wt=xml",
-            "count(/response/lst[@name='field']) = 1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'");
-    
-    assertU(adoc("newfield", "value1 value2", "id", "123"));
-    assertU(commit());
-
-    assertQ("/select?q=newfield:value1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
-            "/response/result[@name='response'][@numFound='1']",
-            "count(/response/result[@name='response']/doc/*) = 1",
-            "/response/result[@name='response']/doc/str[@name='id'][.='123']");
-  }
-
-  @Test
-  public void testAddFieldWithMulipleOptions() throws Exception {
-    assertQ("/schema/fields/newfield?indent=on&wt=xml",
-            "count(/response/lst[@name='field']) = 0",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '404'",
-            "/response/lst[@name='error']/int[@name='code'] = '404'");
-
-    assertJPut("/schema/fields/newfield",
-               json("{'type':'text_en','stored':true,'indexed':false}"),
-               "/responseHeader/status==0");
-
-    File managedSchemaFile = new File(tmpConfDir, "managed-schema");
-    assertTrue(managedSchemaFile.exists());
-    String managedSchemaContents = FileUtils.readFileToString(managedSchemaFile, "UTF-8");
-    Pattern newfieldStoredTrueIndexedFalsePattern 
-        = Pattern.compile( "<field name=\"newfield\" type=\"text_en\" "
-                         + "(?=.*stored=\"true\")(?=.*indexed=\"false\").*/>");
-    assertTrue(newfieldStoredTrueIndexedFalsePattern.matcher(managedSchemaContents).find());
-
-    assertQ("/schema/fields/newfield?indent=on&wt=xml",
-            "count(/response/lst[@name='field']) = 1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
-            "/response/lst[@name='field']/str[@name='name'] = 'newfield'",
-            "/response/lst[@name='field']/str[@name='type'] = 'text_en'",
-            "/response/lst[@name='field']/bool[@name='indexed'] = 'false'",
-            "/response/lst[@name='field']/bool[@name='stored'] = 'true'");
-
-    assertU(adoc("newfield", "value1 value2", "id", "1234"));
-    assertU(commit());
-
-    assertQ("/schema/fields/newfield2?indent=on&wt=xml",
-            "count(/response/lst[@name='field']) = 0",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '404'",
-            "/response/lst[@name='error']/int[@name='code'] = '404'");
-
-    assertJPut("/schema/fields/newfield2",
-               json("{'type':'text_en','stored':true,'indexed':true,'multiValued':true}"),
-               "/responseHeader/status==0");
-
-    managedSchemaContents = FileUtils.readFileToString(managedSchemaFile, "UTF-8");
-    Pattern newfield2StoredTrueIndexedTrueMultiValuedTruePattern
-        = Pattern.compile( "<field name=\"newfield2\" type=\"text_en\" "
-                         + "(?=.*stored=\"true\")(?=.*indexed=\"true\")(?=.*multiValued=\"true\").*/>");
-    assertTrue(newfield2StoredTrueIndexedTrueMultiValuedTruePattern.matcher(managedSchemaContents).find());
-
-    assertQ("/schema/fields/newfield2?indent=on&wt=xml",
-            "count(/response/lst[@name='field']) = 1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
-            "/response/lst[@name='field']/str[@name='name'] = 'newfield2'",
-            "/response/lst[@name='field']/str[@name='type'] = 'text_en'",
-            "/response/lst[@name='field']/bool[@name='indexed'] = 'true'",
-            "/response/lst[@name='field']/bool[@name='stored'] = 'true'",
-            "/response/lst[@name='field']/bool[@name='multiValued'] = 'true'");
-
-    assertU(adoc("newfield2", "value1 value2", "newfield2", "value3 value4", "id", "5678"));
-    assertU(commit());
-
-    assertQ("/select?q=newfield2:value3",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
-            "/response/result[@name='response'][@numFound='1']",
-            "count(/response/result[@name='response']/doc) = 1",
-            "/response/result[@name='response']/doc/str[@name='id'][.='5678']");
-  }
-
-  @Test
-  public void testAddFieldCollectionWithMultipleOptions() throws Exception {
-    assertQ("/schema/fields?indent=on&wt=xml",
-            "count(/response/arr[@name='fields']/lst/str[@name]) > 0",                          // there are fields
-            "count(/response/arr[@name='fields']/lst/str[starts-with(@name,'newfield')]) = 0"); // but none named newfield*
-
-    assertJPost("/schema/fields",
-               json("[{'name':'newfield','type':'text_en','stored':true,'indexed':false}]"),
-               "/responseHeader/status==0");
-
-    File managedSchemaFile = new File(tmpConfDir, "managed-schema");
-    assertTrue(managedSchemaFile.exists());
-    String managedSchemaContents = FileUtils.readFileToString(managedSchemaFile, "UTF-8");
-    Pattern newfieldStoredTrueIndexedFalsePattern
-        = Pattern.compile( "<field name=\"newfield\" type=\"text_en\" "
-                         + "(?=.*stored=\"true\")(?=.*indexed=\"false\").*/>");
-    assertTrue(newfieldStoredTrueIndexedFalsePattern.matcher(managedSchemaContents).find());
-
-    assertQ("/schema/fields?indent=on&wt=xml",
-             "/response/arr[@name='fields']/lst"
-           + "[str[@name='name']='newfield' and str[@name='type']='text_en'"
-           + " and bool[@name='stored']='true' and bool[@name='indexed']='false']");
-
-    assertU(adoc("newfield", "value1 value2", "id", "789"));
-    assertU(commit());
-
-    assertJPost("/schema/fields",
-                json("[{'name':'newfield2','type':'text_en','stored':true,'indexed':true,'multiValued':true}]"),
-                "/responseHeader/status==0");
-
-    managedSchemaContents = FileUtils.readFileToString(managedSchemaFile, "UTF-8");
-    Pattern newfield2StoredTrueIndexedTrueMultiValuedTruePattern
-        = Pattern.compile( "<field name=\"newfield2\" type=\"text_en\" "
-                         + "(?=.*stored=\"true\")(?=.*indexed=\"true\")(?=.*multiValued=\"true\").*/>");
-    assertTrue(newfield2StoredTrueIndexedTrueMultiValuedTruePattern.matcher(managedSchemaContents).find());
-
-    assertQ("/schema/fields?indent=on&wt=xml",
-            "/response/arr[@name='fields']/lst"
-            + "[str[@name='name']='newfield2' and str[@name='type']='text_en'"
-            + " and bool[@name='stored']='true' and bool[@name='indexed']='true' and bool[@name='multiValued']='true']");
-
-    assertU(adoc("newfield2", "value1 value2", "newfield2", "value3 value4", "id", "790"));
-    assertU(commit());
-
-    assertQ("/select?q=newfield2:value3",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
-            "/response/result[@name='response'][@numFound='1']",
-            "count(/response/result[@name='response']/doc) = 1",
-            "/response/result[@name='response']/doc/str[@name='id'][.='790']");
-  }
-
-
-  @Test
-  public void testAddCopyField() throws Exception {
-    assertQ("/schema/fields/newfield2?indent=on&wt=xml",
-            "count(/response/lst[@name='field']) = 0",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '404'",
-            "/response/lst[@name='error']/int[@name='code'] = '404'");
-
-    assertJPut("/schema/fields/fieldA", 
-               json("{'type':'text','stored':false}"),
-               "/responseHeader/status==0");
-    assertJPut("/schema/fields/fieldB",
-               json("{'type':'text','stored':false, 'copyFields':['fieldA']}"),
-               "/responseHeader/status==0");
-    assertJPut("/schema/fields/fieldC",
-               json("{'type':'text','stored':false, 'copyFields':'fieldA'}"),
-               "/responseHeader/status==0");
-
-    assertQ("/schema/fields/fieldB?indent=on&wt=xml",
-            "count(/response/lst[@name='field']) = 1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'");
-    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=fieldB",
-        "count(/response/arr[@name='copyFields']/lst) = 1"
-    );
-    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=fieldC",
-        "count(/response/arr[@name='copyFields']/lst) = 1"
-    );
-    //fine to pass in empty list, just won't do anything
-    assertJPut("/schema/fields/fieldD",
-               json("{'type':'text','stored':false, 'copyFields':[]}"),
-               "/responseHeader/status==0");
-    //some bad usages
-    assertJPut("/schema/fields/fieldF",
-               json("{'type':'text','stored':false, 'copyFields':['some_nonexistent_field_ignore_exception']}"),
-               "/error/msg==\"copyField dest :\\'some_nonexistent_field_ignore_exception\\' is not an explicit field and doesn\\'t match a dynamicField.\"");
-  }
-
-  @Test
-  public void testPostMultipleFields() throws Exception {
-    assertQ("/schema/fields/newfield1?indent=on&wt=xml",
-            "count(/response/lst[@name='field']) = 0",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '404'",
-            "/response/lst[@name='error']/int[@name='code'] = '404'");
-
-    assertQ("/schema/fields/newfield2?indent=on&wt=xml",
-            "count(/response/lst[@name='field']) = 0",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '404'",
-            "/response/lst[@name='error']/int[@name='code'] = '404'");
-
-    assertJPost("/schema/fields",
-                json( "[{'name':'newfield1','type':'text','stored':false},"
-                    + " {'name':'newfield2','type':'text','stored':false}]"),
-                "/responseHeader/status==0");
-
-    assertQ("/schema/fields/newfield1?indent=on&wt=xml",
-            "count(/response/lst[@name='field']) = 1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'");
-
-    assertQ("/schema/fields/newfield2?indent=on&wt=xml",
-            "count(/response/lst[@name='field']) = 1",
-            "/response/lst[@name='responseHeader']/int[@name='status'] = '0'");
-
-    assertU(adoc("newfield1", "value1 value2", "id", "123"));
-    assertU(adoc("newfield2", "value3 value4", "id", "456"));
-    assertU(commit());
-
-    assertQ("/select?q=newfield1:value1",
-        "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
-        "/response/result[@name='response'][@numFound='1']",
-        "count(/response/result[@name='response']/doc/*) = 1",
-        "/response/result[@name='response']/doc/str[@name='id'][.='123']");
-    assertQ("/select?q=newfield2:value3",
-        "/response/lst[@name='responseHeader']/int[@name='status'] = '0'",
-        "/response/result[@name='response'][@numFound='1']",
-        "count(/response/result[@name='response']/doc/*) = 1",
-        "/response/result[@name='response']/doc/str[@name='id'][.='456']");
-  }
-
-  @Test
-  public void testPostCopy() throws Exception {
-    assertJPost("/schema/fields",
-                json( "[{'name':'fieldA','type':'text','stored':false},"
-                    + " {'name':'fieldB','type':'text','stored':false},"
-                    + " {'name':'fieldC','type':'text','stored':false, 'copyFields':['fieldB']}]"),
-                "/responseHeader/status==0");
-    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=fieldC",
-        "count(/response/arr[@name='copyFields']/lst) = 1"
-    );
-    assertJPost("/schema/fields",
-                json( "[{'name':'fieldD','type':'text','stored':false},"
-                    + " {'name':'fieldE','type':'text','stored':false},"
-                    + " {'name':'fieldF','type':'text','stored':false, 'copyFields':['fieldD','fieldE']},"
-                    + " {'name':'fieldG','type':'text','stored':false, 'copyFields':'fieldD'}]"),//single
-                "/responseHeader/status==0");
-    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=fieldF",
-        "count(/response/arr[@name='copyFields']/lst) = 2"
-    );
-    //passing in an empty list is perfectly acceptable, it just won't do anything
-    assertJPost("/schema/fields",
-                json( "[{'name':'fieldX','type':'text','stored':false},"
-                    + " {'name':'fieldY','type':'text','stored':false},"
-                    + " {'name':'fieldZ','type':'text','stored':false, 'copyFields':[]}]"),
-                "/responseHeader/status==0");
-    //some bad usages
-
-    assertJPost("/schema/fields",
-                json( "[{'name':'fieldH','type':'text','stored':false},"
-                    + " {'name':'fieldI','type':'text','stored':false},"
-                    + " {'name':'fieldJ','type':'text','stored':false, 'copyFields':['some_nonexistent_field_ignore_exception']}]"),
-                "/error/msg=='copyField dest :\\'some_nonexistent_field_ignore_exception\\' is not an explicit field and doesn\\'t match a dynamicField.'");
-  }
-
-  @Test
-  public void testPostCopyFields() throws Exception {
-    assertJPost("/schema/fields",
-                json( "[{'name':'fieldA','type':'text','stored':false},"
-                    + " {'name':'fieldB','type':'text','stored':false},"
-                    + " {'name':'fieldC','type':'text','stored':false},"
-                    + " {'name':'fieldD','type':'text','stored':false},"
-                    + " {'name':'fieldE','type':'text','stored':false}]"),
-                "/responseHeader/status==0");
-    assertJPost("/schema/copyfields", 
-                json( "[{'source':'fieldA', 'dest':'fieldB'},"
-                    + " {'source':'fieldD', 'dest':['fieldC', 'fieldE']}]"),
-                "/responseHeader/status==0");
-    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=fieldA",
-        "count(/response/arr[@name='copyFields']/lst) = 1");
-    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=fieldD",
-        "count(/response/arr[@name='copyFields']/lst) = 2");
-    assertJPost("/schema/copyfields", 
-                json("[{'source':'some_nonexistent_field_ignore_exception', 'dest':['fieldA']}]"),
-                "/error/msg=='copyField source :\\'some_nonexistent_field_ignore_exception\\' is not a glob and doesn\\'t match any explicit field or dynamicField.'");
-    assertJPost("/schema/copyfields", 
-                json("[{'source':'fieldD', 'dest':['some_nonexistent_field_ignore_exception']}]"),
-                "/error/msg=='copyField dest :\\'some_nonexistent_field_ignore_exception\\' is not an explicit field and doesn\\'t match a dynamicField.'");
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f2c281ab/solr/core/src/test/org/apache/solr/rest/schema/TestManagedSchemaFieldTypeResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestManagedSchemaFieldTypeResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestManagedSchemaFieldTypeResource.java
deleted file mode 100644
index a0f4e25..0000000
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestManagedSchemaFieldTypeResource.java
+++ /dev/null
@@ -1,350 +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 java.io.File;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.solr.util.RestTestBase;
-import org.eclipse.jetty.servlet.ServletHolder;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.noggit.JSONUtil;
-import org.restlet.ext.servlet.ServerServlet;
-
-public class TestManagedSchemaFieldTypeResource extends RestTestBase {
-  
-  private static File tmpSolrHome;
-  private static File tmpConfDir;
-
-  private static final String collection = "collection1";
-  private static final String confDir = collection + "/conf";
-
-  @Before
-  public void before() throws Exception {
-    tmpSolrHome = createTempDir().toFile();
-    tmpConfDir = new File(tmpSolrHome, confDir);
-    FileUtils.copyDirectory(new File(TEST_HOME()), tmpSolrHome.getAbsoluteFile());
-
-    final SortedMap<ServletHolder,String> extraServlets = new TreeMap<>();
-    final ServletHolder solrRestApi = new ServletHolder("SolrSchemaRestApi", ServerServlet.class);
-    solrRestApi.setInitParameter("org.restlet.application", "org.apache.solr.rest.SolrSchemaRestApi");
-    extraServlets.put(solrRestApi, "/schema/*");  // '/schema/*' matches '/schema', '/schema/', and '/schema/whatever...'
-
-    System.setProperty("managed.schema.mutable", "true");
-    System.setProperty("enable.update.log", "false");
-
-    createJettyAndHarness(tmpSolrHome.getAbsolutePath(), "solrconfig-managed-schema.xml", "schema-rest.xml",
-                          "/solr", true, extraServlets);
-  }
-
-  @After
-  private void after() throws Exception {
-    jetty.stop();
-    jetty = null;
-    System.clearProperty("managed.schema.mutable");
-    System.clearProperty("enable.update.log");
-    
-    if (restTestHarness != null) {
-      restTestHarness.close();
-    }
-    restTestHarness = null;
-  }
-  
-  @Test
-  public void testAddFieldTypes() throws Exception {
-    
-    // name mismatch
-    assertJPut("/schema/fieldtypes/myIntFieldType",
-        json("{'name':'badNameEh','class':'solr.TrieIntField','stored':false}"),
-        "/responseHeader/status==400");    
-    
-    // no class
-    assertJPut("/schema/fieldtypes/myIntFieldType",
-        json("{'stored':false}"),
-        "/responseHeader/status==400");
-    
-    // invalid attribute
-    assertJPut("/schema/fieldtypes/myIntFieldType",
-        json("{'foo':'bar'}"),
-        "/responseHeader/status==400");
-    
-    // empty analyzer
-    String ftdef = "";
-    ftdef += "{";
-    ftdef += "  'class':'solr.TextField','positionIncrementGap':'100',";
-    ftdef += "  'analyzer':''";
-    ftdef += "}";    
-    assertJPut("/schema/fieldtypes/emptyAnalyzerFieldType",
-        json(ftdef),
-        "/responseHeader/status==400");
-
-    // basic field types
-    assertJPut("/schema/fieldtypes/myIntFieldType",
-        json("{'name':'myIntFieldType','class':'solr.TrieIntField','stored':false}"),
-        "/responseHeader/status==0");    
-    checkFieldTypeProps(getExpectedProps("myIntFieldType", "solr.TrieIntField", true, false), 16);
-    
-    assertJPut("/schema/fieldtypes/myDoubleFieldType",
-        json("{'class':'solr.TrieDoubleField','precisionStep':'0','positionIncrementGap':'0'}"),
-        "/responseHeader/status==0");
-    Map<String,Object> expProps = 
-        getExpectedProps("myDoubleFieldType", "solr.TrieDoubleField", true, true);
-    // add some additional expected props for this type
-    expProps.put("precisionStep", "0");
-    expProps.put("positionIncrementGap", "0");
-    checkFieldTypeProps(expProps, 18);
-    
-    assertJPut("/schema/fieldtypes/myBoolFieldType",
-        json("{'class':'solr.BoolField','sortMissingLast':true}"),
-        "/responseHeader/status==0");
-    expProps = getExpectedProps("myBoolFieldType", "solr.BoolField", true, true);
-    expProps.put("sortMissingLast", true);
-    checkFieldTypeProps(expProps, 17);    
-    
-    // a text analyzing field type
-    ftdef = "{";
-    ftdef += "  'class':'solr.TextField','positionIncrementGap':'100',";
-    ftdef += "  'analyzer':{";
-    ftdef += "    'charFilters':[";
-    ftdef += "       {'class':'solr.PatternReplaceCharFilterFactory','replacement':'$1$1','pattern':'([a-zA-Z])\\\\1+'}";
-    ftdef += "    ],";
-    ftdef += "    'tokenizer':{'class':'solr.WhitespaceTokenizerFactory'},";
-    ftdef += "    'filters':[";
-    ftdef += "       {'class':'solr.WordDelimiterFilterFactory','preserveOriginal':'0'},";
-    ftdef += "       {'class':'solr.StopFilterFactory','words':'stopwords.txt','ignoreCase':'true'},";
-    ftdef += "       {'class':'solr.LowerCaseFilterFactory'},";
-    ftdef += "       {'class':'solr.ASCIIFoldingFilterFactory'},";
-    ftdef += "       {'class':'solr.KStemFilterFactory'}";
-    ftdef += "    ]";
-    ftdef += "  }";
-    ftdef += "}";
-  
-    assertJPut("/schema/fieldtypes/myTextFieldType", json(ftdef), "/responseHeader/status==0");
-    
-    expProps = getExpectedProps("myTextFieldType", "solr.TextField", true, true);
-    expProps.put("autoGeneratePhraseQueries", false);
-    expProps.put("omitNorms", false);
-    expProps.put("omitTermFreqAndPositions", false);
-    expProps.put("omitPositions", false);
-    expProps.put("storeOffsetsWithPositions", false);
-    expProps.put("tokenized", true);
-            
-    List<String> analyzerTests = new ArrayList<>();
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='charFilters']/lst[1]/str[@name='class'] = 'solr.PatternReplaceCharFilterFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/lst[@name='tokenizer']/str[@name='class'] = 'solr.WhitespaceTokenizerFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst[1]/str[@name='class'] = 'solr.WordDelimiterFilterFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst[2]/str[@name='class'] = 'solr.StopFilterFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst[3]/str[@name='class'] = 'solr.LowerCaseFilterFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst[4]/str[@name='class'] = 'solr.ASCIIFoldingFilterFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst[5]/str[@name='class'] = 'solr.KStemFilterFactory'");
-    checkFieldTypeProps(expProps, 19, analyzerTests);    
-    
-    // now add a field type that uses managed resources and a field that uses that type
-    
-    String piglatinStopWordEndpoint = "/schema/analysis/stopwords/piglatin";            
-    String piglatinSynonymEndpoint = "/schema/analysis/synonyms/piglatin";    
-    
-    // now define a new FieldType that uses the managed piglatin endpoints
-    // the managed endpoints will be autovivified as needed 
-    ftdef = "{";
-    ftdef += "  'class':'solr.TextField',";
-    ftdef += "  'analyzer':{";
-    ftdef += "    'tokenizer':{'class':'solr.StandardTokenizerFactory'},";
-    ftdef += "    'filters':[";
-    ftdef += "       {'class':'solr.ManagedStopFilterFactory','managed':'piglatin'},";
-    ftdef += "       {'class':'solr.ManagedSynonymFilterFactory','managed':'piglatin'}";
-    ftdef += "    ]";
-    ftdef += "  }";
-    ftdef += "}";
-    assertJPut("/schema/fieldtypes/piglatinFieldType", json(ftdef), "/responseHeader/status==0");
-    
-    expProps = getExpectedProps("piglatinFieldType", "solr.TextField", true, true);
-    expProps.put("autoGeneratePhraseQueries", false);
-    expProps.put("omitNorms", false);
-    expProps.put("omitTermFreqAndPositions", false);
-    expProps.put("omitPositions", false);
-    expProps.put("storeOffsetsWithPositions", false);
-    expProps.put("tokenized", true);
-            
-    analyzerTests = new ArrayList<>();
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/lst[@name='tokenizer']/str[@name='class'] = 'solr.StandardTokenizerFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst[1]/str[@name='class'] = 'solr.ManagedStopFilterFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst[2]/str[@name='class'] = 'solr.ManagedSynonymFilterFactory'");
-    checkFieldTypeProps(expProps, 18, analyzerTests);
-    
-    assertJQ(piglatinSynonymEndpoint, 
-        "/synonymMappings/initArgs/ignoreCase==false",
-        "/synonymMappings/managedMap=={}");
-
-    // add some piglatin synonyms
-    Map<String,List<String>> syns = new HashMap<>();
-    syns.put("appyhay", Arrays.asList("ladgay","oyfuljay"));    
-    assertJPut(piglatinSynonymEndpoint, 
-              JSONUtil.toJSON(syns),
-              "/responseHeader/status==0");    
-    assertJQ(piglatinSynonymEndpoint, 
-            "/synonymMappings/managedMap/appyhay==['ladgay','oyfuljay']");
-    
-    // add some piglatin stopwords
-    assertJPut(piglatinStopWordEndpoint, 
-        JSONUtil.toJSON(Arrays.asList("hetay")),
-        "/responseHeader/status==0");
-   
-    assertJQ(piglatinStopWordEndpoint + "/hetay", "/hetay=='hetay'");
-    
-    // add a field that uses our new type
-    assertJPut("/schema/fields/newManagedField",
-        json("{'type':'piglatinFieldType','stored':false}"),
-        "/responseHeader/status==0");   
-    
-    assertQ("/schema/fields/newManagedField?indent=on&wt=xml",
-        "count(/response/lst[@name='field']) = 1",
-        "/response/lst[@name='responseHeader']/int[@name='status'] = '0'");
-
-    // try to delete the managed synonyms endpoint, which should fail because it is being used
-    assertJDelete(piglatinSynonymEndpoint, "/responseHeader/status==403");
-    
-    // test adding multiple field types at once
-    ftdef = "[";
-    ftdef += "{";
-    ftdef += "  'name':'textFieldType1',";
-    ftdef += "  'class':'solr.TextField','positionIncrementGap':'100',";
-    ftdef += "  'analyzer':{";
-    ftdef += "    'tokenizer':{'class':'solr.WhitespaceTokenizerFactory'},";
-    ftdef += "    'filters':[";
-    ftdef += "       {'class':'solr.WordDelimiterFilterFactory','preserveOriginal':'0'},";
-    ftdef += "       {'class':'solr.StopFilterFactory','words':'stopwords.txt','ignoreCase':'true'},";
-    ftdef += "       {'class':'solr.LowerCaseFilterFactory'}";
-    ftdef += "    ]";
-    ftdef += "  }";
-    ftdef += "},{";
-    ftdef += "  'name':'textFieldType2',";
-    ftdef += "  'class':'solr.TextField','positionIncrementGap':'100',";
-    ftdef += "  'analyzer':{";
-    ftdef += "    'tokenizer':{'class':'solr.WhitespaceTokenizerFactory'},";
-    ftdef += "    'filters':[";
-    ftdef += "       {'class':'solr.WordDelimiterFilterFactory','preserveOriginal':'0'},";
-    ftdef += "       {'class':'solr.StopFilterFactory','words':'stopwords.txt','ignoreCase':'true'},";
-    ftdef += "       {'class':'solr.LowerCaseFilterFactory'},";
-    ftdef += "       {'class':'solr.ASCIIFoldingFilterFactory'}";
-    ftdef += "    ]";
-    ftdef += "  }";
-    ftdef += "}";
-    ftdef += "]";
-  
-    assertJPost("/schema/fieldtypes", json(ftdef), "/responseHeader/status==0");
-    
-    expProps = getExpectedProps("textFieldType1", "solr.TextField", true, true);
-    expProps.put("autoGeneratePhraseQueries", false);
-    expProps.put("omitNorms", false);
-    expProps.put("omitTermFreqAndPositions", false);
-    expProps.put("omitPositions", false);
-    expProps.put("storeOffsetsWithPositions", false);
-    expProps.put("tokenized", true);
-            
-    analyzerTests = new ArrayList<>();
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/lst[@name='tokenizer']/str[@name='class'] = 'solr.WhitespaceTokenizerFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst[1]/str[@name='class'] = 'solr.WordDelimiterFilterFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst[2]/str[@name='class'] = 'solr.StopFilterFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst[3]/str[@name='class'] = 'solr.LowerCaseFilterFactory'");
-    checkFieldTypeProps(expProps, 19, analyzerTests);    
-
-    expProps = getExpectedProps("textFieldType2", "solr.TextField", true, true);
-    expProps.put("autoGeneratePhraseQueries", false);
-    expProps.put("omitNorms", false);
-    expProps.put("omitTermFreqAndPositions", false);
-    expProps.put("omitPositions", false);
-    expProps.put("storeOffsetsWithPositions", false);
-    expProps.put("tokenized", true);
-            
-    analyzerTests = new ArrayList<>();
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/lst[@name='tokenizer']/str[@name='class'] = 'solr.WhitespaceTokenizerFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst[1]/str[@name='class'] = 'solr.WordDelimiterFilterFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst[2]/str[@name='class'] = 'solr.StopFilterFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst[3]/str[@name='class'] = 'solr.LowerCaseFilterFactory'");
-    analyzerTests.add("/response/lst[@name='fieldType']/lst[@name='analyzer']/arr[@name='filters']/lst[4]/str[@name='class'] = 'solr.ASCIIFoldingFilterFactory'");
-    checkFieldTypeProps(expProps, 19, analyzerTests);        
-  }
-  
-  /**
-   * Helper function to check fieldType settings against a set of expected values.
-   */
-  protected void checkFieldTypeProps(Map<String,Object> expected, int expectedChildCount) {
-    checkFieldTypeProps(expected, expectedChildCount, null);
-  }
-  
-  protected void checkFieldTypeProps(Map<String,Object> expected, int expectedChildCount, List<String> addlTests) {
-    String fieldTypeName = (String)expected.get("name");
-    
-    List<String> tests = new ArrayList<>();
-    tests.add("count(/response/lst[@name='fieldType']) = 1");
-    tests.add("count(/response/lst[@name='fieldType']/*) = "+expectedChildCount);
-    tests.add("count(/response/lst[@name='fieldType']/arr[@name='fields']/*) = 0");
-    tests.add("count(/response/lst[@name='fieldType']/arr[@name='dynamicFields']/*) = 0");
-    for (Map.Entry<String,Object> next : expected.entrySet()) {
-      Object val = next.getValue();
-      String pathType = null;
-      if (val instanceof Boolean)
-        pathType = "bool";
-      else if (val instanceof String)
-        pathType = "str";
-      else
-        fail("Unexpected value type "+val.getClass().getName());
-      // NOTE: it seems like the fieldtypes endpoint only returns strings or booleans
-      
-      String xpath = 
-          "/response/lst[@name='fieldType']/"+pathType+"[@name='"+next.getKey()+"']";
-      tests.add(xpath+" = '"+val+"'");
-    }
-    
-    if (addlTests != null)
-      tests.addAll(addlTests);
-    
-    assertQ("/schema/fieldtypes/"+fieldTypeName+"?indent=on&wt=xml&showDefaults=true",
-        tests.toArray(new String[0]));
-  }
-  
-  /**
-   * Builds a map containing expected values for a field type created by this test. 
-   */
-  protected Map<String,Object> getExpectedProps(String name, String className, boolean indexed, boolean stored) {
-    Map<String,Object> map = new HashMap<>();
-    map.put("name", name);
-    map.put("class", className);
-    map.put("indexed", indexed);
-    map.put("stored", stored);      
-    map.put("docValues", false);
-    map.put("termVectors", false);
-    map.put("termPositions", false);
-    map.put("termOffsets", false);
-    map.put("omitNorms", true);
-    map.put("omitTermFreqAndPositions", true);
-    map.put("omitPositions", false);
-    map.put("storeOffsetsWithPositions", false);
-    map.put("multiValued", false);
-    map.put("tokenized", false);
-    return map;
-  }
-}