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

[50/50] [abbrv] lucene-solr:jira/solr-6736: SOLR-6736: Adding config sets with trusted=false, disallow XSLT request handler

SOLR-6736: Adding config sets with trusted=false, disallow XSLT request handler


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

Branch: refs/heads/jira/solr-6736
Commit: f664f1f39bf5c93e20cd19ae83a2538100d21942
Parents: 405da79
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Sun Mar 12 05:47:24 2017 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Sun Mar 12 05:47:24 2017 +0530

----------------------------------------------------------------------
 .../apache/solr/cloud/CreateCollectionCmd.java  |   2 +-
 .../apache/solr/cloud/ZkSolrResourceLoader.java |   3 +
 .../java/org/apache/solr/core/ConfigSet.java    |  12 ++-
 .../org/apache/solr/core/ConfigSetService.java  |  25 +++++-
 .../org/apache/solr/core/CoreContainer.java     |   3 +-
 .../org/apache/solr/core/CoreDescriptor.java    |  17 ++++
 .../org/apache/solr/core/RequestHandlers.java   |  12 ++-
 .../src/java/org/apache/solr/core/SolrCore.java |  10 ++-
 .../solr/handler/admin/ConfigSetsHandler.java   |  10 +--
 .../apache/solr/handler/loader/XMLLoader.java   |   7 ++
 .../solr/configsets/upload/managed-schema       |  25 ++++++
 .../solr/configsets/upload/newzkconf.zip        | Bin 1994 -> 3186 bytes
 .../solr/configsets/upload/schema-minimal.xml   |  25 ------
 .../configsets/upload/solrconfig-minimal.xml    |  59 -------------
 .../solr/configsets/upload/solrconfig.xml       |  61 +++++++++++++
 .../upload/xslt/xsl-update-handler-test.xsl     |  49 +++++++++++
 .../apache/solr/cloud/TestConfigSetsAPI.java    |  88 ++++++++++++++++---
 .../org/apache/solr/core/TestCodecSupport.java  |   2 +-
 .../apache/solr/cloud/MiniSolrCloudCluster.java |  10 +++
 19 files changed, 302 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f664f1f3/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
