You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by th...@apache.org on 2020/10/07 20:17:53 UTC

[lucene-solr] branch branch_8x updated: SOLR-14659: Remove restlet as dependency for the ManagedResource API (backport to 8.x) (#1960)

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 8da6fcf  SOLR-14659: Remove restlet as dependency for the ManagedResource API (backport to 8.x) (#1960)
8da6fcf is described below

commit 8da6fcff9edb1a558a629aeec04917df176151d4
Author: Timothy Potter <th...@gmail.com>
AuthorDate: Wed Oct 7 14:17:37 2020 -0600

    SOLR-14659: Remove restlet as dependency for the ManagedResource API (backport to 8.x) (#1960)
    
    Co-authored-by: noblepaul <no...@gmail.com>
---
 dev-tools/maven/solr/pom.xml.template              |   5 -
 lucene/default-nested-ivy-settings.xml             |   2 -
 lucene/ivy-versions.properties                     |   4 -
 solr/CHANGES.txt                                   |   2 +
 solr/NOTICE.txt                                    |  12 -
 .../test/org/apache/solr/ltr/TestRerankBase.java   |  14 +-
 solr/core/ivy.xml                                  |   2 -
 .../org/apache/solr/handler/SchemaHandler.java     |  39 +++
 .../org/apache/solr/rest/BaseSolrResource.java     | 136 ++-------
 .../src/java/org/apache/solr/rest/DELETEable.java  |  26 --
 .../src/java/org/apache/solr/rest/GETable.java     |  26 --
 .../java/org/apache/solr/rest/ManagedResource.java |  23 +-
 .../apache/solr/rest/ManagedResourceStorage.java   |   4 +-
 .../src/java/org/apache/solr/rest/POSTable.java    |  26 --
 .../src/java/org/apache/solr/rest/PUTable.java     |  26 --
 .../src/java/org/apache/solr/rest/RestManager.java | 323 +++++++--------------
 .../org/apache/solr/rest/SolrSchemaRestApi.java    |  78 -----
 .../java/org/apache/solr/rest/package-info.java    |   2 +-
 .../analysis/ManagedSynonymFilterFactory.java      |   6 +-
 .../analysis/ManagedSynonymGraphFilterFactory.java |   6 +-
 .../solr/rest/schema/analysis/package-info.java    |   3 +-
 .../org/apache/solr/rest/schema/package-info.java  |   2 +-
 .../java/org/apache/solr/servlet/HttpSolrCall.java |  20 --
 .../apache/solr/servlet/SolrDispatchFilter.java    |   4 +-
 .../apache/solr/servlet/SolrRequestParsers.java    |  11 +-
 .../apache/solr/core/TestSolrConfigHandler.java    |   4 -
 .../org/apache/solr/request/TestStreamBody.java    |   4 -
 .../org/apache/solr/rest/SolrRestletTestBase.java  |   9 +-
 .../test/org/apache/solr/rest/TestRestManager.java | 147 +---------
 .../schema/TestSerializedLuceneMatchVersion.java   |   4 -
 .../analysis/TestManagedStopFilterFactory.java     |   4 -
 .../analysis/TestManagedSynonymFilterFactory.java  |   4 -
 .../TestManagedSynonymGraphFilterFactory.java      |   4 -
 .../apache/solr/schema/TestCloudSchemaless.java    |   4 -
 solr/licenses/org.restlet-2.4.3.jar.sha1           |   1 -
 solr/licenses/org.restlet-LICENSE-ASL.txt          | 201 -------------
 solr/licenses/org.restlet-NOTICE.txt               |   2 -
 .../org.restlet.ext.servlet-2.4.3.jar.sha1         |   1 -
 .../org.restlet.ext.servlet-LICENSE-ASL.txt        | 201 -------------
 solr/licenses/org.restlet.ext.servlet-NOTICE.txt   |   2 -
 .../solr/client/solrj/request/SchemaTest.java      |   4 -
 solr/webapp/web/WEB-INF/web.xml                    |  14 -
 42 files changed, 203 insertions(+), 1209 deletions(-)

diff --git a/dev-tools/maven/solr/pom.xml.template b/dev-tools/maven/solr/pom.xml.template
index 827eb26..532b0c4 100644
--- a/dev-tools/maven/solr/pom.xml.template
+++ b/dev-tools/maven/solr/pom.xml.template
@@ -76,11 +76,6 @@
   <inceptionYear>2006</inceptionYear>
   <repositories>
     <repository>
-      <id>maven-restlet</id>
-      <name>Public online Restlet repository</name>
-      <url>https://maven.restlet.com</url>
-    </repository>
-    <repository>
       <id>releases.cloudera.com</id>
       <name>Cloudera Releases</name>
       <url>https://repository.cloudera.com/artifactory/libs-release-local/</url>
diff --git a/lucene/default-nested-ivy-settings.xml b/lucene/default-nested-ivy-settings.xml
index c9fe95b..60bc138 100644
--- a/lucene/default-nested-ivy-settings.xml
+++ b/lucene/default-nested-ivy-settings.xml
@@ -33,7 +33,6 @@
 
   <resolvers>
     <ibiblio name="sonatype-releases" root="https://oss.sonatype.org/content/repositories/releases" m2compatible="true" />
-    <ibiblio name="maven.restlet.com" root="https://maven.restlet.com" m2compatible="true" />
     <ibiblio name="releases.cloudera.com" root="https://repository.cloudera.com/artifactory/libs-release-local" m2compatible="true" />
 
     <filesystem name="local-maven-2" m2compatible="true" local="true">
@@ -47,7 +46,6 @@
       <resolver ref="local"/>
       <!-- <resolver ref="local-maven-2" /> -->
       <resolver ref="main"/>
-      <resolver ref="maven.restlet.com" />
       <resolver ref="sonatype-releases" />
       <resolver ref="releases.cloudera.com"/>
     </chain>
diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index 7bca040..d1eca62 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -308,10 +308,6 @@ org.ow2.asm.version = 8.0.1
 /org.ow2.asm/asm = ${org.ow2.asm.version}
 /org.ow2.asm/asm-commons = ${org.ow2.asm.version}
 
-org.restlet.jee.version = 2.4.3
-/org.restlet.jee/org.restlet = ${org.restlet.jee.version}
-/org.restlet.jee/org.restlet.ext.servlet = ${org.restlet.jee.version}
-
 /org.rrd4j/rrd4j = 3.5
 
 org.slf4j.version = 1.7.24
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a8f290b..8875cd5 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -182,6 +182,8 @@ Other Changes
 * SOLR-14910: Use in-line tags for logger declarations in Gradle ValidateLogCalls that are non-standard,
               change //logok to //nowarn (Erick Erickson)
 
+* SOLR-14659: Remove restlet as dependency for the ManagedResource API (Timothy Potter, noble)
+
 ==================  8.6.2 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
diff --git a/solr/NOTICE.txt b/solr/NOTICE.txt
index f60d6b4..f5d6fcc 100644
--- a/solr/NOTICE.txt
+++ b/solr/NOTICE.txt
@@ -546,18 +546,6 @@ ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
 SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 
 =========================================================================
-==     Restlet Notice                                                  ==
-=========================================================================
-
-Copyright (C) 2005-2014 Restlet S.A.S.
-
-Restlet is a registered trademark of Restlet S.A.S.
-
-This product contains software developed by the Restlet project.
-
-See http://www.restlet.org/
-
-=========================================================================
 ==     Protocol Buffers Notice                                         ==
 =========================================================================
 
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestRerankBase.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestRerankBase.java
index 22c0201..5dd7cf0 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestRerankBase.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestRerankBase.java
@@ -44,11 +44,8 @@ import org.apache.solr.ltr.model.ModelException;
 import org.apache.solr.ltr.store.FeatureStore;
 import org.apache.solr.ltr.store.rest.ManagedFeatureStore;
 import org.apache.solr.ltr.store.rest.ManagedModelStore;
-import org.apache.solr.rest.ManagedResourceStorage;
-import org.apache.solr.rest.SolrSchemaRestApi;
 import org.apache.solr.util.RestTestBase;
 import org.eclipse.jetty.servlet.ServletHolder;
-import org.restlet.ext.servlet.ServerServlet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -178,17 +175,8 @@ public class TestRerankBase extends RestTestBase {
               + "/collection1/conf/schema.xml"));
     }
 
-    final SortedMap<ServletHolder,String> extraServlets = new TreeMap<>();
-    final ServletHolder solrRestApi = new ServletHolder("SolrSchemaRestApi",
-        ServerServlet.class);
-    solrRestApi.setInitParameter("org.restlet.application",
-        SolrSchemaRestApi.class.getCanonicalName());
-    solrRestApi.setInitParameter("storageIO",
-        ManagedResourceStorage.InMemoryStorageIO.class.getCanonicalName());
-    extraServlets.put(solrRestApi, PARENT_ENDPOINT);
-
     System.setProperty("managed.schema.mutable", "true");
-
+    final SortedMap<ServletHolder,String> extraServlets = new TreeMap<>();
     return extraServlets;
   }
 
