You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cp...@apache.org on 2017/03/30 17:25:28 UTC

[59/73] [abbrv] lucene-solr:jira/solr-6203: SOLR-6736: Adding support for uploading zipped configsets using ConfigSets API

SOLR-6736: Adding support for uploading zipped configsets using ConfigSets API


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

Branch: refs/heads/jira/solr-6203
Commit: 6b0217b7cbff1216bb4ffbecdba02eb8c5dd3df6
Parents: 0322068
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Wed Mar 29 00:44:27 2017 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Wed Mar 29 00:44:27 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../dataimport/EntityProcessorWrapper.java      |  23 ++
 .../apache/solr/cloud/ZkSolrResourceLoader.java |   2 +-
 .../java/org/apache/solr/core/ConfigSet.java    |  10 +-
 .../org/apache/solr/core/ConfigSetService.java  |  29 +-
 .../org/apache/solr/core/CoreContainer.java     |   3 +-
 .../org/apache/solr/core/CoreDescriptor.java    |  17 +
 .../apache/solr/core/RunExecutableListener.java |  13 +-
 .../src/java/org/apache/solr/core/SolrCore.java |   1 +
 .../solr/handler/admin/ConfigSetsHandler.java   |  93 ++++-
 .../apache/solr/handler/loader/XMLLoader.java   |   7 +
 .../solr/security/PermissionNameProvider.java   |   2 +-
 .../org/apache/solr/update/UpdateHandler.java   |  16 +
 .../StatelessScriptUpdateProcessorFactory.java  |   6 +
 .../dih-script-transformer/managed-schema       |  25 ++
 .../dih-script-transformer/solrconfig.xml       |  61 +++
 .../configsets/upload/regular/managed-schema    |  25 ++
 .../configsets/upload/regular/solrconfig.xml    |  61 +++
 .../regular/xslt/xsl-update-handler-test.xsl    |  49 +++
 .../with-run-executable-listener/managed-schema |  25 ++
 .../with-run-executable-listener/solrconfig.xml |  69 ++++
 .../upload/with-script-processor/managed-schema |  25 ++
 ...missleading.extension.updateprocessor.js.txt |  23 ++
 .../upload/with-script-processor/solrconfig.xml |  65 +++
 .../apache/solr/cloud/TestConfigSetsAPI.java    | 401 ++++++++++++++++++-
 .../org/apache/solr/core/TestCodecSupport.java  |   2 +-
 .../apache/solr/core/TestDynamicLoading.java    |  11 +-
 .../solr/common/params/ConfigSetParams.java     |   1 +
 .../apache/solr/cloud/MiniSolrCloudCluster.java |   6 +-
 29 files changed, 1056 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 8875160..49300fe 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -123,6 +123,9 @@ New Features
 
 * SOLR-10076: Hide keystore and truststore passwords from /admin/info/* outputs. (Mano Kovacs via Mark Miller)
 
+* SOLR-6736: Adding support for uploading zipped configsets using ConfigSets API (Varun Rajput, Ishan Chattopadhyaya,
+  Noble Paul, Anshum Gupta, Gregory Chanan)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EntityProcessorWrapper.java
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EntityProcessorWrapper.java b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EntityProcessorWrapper.java
index 6370d24..8a76e11 100644
--- a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EntityProcessorWrapper.java
+++ b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EntityProcessorWrapper.java
@@ -17,6 +17,8 @@
 package org.apache.solr.handler.dataimport;
 
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.dataimport.config.ConfigNameConstants;
 import org.apache.solr.handler.dataimport.config.Entity;
 
@@ -98,6 +100,9 @@ public class EntityProcessorWrapper extends EntityProcessor {
     for (String aTransArr : transArr) {
       String trans = aTransArr.trim();
       if (trans.startsWith("script:")) {
+        // The script transformer is a potential vulnerability, esp. when the script is
+        // provided from an untrusted source. Check and don't proceed if source is untrusted.
+        checkIfTrusted(trans);
         String functionName = trans.substring("script:".length());
         ScriptTransformer scriptTransformer = new ScriptTransformer();
         scriptTransformer.setFunctionName(functionName);
@@ -126,6 +131,24 @@ public class EntityProcessorWrapper extends EntityProcessor {
 
   }
 
+  private void checkIfTrusted(String trans) {
+    if (docBuilder != null) {
+      SolrCore core = docBuilder.dataImporter.getCore();
+      boolean trusted = (core != null)? core.getCoreDescriptor().isConfigSetTrusted(): true;
+      if (!trusted) {
+        Exception ex = new SolrException(ErrorCode.UNAUTHORIZED, "The configset for this collection was uploaded "
+            + "without any authentication in place,"
+            + " and this transformer is not available for collections with untrusted configsets. To use this transformer,"
+            + " re-upload the configset after enabling authentication and authorization.");
+        String msg = "Transformer: "
+            + trans
+            + ". " + ex.getMessage();
+        log.error(msg);
+        wrapAndThrow(SEVERE, ex, msg);
+      }
+    }
+  }
+
   @SuppressWarnings("unchecked")
   static class ReflectionTransformer extends Transformer {
     final Method meth;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
index b4137b3..5f32ef2 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
@@ -81,7 +81,7 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
   @Override
   public InputStream openResource(String resource) throws IOException {
     InputStream is;
-    String file = configSetZkPath + "/" + resource;
+    String file = (".".equals(resource)) ? configSetZkPath : configSetZkPath + "/" + resource;
     int maxTries = 10;
     Exception exception = null;
     while (maxTries -- > 0) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/java/org/apache/solr/core/ConfigSet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/ConfigSet.java b/solr/core/src/java/org/apache/solr/core/ConfigSet.java
index c406506..e0c9fe4 100644
--- a/solr/core/src/java/org/apache/solr/core/ConfigSet.java
+++ b/solr/core/src/java/org/apache/solr/core/ConfigSet.java
@@ -32,11 +32,15 @@ public class ConfigSet {
 
   private final NamedList properties;
 
-  public ConfigSet(String name, SolrConfig solrConfig, IndexSchema indexSchema, NamedList properties) {
+  private final boolean trusted;
+
+  public ConfigSet(String name, SolrConfig solrConfig, IndexSchema indexSchema,
+      NamedList properties, boolean trusted) {
     this.name = name;
     this.solrconfig = solrConfig;
     this.indexSchema = indexSchema;
     this.properties = properties;
+    this.trusted = trusted;
   }
 
   public String getName() {
@@ -54,4 +58,8 @@ public class ConfigSet {
   public NamedList getProperties() {
     return properties;
   }
+  
+  public boolean isTrusted() {
+    return trusted;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
index e4a135e..13ac9ce 100644
--- a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
@@ -28,6 +28,7 @@ import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
 import org.apache.solr.cloud.CloudConfigSetService;
 import org.apache.solr.cloud.ZkController;
+import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.schema.IndexSchema;
@@ -74,10 +75,22 @@ public abstract class ConfigSetService {
     SolrResourceLoader coreLoader = createCoreResourceLoader(dcore);
 
     try {
+
+      // ConfigSet properties are loaded from ConfigSetProperties.DEFAULT_FILENAME file.
+      // ConfigSet flags are loaded from the metadata of the ZK node of the configset.
+      NamedList properties = createConfigSetProperties(dcore, coreLoader);
+      NamedList flags = getConfigSetFlags(dcore, coreLoader);
+
+      boolean trusted =
+          (coreLoader instanceof ZkSolrResourceLoader
+              && flags != null
+              && flags.get("trusted") != null
+              && !flags.getBooleanArg("trusted")
+              ) ? false: true;
+
       SolrConfig solrConfig = createSolrConfig(dcore, coreLoader);
       IndexSchema schema = createIndexSchema(dcore, solrConfig);
-      NamedList properties = createConfigSetProperties(dcore, coreLoader);
-      return new ConfigSet(configName(dcore), solrConfig, schema, properties);
+      return new ConfigSet(configName(dcore), solrConfig, schema, properties, trusted);
     } catch (Exception e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
           "Could not load conf for core " + dcore.getName() +
@@ -116,6 +129,18 @@ public abstract class ConfigSetService {
     return ConfigSetProperties.readFromResourceLoader(loader, cd.getConfigSetPropertiesName());
   }
 
+  protected NamedList getConfigSetFlags(CoreDescriptor cd, SolrResourceLoader loader) {
+    if (loader instanceof ZkSolrResourceLoader) {
+      try {
+        return ConfigSetProperties.readFromResourceLoader(loader, ".");
+      } catch (Exception ex) {
+        return null;
+      }
+    } else {
+      return null;
+    }
+  }
+
   /**
    * Create a SolrResourceLoader for a core
    * @param cd the core's CoreDescriptor

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 9db3261..5ec34ba 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -945,7 +945,8 @@ public class CoreContainer {
       }
 
       ConfigSet coreConfig = coreConfigService.getConfig(dcore);
-      log.info("Creating SolrCore '{}' using configuration from {}", dcore.getName(), coreConfig.getName());
+      dcore.setConfigSetTrusted(coreConfig.isTrusted());
+      log.info("Creating SolrCore '{}' using configuration from {}, trusted={}", dcore.getName(), coreConfig.getName(), dcore.isConfigSetTrusted());
       try {
         core = new SolrCore(dcore, coreConfig);
       } catch (SolrException e) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java b/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
index a58de48..0dc2c71 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
@@ -66,6 +66,14 @@ public class CoreDescriptor {
   public static final String DEFAULT_EXTERNAL_PROPERTIES_FILE = "conf" + File.separator + "solrcore.properties";
 
   /**
+   * Whether this core was configured using a configSet that was trusted.
+   * This helps in avoiding the loading of plugins that have potential
+   * vulnerabilities, when the configSet was not uploaded from a trusted
+   * user.
+   */
+  private boolean trustedConfigSet = true;
+
+  /**
    * Get the standard properties in persistable form
    * @return the standard core properties in persistable form
    */