index a1bb70e..f426658 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
@@ -246,7 +246,7 @@ public class CreateCollectionCmd implements Cmd {
         ShardRequest sreq = new ShardRequest();
         sreq.nodeName = nodeName;
         params.set("qt", ocmh.adminPath);
-        sreq.purpose = 1;
+        sreq.purpose = ShardRequest.PURPOSE_PRIVATE;
         sreq.shards = new String[]{baseUrl};
         sreq.actualShards = sreq.shards;
         sreq.params = params;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f664f1f3/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..c2b6bbb 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
@@ -82,6 +82,9 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
   public InputStream openResource(String resource) throws IOException {
     InputStream is;
     String file = configSetZkPath + "/" + resource;
+    if (file.endsWith("/")) {
+      file = file.substring(0, file.length()-1);
+    }
     int maxTries = 10;
     Exception exception = null;
     while (maxTries -- > 0) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f664f1f3/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..e94a892 100644
--- a/solr/core/src/java/org/apache/solr/core/ConfigSet.java
+++ b/solr/core/src/java/org/apache/solr/core/ConfigSet.java
@@ -31,12 +31,16 @@ public class ConfigSet {
   private final IndexSchema indexSchema;
 
   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/f664f1f3/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..5e68b70 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;
@@ -76,8 +77,18 @@ public abstract class ConfigSetService {
     try {
       SolrConfig solrConfig = createSolrConfig(dcore, coreLoader);
       IndexSchema schema = createIndexSchema(dcore, solrConfig);
+      
+      // nocommit javadocs difference between properties and flags
       NamedList properties = createConfigSetProperties(dcore, coreLoader);
-      return new ConfigSet(configName(dcore), solrConfig, schema, properties);
+      NamedList flags = getConfigSetFlags(dcore, coreLoader);
+
+      boolean trusted = 
+          (coreLoader instanceof ZkSolrResourceLoader 
+              && flags != null
+              && flags.get("trusted") != null
+              && !flags.getBooleanArg("trusted")
+          ) ? false: true;
+      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 +127,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/f664f1f3/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 b9597ae..5a6f98f 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -944,7 +944,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/f664f1f3/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..5dd214a 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
+   * source.
+   */
+  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/f664f1f3/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/RequestHandlers.java b/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
index dd06fa5..c91e711 100644
--- a/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
+++ b/solr/core/src/java/org/apache/solr/core/RequestHandlers.java
@@ -114,7 +114,7 @@ public final class RequestHandlers {
    * Handlers will be registered and initialized in the order they appear in solrconfig.xml
    */
 
-  void initHandlersFromConfig(SolrConfig config) {
+  void initHandlersFromConfig(SolrConfig config, boolean isConfigSetTrusted) {
     List<PluginInfo> implicits = core.getImplicitHandlers();
     // use link map so we iterate in the same order
     Map<String, PluginInfo> infoMap= new LinkedHashMap<>();
@@ -125,8 +125,9 @@ public final class RequestHandlers {
 
     List<PluginInfo> modifiedInfos = new ArrayList<>();
     for (PluginInfo info : infos) {
-      modifiedInfos.add(applyInitParams(config, info));
+      modifiedInfos.add(applyInitParams(config, isConfigSetTrusted, info));
     }
+    System.out.println("Handlers infos: "+modifiedInfos); // nocommit
     handlers.init(Collections.emptyMap(),core, modifiedInfos);
     handlers.alias(handlers.getDefault(), "");
     log.debug("Registered paths: {}" , StrUtils.join(new ArrayList<>(handlers.keySet()) , ',' ));
@@ -137,7 +138,7 @@ public final class RequestHandlers {
     }
   }
 
-  private PluginInfo applyInitParams(SolrConfig config, PluginInfo info) {
+  private PluginInfo applyInitParams(SolrConfig config, boolean isConfigSetTrusted, PluginInfo info) {
     List<InitParams> ags = new ArrayList<>();
     String p = info.attributes.get(InitParams.TYPE);
     if(p!=null) {
@@ -148,12 +149,17 @@ public final class RequestHandlers {
     }
     for (InitParams args : config.getInitParams().values())
       if(args.matchPath(info.name)) ags.add(args);
+
+    // nocommit review to make sure no plugin actually uses that keyname
+    info.initArgs.add("trusted", isConfigSetTrusted);
+    
     if(!ags.isEmpty()){
       info = info.copy();
       for (InitParams initParam : ags) {
         initParam.apply(info);
       }
     }
+    
     return info;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f664f1f3/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 13c3bdd..dd7645d 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -187,6 +187,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   private final SolrResourceLoader resourceLoader;
   private volatile IndexSchema schema;
   private final NamedList configSetProperties;
+  private final boolean isConfigSetTrusted;
   private final String dataDir;
   private final String ulogDir;
   private final UpdateHandler updateHandler;
@@ -635,7 +636,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
         CoreDescriptor cd = new CoreDescriptor(coreDescriptor.getName(), coreDescriptor);
         cd.loadExtraProperties(); //Reload the extra properties
         core = new SolrCore(getName(), getDataDir(), coreConfig.getSolrConfig(),
-            coreConfig.getIndexSchema(), coreConfig.getProperties(),
+            coreConfig.getIndexSchema(), coreConfig.getProperties(), coreConfig.isTrusted(),
             cd, updateHandler, solrDelPolicy, currentCore, true);
         
         // we open a new IndexWriter to pick up the latest config
@@ -827,7 +828,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
   public SolrCore(CoreDescriptor cd, ConfigSet coreConfig) {
     this(cd.getName(), null, coreConfig.getSolrConfig(), coreConfig.getIndexSchema(), coreConfig.getProperties(),
-        cd, null, null, null, false);
+        coreConfig.isTrusted(), cd, null, null, null, false);
   }
 
   
@@ -845,7 +846,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
    * @since solr 1.3
    */
   public SolrCore(String name, String dataDir, SolrConfig config,
-      IndexSchema schema, NamedList configSetProperties,
+      IndexSchema schema, NamedList configSetProperties, boolean isConfigSetTrusted,
       CoreDescriptor coreDescriptor, UpdateHandler updateHandler,
       IndexDeletionPolicyWrapper delPolicy, SolrCore prev, boolean reload) {
     
@@ -858,6 +859,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     resourceLoader = config.getResourceLoader();
     this.solrConfig = config;
     this.configSetProperties = configSetProperties;
+    this.isConfigSetTrusted = isConfigSetTrusted;
     // Initialize the metrics manager
     this.coreMetricManager = initCoreMetricManager(config);
     this.coreMetricManager.loadReporters();
@@ -922,7 +924,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
       // Processors initialized before the handlers
       updateProcessorChains = loadUpdateProcessorChains();
       reqHandlers = new RequestHandlers(this);
-      reqHandlers.initHandlersFromConfig(solrConfig);
+      reqHandlers.initHandlersFromConfig(solrConfig, isConfigSetTrusted);
 
       statsCache = initStatsCache();
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f664f1f3/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 cc15239..17e52ea 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
@@ -144,12 +144,6 @@ public class ConfigSetsHandler extends RequestHandlerBase {
 
   private void handleConfigUploadRequest(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
 
-    String httpMethod = (String) req.getContext().get(ConfigSetParams.HTTP_METHOD);
-    if (!"POST".equals(httpMethod)) {
-      throw new SolrException(ErrorCode.BAD_REQUEST,
-          "The upload action supports POST requests only");
-    }
-
     String configSetName = req.getParams().get(NAME);
     if (StringUtils.isBlank(configSetName)) {
       throw new SolrException(ErrorCode.BAD_REQUEST,
@@ -173,8 +167,8 @@ public class ConfigSetsHandler extends RequestHandlerBase {
 
     InputStream inputStream = contentStreamsIterator.next().getStream();
 
-    // Create a node for the configuration in zookeeper
-    zkClient.makePath(configPathInZk, true);
+    // Create a node for the configuration in zookeeper nocommit: do this only if /admin is not protected by authz/authc
+    zkClient.makePath(configPathInZk, "{\"trusted\": false}".getBytes(StandardCharsets.UTF_8), true);
 
     ZipInputStream zis = new ZipInputStream(inputStream, StandardCharsets.UTF_8);
     ZipEntry zipEntry = null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f664f1f3/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 038ed9f..c283f6a 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
@@ -41,6 +41,7 @@ import com.google.common.collect.Lists;
 import org.apache.commons.io.IOUtils;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 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;
@@ -132,6 +133,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 authorization in place,"
+            + " and this operation is not available for collections with untrusted configsets. To have this feature, re-upload the configset"
+            + " after enabling authentication and authorization for the /admin endpoints.");
+      }
+
       final Transformer t = getTransformer(tr,req);
       final DOMResult result = new DOMResult();
       

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f664f1f3/solr/core/src/test-files/solr/configsets/upload/managed-schema
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/managed-schema b/solr/core/src/test-files/solr/configsets/upload/managed-schema
new file mode 100644
index 0000000..9e2f947
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/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/f664f1f3/solr/core/src/test-files/solr/configsets/upload/newzkconf.zip
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/newzkconf.zip b/solr/core/src/test-files/solr/configsets/upload/newzkconf.zip
index e9db927..1577bd1 100644
Binary files a/solr/core/src/test-files/solr/configsets/upload/newzkconf.zip and b/solr/core/src/test-files/solr/configsets/upload/newzkconf.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f664f1f3/solr/core/src/test-files/solr/configsets/upload/schema-minimal.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/schema-minimal.xml b/solr/core/src/test-files/solr/configsets/upload/schema-minimal.xml
deleted file mode 100644
index 9e2f947..0000000
--- a/solr/core/src/test-files/solr/configsets/upload/schema-minimal.xml
+++ /dev/null
@@ -1,25 +0,0 @@
-<?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/f664f1f3/solr/core/src/test-files/solr/configsets/upload/solrconfig-minimal.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/solrconfig-minimal.xml b/solr/core/src/test-files/solr/configsets/upload/solrconfig-minimal.xml
deleted file mode 100644
index 2f9609d..0000000
--- a/solr/core/src/test-files/solr/configsets/upload/solrconfig-minimal.xml
+++ /dev/null
@@ -1,59 +0,0 @@
-<?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="/admin/" class="org.apache.solr.handler.admin.AdminHandlers" />
-
-  <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
-</config>
-

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f664f1f3/solr/core/src/test-files/solr/configsets/upload/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/solrconfig.xml b/solr/core/src/test-files/solr/configsets/upload/solrconfig.xml
new file mode 100644
index 0000000..82d0cc9
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/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/f664f1f3/solr/core/src/test-files/solr/configsets/upload/xslt/xsl-update-handler-test.xsl
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/upload/xslt/xsl-update-handler-test.xsl b/solr/core/src/test-files/solr/configsets/upload/xslt/xsl-update-handler-test.xsl
new file mode 100644
index 0000000..2e7359a
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/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/f664f1f3/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 aa78c1d..fc325fb 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
@@ -40,19 +40,27 @@ import org.apache.http.entity.ByteArrayEntity;
 import org.apache.http.util.EntityUtils;
 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.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.ConfigSetAdminRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
 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.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.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkConfigManager;
+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.params.CollectionParams.CollectionAction;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.ConfigSetProperties;
@@ -317,23 +325,77 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
       long statusCode = (long) getObjectByPath(map, false, Arrays.asList("responseHeader", "status"));
       assertEquals(0l, statusCode);
 
-      assertTrue("schema-minimal.xml file should have been uploaded",
-          zkClient.exists("/configs/newzkconf/schema-minimal.xml", true));
-      assertTrue("schema-minimal.xml file contents on zookeeper are not exactly same as that of the file uploaded in config",
-          Arrays.equals(zkClient.getData("/configs/newzkconf/schema-minimal.xml", null, null, true),
-              readFile("solr/configsets/upload/schema-minimal.xml")));
-
-      assertTrue("solrconfig-minimal.xml file should have been uploaded",
-          zkClient.exists("/configs/newzkconf/solrconfig-minimal.xml", true));
-      assertTrue("solrconfig-minimal.xml file contents on zookeeper are not exactly same as that of the file uploaded in config",
-          Arrays.equals(zkClient.getData("/configs/newzkconf/solrconfig-minimal.xml", null, null, true),
-              readFile("solr/configsets/upload/solrconfig-minimal.xml")));
+      assertTrue("managed-schema file should have been uploaded",
+          zkClient.exists("/configs/newzkconf/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/newzkconf/managed-schema", null, null, true),
+              readFile("solr/configsets/upload/managed-schema")));
+
+      assertTrue("solrconfig.xml file should have been uploaded",
+          zkClient.exists("/configs/newzkconf/solrconfig.xml", true));
+      byte data[] = zkClient.getData("/configs/newzkconf", 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/newzkconf/solrconfig.xml", null, null, true),
+              readFile("solr/configsets/upload/solrconfig.xml")));
+      
+      // try to create a collection with the uploaded configset
+      createCollection("newcollection", "newzkconf", 1, 1, solrClient);
+      xsltRequest("newcollection");
     } finally {
       zkClient.close();
     }
     solrClient.close();
   }
+  
+  public 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());
+      }
 
+      //System.out.println("Results: "+client.query(params("q", "*:*")));
+    }
+  }
+  
+
+  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) throws IOException {
     HttpPost httpPost = null;
@@ -462,7 +524,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/f664f1f3/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/f664f1f3/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..b129193 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
@@ -438,6 +438,16 @@ public class MiniSolrCloudCluster {
       }
     }
   }
+  
+  /*public void createCollection(String collectionName, String configName, int numShards,
+      int replicationFactor) throws Exception {
+    try (ZkStateReader reader = new ZkStateReader(solrClient.getZkStateReader().getZkClient())) {
+      reader.createClusterStateWatchersAndUpdate();
+      for (String collection : reader.getClusterState().getCollectionStates().keySet()) {
+        CollectionAdminRequest.deleteCollection(collection).process(solrClient);
+      }
+    }
+  }*/
 
   /**
    * Shut down the cluster, including all Solr nodes and ZooKeeper