diff --git a/solr/core/ivy.xml b/solr/core/ivy.xml
index 9078012..e2cb029 100644
--- a/solr/core/ivy.xml
+++ b/solr/core/ivy.xml
@@ -47,8 +47,6 @@
     <dependency org="org.apache.commons" name="commons-math3" rev="${/org.apache.commons/commons-math3}" conf="compile"/>
     <dependency org="org.ow2.asm" name="asm" rev="${/org.ow2.asm/asm}" conf="compile"/>
     <dependency org="org.ow2.asm" name="asm-commons" rev="${/org.ow2.asm/asm-commons}" conf="compile"/>
-    <dependency org="org.restlet.jee" name="org.restlet" rev="${/org.restlet.jee/org.restlet}" conf="compile"/>
-    <dependency org="org.restlet.jee" name="org.restlet.ext.servlet" rev="${/org.restlet.jee/org.restlet.ext.servlet}" conf="compile"/>
     <dependency org="com.carrotsearch" name="hppc" rev="${/com.carrotsearch/hppc}" conf="compile"/>
     <dependency org="io.sgr" name="s2-geometry-library-java" rev="${/io.sgr/s2-geometry-library-java}" conf="compile"/>
 
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 12733ec..a0e6fda 100644
--- a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
@@ -39,6 +39,7 @@ import org.apache.solr.pkg.PackageListeningClassLoader;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.rest.RestManager;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.ManagedIndexSchema;
 import org.apache.solr.schema.SchemaManager;