@@ -170,6 +178,7 @@ public class CoreDescriptor {
     this.coreProperties.setProperty(CORE_NAME, coreName);
     this.originalCoreProperties.setProperty(CORE_NAME, coreName);
     this.substitutableProperties.setProperty(SOLR_CORE_PROP_PREFIX + CORE_NAME, coreName);
+    this.trustedConfigSet = other.trustedConfigSet;
   }
 
   /**
@@ -372,4 +381,12 @@ public class CoreDescriptor {
   public String getConfigSetPropertiesName() {
     return coreProperties.getProperty(CORE_CONFIGSET_PROPERTIES);
   }
+
+  public boolean isConfigSetTrusted() {
+    return trustedConfigSet;
+  }
+
+  public void setConfigSetTrusted(boolean trusted) {
+    this.trustedConfigSet = trusted;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/java/org/apache/solr/core/RunExecutableListener.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/RunExecutableListener.java b/solr/core/src/java/org/apache/solr/core/RunExecutableListener.java
index ba0046e..c6d0090 100644
--- a/solr/core/src/java/org/apache/solr/core/RunExecutableListener.java
+++ b/solr/core/src/java/org/apache/solr/core/RunExecutableListener.java
@@ -17,8 +17,10 @@
 package org.apache.solr.core;
 
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -30,7 +32,7 @@ import java.util.ArrayList;
 
 /**
  */