@@ -60,6 +61,7 @@ import static org.apache.solr.schema.IndexSchema.SchemaProps.Handler.FIELD_TYPES
 public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware, PermissionNameProvider {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private boolean isImmutableConfigSet = false;
+  private SolrRequestHandler managedResourceRequestHandler;
 
   private static final Map<String, String> level2;
 
@@ -106,6 +108,8 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
     switch (ctx.getHttpMethod()) {
       case "GET":
         return PermissionNameProvider.Name.SCHEMA_READ_PERM;
+      case "PUT":
+      case "DELETE":
       case "POST":
         return PermissionNameProvider.Name.SCHEMA_EDIT_PERM;
       default:
@@ -261,6 +265,8 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
     String prefix =  parts.get(0);
     if(subPaths.contains(prefix)) return this;
 
+    if(managedResourceRequestHandler != null) return managedResourceRequestHandler;
+
     return null;
   }
 
@@ -277,6 +283,7 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
   @Override
   public void inform(SolrCore core) {
     isImmutableConfigSet = SolrConfigHandler.getImmutable(core);
+    this.managedResourceRequestHandler =  new ManagedResourceRequestHandler(core.getRestManager());
   }
 
   @Override
@@ -294,4 +301,36 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
   public Boolean registerV2() {
     return Boolean.TRUE;
   }
+
+  private  class ManagedResourceRequestHandler extends RequestHandlerBase implements PermissionNameProvider {
+
+
+    private final RestManager restManager;
+
+    private ManagedResourceRequestHandler(RestManager restManager) {
+      this.restManager = restManager;
+    }
+
+    @Override
+    public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) {
+      RestManager.ManagedEndpoint me = new RestManager.ManagedEndpoint(restManager);
+      me.doInit(req, rsp);
+      me.delegateRequestToManagedResource();
+    }
+
+    @Override
+    public Name getPermissionName(AuthorizationContext ctx) {
+      return SchemaHandler.this.getPermissionName(ctx);
+    }
+
+    @Override
+    public String getName() {
+      return null;
+    }
+
+    @Override
+    public String getDescription() {
+      return null;
+    }
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/rest/BaseSolrResource.java b/solr/core/src/java/org/apache/solr/rest/BaseSolrResource.java
index 5b765c8..f07bb27 100644
--- a/solr/core/src/java/org/apache/solr/rest/BaseSolrResource.java
+++ b/solr/core/src/java/org/apache/solr/rest/BaseSolrResource.java
@@ -15,37 +15,28 @@
  * limitations under the License.
  */
 package org.apache.solr.rest;
-import java.io.IOException;
-import java.io.OutputStream;
+
 import java.io.UnsupportedEncodingException;
 import java.net.URLDecoder;
 
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
-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.servlet.ResponseUtils;
-import org.restlet.data.MediaType;
-import org.restlet.data.Method;
-import org.restlet.data.Status;
-import org.restlet.representation.OutputRepresentation;
-import org.restlet.resource.ResourceException;
-import org.restlet.resource.ServerResource;
 import org.slf4j.Logger;
-
 import static org.apache.solr.common.params.CommonParams.JSON;
 
 /**
- * Base class of all Solr Restlet server resource classes.
+ * Base class for delegating REST-oriented requests to ManagedResources. ManagedResources are heavy-weight and
+ * should not be created for every request, so this class serves as a gateway between a REST call and the resource.
  */
-public abstract class BaseSolrResource extends ServerResource {
+public abstract class BaseSolrResource {
   protected static final String SHOW_DEFAULTS = "showDefaults";
   public static final String UPDATE_TIMEOUT_SECS = "updateTimeoutSecs";
 
@@ -56,6 +47,7 @@ public abstract class BaseSolrResource extends ServerResource {
   private QueryResponseWriter responseWriter;
   private String contentType;
   private int updateTimeoutSecs = -1;
+  private int statusCode = -1;
 
   public SolrCore getSolrCore() { return solrCore; }
   public IndexSchema getSchema() { return schema; }
@@ -73,95 +65,33 @@ public abstract class BaseSolrResource extends ServerResource {
    * from the SolrRequestInfo thread local, then gets the SolrCore
    * and IndexSchema and sets up the response.
    * writer.
-   * <p>
-   * If an error occurs during initialization, setExisting(false) is
-   * called and an error status code and message is set; in this case,
-   * Restlet will not continue servicing the request (by calling the
-   * method annotated to associate it with GET, etc., but rather will
-   * send an error response.
    */
-  @Override
-  public void doInit() throws ResourceException {
-    super.doInit();
-    setNegotiated(false); // Turn off content negotiation for now
-    if (isExisting()) {
-      try {
-        SolrRequestInfo solrRequestInfo = SolrRequestInfo.getRequestInfo();
-        if (null == solrRequestInfo) {
-          final String message = "No handler or core found in " + getRequest().getOriginalRef().getPath();
-          doError(Status.CLIENT_ERROR_BAD_REQUEST, message);
-          setExisting(false);
-        } else {
-          solrRequest = solrRequestInfo.getReq();
-          if (null == solrRequest) {
-            final String message = "No handler or core found in " + getRequest().getOriginalRef().getPath();
-            doError(Status.CLIENT_ERROR_BAD_REQUEST, message);
-            setExisting(false);
-          } else {
-            solrResponse = solrRequestInfo.getRsp();
-            solrCore = solrRequest.getCore();
-            schema = solrRequest.getSchema();
-            String responseWriterName = solrRequest.getParams().get(CommonParams.WT);
-            if (null == responseWriterName) {
-              responseWriterName = JSON; // Default to json writer
-            }
-            String indent = solrRequest.getParams().get("indent");
-            if (null == indent || ! ("off".equals(indent) || "false".equals(indent))) {
-              // indent by default
-              ModifiableSolrParams newParams = new ModifiableSolrParams(solrRequest.getParams());
-              newParams.remove(indent);
-              newParams.add("indent", "on");
-              solrRequest.setParams(newParams);
-            }
-            responseWriter = solrCore.getQueryResponseWriter(responseWriterName);
-            contentType = responseWriter.getContentType(solrRequest, solrResponse);
-            final String path = getRequest().getRootRef().getPath();
-            if ( ! RestManager.SCHEMA_BASE_PATH.equals(path)) {
-              // don't set webapp property on the request when context and core/collection are excluded 
-              final int cutoffPoint = path.indexOf("/", 1);
-              final String firstPathElement = -1 == cutoffPoint ? path : path.substring(0, cutoffPoint);
-              solrRequest.getContext().put("webapp", firstPathElement); // Context path
-            }
-            SolrCore.preDecorateResponse(solrRequest, solrResponse);
-
-            // client application can set a timeout for update requests
-            Object updateTimeoutSecsParam = getSolrRequest().getParams().get(UPDATE_TIMEOUT_SECS);
-            if (updateTimeoutSecsParam != null)
-              updateTimeoutSecs = (updateTimeoutSecsParam instanceof Number)
-                  ? ((Number) updateTimeoutSecsParam).intValue()
-                  : Integer.parseInt(updateTimeoutSecsParam.toString());
-
-          }
-        }
-      } catch (Throwable t) {
-        if (t instanceof OutOfMemoryError) {
-          throw (OutOfMemoryError) t;
-        }
-        setExisting(false);
-        throw new ResourceException(t);
+  public void doInit(SolrQueryRequest solrRequest, SolrQueryResponse solrResponse) {
+    try {
+      this.solrRequest = solrRequest;
+      this.solrResponse = solrResponse;
+      solrCore = solrRequest.getCore();
+      schema = solrRequest.getSchema();
+      String responseWriterName = solrRequest.getParams().get(CommonParams.WT, JSON);
+      responseWriter = solrCore.getQueryResponseWriter(responseWriterName);
+      contentType = responseWriter.getContentType(solrRequest, solrResponse);
+      final String path = solrRequest.getPath();
+      if ( ! RestManager.SCHEMA_BASE_PATH.equals(path)) {
+        // don't set webapp property on the request when context and core/collection are excluded
+        final int cutoffPoint = path.indexOf("/", 1);
+        final String firstPathElement = -1 == cutoffPoint ? path : path.substring(0, cutoffPoint);
+        solrRequest.getContext().put("webapp", firstPathElement); // Context path
       }
-    }
-  }
 
-  /**
-   * This class serves as an adapter between Restlet and Solr's response writers. 
-   */
-  public class SolrOutputRepresentation extends OutputRepresentation {
-    
-    public SolrOutputRepresentation() {
-      // No normalization, in case of a custom media type
-      super(MediaType.valueOf(contentType));
-      // TODO: For now, don't send the Vary: header, but revisit if/when content negotiation is added
-      getDimensions().clear();
-    }
-    
-    
-    /** Called by Restlet to get the response body */
-    @Override
-    public void write(OutputStream outputStream) throws IOException {
-      if (getRequest().getMethod() != Method.HEAD) {
-        QueryResponseWriterUtil.writeQueryResponse(outputStream, responseWriter, solrRequest, solrResponse, contentType);
+      // client application can set a timeout for update requests
+      String updateTimeoutSecsParam = solrRequest.getParams().get(UPDATE_TIMEOUT_SECS);
+      if (updateTimeoutSecsParam != null)
+        updateTimeoutSecs = Integer.parseInt(updateTimeoutSecsParam);
+    } catch (Throwable t) {
+      if (t instanceof OutOfMemoryError) {
+        throw (OutOfMemoryError) t;
       }
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, t);
     }
   }
 
@@ -172,9 +102,7 @@ public abstract class BaseSolrResource extends ServerResource {
   protected void handlePostExecution(Logger log) {
     
     handleException(log);
-    
-    // TODO: should status=0 (success?) be left as-is in the response header?
-    SolrCore.postDecorateResponse(null, solrRequest, solrResponse);
+
     addDeprecatedWarning();
 
     if (log.isInfoEnabled() && solrResponse.getToLog().size() > 0) {
@@ -184,7 +112,6 @@ public abstract class BaseSolrResource extends ServerResource {
 
   protected void addDeprecatedWarning(){
     solrResponse.add("warn","This API is deprecated");
-
   }
 
   /**
@@ -200,8 +127,7 @@ public abstract class BaseSolrResource extends ServerResource {
     if (null != exception) {
       @SuppressWarnings({"rawtypes"})
       NamedList info = new SimpleOrderedMap();
-      int code = ResponseUtils.getErrorInfo(exception, info, log);
-      setStatus(Status.valueOf(code));
+      this.statusCode = ResponseUtils.getErrorInfo(exception, info, log);
       getSolrResponse().add("error", info);
       String message = (String)info.get("msg");
       if (null != message && ! message.trim().isEmpty()) {
diff --git a/solr/core/src/java/org/apache/solr/rest/DELETEable.java b/solr/core/src/java/org/apache/solr/rest/DELETEable.java
deleted file mode 100644
index 591f35c..0000000
--- a/solr/core/src/java/org/apache/solr/rest/DELETEable.java
+++ /dev/null
@@ -1,26 +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;
-
-import org.restlet.representation.Representation;
-import org.restlet.resource.Delete;
-
-/** Marker interface for resource classes that handle DELETE requests. */
-public interface DELETEable {
-  @Delete
-  public Representation delete();
-}
diff --git a/solr/core/src/java/org/apache/solr/rest/GETable.java b/solr/core/src/java/org/apache/solr/rest/GETable.java
deleted file mode 100644
index 131ffe9..0000000
--- a/solr/core/src/java/org/apache/solr/rest/GETable.java
+++ /dev/null
@@ -1,26 +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;
-
-import org.restlet.representation.Representation;
-import org.restlet.resource.Get;
-
-/** Marker interface for resource classes that handle GET requests. */
-public interface GETable {
-  @Get
-  public Representation get();
-}
diff --git a/solr/core/src/java/org/apache/solr/rest/ManagedResource.java b/solr/core/src/java/org/apache/solr/rest/ManagedResource.java
index f9da549..c6e7adf 100644
--- a/solr/core/src/java/org/apache/solr/rest/ManagedResource.java
+++ b/solr/core/src/java/org/apache/solr/rest/ManagedResource.java
@@ -31,9 +31,6 @@ import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.rest.ManagedResourceStorage.StorageIO;
-import org.restlet.data.Status;
-import org.restlet.representation.Representation;
-import org.restlet.resource.ResourceException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -154,7 +151,7 @@ public abstract class ManagedResource {
   }
 
   /**
-   * Called from {@link #doPut(BaseSolrResource,Representation,Object)}
+   * Called from {@link #doPut(BaseSolrResource,Object)}
    * to update this resource's init args using the given updatedArgs
    */
   @SuppressWarnings("unchecked")
@@ -277,7 +274,7 @@ public abstract class ManagedResource {
           "Failed to store data for %s due to: %s",
           resourceId, storeErr.toString());
       log.error(errMsg, storeErr);
-      throw new ResourceException(Status.SERVER_ERROR_INTERNAL, errMsg, storeErr);
+      throw new SolrException(ErrorCode.SERVER_ERROR, errMsg, storeErr);
     }
   }
 
@@ -352,18 +349,18 @@ public abstract class ManagedResource {
   }
   
   /**
-   * Just calls {@link #doPut(BaseSolrResource,Representation,Object)};
+   * Just calls {@link #doPut(BaseSolrResource,Object)};
    * override to change the behavior of POST handling.
    */
-  public void doPost(BaseSolrResource endpoint, Representation entity, Object json) {
-    doPut(endpoint, entity, json);
+  public void doPost(BaseSolrResource endpoint, Object json) {
+    doPut(endpoint, json);
   }
   
   /**
    * Applies changes to initArgs or managed data.
    */
   @SuppressWarnings("unchecked")
-  public synchronized void doPut(BaseSolrResource endpoint, Representation entity, Object json) {
+  public synchronized void doPut(BaseSolrResource endpoint, Object json) {
 
     if (log.isInfoEnabled()) {
       log.info("Processing update to {}: {} is a {}", getResourceId(), json, json.getClass().getName());
@@ -392,7 +389,7 @@ public abstract class ManagedResource {
     } else if (json instanceof List) {
       managedData = json;
     } else {
-      throw new ResourceException(Status.CLIENT_ERROR_BAD_REQUEST, 
+      throw new SolrException(ErrorCode.BAD_REQUEST,
           "Unsupported update format "+json.getClass().getName());
     }
         
@@ -425,15 +422,13 @@ public abstract class ManagedResource {
   protected abstract Object applyUpdatesToManagedData(Object updates);
 
   /**
-   * Called by {@link RestManager.ManagedEndpoint#delete()}
-   * to delete a named part (the given childId) of the
+   * Called to delete a named part (the given childId) of the
    * resource at the given endpoint
    */
   public abstract void doDeleteChild(BaseSolrResource endpoint, String childId);
 
   /**
-   * Called by {@link RestManager.ManagedEndpoint#get()}
-   * to retrieve a named part (the given childId) of the
+   * Called to retrieve a named part (the given childId) of the
    * resource at the given endpoint
    */
   public abstract void doGet(BaseSolrResource endpoint, String childId);
diff --git a/solr/core/src/java/org/apache/solr/rest/ManagedResourceStorage.java b/solr/core/src/java/org/apache/solr/rest/ManagedResourceStorage.java
index 52ad830..e9102a2 100644
--- a/solr/core/src/java/org/apache/solr/rest/ManagedResourceStorage.java
+++ b/solr/core/src/java/org/apache/solr/rest/ManagedResourceStorage.java
@@ -44,8 +44,6 @@ import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.SolrResourceLoader;
-import org.restlet.data.Status;
-import org.restlet.resource.ResourceException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -302,7 +300,7 @@ public abstract class ManagedResourceStorage {
             if (e instanceof RuntimeException) {
               throw (RuntimeException)e;              
             } else {
-              throw new ResourceException(Status.SERVER_ERROR_INTERNAL, 
+              throw new SolrException(ErrorCode.SERVER_ERROR,
                   "Failed to save data to ZooKeeper znode: "+znodePath+" due to: "+e, e);
             }
           }
diff --git a/solr/core/src/java/org/apache/solr/rest/POSTable.java b/solr/core/src/java/org/apache/solr/rest/POSTable.java
deleted file mode 100644
index 5b7fd4e..0000000
--- a/solr/core/src/java/org/apache/solr/rest/POSTable.java
+++ /dev/null
@@ -1,26 +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;
-
-import org.restlet.representation.Representation;
-import org.restlet.resource.Post;
-
-/** Marker interface for resource classes that handle POST requests. */
-public interface POSTable {
-  @Post
-  public Representation post(Representation representation);
-}
diff --git a/solr/core/src/java/org/apache/solr/rest/PUTable.java b/solr/core/src/java/org/apache/solr/rest/PUTable.java
deleted file mode 100644
index e8b27b4..0000000
--- a/solr/core/src/java/org/apache/solr/rest/PUTable.java
+++ /dev/null
@@ -1,26 +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;
-
-import org.restlet.representation.Representation;
-import org.restlet.resource.Put;
-
-/** Marker interface for resource classes that handle PUT requests. */
-public interface PUTable {
-  @Put
-  public Representation put(Representation entity);
-}
diff --git a/solr/core/src/java/org/apache/solr/rest/RestManager.java b/solr/core/src/java/org/apache/solr/rest/RestManager.java
index addee77..ea1d3c0 100644
--- a/solr/core/src/java/org/apache/solr/rest/RestManager.java
+++ b/solr/core/src/java/org/apache/solr/rest/RestManager.java
@@ -18,13 +18,17 @@ package org.apache.solr.rest;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.Reader;
+import java.io.UnsupportedEncodingException;
 import java.lang.invoke.MethodHandles;
 import java.lang.reflect.Constructor;
+import java.net.URLDecoder;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Locale;
@@ -36,22 +40,15 @@ import java.util.regex.Pattern;
 
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.request.SolrRequestInfo;
+import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.rest.ManagedResourceStorage.StorageIO;
-import org.restlet.Request;
-import org.restlet.data.MediaType;
-import org.restlet.data.Method;
-import org.restlet.data.Status;
-import org.restlet.representation.Representation;
-import org.restlet.resource.ResourceException;
-import org.restlet.routing.Router;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.common.util.Utils.fromJSONString;
+import static org.apache.solr.common.util.Utils.fromJSON;
 
 /**
  * Supports runtime mapping of REST API endpoints to ManagedResource 
@@ -122,11 +119,6 @@ public class RestManager {
 
     public Registry() {
       reservedEndpoints.add(SCHEMA_BASE_PATH + MANAGED_ENDPOINT);
-
-      for (String reservedEndpoint : SolrSchemaRestApi.getReservedEndpoints()) {
-        reservedEndpoints.add(reservedEndpoint);
-      }
-
       reservedEndpointsPattern = getReservedEndpointsPattern();
     }
 
@@ -210,7 +202,7 @@ public class RestManager {
       // it's ok to re-register the same class for an existing path
       ManagedResourceRegistration reg = registered.get(resourceId);
       if (reg != null) {
-        if (!reg.implClass.equals(implClass)) {
+        if (!implClass.equals(reg.implClass)) {
           String errMsg = String.format(Locale.ROOT,
               "REST API path %s already registered to instances of %s",
               resourceId, reg.implClass.getName());
@@ -240,46 +232,42 @@ public class RestManager {
   }  
 
   /**
-   * Locates the RestManager using ThreadLocal SolrRequestInfo.
+   * Request handling needs a lightweight object to delegate a request to.
+   * ManagedResource implementations are heavy-weight objects that live for the duration of
+   * a SolrCore, so this class acts as the proxy between the request handler and a
+   * ManagedResource when doing request processing.
    */
-  public static RestManager getRestManager(SolrRequestInfo solrRequestInfo) {
-    if (solrRequestInfo == null)
-      throw new ResourceException(Status.SERVER_ERROR_INTERNAL, 
-          "No SolrRequestInfo in this Thread!");
-
-    SolrQueryRequest req = solrRequestInfo.getReq();
-    RestManager restManager = 
-        (req != null) ? req.getCore().getRestManager() : null;
-    
-    if (restManager == null)
-      throw new ResourceException(Status.SERVER_ERROR_INTERNAL, 
-          "No RestManager found!");
-    
-    return restManager;
-  }
-  
-  /**
-   * The Restlet router needs a lightweight extension of ServerResource to delegate a request
-   * to. ManagedResource implementations are heavy-weight objects that live for the duration of
-   * a SolrCore, so this class acts as the proxy between Restlet and a ManagedResource when
-   * doing request processing.
-   *
-   */
-  public static class ManagedEndpoint extends BaseSolrResource
-      implements GETable, PUTable, POSTable, DELETEable
-  {
+  public static class ManagedEndpoint extends BaseSolrResource {
+
+    final RestManager restManager;
+
+    public ManagedEndpoint(RestManager restManager) {
+      this.restManager = restManager;
+    }
+
     /**
-     * Determines the ManagedResource resourceId from the Restlet request.
+     * Determines the ManagedResource resourceId from the request path.
      */
-    public static String resolveResourceId(Request restletReq)  {
-      String resourceId = restletReq.getResourceRef().
-          getRelativeRef(restletReq.getRootRef().getParentRef()).getPath(DECODE);
-      
+    public static String resolveResourceId(final String path)  {
+      String resourceId;
+      try {
+        resourceId = URLDecoder.decode(path, "UTF-8");
+      } catch (UnsupportedEncodingException e) {
+        throw new RuntimeException(e); // shouldn't happen
+      }
+
+      int at = resourceId.indexOf("/schema");
+      if (at == -1) {
+        at = resourceId.indexOf("/config");
+      }
+      if (at > 0) {
+        resourceId = resourceId.substring(at);
+      }
+
       // all resources are registered with the leading slash
       if (!resourceId.startsWith("/"))
         resourceId = "/"+resourceId;
 
-
       return resourceId;
     }
     
@@ -292,18 +280,11 @@ public class RestManager {
      * dynamically locate the ManagedResource associated with the request URI.
      */
     @Override
-    public void doInit() throws ResourceException {
-      super.doInit();      
-      
-      // get the relative path to the requested resource, which is
-      // needed to locate ManagedResource impls at runtime
-      String resourceId = resolveResourceId(getRequest());
+    public void doInit(SolrQueryRequest solrRequest, SolrQueryResponse solrResponse) {
+      super.doInit(solrRequest, solrResponse);
 
-      // supports a request for a registered resource or its child
-      RestManager restManager = 
-          RestManager.getRestManager(SolrRequestInfo.getRequestInfo());
-      
-      managedResource = restManager.getManagedResourceOrNull(resourceId);      
+      final String resourceId = resolveResourceId(solrRequest.getPath());
+      managedResource = restManager.getManagedResourceOrNull(resourceId);
       if (managedResource == null) {
         // see if we have a registered endpoint one-level up ...
         int lastSlashAt = resourceId.lastIndexOf('/');
@@ -317,7 +298,7 @@ public class RestManager {
             if (!(managedResource instanceof ManagedResource.ChildResourceSupport)) {
               String errMsg = String.format(Locale.ROOT,
                   "%s does not support child resources!", managedResource.getResourceId());
-              throw new ResourceException(Status.CLIENT_ERROR_BAD_REQUEST, errMsg);
+              throw new SolrException(ErrorCode.BAD_REQUEST, errMsg);
             }
             
             childId = resourceId.substring(lastSlashAt+1);
@@ -326,44 +307,46 @@ public class RestManager {
           }
         }
       }    
-      
+
       if (managedResource == null) {
-        if (Method.PUT.equals(getMethod()) || Method.POST.equals(getMethod())) {
+        final String method = getSolrRequest().getHttpMethod();
+        if ("PUT".equals(method) || "POST".equals(method)) {
           // delegate create requests to the RestManager
           managedResource = restManager.endpoint;
-        } else {        
-          throw new ResourceException(Status.CLIENT_ERROR_NOT_FOUND, 
+        } else {
+          throw new SolrException(ErrorCode.BAD_REQUEST,
               "No REST managed resource registered for path "+resourceId);
         }
       }
-      
+
       log.info("Found ManagedResource [{}] for {}", managedResource, resourceId);
-    }    
-    
-    @Override
-    public Representation put(Representation entity) {
-      try {
-        managedResource.doPut(this, entity, parseJsonFromRequestBody(entity));
-      } catch (Exception e) {
-        getSolrResponse().setException(e);        
-      }
-      handlePostExecution(log);
-      return new SolrOutputRepresentation();    
     }
-    
-    @Override
-    public Representation post(Representation entity) {
+
+    public void delegateRequestToManagedResource() {
+      SolrQueryRequest req = getSolrRequest();
+      final String method = req.getHttpMethod();
       try {
-        managedResource.doPost(this, entity, parseJsonFromRequestBody(entity));
+        switch (method) {
+          case "GET":
+            managedResource.doGet(this, childId);
+            break;
+          case "PUT":
+            managedResource.doPut(this, parseJsonFromRequestBody(req));
+            break;
+          case "POST":
+            managedResource.doPost(this, parseJsonFromRequestBody(req));
+            break;
+          case "DELETE":
+            doDelete();
+            break;
+        }
       } catch (Exception e) {
-        getSolrResponse().setException(e);        
+        getSolrResponse().setException(e);
       }
       handlePostExecution(log);
-      return new SolrOutputRepresentation();    
-    }    
+    }
 
-    @Override
-    public Representation delete() {
+    protected void doDelete() {
       // only delegate delete child resources to the ManagedResource
       // as deleting the actual resource is best handled by the
       // RestManager
@@ -375,68 +358,24 @@ public class RestManager {
         }
       } else {
         try {
-          RestManager restManager = 
-              RestManager.getRestManager(SolrRequestInfo.getRequestInfo());
           restManager.deleteManagedResource(managedResource);
         } catch (Exception e) {
           getSolrResponse().setException(e);        
         }
       }
       handlePostExecution(log);
-      return new SolrOutputRepresentation();    
-    }    
-        
-    @Override
-    public Representation get() { 
-      try {
-        managedResource.doGet(this, childId);
-      } catch (Exception e) {
-        getSolrResponse().setException(e);        
-      }
-      handlePostExecution(log);
-      return new SolrOutputRepresentation();    
-    }     
-    
-    /**
-     * Parses and validates the JSON passed from the to the ManagedResource. 
-     */
-    protected Object parseJsonFromRequestBody(Representation entity) {
-      if (entity.getMediaType() == null) {
-        entity.setMediaType(MediaType.APPLICATION_JSON);
-      }
-      
-      if (!entity.getMediaType().equals(MediaType.APPLICATION_JSON, true)) {
-        String errMsg = String.format(Locale.ROOT,
-            "Invalid content type %s; only %s is supported.",
-            entity.getMediaType(), MediaType.APPLICATION_JSON.toString());
-        log.error(errMsg);
-        throw new ResourceException(Status.CLIENT_ERROR_BAD_REQUEST, errMsg);
-      }
-      
-      String text = null;
-      try {
-        text = entity.getText();
-      } catch (IOException ioExc) {
-        String errMsg = "Failed to read entity text due to: "+ioExc;
-        log.error(errMsg, ioExc);
-        throw new ResourceException(Status.SERVER_ERROR_INTERNAL, errMsg, ioExc);
-      }
-      
-      if (text == null || text.trim().length() == 0) {
-        throw new ResourceException(Status.CLIENT_ERROR_BAD_REQUEST, "Empty request body!");      
-      }
+    }
 
-      Object parsedJson = null;
-      try {
-        parsedJson = fromJSONString(text);
-      } catch (Exception ioExc) {
-        String errMsg = String.format(Locale.ROOT,
-            "Failed to parse request [%s] into JSON due to: %s",
-            text, ioExc.toString());
-        log.error(errMsg, ioExc);
-        throw new ResourceException(Status.CLIENT_ERROR_BAD_REQUEST, errMsg, ioExc);
+    protected Object parseJsonFromRequestBody(SolrQueryRequest req) {
+      Iterator<ContentStream> iter = req.getContentStreams().iterator();
+      if (iter.hasNext()) {
+        try (Reader reader = iter.next().getReader()) {
+          return fromJSON(reader);
+        } catch (IOException ioExc) {
+          throw new SolrException(ErrorCode.SERVER_ERROR, ioExc);
+        }
       }
-      return parsedJson;
+      throw new SolrException(ErrorCode.BAD_REQUEST, "No JSON body found in request!");
     }
 
     @Override
@@ -518,16 +457,16 @@ public class RestManager {
      */
     @SuppressWarnings("unchecked")
     @Override
-    public synchronized void doPut(BaseSolrResource endpoint, Representation entity, Object json) {      
+    public synchronized void doPut(BaseSolrResource endpoint, Object json) {
       if (json instanceof Map) {
-        String resourceId = ManagedEndpoint.resolveResourceId(endpoint.getRequest());
+        String resourceId = ManagedEndpoint.resolveResourceId(endpoint.getSolrRequest().getPath());
         Map<String,String> info = (Map<String,String>)json;
         info.put("resourceId", resourceId);
         storeManagedData(applyUpdatesToManagedData(json));
       } else {
-        throw new ResourceException(Status.CLIENT_ERROR_BAD_REQUEST, 
+        throw new SolrException(ErrorCode.BAD_REQUEST,
             "Expected Map to create a new ManagedResource but received a "+json.getClass().getName());
-      }          
+      }
       // PUT just returns success status code with an empty body
     }
 
@@ -539,15 +478,15 @@ public class RestManager {
     @SuppressWarnings("unchecked")
     @Override
     protected Object applyUpdatesToManagedData(Object updates) {
-      Map<String,String> info = (Map<String,String>)updates;      
+      Map<String,String> info = (Map<String,String>)updates;
       // this is where we'd register a new ManagedResource
       String implClass = info.get("class");
       String resourceId = info.get("resourceId");
-      log.info("Creating a new ManagedResource of type {} at path {}", 
+      log.info("Creating a new ManagedResource of type {} at path {}",
           implClass, resourceId);
-      Class<? extends ManagedResource> clazz = 
+      Class<? extends ManagedResource> clazz =
           solrResourceLoader.findClass(implClass, ManagedResource.class);
-      
+
       // add this new resource to the RestManager
       restManager.addManagedResource(resourceId, clazz);
 
@@ -558,7 +497,7 @@ public class RestManager {
         if (reg.observers.isEmpty()) {
           managedList.add(reg.getInfo());
         }
-      }          
+      }
       return managedList;
     }
 
@@ -567,18 +506,18 @@ public class RestManager {
      */
     @Override
     public void doDeleteChild(BaseSolrResource endpoint, String childId) {
-      throw new ResourceException(Status.SERVER_ERROR_NOT_IMPLEMENTED);
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Delete child resource not supported!");
     }
 
     @Override
     public void doGet(BaseSolrResource endpoint, String childId) {
       
       // filter results by /schema or /config
-      String path = ManagedEndpoint.resolveResourceId(endpoint.getRequest());
+      String path = ManagedEndpoint.resolveResourceId(endpoint.getSolrRequest().getPath());
       Matcher resourceIdMatcher = resourceIdRegex.matcher(path);
       if (!resourceIdMatcher.matches()) {
         // extremely unlikely but didn't want to squelch it either
-        throw new ResourceException(Status.SERVER_ERROR_NOT_IMPLEMENTED, path);
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Requests to path "+path+" not supported!");
       }
       
       String filter = resourceIdMatcher.group(1);
@@ -603,11 +542,7 @@ public class RestManager {
   protected Map<String,ManagedResource> managed = new TreeMap<>();
   protected RestManagerManagedResource endpoint;
   protected SolrResourceLoader loader;
-  
-  // refs to these are needed to bind new ManagedResources created using the API
-  protected Router schemaRouter;
-  protected Router configRouter;
-  
+
   /**
    * Initializes the RestManager with the storageIO being optionally created outside of this implementation
    * such as to use ZooKeeper instead of the local FS. 
@@ -625,7 +560,7 @@ public class RestManager {
     
     this.storageIO = storageIO;
     this.loader = loader;
-    
+
     registry = loader.getManagedResourceRegistry();
     
     // the RestManager provides metadata about managed resources via the /managed endpoint
@@ -651,8 +586,7 @@ public class RestManager {
 
   /**
    * If not already registered, registers the given {@link ManagedResource} subclass
-   * at the given resourceId, creates an instance, and attaches it to the appropriate
-   * Restlet router.  Returns the corresponding instance.
+   * at the given resourceId, creates an instance. Returns the corresponding instance.
    */
   public synchronized ManagedResource addManagedResource(String resourceId, Class<? extends ManagedResource> clazz) {
     final ManagedResource res;
@@ -665,31 +599,16 @@ public class RestManager {
     }
     return res;
   }
-  
-  // used internally to create and attach a ManagedResource to the Restlet router
-  // the registry also uses this method directly, which is slightly hacky but necessary
-  // in order to support dynamic adding of new fieldTypes using the managed-schema API
+
+  // cache a mapping of path to ManagedResource
   private synchronized ManagedResource addRegisteredResource(ManagedResourceRegistration reg) {
     String resourceId = reg.resourceId;
     ManagedResource res = createManagedResource(reg);
     managed.put(resourceId, res);
     log.info("Registered new managed resource {}", resourceId);
-    
-    // attach this new resource to the Restlet router
-    Matcher resourceIdValidator = resourceIdRegex.matcher(resourceId);
-    boolean validated = resourceIdValidator.matches();
-    assert validated : "managed resourceId '" + resourceId
-                     + "' should already be validated by registerManagedResource()";
-    String routerPath = resourceIdValidator.group(1);      
-    String path = resourceIdValidator.group(2);
-    Router router = SCHEMA_BASE_PATH.equals(routerPath) ? schemaRouter : configRouter;
-    if (router != null) {
-      attachManagedResource(res, path, router);
-    }
     return res;
   }
 
-
   /**
    * Creates a ManagedResource using registration information. 
    */
@@ -714,14 +633,13 @@ public class RestManager {
    * Returns the {@link ManagedResource} subclass instance corresponding
    * to the given resourceId from the registry.
    *
-   * @throws ResourceException if no managed resource is registered with
+   * @throws SolrException if no managed resource is registered with
    *  the given resourceId.
    */
   public ManagedResource getManagedResource(String resourceId) {
     ManagedResource res = getManagedResourceOrNull(resourceId);
     if (res == null) {
-      throw new ResourceException(Status.SERVER_ERROR_INTERNAL, 
-          "No ManagedResource registered for path: "+resourceId);
+      throw new SolrException(ErrorCode.NOT_FOUND, "No ManagedResource registered for path: "+resourceId);
     }
     return res;
   }
@@ -759,48 +677,5 @@ public class RestManager {
       log.error("Error when trying to clean-up after deleting {}",resourceId, e);
     }
   }
-      
-  /**
-   * Attach managed resource paths to the given Restlet Router. 
-   * @param router - Restlet Router
-   */
-  public synchronized void attachManagedResources(String routerPath, Router router) {
-    if (SCHEMA_BASE_PATH.equals(routerPath)) {
-      this.schemaRouter = router;
-    } else {
-      throw new SolrException(ErrorCode.SERVER_ERROR, 
-          routerPath+" not supported by the RestManager");
-    }      
-    
-    int numAttached = 0;
-    for (Map.Entry<String, ManagedResource> entry : managed.entrySet()) {
-      String resourceId = entry.getKey();
-      if (resourceId.startsWith(routerPath)) {
-        // the way restlet works is you attach a path w/o the routerPath
-        String path = resourceId.substring(routerPath.length());
-        attachManagedResource(entry.getValue(), path, router);
-        ++numAttached;
-      }
-    }
-    
-    log.info("Attached {} ManagedResource endpoints to Restlet router: {}", 
-        numAttached, routerPath);
-  }
-  
-  /**
-   * Attaches a ManagedResource and optionally a path for child resources
-   * to the given Restlet Router.
-   */
-  protected void attachManagedResource(ManagedResource res, String path, Router router) {
-    router.attach(path, res.getServerResourceClass());
-    log.info("Attached managed resource at path: {}",path);
-    
-    // Determine if we should also route requests for child resources
-    // ManagedResource.ChildResourceSupport is a marker interface that
-    // indicates the ManagedResource also manages child resources at
-    // a path one level down from the main resourceId
-    if (ManagedResource.ChildResourceSupport.class.isAssignableFrom(res.getClass())) {
-      router.attach(path+"/{child}", res.getServerResourceClass());
-    }    
-  }
+
 }
diff --git a/solr/core/src/java/org/apache/solr/rest/SolrSchemaRestApi.java b/solr/core/src/java/org/apache/solr/rest/SolrSchemaRestApi.java
deleted file mode 100644
index aa7826a..0000000
--- a/solr/core/src/java/org/apache/solr/rest/SolrSchemaRestApi.java
+++ /dev/null
@@ -1,78 +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;
-
-import java.lang.invoke.MethodHandles;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.solr.request.SolrRequestInfo;
-import org.restlet.Application;
-import org.restlet.Restlet;
-import org.restlet.routing.Router;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * 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());
-
-
-  /**
-   * Returns reserved endpoints under /schema
-   */
-  public static Set<String> getReservedEndpoints() {
-    Set<String> reservedEndpoints = new HashSet<>();
-    return Collections.unmodifiableSet(reservedEndpoints);
-  }
-
-  private Router router;
-
-  public SolrSchemaRestApi() {
-    router = new Router(getContext());
-  }
-
-  @Override
-  public void stop() throws Exception {
-    if (null != router) {
-      router.stop();
-    }
-  }
-
-  /**
-   * Bind URL paths to the appropriate ServerResource subclass. 
-   */
-  @Override
-  public synchronized Restlet createInboundRoot() {
-
-    log.info("createInboundRoot started for /schema");
-
-
-    router.attachDefault(RestManager.ManagedEndpoint.class);
-    
-    // attach all the dynamically registered schema resources
-    RestManager.getRestManager(SolrRequestInfo.getRequestInfo())
-        .attachManagedResources(RestManager.SCHEMA_BASE_PATH, router);
-
-    log.info("createInboundRoot complete for /schema");
-
-    return router;
-  }  
-}
diff --git a/solr/core/src/java/org/apache/solr/rest/package-info.java b/solr/core/src/java/org/apache/solr/rest/package-info.java
index b903928..ef642aa 100644
--- a/solr/core/src/java/org/apache/solr/rest/package-info.java
+++ b/solr/core/src/java/org/apache/solr/rest/package-info.java
@@ -16,7 +16,7 @@
  */
  
 /** 
- * Solr RESTful APIs via Restlet.
+ * Solr RESTful APIs.
  */
 package org.apache.solr.rest;
 
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymFilterFactory.java b/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymFilterFactory.java
index c0a8fd2..f5c7e2f 100644
--- a/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymFilterFactory.java
+++ b/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymFilterFactory.java
@@ -43,8 +43,6 @@ import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.rest.BaseSolrResource;
 import org.apache.solr.rest.ManagedResource;
 import org.apache.solr.rest.ManagedResourceStorage.StorageIO;
-import org.restlet.data.Status;
-import org.restlet.resource.ResourceException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -180,7 +178,7 @@ public class ManagedSynonymFilterFactory extends BaseManagedTokenFilterFactory {
       } else if (updates instanceof Map) {
         madeChanges = applyMapUpdates((Map<String,Object>)updates, ignoreCase);
       } else {
-        throw new ResourceException(Status.CLIENT_ERROR_BAD_REQUEST,
+        throw new SolrException(ErrorCode.BAD_REQUEST,
             "Unsupported data format (" + updates.getClass().getName() + "); expected a JSON object (Map or List)!");
       }
       return madeChanges ? getStoredView() : null;
@@ -250,7 +248,7 @@ public class ManagedSynonymFilterFactory extends BaseManagedTokenFilterFactory {
           }
 
         } else {
-          throw new ResourceException(Status.CLIENT_ERROR_BAD_REQUEST, "Unsupported value "+val+
+          throw new SolrException(ErrorCode.BAD_REQUEST, "Unsupported value "+val+
               " for "+term+"; expected single value or a JSON array!");
         }
 
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymGraphFilterFactory.java b/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymGraphFilterFactory.java
index 62434e3..7cf1ff7 100644
--- a/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymGraphFilterFactory.java
+++ b/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymGraphFilterFactory.java
@@ -42,8 +42,6 @@ import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.rest.BaseSolrResource;
 import org.apache.solr.rest.ManagedResource;
 import org.apache.solr.rest.ManagedResourceStorage.StorageIO;
-import org.restlet.data.Status;
-import org.restlet.resource.ResourceException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -175,7 +173,7 @@ public class ManagedSynonymGraphFilterFactory extends BaseManagedTokenFilterFact
       } else if (updates instanceof Map) {
         madeChanges = applyMapUpdates((Map<String,Object>)updates, ignoreCase);
       } else {
-        throw new ResourceException(Status.CLIENT_ERROR_BAD_REQUEST,
+        throw new SolrException(ErrorCode.BAD_REQUEST,
             "Unsupported data format (" + updates.getClass().getName() + "); expected a JSON object (Map or List)!");
       }
       return madeChanges ? getStoredView() : null;
@@ -245,7 +243,7 @@ public class ManagedSynonymGraphFilterFactory extends BaseManagedTokenFilterFact
           }
 
         } else {
-          throw new ResourceException(Status.CLIENT_ERROR_BAD_REQUEST, "Unsupported value "+val+
+          throw new SolrException(ErrorCode.BAD_REQUEST, "Unsupported value "+val+
               " for "+term+"; expected single value or a JSON array!");
         }
 
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/analysis/package-info.java b/solr/core/src/java/org/apache/solr/rest/schema/analysis/package-info.java
index 04d4428..8124188 100644
--- a/solr/core/src/java/org/apache/solr/rest/schema/analysis/package-info.java
+++ b/solr/core/src/java/org/apache/solr/rest/schema/analysis/package-info.java
@@ -16,7 +16,8 @@
  */
  
 /** 
- * Analysis-related functionality for RESTful API access to the Solr Schema using Restlet.
+ * Analysis-related functionality for RESTful API access to managed resources related to the schema, such
+ * as stopwords, protected words, and synonyms.
  */
 package org.apache.solr.rest.schema.analysis;
 
diff --git a/solr/core/src/java/org/apache/solr/rest/schema/package-info.java b/solr/core/src/java/org/apache/solr/rest/schema/package-info.java
index ca5fd2c..3712359 100644
--- a/solr/core/src/java/org/apache/solr/rest/schema/package-info.java
+++ b/solr/core/src/java/org/apache/solr/rest/schema/package-info.java
@@ -16,7 +16,7 @@
  */
  
 /** 
- * Provides RESTful API access to the Solr Schema using Restlet.
+ * Provides RESTful API access to managed resources in the Solr Schema.
  */
 package org.apache.solr.rest.schema;
 
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 fa9ea04..2900184 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -408,26 +408,6 @@ public class HttpSolrCall {
   protected void extractHandlerFromURLPath(SolrRequestParsers parser) throws Exception {
     if (handler == null && path.length() > 1) { // don't match "" or "/" as valid path
       handler = core.getRequestHandler(path);
-
-      if (handler == null) {
-        //may be a restlet path
-        // Handle /schema/* paths via Restlet
-        if (path.equals("/schema") || path.startsWith("/schema/")) {
-          solrReq = parser.parse(core, path, req);
-          SolrRequestInfo.setRequestInfo(new SolrRequestInfo(solrReq, new SolrQueryResponse()));
-          mustClearSolrRequestInfo = true;
-          if (path.equals(req.getServletPath())) {
-            // avoid endless loop - pass through to Restlet via webapp
-            action = PASSTHROUGH;
-          } else {
-            // forward rewritten URI (without path prefix and core/collection name) to Restlet
-            action = FORWARD;
-          }
-          SolrRequestInfo.getRequestInfo().setAction(action);
-          return;
-        }
-      }
-
       // no handler yet but <requestDispatcher> allows us to handle /select with a 'qt' param
       if (handler == null && parser.isHandleSelect()) {
         if ("/select".equals(path) || "/select/".equals(path)) {
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 4583692..9adc764 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -116,8 +116,8 @@ public class SolrDispatchFilter extends BaseSolrFilter {
 
   /**
    * Enum to define action that needs to be processed.
-   * PASSTHROUGH: Pass through to Restlet via webapp.
-   * FORWARD: Forward rewritten URI (without path prefix and core/collection name) to Restlet
+   * PASSTHROUGH: Pass through to another filter via webapp.
+   * FORWARD: Forward rewritten URI (without path prefix and core/collection name) to another filter in the chain
    * RETURN: Returns the control, and no further specific processing is needed.
    *  This is generally when an error is set and returned.
    * RETRY:Retry the request. In cases when a core isn't found to work with, this is set.
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
index 4a8cffa..2592950 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
@@ -768,8 +768,7 @@ public class SolrRequestParsers {
 
       // According to previous StandardRequestParser logic (this is a re-written version),
       // POST was handled normally, but other methods (PUT/DELETE)
-      // were handled by restlet if the URI contained /schema or /config
-      // "handled by restlet" means that we don't attempt to handle any request body here.
+      // were handled by the RestManager classes if the URI contained /schema or /config
       if (!isPost) {
         if (isV2) {
           return raw.parseParamsAndFillStreams(req, streams);
@@ -780,14 +779,14 @@ public class SolrRequestParsers {
 
         // OK, we have a BODY at this point
 
-        boolean restletPath = false;
+        boolean schemaRestPath = false;
         int idx = uri.indexOf("/schema");
         if (idx >= 0 && uri.endsWith("/schema") || uri.contains("/schema/")) {
-          restletPath = true;
+          schemaRestPath = true;
         }
 
-        if (restletPath) {
-          return parseQueryString(req.getQueryString());
+        if (schemaRestPath) {
+          return raw.parseParamsAndFillStreams(req, streams);
         }
 
         if ("PUT".equals(method) || "DELETE".equals(method)) {
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
index 868561c..ce7580d 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
@@ -49,7 +49,6 @@ import org.eclipse.jetty.servlet.ServletHolder;
 import org.junit.After;
 import org.junit.Before;
 import org.noggit.JSONParser;
-import org.restlet.ext.servlet.ServerServlet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -75,9 +74,6 @@ public class TestSolrConfigHandler extends RestTestBase {
     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");
diff --git a/solr/core/src/test/org/apache/solr/request/TestStreamBody.java b/solr/core/src/test/org/apache/solr/request/TestStreamBody.java
index ab4648d..181a008 100644
--- a/solr/core/src/test/org/apache/solr/request/TestStreamBody.java
+++ b/solr/core/src/test/org/apache/solr/request/TestStreamBody.java
@@ -32,7 +32,6 @@ 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 org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -50,9 +49,6 @@ public class TestStreamBody extends RestTestBase {
     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");
diff --git a/solr/core/src/test/org/apache/solr/rest/SolrRestletTestBase.java b/solr/core/src/test/org/apache/solr/rest/SolrRestletTestBase.java
index 5ce6a9e..354d316 100644
--- a/solr/core/src/test/org/apache/solr/rest/SolrRestletTestBase.java
+++ b/solr/core/src/test/org/apache/solr/rest/SolrRestletTestBase.java
@@ -18,7 +18,6 @@ package org.apache.solr.rest;
 import org.apache.solr.util.RestTestBase;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.junit.BeforeClass;
-import org.restlet.ext.servlet.ServerServlet;
 
 import java.nio.file.Path;
 import java.util.Properties;
@@ -26,9 +25,8 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 
 /**
- * Base class for Solr Restlet-based tests. Creates jetty and test harness
- * with solrconfig.xml and schema-rest.xml, including "extra" servlets for
- * all Solr Restlet Application subclasses.
+ * Base class for Solr Rest-oriented API tests. Creates jetty and test harness
+ * with solrconfig.xml and schema-rest.xml.
  *
  * Use RestTestBase instead if you need to specialize the solrconfig,
  * the schema, or jetty/test harness creation; otherwise you'll get
@@ -51,9 +49,6 @@ abstract public class SolrRestletTestBase extends RestTestBase {
     System.setProperty("configSetBaseDir", TEST_HOME());
 
     final SortedMap<ServletHolder,String> extraServlets = new TreeMap<>();
-    final ServletHolder solrSchemaRestApi = new ServletHolder("SolrSchemaRestApi", ServerServlet.class);
-    solrSchemaRestApi.setInitParameter("org.restlet.application", "org.apache.solr.rest.SolrSchemaRestApi");
-    extraServlets.put(solrSchemaRestApi, "/schema/*");  // '/schema/*' matches '/schema', '/schema/', and '/schema/whatever...'
 
     Properties props = new Properties();
     props.setProperty("name", DEFAULT_TEST_CORENAME);
diff --git a/solr/core/src/test/org/apache/solr/rest/TestRestManager.java b/solr/core/src/test/org/apache/solr/rest/TestRestManager.java
index b4aff0b..bfe1f11 100644
--- a/solr/core/src/test/org/apache/solr/rest/TestRestManager.java
+++ b/solr/core/src/test/org/apache/solr/rest/TestRestManager.java
@@ -19,146 +19,19 @@ package org.apache.solr.rest;
 import java.io.File;
 import java.nio.file.Paths;
 import java.util.Arrays;
-import java.util.Locale;
-import java.util.Set;
 
-import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.rest.ManagedResourceStorage.StorageIO;
 import org.apache.solr.rest.schema.analysis.ManagedWordSetResource;
-import org.junit.Ignore;
 import org.junit.Test;
-import org.restlet.Request;
-import org.restlet.data.Reference;
 
 /**
  * Tests {@link RestManager} functionality, including resource registration,
  * and REST API requests and responses.
  */
 public class TestRestManager extends SolrRestletTestBase {
-  
-  private class BogusManagedResource extends ManagedResource {
-
-    protected BogusManagedResource(String resourceId,
-        SolrResourceLoader loader, StorageIO storageIO) throws SolrException {
-      super(resourceId, loader, storageIO);
-    }
-
-    @Override
-    protected void onManagedDataLoadedFromStorage(NamedList<?> managedInitArgs, Object managedData)
-        throws SolrException {}
-
-    @Override
-    protected Object applyUpdatesToManagedData(Object updates) {
-      return null;
-    }
-
-    @Override
-    public void doDeleteChild(BaseSolrResource endpoint, String childId) {}
-
-    @Override
-    public void doGet(BaseSolrResource endpoint, String childId) {}
-    
-  }
-  
-  private static class MockAnalysisComponent implements ManagedResourceObserver {
-
-    @Override
-    public void onManagedResourceInitialized(NamedList<?> args, ManagedResource res)
-        throws SolrException {
-      assertTrue(res instanceof ManagedWordSetResource);      
-    }
-  }
-  
-  /**
-   * Test RestManager initialization and handling of registered ManagedResources. 
-   */
-  @Test
-  @Ignore
-  public void testManagedResourceRegistrationAndInitialization() throws Exception {
-    // first, we need to register some ManagedResources, which is done with the registry
-    // provided by the SolrResourceLoader
-    SolrResourceLoader loader = new SolrResourceLoader(Paths.get("./"));
-    
-    RestManager.Registry registry = loader.getManagedResourceRegistry();
-    assertNotNull("Expected a non-null RestManager.Registry from the SolrResourceLoader!", registry);
-    
-    String resourceId = "/config/test/foo";
-    registry.registerManagedResource(resourceId, 
-                                     ManagedWordSetResource.class, 
-                                     new MockAnalysisComponent());
-    
-    // verify the two different components can register the same ManagedResource in the registry
-    registry.registerManagedResource(resourceId, 
-                                     ManagedWordSetResource.class, 
-                                     new MockAnalysisComponent());
-    
-    // verify we can register another resource under a different resourceId
-    registry.registerManagedResource("/config/test/foo2", 
-                                     ManagedWordSetResource.class, 
-                                     new MockAnalysisComponent());
-
-    ignoreException("REST API path .* already registered to instances of ");
-
-    String failureMessage = "Should not be able to register a different"
-                          + " ManagedResource implementation for {}";
-
-    // verify that some other hooligan cannot register another type
-    // of ManagedResource implementation under the same resourceId
-    try {
-      registry.registerManagedResource(resourceId, 
-                                       BogusManagedResource.class, 
-                                       new MockAnalysisComponent());
-      fail(String.format(Locale.ROOT, failureMessage, resourceId));
-    } catch (SolrException solrExc) {
-      // expected output
-    }
-
-    resetExceptionIgnores();
-
-    ignoreException("is a reserved endpoint used by the Solr REST API!");
-
-    failureMessage = "Should not be able to register reserved endpoint {}";
-
-    // verify that already-spoken-for REST API endpoints can't be registered
-    Set<String> reservedEndpoints = registry.getReservedEndpoints();
-    assertTrue(reservedEndpoints.size() > 2);
-    assertTrue(reservedEndpoints.contains(RestManager.SCHEMA_BASE_PATH + RestManager.MANAGED_ENDPOINT));
-    for (String endpoint : reservedEndpoints) {
-
-      try {
-        registry.registerManagedResource
-            (endpoint, BogusManagedResource.class, new MockAnalysisComponent());
-        fail(String.format(Locale.ROOT, failureMessage, endpoint));
-      } catch (SolrException solrExc) {
-        // expected output
-      }
-
-      // also try to register already-spoken-for REST API endpoints with a child segment
-      endpoint += "/kid";
-      try {
-        registry.registerManagedResource
-            (endpoint, BogusManagedResource.class, new MockAnalysisComponent());
-        fail(String.format(Locale.ROOT, failureMessage, endpoint));
-      } catch (SolrException solrExc) {
-        // expected output
-      }
-    }
-
-    resetExceptionIgnores();
-    
-    NamedList<String> initArgs = new NamedList<>();
-    RestManager restManager = new RestManager();
-    restManager.init(loader, initArgs, new ManagedResourceStorage.InMemoryStorageIO());
-    
-    ManagedResource res = restManager.getManagedResource(resourceId);
-    assertTrue(res instanceof ManagedWordSetResource);    
-    assertEquals(res.getResourceId(), resourceId);
-    
-    restManager.getManagedResource("/config/test/foo2"); // exception if it isn't registered
-  }
 
   /**
    * Tests {@link RestManager}'s responses to REST API requests on /config/managed
@@ -255,27 +128,15 @@ public class TestRestManager extends SolrRestletTestBase {
 
   @Test
   public void testResolveResourceId () throws Exception {
-    Request testRequest = new Request();
-    Reference rootRef = new Reference("http://solr.apache.org/");
-    testRequest.setRootRef(rootRef);
-
-    Reference resourceRef = new Reference("http://solr.apache.org/schema/analysis/synonyms/de");
-    testRequest.setResourceRef(resourceRef);
-
-    String resourceId = RestManager.ManagedEndpoint.resolveResourceId(testRequest);
+    String path = "http://solr.apache.org/schema/analysis/synonyms/de";
+    String resourceId = RestManager.ManagedEndpoint.resolveResourceId(path);
     assertEquals(resourceId, "/schema/analysis/synonyms/de");
   }
 
   @Test
   public void testResolveResourceIdDecodeUrlEntities () throws Exception {
-    Request testRequest = new Request();
-    Reference rootRef = new Reference("http://solr.apache.org/");
-    testRequest.setRootRef(rootRef);
-
-    Reference resourceRef = new Reference("http://solr.apache.org/schema/analysis/synonyms/de/%C3%84ndern");
-    testRequest.setResourceRef(resourceRef);
-
-    String resourceId = RestManager.ManagedEndpoint.resolveResourceId(testRequest);
+    String path = "http://solr.apache.org/schema/analysis/synonyms/de/%C3%84ndern";
+    String resourceId = RestManager.ManagedEndpoint.resolveResourceId(path);
     assertEquals(resourceId, "/schema/analysis/synonyms/de/Ă„ndern");
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestSerializedLuceneMatchVersion.java b/solr/core/src/test/org/apache/solr/rest/schema/TestSerializedLuceneMatchVersion.java
index f50d669..4750b0f 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestSerializedLuceneMatchVersion.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestSerializedLuceneMatchVersion.java
@@ -19,7 +19,6 @@ import org.apache.solr.util.RestTestBase;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.restlet.ext.servlet.ServerServlet;
 
 import java.util.SortedMap;
 import java.util.TreeMap;
@@ -30,9 +29,6 @@ public class TestSerializedLuceneMatchVersion extends RestTestBase {
   @BeforeClass
   public static void init() throws Exception {
     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...'
 
     createJettyAndHarness(TEST_HOME(), "solrconfig-minimal.xml", "schema-rest-lucene-match-version.xml",
                           "/solr", true, extraServlets);
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 4950ac4..d411325 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
@@ -28,7 +28,6 @@ import org.eclipse.jetty.servlet.ServletHolder;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-import org.restlet.ext.servlet.ServerServlet;
 
 /**
  * Test the REST API for managing stop words, which is pretty basic:
@@ -49,9 +48,6 @@ public class TestManagedStopFilterFactory extends RestTestBase {
     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");
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 603249b..8740cec 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
@@ -32,7 +32,6 @@ 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 static org.apache.solr.common.util.Utils.toJSONString;
 
@@ -49,9 +48,6 @@ public class TestManagedSynonymFilterFactory extends RestTestBase {
     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/*");
 
     System.setProperty("managed.schema.mutable", "true");
     System.setProperty("enable.update.log", "false");
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymGraphFilterFactory.java b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymGraphFilterFactory.java
index 66e9efe..778b19a 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymGraphFilterFactory.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymGraphFilterFactory.java
@@ -33,7 +33,6 @@ 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 static org.apache.solr.common.util.Utils.toJSONString;
 
@@ -51,9 +50,6 @@ public class TestManagedSynonymGraphFilterFactory extends RestTestBase {
     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/*");
 
     System.setProperty("managed.schema.mutable", "true");
     System.setProperty("enable.update.log", "false");
diff --git a/solr/core/src/test/org/apache/solr/schema/TestCloudSchemaless.java b/solr/core/src/test/org/apache/solr/schema/TestCloudSchemaless.java
index b281e34..cc21ccc 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestCloudSchemaless.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestCloudSchemaless.java
@@ -36,7 +36,6 @@ import org.apache.solr.util.RestTestHarness;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.junit.After;
 import org.junit.Test;
-import org.restlet.ext.servlet.ServerServlet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -68,9 +67,6 @@ public class TestCloudSchemaless extends AbstractFullDistribZkTestBase {
   @Override
   public SortedMap<ServletHolder,String> getExtraServlets() {
     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...'
     return extraServlets;
   }
 
diff --git a/solr/licenses/org.restlet-2.4.3.jar.sha1 b/solr/licenses/org.restlet-2.4.3.jar.sha1
deleted file mode 100644
index 67a7f5c..0000000
--- a/solr/licenses/org.restlet-2.4.3.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-fb9441cfe1e17b04976bed9b0dfd8c4a39c41b78
diff --git a/solr/licenses/org.restlet-LICENSE-ASL.txt b/solr/licenses/org.restlet-LICENSE-ASL.txt
deleted file mode 100644
index 261eeb9..0000000
--- a/solr/licenses/org.restlet-LICENSE-ASL.txt
+++ /dev/null
@@ -1,201 +0,0 @@
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   Licensed 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.
diff --git a/solr/licenses/org.restlet-NOTICE.txt b/solr/licenses/org.restlet-NOTICE.txt
deleted file mode 100644
index c7839b5..0000000
--- a/solr/licenses/org.restlet-NOTICE.txt
+++ /dev/null
@@ -1,2 +0,0 @@
-This product includes software developed by
-the Restlet project (http://www.restlet.org).
\ No newline at end of file
diff --git a/solr/licenses/org.restlet.ext.servlet-2.4.3.jar.sha1 b/solr/licenses/org.restlet.ext.servlet-2.4.3.jar.sha1
deleted file mode 100644
index b0aa84b..0000000
--- a/solr/licenses/org.restlet.ext.servlet-2.4.3.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-5e805b9c6c07cd21958288805451236895316f56
diff --git a/solr/licenses/org.restlet.ext.servlet-LICENSE-ASL.txt b/solr/licenses/org.restlet.ext.servlet-LICENSE-ASL.txt
deleted file mode 100644
index 261eeb9..0000000
--- a/solr/licenses/org.restlet.ext.servlet-LICENSE-ASL.txt
+++ /dev/null
@@ -1,201 +0,0 @@
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   Licensed 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.
diff --git a/solr/licenses/org.restlet.ext.servlet-NOTICE.txt b/solr/licenses/org.restlet.ext.servlet-NOTICE.txt
deleted file mode 100644
index 154ac0a..0000000
--- a/solr/licenses/org.restlet.ext.servlet-NOTICE.txt
+++ /dev/null
@@ -1,2 +0,0 @@
-This product includes software developed by
-the SimpleXML project (http://simple.sourceforge.net).
\ No newline at end of file
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 284d2e6..4675f0f 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
@@ -49,7 +49,6 @@ import org.eclipse.jetty.servlet.ServletHolder;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-import org.restlet.ext.servlet.ServerServlet;
 
 /**
  * Test the functionality (accuracy and failure) of the methods exposed by the classes
@@ -106,9 +105,6 @@ public class SchemaTest extends RestTestBase {
     FileUtils.copyDirectory(new File(getFile("solrj/solr/collection1").getParent()), 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");
diff --git a/solr/webapp/web/WEB-INF/web.xml b/solr/webapp/web/WEB-INF/web.xml
index 53ab57a..2599420 100644
--- a/solr/webapp/web/WEB-INF/web.xml
+++ b/solr/webapp/web/WEB-INF/web.xml
@@ -47,25 +47,11 @@
     <servlet-class>org.apache.solr.servlet.LoadAdminUiServlet</servlet-class>
   </servlet>
 
-  <servlet>
-    <servlet-name>SolrRestApi</servlet-name>
-    <servlet-class>org.restlet.ext.servlet.ServerServlet</servlet-class>
-    <init-param>
-      <param-name>org.restlet.application</param-name>
-      <param-value>org.apache.solr.rest.SolrSchemaRestApi</param-value>
-    </init-param>
-  </servlet>
-
   <servlet-mapping>
     <servlet-name>LoadAdminUI</servlet-name>
     <url-pattern>/index.html</url-pattern>
   </servlet-mapping>
 
-  <servlet-mapping>
-    <servlet-name>SolrRestApi</servlet-name>
-    <url-pattern>/schema/*</url-pattern>
-  </servlet-mapping>
-  
   <mime-mapping>
     <extension>.xsl</extension>
     <!-- per http://www.w3.org/TR/2006/PR-xslt20-20061121/ -->