-class RunExecutableListener extends AbstractSolrEventListener {
+class RunExecutableListener extends AbstractSolrEventListener implements SolrCoreAware {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   
   public RunExecutableListener(SolrCore core) {
@@ -66,6 +68,15 @@ class RunExecutableListener extends AbstractSolrEventListener {
     if ("false".equals(args.get("wait")) || Boolean.FALSE.equals(args.get("wait"))) wait=false;
   }
 
+  @Override
+  public void inform(SolrCore core) {
+    if (!core.getCoreDescriptor().isConfigSetTrusted()) {
+      throw new SolrException(ErrorCode.UNAUTHORIZED, "The configset for this collection was uploaded without any authentication in place,"
+          + " and this operation is not available for collections with untrusted configsets. To have this component, re-upload the configset"
+          + " after enabling authentication and authorization.");
+    }
+  }
+
   /**
    * External executable listener.
    * 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index 9d77c7e..a6ba2dc 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -962,6 +962,7 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
       // Finally tell anyone who wants to know
       resourceLoader.inform(resourceLoader);
       resourceLoader.inform(this); // last call before the latch is released.
+      this.updateHandler.informEventListeners(this);
     } catch (Throwable e) {
       // release the latch, otherwise we block trying to do the close. This
       // should be fine, since counting down on a latch of 0 is still fine

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
index 5d6f02c..d3489df 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
@@ -16,14 +16,20 @@
  */
 package org.apache.solr.handler.admin;
 
+import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
-
+import java.nio.charset.StandardCharsets;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.fs.Path;
 import org.apache.solr.api.Api;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.cloud.OverseerSolrResponse;
@@ -36,6 +42,7 @@ import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.ConfigSetParams;
 import org.apache.solr.common.params.ConfigSetParams.ConfigSetAction;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.Utils;
@@ -43,6 +50,10 @@ import org.apache.solr.core.CoreContainer;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.AuthenticationPlugin;
+import org.apache.solr.security.AuthorizationContext;
+import org.apache.solr.security.PermissionNameProvider;
+import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.BASE_CONFIGSET;
 import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.CONFIGSETS_ACTION_PREFIX;
@@ -59,7 +70,7 @@ import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
 /**
  * A {@link org.apache.solr.request.SolrRequestHandler} for ConfigSets API requests.
  */
-public class ConfigSetsHandler extends RequestHandlerBase {
+public class ConfigSetsHandler extends RequestHandlerBase implements PermissionNameProvider {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected final CoreContainer coreContainer;
   public static long DEFAULT_ZK_TIMEOUT = 300*1000;
@@ -95,6 +106,10 @@ public class ConfigSetsHandler extends RequestHandlerBase {
       ConfigSetAction action = ConfigSetAction.get(a);
       if (action == null)
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown action: " + a);
+      if (action == ConfigSetAction.UPLOAD) {
+        handleConfigUploadRequest(req, rsp);
+        return;
+      }
       invokeAction(req, rsp, action);
     } else {
       throw new SolrException(ErrorCode.BAD_REQUEST, "action is a required param");
@@ -121,6 +136,68 @@ public class ConfigSetsHandler extends RequestHandlerBase {
     }
   }
 
+  private void handleConfigUploadRequest(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    String configSetName = req.getParams().get(NAME);
+    if (StringUtils.isBlank(configSetName)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+          "The configuration name should be provided in the \"name\" parameter");
+    }
+
+    SolrZkClient zkClient = coreContainer.getZkController().getZkClient();
+    String configPathInZk = ZkConfigManager.CONFIGS_ZKNODE + Path.SEPARATOR + configSetName;
+
+    if (zkClient.exists(configPathInZk, true)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+          "The configuration " + configSetName + " already exists in zookeeper");
+    }
+
+    Iterator<ContentStream> contentStreamsIterator = req.getContentStreams().iterator();
+
+    if (!contentStreamsIterator.hasNext()) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+          "No stream found for the config data to be uploaded");
+    }
+
+    InputStream inputStream = contentStreamsIterator.next().getStream();
+
+    // Create a node for the configuration in zookeeper
+    boolean trusted = getTrusted(req);
+    zkClient.makePath(configPathInZk, ("{\"trusted\": " + Boolean.toString(trusted) + "}").
+        getBytes(StandardCharsets.UTF_8), true);
+
+    ZipInputStream zis = new ZipInputStream(inputStream, StandardCharsets.UTF_8);
+    ZipEntry zipEntry = null;
+    while ((zipEntry = zis.getNextEntry()) != null) {
+      String filePathInZk = configPathInZk + "/" + zipEntry.getName();
+      if (zipEntry.isDirectory()) {
+        zkClient.makePath(filePathInZk, true);
+      } else {
+        createZkNodeIfNotExistsAndSetData(zkClient, filePathInZk,
+            IOUtils.toByteArray(zis));
+      }
+    }
+    zis.close();
+  }
+
+  boolean getTrusted(SolrQueryRequest req) {
+    AuthenticationPlugin authcPlugin = coreContainer.getAuthenticationPlugin();
+    log.info("Trying to upload a configset. authcPlugin: {}, user principal: {}",
+        authcPlugin, req.getUserPrincipal());
+    if (authcPlugin != null && req.getUserPrincipal() != null) {
+      return true;
+    }
+    return false;
+  }
+
+  private void createZkNodeIfNotExistsAndSetData(SolrZkClient zkClient,
+      String filePathInZk, byte[] data) throws Exception {
+    if (!zkClient.exists(filePathInZk, true)) {
+      zkClient.create(filePathInZk, data, CreateMode.PERSISTENT, true);
+    } else {
+      zkClient.setData(filePathInZk, data, true);
+    }
+  }
+
   private void handleResponse(String operation, ZkNodeProps m,
       SolrQueryResponse rsp, long timeout) throws KeeperException, InterruptedException {
     long time = System.nanoTime();
@@ -226,4 +303,16 @@ public class ConfigSetsHandler extends RequestHandlerBase {
   public Boolean registerV2() {
     return Boolean.TRUE;
   }
+
+  @Override
+  public Name getPermissionName(AuthorizationContext ctx) {
+    switch (ctx.getHttpMethod()) {
+      case "GET":
+        return Name.CONFIG_READ_PERM;
+      case "POST":
+        return Name.CONFIG_EDIT_PERM;
+      default:
+        return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java b/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java
index 000edee..a07aff2 100644
--- a/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java
+++ b/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java
@@ -42,6 +42,7 @@ import com.google.common.collect.Lists;
 import org.apache.commons.io.IOUtils;
 import org.apache.solr.common.EmptyEntityResolver;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -134,6 +135,12 @@ public class XMLLoader extends ContentStreamLoader {
 
     String tr = req.getParams().get(CommonParams.TR,null);
     if(tr!=null) {
+      if (req.getCore().getCoreDescriptor().isConfigSetTrusted() == false) {
+          throw new SolrException(ErrorCode.UNAUTHORIZED, "The configset for this collection was uploaded without any authentication in place,"
+                  + " and this operation is not available for collections with untrusted configsets. To use this feature, re-upload the configset"
+                  + " after enabling authentication and authorization.");
+      }
+
       final Transformer t = getTransformer(tr,req);
       final DOMResult result = new DOMResult();
       

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
index 425be38..9f797487 100644
--- a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
+++ b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
@@ -40,7 +40,7 @@ public interface PermissionNameProvider {
     CORE_EDIT_PERM("core-admin-edit", null),
     READ_PERM("read", "*"),
     UPDATE_PERM("update", "*"),
-    CONFIG_EDIT_PERM("config-edit", "*"),
+    CONFIG_EDIT_PERM("config-edit", unmodifiableSet(new HashSet<>(asList("*", null)))),
     CONFIG_READ_PERM("config-read", "*"),
     SCHEMA_READ_PERM("schema-read", "*"),
     SCHEMA_EDIT_PERM("schema-edit", "*"),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateHandler.java b/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
index 1cf8a3f..cbfb0d5 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
@@ -72,6 +72,22 @@ public abstract class UpdateHandler implements SolrInfoMBean {
     }
   }
 
+  /**
+   * Call the {@link SolrCoreAware#inform(SolrCore)} on all the applicable registered listeners.
+   */
+  public void informEventListeners(SolrCore core) {
+    for (SolrEventListener listener: commitCallbacks) {
+      if (listener instanceof SolrCoreAware) {
+        ((SolrCoreAware) listener).inform(core);
+      }
+    }
+    for (SolrEventListener listener: optimizeCallbacks) {
+      if (listener instanceof SolrCoreAware) {
+        ((SolrCoreAware) listener).inform(core);
+      }
+    }
+  }
+
   protected void callPostCommitCallbacks() {
     for (SolrEventListener listener : commitCallbacks) {
       listener.postCommit();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java
index 72da7b5..190501d 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java
@@ -17,6 +17,7 @@
 package org.apache.solr.update.processor;
 
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.core.SolrCore;
@@ -217,6 +218,11 @@ public class StatelessScriptUpdateProcessorFactory extends UpdateRequestProcesso
 
   @Override
   public void inform(SolrCore core) {
+    if (!core.getCoreDescriptor().isConfigSetTrusted()) {
+      throw new SolrException(ErrorCode.UNAUTHORIZED, "The configset for this collection was uploaded without any authentication in place,"
+          + " and this operation is not available for collections with untrusted configsets. To use this component, re-upload the configset"
+          + " after enabling authentication and authorization.");
+    }
     resourceLoader = core.getResourceLoader();
 
     // test that our engines & scripts are valid

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/managed-schema
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/managed-schema b/solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/managed-schema
new file mode 100644
index 0000000..9e2f947
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/managed-schema
@@ -0,0 +1,25 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+<schema name="minimal" version="1.1">
+ <types>
+  <fieldType name="string" class="solr.StrField"/>
+ </types>
+ <fields>
+   <dynamicField name="*" type="string" indexed="true" stored="true" />
+ </fields>
+</schema>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/solrconfig.xml b/solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/solrconfig.xml
new file mode 100644
index 0000000..82d0cc9
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/solrconfig.xml
@@ -0,0 +1,61 @@
+<?xml version="1.0" ?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- This is a "kitchen sink" config file that tests can use.
+     When writting a new test, feel free to add *new* items (plugins,
+     config options, etc...) as long as they don't break any existing
+     tests.  if you need to test something esoteric please add a new
+     "solrconfig-your-esoteric-purpose.xml" config file.
+
+     Note in particular that this test is used by MinimalSchemaTest so
+     Anything added to this file needs to work correctly even if there
+     is now uniqueKey or defaultSearch Field.
+  -->
+
+<config>
+
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <directoryFactory name="DirectoryFactory"
+                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+    <commitWithin>
+      <softCommit>${solr.commitwithin.softcommit:true}</softCommit>
+    </commitWithin>
+
+  </updateHandler>
+  <requestHandler name="/select" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+      <str name="indent">true</str>
+      <str name="df">text</str>
+    </lst>
+
+  </requestHandler>
+
+  <requestHandler name="/update/xslt"
+                   startup="lazy"
+                   class="solr.XsltUpdateRequestHandler"/>
+
+  <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
+</config>
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/test-files/solr/configsets/upload/regular/managed-schema
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/regular/managed-schema b/solr/core/src/test-files/solr/configsets/upload/regular/managed-schema
new file mode 100644
index 0000000..9e2f947
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/regular/managed-schema
@@ -0,0 +1,25 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+<schema name="minimal" version="1.1">
+ <types>
+  <fieldType name="string" class="solr.StrField"/>
+ </types>
+ <fields>
+   <dynamicField name="*" type="string" indexed="true" stored="true" />
+ </fields>
+</schema>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/test-files/solr/configsets/upload/regular/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/regular/solrconfig.xml b/solr/core/src/test-files/solr/configsets/upload/regular/solrconfig.xml
new file mode 100644
index 0000000..82d0cc9
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/regular/solrconfig.xml
@@ -0,0 +1,61 @@
+<?xml version="1.0" ?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- This is a "kitchen sink" config file that tests can use.
+     When writting a new test, feel free to add *new* items (plugins,
+     config options, etc...) as long as they don't break any existing
+     tests.  if you need to test something esoteric please add a new
+     "solrconfig-your-esoteric-purpose.xml" config file.
+
+     Note in particular that this test is used by MinimalSchemaTest so
+     Anything added to this file needs to work correctly even if there
+     is now uniqueKey or defaultSearch Field.
+  -->
+
+<config>
+
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <directoryFactory name="DirectoryFactory"
+                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+    <commitWithin>
+      <softCommit>${solr.commitwithin.softcommit:true}</softCommit>
+    </commitWithin>
+
+  </updateHandler>
+  <requestHandler name="/select" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+      <str name="indent">true</str>
+      <str name="df">text</str>
+    </lst>
+
+  </requestHandler>
+
+  <requestHandler name="/update/xslt"
+                   startup="lazy"
+                   class="solr.XsltUpdateRequestHandler"/>
+
+  <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
+</config>
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/test-files/solr/configsets/upload/regular/xslt/xsl-update-handler-test.xsl
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/regular/xslt/xsl-update-handler-test.xsl b/solr/core/src/test-files/solr/configsets/upload/regular/xslt/xsl-update-handler-test.xsl
new file mode 100644
index 0000000..2e7359a
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/regular/xslt/xsl-update-handler-test.xsl
@@ -0,0 +1,49 @@
+<?xml version='1.0' encoding='UTF-8'?>
+
+<!-- 
+ * 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.
+ -->
+
+<!-- 
+
+
+XSL transform used to test the XSLTUpdateRequestHandler.
+Transforms a test XML into standard Solr <add><doc/></add> format.
+
+ -->
+<xsl:stylesheet xmlns:xsl="http://www.w3.org/1999/XSL/Transform" version="1.0">
+  <xsl:template match="/">
+    <add>
+      <xsl:apply-templates select="/random/document"/>
+    </add>
+  </xsl:template>
+
+  <xsl:template match="document">
+    <doc boost="5.5">
+      <xsl:apply-templates select="*"/>
+    </doc>
+  </xsl:template>
+
+  <xsl:template match="node">
+    <field name="{@name}">
+      <xsl:if test="@enhance!=''">
+        <xsl:attribute name="boost"><xsl:value-of select="@enhance"/></xsl:attribute>
+      </xsl:if>
+      <xsl:value-of select="@value"/>
+    </field>
+  </xsl:template>
+
+</xsl:stylesheet>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/managed-schema
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/managed-schema b/solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/managed-schema
new file mode 100644
index 0000000..9e2f947
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/managed-schema
@@ -0,0 +1,25 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+<schema name="minimal" version="1.1">
+ <types>
+  <fieldType name="string" class="solr.StrField"/>
+ </types>
+ <fields>
+   <dynamicField name="*" type="string" indexed="true" stored="true" />
+ </fields>
+</schema>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/solrconfig.xml b/solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/solrconfig.xml
new file mode 100644
index 0000000..4683841
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/solrconfig.xml
@@ -0,0 +1,69 @@
+<?xml version="1.0" ?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- This is a "kitchen sink" config file that tests can use.
+     When writting a new test, feel free to add *new* items (plugins,
+     config options, etc...) as long as they don't break any existing
+     tests.  if you need to test something esoteric please add a new
+     "solrconfig-your-esoteric-purpose.xml" config file.
+
+     Note in particular that this test is used by MinimalSchemaTest so
+     Anything added to this file needs to work correctly even if there
+     is now uniqueKey or defaultSearch Field.
+  -->
+
+<config>
+
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <directoryFactory name="DirectoryFactory"
+                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+    <commitWithin>
+      <softCommit>${solr.commitwithin.softcommit:true}</softCommit>
+    </commitWithin>
+
+    <listener event="postCommit" class="solr.RunExecutableListener">
+      <str name="exe">/var/opt/resin3/__PORT__/scripts/solr/snapshooter</str>
+      <str name="dir">/var/opt/resin3/__PORT__</str>
+      <bool name="wait">true</bool>
+      <arr name="args"> <str>arg1</str> <str>arg2</str> </arr>
+      <arr name="env"> <str>MYVAR=val1</str> </arr>
+    </listener>
+
+  </updateHandler>
+  <requestHandler name="/select" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+      <str name="indent">true</str>
+      <str name="df">text</str>
+    </lst>
+
+  </requestHandler>
+
+  <requestHandler name="/update/xslt"
+                   startup="lazy"
+                   class="solr.XsltUpdateRequestHandler"/>
+
+  <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
+</config>
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/test-files/solr/configsets/upload/with-script-processor/managed-schema
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/with-script-processor/managed-schema b/solr/core/src/test-files/solr/configsets/upload/with-script-processor/managed-schema
new file mode 100644
index 0000000..9e2f947
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/with-script-processor/managed-schema
@@ -0,0 +1,25 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+<schema name="minimal" version="1.1">
+ <types>
+  <fieldType name="string" class="solr.StrField"/>
+ </types>
+ <fields>
+   <dynamicField name="*" type="string" indexed="true" stored="true" />
+ </fields>
+</schema>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/test-files/solr/configsets/upload/with-script-processor/missleading.extension.updateprocessor.js.txt
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/with-script-processor/missleading.extension.updateprocessor.js.txt b/solr/core/src/test-files/solr/configsets/upload/with-script-processor/missleading.extension.updateprocessor.js.txt
new file mode 100644
index 0000000..984e1d8
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/with-script-processor/missleading.extension.updateprocessor.js.txt
@@ -0,0 +1,23 @@
+function processAdd(cmd) {
+    // Integer.valueOf is needed here to get a tru java object, because 
+    // all javascript numbers are floating point (ie: java.lang.Double)
+    cmd.getSolrInputDocument().addField("script_added_i", 
+                                        java.lang.Integer.valueOf(42));
+    cmd.getSolrInputDocument().addField("script_added_d", 42.3);
+    
+}
+function processDelete() {
+    // NOOP
+}
+function processCommit() { 
+    // NOOP
+}
+function processRollback() {
+    // NOOP
+}
+function processMergeIndexes() {
+    // NOOP
+}
+function finish() { 
+    // NOOP
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/test-files/solr/configsets/upload/with-script-processor/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/with-script-processor/solrconfig.xml b/solr/core/src/test-files/solr/configsets/upload/with-script-processor/solrconfig.xml
new file mode 100644
index 0000000..1c62889
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/with-script-processor/solrconfig.xml
@@ -0,0 +1,65 @@
+<?xml version="1.0" ?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- This is a "kitchen sink" config file that tests can use.
+     When writting a new test, feel free to add *new* items (plugins,
+     config options, etc...) as long as they don't break any existing
+     tests.  if you need to test something esoteric please add a new
+     "solrconfig-your-esoteric-purpose.xml" config file.
+
+     Note in particular that this test is used by MinimalSchemaTest so
+     Anything added to this file needs to work correctly even if there
+     is now uniqueKey or defaultSearch Field.
+  -->
+
+<config>
+
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <directoryFactory name="DirectoryFactory"
+                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+    <commitWithin>
+      <softCommit>${solr.commitwithin.softcommit:true}</softCommit>
+    </commitWithin>
+
+  </updateHandler>
+  <requestHandler name="/select" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+      <str name="indent">true</str>
+      <str name="df">text</str>
+    </lst>
+
+  </requestHandler>
+
+  <updateRequestProcessorChain name="force-script-engine" default="true">
+    <processor class="solr.StatelessScriptUpdateProcessorFactory">
+      <str name="engine">javascript</str>
+      <str name="script">missleading.extension.updateprocessor.js.txt</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+  <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
+</config>
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
index 2027a6d..6c20ccc 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
@@ -16,50 +16,96 @@
  */
 package org.apache.solr.cloud;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.BASE_CONFIGSET;
+import static org.apache.solr.common.params.CommonParams.NAME;
+import static org.apache.solr.core.ConfigSetProperties.DEFAULT_FILENAME;
+
 import java.io.ByteArrayInputStream;
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.StringReader;
+import java.lang.invoke.MethodHandles;
+import java.net.URI;
+import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Deque;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
 
-import com.google.common.collect.ImmutableMap;
 import org.apache.commons.io.FileUtils;
+import org.apache.http.HttpEntity;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.entity.ByteArrayEntity;
+import org.apache.http.message.BasicHeader;
+import org.apache.http.util.EntityUtils;
+import org.apache.lucene.util.TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.ConfigSetAdminRequest;
 import org.apache.solr.client.solrj.request.ConfigSetAdminRequest.Create;
 import org.apache.solr.client.solrj.request.ConfigSetAdminRequest.Delete;
-import org.apache.solr.client.solrj.request.ConfigSetAdminRequest.List;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.client.solrj.response.ConfigSetAdminResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkConfigManager;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ConfigSetParams;
 import org.apache.solr.common.params.ConfigSetParams.ConfigSetAction;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.Base64;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.ConfigSetProperties;
+import org.apache.solr.core.TestDynamicLoading;
+import org.apache.solr.security.BasicAuthIntegrationTest;
+import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.noggit.JSONParser;
+import org.noggit.ObjectBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.BASE_CONFIGSET;
-import static org.apache.solr.common.params.CommonParams.NAME;
-import static org.apache.solr.core.ConfigSetProperties.DEFAULT_FILENAME;
+import com.google.common.collect.ImmutableMap;
 
 /**
  * Simple ConfigSets API tests on user errors and simple success cases.
  */
 public class TestConfigSetsAPI extends SolrTestCaseJ4 {
 
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private MiniSolrCloudCluster solrCluster;
 
   @Override
@@ -232,6 +278,349 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
   }
 
   @Test
+  public void testUploadErrors() throws Exception {
+    final SolrClient solrClient = new HttpSolrClient(
+        solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString());
+
+    ByteBuffer emptyData = ByteBuffer.allocate(0);
+
+    // Checking error when no configuration name is specified in request
+    Map map = postDataAndGetResponse(solrCluster.getSolrClient(),
+        solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString()
+        + "/admin/configs?action=UPLOAD&wt=json", emptyData, null, null);
+    assertNotNull(map);
+    long statusCode = (long) getObjectByPath(map, false,
+        Arrays.asList("responseHeader", "status"));
+    assertEquals(400l, statusCode);
+
+    SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(),
+        AbstractZkTestCase.TIMEOUT, 45000, null);
+
+    // Create dummy config files in zookeeper
+    zkClient.makePath("/configs/myconf", true);
+    zkClient.create("/configs/myconf/firstDummyFile",
+        "first dummy content".getBytes(StandardCharsets.UTF_8), CreateMode.PERSISTENT, true);
+    zkClient.create("/configs/myconf/anotherDummyFile",
+        "second dummy content".getBytes(StandardCharsets.UTF_8), CreateMode.PERSISTENT, true);
+
+    // Checking error when configuration name specified already exists
+    map = postDataAndGetResponse(solrCluster.getSolrClient(),
+        solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString()
+        + "/admin/configs?action=UPLOAD&wt=json&name=myconf", emptyData, null, null);
+    assertNotNull(map);
+    statusCode = (long) getObjectByPath(map, false,
+        Arrays.asList("responseHeader", "status"));
+    assertEquals(400l, statusCode);
+    assertTrue("Expected file doesnt exist in zk. It's possibly overwritten",
+        zkClient.exists("/configs/myconf/firstDummyFile", true));
+    assertTrue("Expected file doesnt exist in zk. It's possibly overwritten",
+        zkClient.exists("/configs/myconf/anotherDummyFile", true));
+
+    zkClient.close();
+    solrClient.close();
+  }
+
+  @Test
+  public void testUpload() throws Exception {
+    String suffix = "-untrusted";
+    uploadConfigSet("regular", suffix, null, null);
+    // try to create a collection with the uploaded configset
+    createCollection("newcollection", "regular" + suffix, 1, 1, solrCluster.getSolrClient());
+    xsltRequest("newcollection");
+  }
+  
+  @Test
+  public void testUploadWithRunExecutableListener() throws Exception {
+    String suffix = "-untrusted";
+    uploadConfigSet("with-run-executable-listener", suffix, null, null);
+    // try to create a collection with the uploaded configset
+    CollectionAdminResponse resp = createCollection("newcollection3", "with-run-executable-listener" + suffix, 1, 1, solrCluster.getSolrClient());
+    log.info("Client saw errors: "+resp.getErrorMessages());
+    assertTrue(resp.getErrorMessages() != null && resp.getErrorMessages().size() > 0);
+    assertTrue(resp.getErrorMessages().getVal(0).
+        contains("The configset for this collection was uploaded without any authentication"));
+  }
+
+  @Test
+  public void testUploadWithScriptUpdateProcessor() throws Exception {
+    for (boolean withAuthorization: Arrays.asList(false, true)) {
+      String suffix;
+      if (withAuthorization) {
+        suffix = "-trusted";
+        protectConfigsHandler();
+        uploadConfigSet("with-script-processor", suffix, "solr", "SolrRocks");
+      } else {
+        suffix = "-untrusted";
+        uploadConfigSet("with-script-processor", suffix, null, null);
+      }
+      // try to create a collection with the uploaded configset
+      CollectionAdminResponse resp = createCollection("newcollection2", "with-script-processor"+suffix,
+          1, 1, solrCluster.getSolrClient());
+      
+      if (withAuthorization) {
+        scriptRequest("newcollection2");
+      } else {
+        log.info("Client saw errors: "+resp.getErrorMessages());
+        assertTrue(resp.getErrorMessages() != null && resp.getErrorMessages().size() > 0);
+        assertTrue(resp.getErrorMessages().getVal(0).
+            contains("The configset for this collection was uploaded without any authentication"));
+      }
+    }
+  }
+
+  protected SolrZkClient zkClient() {
+    ZkStateReader reader = solrCluster.getSolrClient().getZkStateReader();
+    if (reader == null)
+      solrCluster.getSolrClient().connect();
+    return solrCluster.getSolrClient().getZkStateReader().getZkClient();
+  }
+
+  private void protectConfigsHandler() throws Exception {
+    String authcPrefix = "/admin/authentication";
+    String authzPrefix = "/admin/authorization";
+
+    String securityJson = "{\n" +
+        "  'authentication':{\n" +
+        "    'class':'solr.BasicAuthPlugin',\n" +
+        "    'credentials':{'solr':'orwp2Ghgj39lmnrZOTm7Qtre1VqHFDfwAEzr0ApbN3Y= Ju5osoAqOX8iafhWpPP01E5P+sg8tK8tHON7rCYZRRw='}},\n" +
+        "  'authorization':{\n" +
+        "    'class':'solr.RuleBasedAuthorizationPlugin',\n" +
+        "    'user-role':{'solr':'admin'},\n" +
+        "    'permissions':[{'name':'security-edit','role':'admin'}, {'name':'config-edit','role':'admin'}]}}";
+
+    HttpClient cl = null;
+    try {
+      cl = HttpClientUtil.createClient(null);
+      JettySolrRunner randomJetty = solrCluster.getRandomJetty(random());
+      String baseUrl = randomJetty.getBaseUrl().toString();
+
+      zkClient().setData("/security.json", securityJson.replaceAll("'", "\"").getBytes(UTF_8), true);
+      BasicAuthIntegrationTest.verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication/class", "solr.BasicAuthPlugin", 50);
+      BasicAuthIntegrationTest.verifySecurityStatus(cl, baseUrl + authzPrefix, "authorization/class", "solr.RuleBasedAuthorizationPlugin", 50);
+    } finally {
+      if (cl != null) {
+        HttpClientUtil.close(cl);
+      }
+    }
+    Thread.sleep(5000); // TODO: Without a delay, the test fails. Some problem with Authc/Authz framework?
+  }
+
+  private void uploadConfigSet(String configSetName, String suffix, String username, String password) throws Exception {
+    // Read zipped sample config
+    ByteBuffer sampleZippedConfig = TestDynamicLoading
+        .getFileContent(
+            createTempZipFile("solr/configsets/upload/"+configSetName), false);
+
+    SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(),
+        AbstractZkTestCase.TIMEOUT, 45000, null);
+    try {
+      ZkConfigManager configManager = new ZkConfigManager(zkClient);
+      assertFalse(configManager.configExists(configSetName+suffix));
+
+      Map map = postDataAndGetResponse(solrCluster.getSolrClient(),
+          solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString() + "/admin/configs?action=UPLOAD&wt=json&name="+configSetName+suffix,
+          sampleZippedConfig, username, password);
+      assertNotNull(map);
+      long statusCode = (long) getObjectByPath(map, false, Arrays.asList("responseHeader", "status"));
+      assertEquals(0l, statusCode);
+
+      assertTrue("managed-schema file should have been uploaded",
+          zkClient.exists("/configs/"+configSetName+suffix+"/managed-schema", true));
+      assertTrue("managed-schema file contents on zookeeper are not exactly same as that of the file uploaded in config",
+          Arrays.equals(zkClient.getData("/configs/"+configSetName+suffix+"/managed-schema", null, null, true),
+              readFile("solr/configsets/upload/"+configSetName+"/managed-schema")));
+
+      assertTrue("solrconfig.xml file should have been uploaded",
+          zkClient.exists("/configs/"+configSetName+suffix+"/solrconfig.xml", true));
+      byte data[] = zkClient.getData("/configs/"+configSetName+suffix, null, null, true);
+      //assertEquals("{\"trusted\": false}", new String(data, StandardCharsets.UTF_8));
+      assertTrue("solrconfig.xml file contents on zookeeper are not exactly same as that of the file uploaded in config",
+          Arrays.equals(zkClient.getData("/configs/"+configSetName+suffix+"/solrconfig.xml", null, null, true),
+              readFile("solr/configsets/upload/"+configSetName+"/solrconfig.xml")));
+    } finally {
+      zkClient.close();
+    }
+  }
+  
+  /**
+   * Create a zip file (in the temp directory) containing all the files within the specified directory
+   * and return the path for the zip file.
+   */
+  private String createTempZipFile(String directoryPath) {
+    File zipFile = new File(solrCluster.getBaseDir().toFile().getAbsolutePath() +
+        File.separator + TestUtil.randomSimpleString(random(), 6, 8) + ".zip");
+
+    File directory = TestDynamicLoading.getFile(directoryPath);
+    log.info("Directory: "+directory.getAbsolutePath());
+    try {
+      zip (directory, zipFile);
+      log.info("Zipfile: "+zipFile.getAbsolutePath());
+      return zipFile.getAbsolutePath();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void zip(File directory, File zipfile) throws IOException {
+    URI base = directory.toURI();
+    Deque<File> queue = new LinkedList<File>();
+    queue.push(directory);
+    OutputStream out = new FileOutputStream(zipfile);
+    ZipOutputStream zout = new ZipOutputStream(out);
+    try {
+      while (!queue.isEmpty()) {
+        directory = queue.pop();
+        for (File kid : directory.listFiles()) {
+          String name = base.relativize(kid.toURI()).getPath();
+          if (kid.isDirectory()) {
+            queue.push(kid);
+            name = name.endsWith("/") ? name : name + "/";
+            zout.putNextEntry(new ZipEntry(name));
+          } else {
+            zout.putNextEntry(new ZipEntry(name));
+
+            InputStream in = new FileInputStream(kid);
+            try {
+              byte[] buffer = new byte[1024];
+              while (true) {
+                int readCount = in.read(buffer);
+                if (readCount < 0) {
+                  break;
+                }
+                zout.write(buffer, 0, readCount);
+              }
+            } finally {
+              in.close();
+            }
+
+            zout.closeEntry();
+          }
+        }
+      }
+    } finally {
+      zout.close();
+    }
+  }
+
+  private void xsltRequest(String collection) throws SolrServerException, IOException {
+    String baseUrl = solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString();
+    try (HttpSolrClient client = new HttpSolrClient(baseUrl + "/" + collection)) {
+      String xml = 
+          "<random>" +
+              " <document>" +
+              "  <node name=\"id\" value=\"12345\"/>" +
+              "  <node name=\"name\" value=\"kitten\"/>" +
+              "  <node name=\"text\" enhance=\"3\" value=\"some other day\"/>" +
+              "  <node name=\"title\" enhance=\"4\" value=\"A story\"/>" +
+              "  <node name=\"timestamp\" enhance=\"5\" value=\"2011-07-01T10:31:57.140Z\"/>" +
+              " </document>" +
+              "</random>";
+
+      SolrQuery query = new SolrQuery();
+      query.setQuery( "*:*" );//for anything
+      query.add("qt","/update");
+      query.add(CommonParams.TR, "xsl-update-handler-test.xsl");
+      query.add("stream.body", xml);
+      query.add("commit", "true");
+      try {
+        client.query(query);
+        fail("This should've returned a 401.");
+      } catch (SolrException ex) {
+        assertEquals(ErrorCode.UNAUTHORIZED.code, ex.code());
+      }
+    }
+  }
+  
+  public void scriptRequest(String collection) throws SolrServerException, IOException {
+    SolrClient client = solrCluster.getSolrClient();
+    SolrInputDocument doc = sdoc("id", "4055", "subject", "Solr");
+    client.add(collection, doc);
+    client.commit(collection);
+
+    assertEquals("42", client.query(collection, params("q", "*:*")).getResults().get(0).get("script_added_i"));
+  }
+
+  protected CollectionAdminResponse createCollection(String collectionName, String confSetName, int numShards,
+      int replicationFactor, SolrClient client)  throws SolrServerException, IOException {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+    params.set("collection.configName", confSetName);
+    params.set("name", collectionName);
+    params.set("numShards", numShards);
+    params.set("replicationFactor", replicationFactor);
+    SolrRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    CollectionAdminResponse res = new CollectionAdminResponse();
+    res.setResponse(client.request(request));
+    return res;
+  }
+  
+  public static Map postDataAndGetResponse(CloudSolrClient cloudClient,
+      String uri, ByteBuffer bytarr, String username, String password) throws IOException {
+    HttpPost httpPost = null;
+    HttpEntity entity;
+    String response = null;
+    Map m = null;
+    
+    try {
+      httpPost = new HttpPost(uri);
+      
+      if (username != null) {
+        String userPass = username + ":" + password;
+        String encoded = Base64.byteArrayToBase64(userPass.getBytes(UTF_8));
+        BasicHeader header = new BasicHeader("Authorization", "Basic " + encoded);
+        httpPost.setHeader(header);
+      }
+
+      httpPost.setHeader("Content-Type", "application/octet-stream");
+      httpPost.setEntity(new ByteArrayEntity(bytarr.array(), bytarr
+          .arrayOffset(), bytarr.limit()));
+      entity = cloudClient.getLbClient().getHttpClient().execute(httpPost)
+          .getEntity();
+      try {
+        response = EntityUtils.toString(entity, StandardCharsets.UTF_8);
+        m = (Map) ObjectBuilder.getVal(new JSONParser(
+            new StringReader(response)));
+      } catch (JSONParser.ParseException e) {
+        fail(e.getMessage());
+      }
+    } finally {
+      httpPost.releaseConnection();
+    }
+    return m;
+  }
+
+  private static Object getObjectByPath(Map root, boolean onlyPrimitive, java.util.List<String> hierarchy) {
+    Map obj = root;
+    for (int i = 0; i < hierarchy.size(); i++) {
+      String s = hierarchy.get(i);
+      if (i < hierarchy.size() - 1) {
+        if (!(obj.get(s) instanceof Map)) return null;
+        obj = (Map) obj.get(s);
+        if (obj == null) return null;
+      } else {
+        Object val = obj.get(s);
+        if (onlyPrimitive && val instanceof Map) {
+          return null;
+        }
+        return val;
+      }
+    }
+
+    return false;
+  }
+
+  private byte[] readFile(String fname) throws IOException {
+    byte[] buf = null;
+    try (FileInputStream fis = new FileInputStream(getFile(fname))) {
+      buf = new byte[fis.available()];
+      fis.read(buf);
+    }
+    return buf;
+  }
+  
+  @Test
   public void testDeleteErrors() throws Exception {
     final String baseUrl = solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString();
     final SolrClient solrClient = getHttpSolrClient(baseUrl);
@@ -304,7 +693,7 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
         AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT, null);
     try {
       // test empty
-      List list = new List();
+      ConfigSetAdminRequest.List list = new ConfigSetAdminRequest.List();
       ConfigSetAdminResponse.List response = list.process(solrClient);
       Collection<String> actualConfigSets = response.getConfigSets();
       assertEquals(0, actualConfigSets.size());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java b/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
index b6097ab..7d2f174 100644
--- a/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
+++ b/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
@@ -218,7 +218,7 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
     
     try {
       c = new SolrCore(new CoreDescriptor(h.getCoreContainer(), newCoreName, testSolrHome.resolve(newCoreName)), 
-          new ConfigSet("fakeConfigset", config, schema, null));
+          new ConfigSet("fakeConfigset", config, schema, null, true));
       assertNull(h.getCoreContainer().registerCore(newCoreName, c, false, false));
       h.coreName = newCoreName;
       assertEquals("We are not using the correct core", "solrconfig_codec2.xml", h.getCore().getConfigResource());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java b/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
index 306b4b2..9bbe09f 100644
--- a/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
+++ b/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
@@ -25,6 +25,7 @@ import org.apache.solr.util.SimplePostTool;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -257,8 +258,16 @@ public class TestDynamicLoading extends AbstractFullDistribZkTestBase {
   }
 
   public static ByteBuffer getFileContent(String f) throws IOException {
+    return getFileContent(f, true);
+  }
+  /**
+   * @param loadFromClassPath if true, it will look in the classpath to find the file,
+   *        otherwise load from absolute filesystem path.
+   */
+  public static ByteBuffer getFileContent(String f, boolean loadFromClassPath) throws IOException {
     ByteBuffer jar;
-    try (FileInputStream fis = new FileInputStream(getFile(f))) {
+    File file = loadFromClassPath ? getFile(f): new File(f);
+    try (FileInputStream fis = new FileInputStream(file)) {
       byte[] buf = new byte[fis.available()];
       fis.read(buf);
       jar = ByteBuffer.wrap(buf);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java b/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java
index 49c39ec..fde7e57 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java
@@ -27,6 +27,7 @@ public interface ConfigSetParams
 
   public enum ConfigSetAction {
     CREATE,
+    UPLOAD,
     DELETE,
     LIST;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b0217b7/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
index e8a0c08..15895d3 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
@@ -470,7 +470,11 @@ public class MiniSolrCloudCluster {
       }
     }
   }
-  
+
+  public Path getBaseDir() {
+    return baseDir;
+  }
+
   public CloudSolrClient getSolrClient() {
     return solrClient;
   }