You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by an...@apache.org on 2016/02/19 00:19:10 UTC

lucene-solr git commit: SOLR-8677: Restrict creation of shards with invalid names

Repository: lucene-solr
Updated Branches:
  refs/heads/master 8bcac81a2 -> d01230d63


SOLR-8677: Restrict creation of shards with invalid names


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

Branch: refs/heads/master
Commit: d01230d6394b29fa6fd42377404c0c03d6e8a4d9
Parents: 8bcac81
Author: anshum <an...@apache.org>
Authored: Thu Feb 18 14:00:10 2016 -0800
Committer: anshum <an...@apache.org>
Committed: Thu Feb 18 15:18:44 2016 -0800

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  6 +-
 .../org/apache/solr/core/CoreContainer.java     | 12 ++-
 .../solr/handler/admin/CollectionsHandler.java  | 31 ++++++-
 .../solr/util/SolrIdentifierValidator.java      | 47 -----------
 .../apache/solr/cloud/TestCollectionAPI.java    | 71 ++++++++++++++--
 .../handler/admin/CoreAdminHandlerTest.java     |  4 +-
 .../solrj/request/CollectionAdminRequest.java   | 79 ++++++++++++++++--
 .../client/solrj/request/CoreAdminRequest.java  | 33 +++++++-
 .../solrj/util/SolrIdentifierValidator.java     | 51 ++++++++++++
 .../request/TestCollectionAdminRequest.java     | 85 ++++++++++++++++++++
 .../client/solrj/request/TestCoreAdmin.java     | 38 ++++++++-
 11 files changed, 380 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index e6635f1..3fba69d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -266,6 +266,11 @@ Other Changes
 * SOLR-5730: Make Lucene's SortingMergePolicy and EarlyTerminatingSortingCollector configurable in Solr.
   (Christine Poerschke, hossmann, Tomás Fernández Löbbe, Shai Erera)
 
+Other Changes
+----------------------
+* SOLR-8677: Prevent shards containing invalid characters from being created.  Checks added server-side
+  and in SolrJ.  (Shai Erera, Jason Gerlowski, Anshum Gupta)
+
 ======================= 5.5.0 =======================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release
@@ -1018,7 +1023,6 @@ Bug Fixes
 
 * SOLR-8355: update permissions were failing node recovery (noble , Anshum Gupta)
 
-
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/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 426a493..7703721 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -36,6 +36,7 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import org.apache.solr.client.solrj.impl.HttpClientConfigurer;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
+import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
@@ -63,7 +64,6 @@ import org.apache.solr.security.PKIAuthenticationPlugin;
 import org.apache.solr.security.SecurityPluginHolder;
 import org.apache.solr.update.UpdateShardHandler;
 import org.apache.solr.util.DefaultSolrThreadFactory;
-import org.apache.solr.util.SolrIdentifierValidator;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -804,7 +804,10 @@ public class CoreContainer {
     SolrCore core = null;
     try {
       MDCLoggingContext.setCore(core);
-      SolrIdentifierValidator.validateCoreName(dcore.getName());
+      if (!SolrIdentifierValidator.validateCoreName(dcore.getName())) {
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid core: " + dcore.getName()
+            + ". Core names must consist entirely of periods, underscores, and alphanumerics");
+      }
       if (zkSys.getZkController() != null) {
         zkSys.getZkController().preRegister(dcore);
       }
@@ -1007,7 +1010,10 @@ public class CoreContainer {
   }
 
   public void rename(String name, String toName) {
-    SolrIdentifierValidator.validateCoreName(toName);
+    if(!SolrIdentifierValidator.validateCoreName(toName)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid core: " + toName
+          + ". Core names must consist entirely of periods, underscores, and alphanumerics");
+    }
     try (SolrCore core = getCore(name)) {
       if (core != null) {
         registerCore(toName, core, true);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index d04786f..68fac7d 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -69,6 +69,7 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CoreAdminRequest;
 import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestSyncShard;
 import org.apache.solr.client.solrj.response.RequestStatusState;
+import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
 import org.apache.solr.cloud.DistributedMap;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.OverseerCollectionMessageHandler;
@@ -109,7 +110,6 @@ import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.SolrIdentifierValidator;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -347,7 +347,16 @@ public class CollectionsHandler extends RequestHandlerBase {
         addMapObject(props, SNITCH);
         verifyRuleParams(h.coreContainer, props);
         final String collectionName = (String) props.get(NAME);
-        SolrIdentifierValidator.validateCollectionName(collectionName);
+        if (!SolrIdentifierValidator.validateCollectionName(collectionName)) {
+          throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid collection: " + collectionName
+          + ". Collection names must consist entirely of periods, underscores, and alphanumerics");
+        }
+        final String shardsParam = (String) props.get(SHARDS_PROP);
+        if (StringUtils.isNotEmpty(shardsParam)) {
+          log.info("Validating shards param!!!!!!!!" + shardsParam);
+          verifyShardsParam(shardsParam);
+          log.info("Validating shards param!!!!!!! done" + shardsParam);
+        }
         if (SYSTEM_COLL.equals(collectionName)) {
           //We must always create a .system collection with only a single shard
           props.put(NUM_SLICES, 1);
@@ -419,7 +428,10 @@ public class CollectionsHandler extends RequestHandlerBase {
       Map<String, Object> call(SolrQueryRequest req, SolrQueryResponse rsp, CollectionsHandler handler)
           throws Exception {
         final String aliasName = req.getParams().get(NAME);
-        SolrIdentifierValidator.validateCollectionName(aliasName);
+        if (!SolrIdentifierValidator.validateCollectionName(aliasName)) {
+          throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid alias: " + aliasName
+              + ". Aliases must consist entirely of periods, underscores, and alphanumerics");
+        }
         return req.getParams().required().getAll(null, NAME, "collections");
       }
     },
@@ -483,6 +495,11 @@ public class CollectionsHandler extends RequestHandlerBase {
             COLLECTION_PROP,
             SHARD_ID_PROP);
         ClusterState clusterState = handler.coreContainer.getZkController().getClusterState();
+        final String newShardName = req.getParams().get(SHARD_ID_PROP);
+        if (!SolrIdentifierValidator.validateShardName(newShardName)) {
+          throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid shard: " + newShardName
+              + ". Shard names must consist entirely of periods, underscores and alphanumerics");
+        }
         if (!ImplicitDocRouter.NAME.equals(((Map) clusterState.getCollection(req.getParams().get(COLLECTION_PROP)).get(DOC_ROUTER)).get(NAME)))
           throw new SolrException(ErrorCode.BAD_REQUEST, "shards can be added only to 'implicit' collections");
         req.getParams().getAll(map,
@@ -963,6 +980,14 @@ public class CollectionsHandler extends RequestHandlerBase {
     }
     return props;
   }
+  
+  private static void verifyShardsParam(String shardsParam) {
+    for (String shard : shardsParam.split(",")) {
+      if (!SolrIdentifierValidator.validateShardName(shard))
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid shard: " + shard
+            + ". Shard names must consist entirely of periods, underscores and alphanumerics");;
+    }
+  }
 
   public static final List<String> MODIFIABLE_COLL_PROPS = ImmutableList.of(
       RULE,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/core/src/java/org/apache/solr/util/SolrIdentifierValidator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/SolrIdentifierValidator.java b/solr/core/src/java/org/apache/solr/util/SolrIdentifierValidator.java
deleted file mode 100644
index dd6133d..0000000
--- a/solr/core/src/java/org/apache/solr/util/SolrIdentifierValidator.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.util;
-
-import java.lang.invoke.MethodHandles;
-import java.util.regex.Pattern;
-
-import org.apache.solr.common.SolrException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Ensures that provided identifiers align with Solr's recommendations/requirements for choosing
- * collection, core, etc identifiers.
- *  
- * Identifiers are allowed to contain underscores, periods, and alphanumeric characters. 
- */
-public class SolrIdentifierValidator {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  final static Pattern identifierPattern = Pattern.compile("^[\\._A-Za-z0-9]*$");
-  
-  public static void validateCollectionName(String collectionName) throws SolrException {
-    validateCoreName(collectionName);
-  }
-  
-  public static void validateCoreName(String name) throws SolrException {
-    if (name == null || !identifierPattern.matcher(name).matches()) {
-      log.info("Validation failed on the invalid identifier [{}].  Throwing SolrException to indicate a BAD REQUEST.", name);
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-          "Invalid name: '" + name + "' Identifiers must consist entirely of periods, underscores and alphanumerics");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java b/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
index b19c1c1..353c708 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
@@ -44,6 +44,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.zookeeper.KeeperException;
 import org.junit.Test;
 
+import static org.apache.solr.cloud.OverseerCollectionMessageHandler.ROUTER;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.SHARD_UNIQUE;
 
 public class TestCollectionAPI extends ReplicaPropertiesBase {
@@ -79,8 +80,10 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
     replicaPropTest();
     clusterStatusZNodeVersion();
     testClusterStateMigration();
-    testCollectionCreationNameValidation();
+    testCollectionCreationCollectionNameValidation();
+    testCollectionCreationShardNameValidation();
     testAliasCreationNameValidation();
+    testShardCreationNameValidation();
   }
 
   private void clusterStatusWithCollectionAndShard() throws IOException, SolrServerException {
@@ -631,7 +634,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
     }
   }
   
-  private void testCollectionCreationNameValidation() throws Exception {
+  private void testCollectionCreationCollectionNameValidation() throws Exception {
     try (CloudSolrClient client = createCloudClient(null)) {
       ModifiableSolrParams params = new ModifiableSolrParams();
       params.set("action", CollectionParams.CollectionAction.CREATE.toString());
@@ -644,9 +647,32 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
         fail();
       } catch (RemoteSolrException e) {
         final String errorMessage = e.getMessage();
-        assertTrue(errorMessage.contains("Invalid name"));
+        assertTrue(errorMessage.contains("Invalid collection"));
         assertTrue(errorMessage.contains("invalid@name#with$weird%characters"));
-        assertTrue(errorMessage.contains("Identifiers must consist entirely of"));
+        assertTrue(errorMessage.contains("Collection names must consist entirely of"));
+      }
+    }
+  }
+  
+  private void testCollectionCreationShardNameValidation() throws Exception {
+    try (CloudSolrClient client = createCloudClient(null)) {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.set("action", CollectionParams.CollectionAction.CREATE.toString());
+      params.set("name", "valid_collection_name");
+      params.set("router.name", "implicit");
+      params.set("numShards", "1");
+      params.set("shards", "invalid@name#with$weird%characters");
+      SolrRequest request = new QueryRequest(params);
+      request.setPath("/admin/collections");
+
+      try {
+        client.request(request);
+        fail();
+      } catch (RemoteSolrException e) {
+        final String errorMessage = e.getMessage();
+        assertTrue(errorMessage.contains("Invalid shard"));
+        assertTrue(errorMessage.contains("invalid@name#with$weird%characters"));
+        assertTrue(errorMessage.contains("Shard names must consist entirely of"));
       }
     }
   }
@@ -665,9 +691,42 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
         fail();
       } catch (RemoteSolrException e) {
         final String errorMessage = e.getMessage();
-        assertTrue(errorMessage.contains("Invalid name"));
+        assertTrue(errorMessage.contains("Invalid alias"));
+        assertTrue(errorMessage.contains("invalid@name#with$weird%characters"));
+        assertTrue(errorMessage.contains("Aliases must consist entirely of"));
+      }
+    }
+  }
+
+  private void testShardCreationNameValidation() throws Exception {
+    try (CloudSolrClient client = createCloudClient(null)) {
+      client.connect();
+      // Create a collection w/ implicit router
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.set("action", CollectionParams.CollectionAction.CREATE.toString());
+      params.set("name", "valid_collection_name");
+      params.set("shards", "a");
+      params.set("router.name", "implicit");
+      SolrRequest request = new QueryRequest(params);
+      request.setPath("/admin/collections");
+      client.request(request);
+
+      params = new ModifiableSolrParams();
+      params.set("action", CollectionParams.CollectionAction.CREATESHARD.toString());
+      params.set("collection", "valid_collection_name");
+      params.set("shard", "invalid@name#with$weird%characters");
+
+      request = new QueryRequest(params);
+      request.setPath("/admin/collections");
+
+      try {
+        client.request(request);
+        fail();
+      } catch (RemoteSolrException e) {
+        final String errorMessage = e.getMessage();
+        assertTrue(errorMessage.contains("Invalid shard"));
         assertTrue(errorMessage.contains("invalid@name#with$weird%characters"));
-        assertTrue(errorMessage.contains("Identifiers must consist entirely of"));
+        assertTrue(errorMessage.contains("Shard names must consist entirely of"));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
index 2159325..89f4b2b 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
@@ -139,7 +139,7 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
               resp);
 
     } catch (SolrException se) {
-      assertTrue("Expected error message for bad core name.", se.toString().contains("Invalid name"));
+      assertTrue("Expected error message for bad core name.", se.toString().contains("Invalid core"));
     }
     CoreDescriptor cd = cores.getCoreDescriptor("ugly$core=name");
     assertNull("Should NOT have added this core!", cd);
@@ -228,7 +228,7 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
               CoreAdminParams.OTHER, "bad$name"),
               resp);
     } catch (SolrException e) { // why the heck does create return a SolrException (admittedly wrapping an IAE)
-      assertTrue("Expected error message for bad core name.", e.getMessage().contains("Invalid name"));
+      assertTrue("Expected error message for bad core name.", e.getMessage().contains("Invalid core"));
     }
 
     cd = cores.getCoreDescriptor("bad$name");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
index e3446f7..700954f 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
@@ -16,9 +16,16 @@
  */
 package org.apache.solr.client.solrj.request;
 
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Properties;
+
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -31,12 +38,6 @@ import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ContentStream;
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Properties;
-
 /**
  * This class is experimental and subject to change.
  *
@@ -122,7 +123,7 @@ public abstract class CollectionAdminRequest <Q extends CollectionAdminRequest<Q
   protected abstract static class CollectionSpecificAdminRequest <T extends CollectionAdminRequest<T>> extends CollectionAdminRequest<T> {
     protected String collection = null;
 
-    public final T setCollectionName(String collectionName) {
+    public T setCollectionName(String collectionName) {
       this.collection = collectionName;
       return getThis();
     }
@@ -277,7 +278,6 @@ public abstract class CollectionAdminRequest <Q extends CollectionAdminRequest<Q
     public Create setConfigName(String config) { this.configName = config; return this; }
     public Create setCreateNodeSet(String nodeSet) { this.createNodeSet = nodeSet; return this; }
     public Create setRouterName(String routerName) { this.routerName = routerName; return this; }
-    public Create setShards(String shards) { this.shards = shards; return this; }
     public Create setRouterField(String routerField) { this.routerField = routerField; return this; }
     public Create setNumShards(Integer numShards) {this.numShards = numShards; return this; }
     public Create setMaxShardsPerNode(Integer numShards) { this.maxShardsPerNode = numShards; return this; }
@@ -296,6 +296,41 @@ public abstract class CollectionAdminRequest <Q extends CollectionAdminRequest<Q
     public Integer getReplicationFactor() { return replicationFactor; }
     public Boolean getAutoAddReplicas() { return autoAddReplicas; }
     public Integer getStateFormat() { return stateFormat; }
+    
+    /**
+     * Provide the name of the shards to be created, separated by commas
+     * 
+     * Shard names must consist entirely of periods, underscores and alphanumerics.  Other characters are not allowed.
+     * 
+     * @throws IllegalArgumentException if any of the shard names contain invalid characters.
+     */
+    public Create setShards(String shards) {
+      for (String shard : shards.split(",")) {
+        if (!SolrIdentifierValidator.validateShardName(shard)) {
+          throw new IllegalArgumentException("Invalid shard: " + shard
+              + ". Shard names must consist entirely of periods, underscores and alphanumerics");
+        }
+      }
+      this.shards = shards;
+      return this;
+    }
+    
+    /**
+     * Provide the name of the collection to be created.
+     * 
+     * Collection names must consist entirely of periods, underscores and alphanumerics.  Other characters are not allowed.
+     * 
+     * @throws IllegalArgumentException if the collection name contains invalid characters.
+     */
+    @Override
+    public Create setCollectionName(String collectionName) throws SolrException {
+      if (!SolrIdentifierValidator.validateCollectionName(collectionName)) {
+        throw new IllegalArgumentException("Invalid collection: " + collectionName
+            + ". Collection names must consist entirely of periods, underscores, and alphanumerics");
+      }
+      this.collection = collectionName;
+      return this;
+    }
 
     public Properties getProperties() {
       return properties;
@@ -409,6 +444,23 @@ public abstract class CollectionAdminRequest <Q extends CollectionAdminRequest<Q
     public CreateShard() {
       action = CollectionAction.CREATESHARD;
     }
+    
+    /**
+     * Provide the name of the shard to be created.
+     * 
+     * Shard names must consist entirely of periods, underscores and alphanumerics.  Other characters are not allowed.
+     * 
+     * @throws IllegalArgumentException if the shard name contains invalid characters.
+     */
+    @Override
+    public CreateShard setShardName(String shardName) {
+      if (!SolrIdentifierValidator.validateShardName(shardName)) {
+        throw new IllegalArgumentException("Invalid shard: " + shardName
+            + ". Shard names must consist entirely of periods, underscores and alphanumerics");
+      }
+      this.shardName = shardName;
+      return this;
+    }
 
     @Override
     public SolrParams getParams() {
@@ -588,7 +640,18 @@ public abstract class CollectionAdminRequest <Q extends CollectionAdminRequest<Q
       action = CollectionAction.CREATEALIAS;
     }
 
+    /**
+     * Provide the name of the alias to be created.
+     * 
+     * Alias names must consist entirely of periods, underscores and alphanumerics.  Other characters are not allowed.
+     * 
+     * @throws IllegalArgumentException if the alias name contains invalid characters.
+     */
     public CreateAlias setAliasName(String aliasName) {
+      if (!SolrIdentifierValidator.validateCollectionName(aliasName)) {
+        throw new IllegalArgumentException("Invalid alias: " + aliasName
+            + ". Aliases must consist entirely of periods, underscores, and alphanumerics");
+      }
       this.aliasName = aliasName;
       return this;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
index 188a9ea..2a2d4ae 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
@@ -20,6 +20,7 @@ import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.response.CoreAdminResponse;
+import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
@@ -100,6 +101,22 @@ public class CoreAdminRequest extends SolrRequest<CoreAdminResponse> {
     public Boolean getIsTransient() { return isTransient; }
     public String getCollectionConfigName() { return collectionConfigName;}
     
+    /**
+     * Provide the name of the core to be created.
+     * 
+     * Core names must consist entirely of periods, underscores and alphanumerics.  Other characters are not allowed.
+     * 
+     * @throws IllegalArgumentException if the core name contains invalid characters.
+     */
+    @Override
+    public void setCoreName(String coreName) {
+      if (!SolrIdentifierValidator.validateCoreName(coreName)) {
+        throw new IllegalArgumentException("Invalid collection: " + coreName
+            + ". Core names must consist entirely of periods, underscores, and alphanumerics");
+      }
+      this.core = coreName;
+    }
+    
     @Override
     public SolrParams getParams() {
       if( action == null ) {
@@ -450,7 +467,7 @@ public class CoreAdminRequest extends SolrRequest<CoreAdminResponse> {
     super( METHOD.GET, path );
   }
 
-  public final void setCoreName( String coreName )
+  public void setCoreName( String coreName )
   {
     this.core = coreName;
   }
@@ -535,8 +552,18 @@ public class CoreAdminRequest extends SolrRequest<CoreAdminResponse> {
     return req.process(client);
   }
 
-  public static CoreAdminResponse renameCore(String coreName, String newName, SolrClient client ) throws SolrServerException, IOException
-  {
+  /**
+   * Rename an existing core.
+   * 
+   * @throws IllegalArgumentException if the new core name contains invalid characters.
+   */
+  public static CoreAdminResponse renameCore(String coreName, String newName, SolrClient client )
+      throws SolrServerException, IOException {
+    if (!SolrIdentifierValidator.validateCoreName(newName)) {
+      throw new IllegalArgumentException("Invalid collection: " + newName
+          + ". Core names must consist entirely of periods, underscores, and alphanumerics");
+    }
+    
     CoreAdminRequest req = new CoreAdminRequest();
     req.setCoreName(coreName);
     req.setOtherCoreName(newName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/solrj/src/java/org/apache/solr/client/solrj/util/SolrIdentifierValidator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/util/SolrIdentifierValidator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/util/SolrIdentifierValidator.java
new file mode 100644
index 0000000..00d9b83
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/util/SolrIdentifierValidator.java
@@ -0,0 +1,51 @@
+package org.apache.solr.client.solrj.util;
+
+import java.util.regex.Pattern;
+
+/*
+ * 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.
+ */
+
+/**
+ * Ensures that provided identifiers align with Solr's recommendations/requirements for choosing
+ * collection, core, etc identifiers.
+ *  
+ * Identifiers are allowed to contain underscores, periods, and alphanumeric characters. 
+ */
+public class SolrIdentifierValidator {
+  final static Pattern identifierPattern = Pattern.compile("^[\\._A-Za-z0-9]*$");
+  
+  public static boolean validateShardName(String shardName) {
+    return validateIdentifier(shardName);
+  }
+  
+  public static boolean validateCollectionName(String collectionName) {
+    return validateIdentifier(collectionName);
+  }
+  
+  public static boolean validateCoreName(String name) {
+    return validateIdentifier(name);
+  }
+  
+  private static boolean validateIdentifier(String identifier) {
+    if (identifier == null || ! identifierPattern.matcher(identifier).matches()) {
+      return false;
+    }
+    return true;
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCollectionAdminRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCollectionAdminRequest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCollectionAdminRequest.java
new file mode 100644
index 0000000..6fef14c
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCollectionAdminRequest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.client.solrj.request;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest.CreateAlias;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest.CreateShard;
+import org.junit.Test;
+
+/**
+ * Unit tests for {@link CollectionAdminRequest}.
+ */
+public class TestCollectionAdminRequest extends LuceneTestCase {
+  
+  @Test
+  public void testInvalidCollectionNameRejectedWhenCreatingCollection() {
+    final Create createRequest = new Create();
+    try {
+      createRequest.setCollectionName("invalid$collection@name");
+      fail();
+    } catch (IllegalArgumentException e) {
+      final String exceptionMessage = e.getMessage();
+      assertTrue(exceptionMessage.contains("Invalid collection"));
+      assertTrue(exceptionMessage.contains("invalid$collection@name"));
+      assertTrue(exceptionMessage.contains("must consist entirely of periods, underscores, and alphanumerics"));
+    }
+  }
+  
+  @Test
+  public void testInvalidShardNamesRejectedWhenCreatingCollection() {
+    final Create createRequest = new Create();
+    try {
+      createRequest.setShards("invalid$shard@name");
+      fail();
+    } catch (IllegalArgumentException e) {
+      final String exceptionMessage = e.getMessage();
+      assertTrue(exceptionMessage.contains("Invalid shard"));
+      assertTrue(exceptionMessage.contains("invalid$shard@name"));
+      assertTrue(exceptionMessage.contains("must consist entirely of periods, underscores, and alphanumerics"));
+    }
+  }
+  
+  @Test
+  public void testInvalidAliasNameRejectedWhenCreatingAlias() {
+    final CreateAlias createAliasRequest = new CreateAlias();
+    try {
+      createAliasRequest.setAliasName("invalid$alias@name");
+      fail();
+    } catch (IllegalArgumentException e) {
+      final String exceptionMessage = e.getMessage();
+      assertTrue(exceptionMessage.contains("Invalid collection"));
+      assertTrue(exceptionMessage.contains("invalid$alias@name"));
+      assertTrue(exceptionMessage.contains("must consist entirely of periods, underscores, and alphanumerics"));
+    }
+  }
+  
+  @Test
+  public void testInvalidShardNameRejectedWhenCreatingShard() {
+    final CreateShard createShardRequest = new CreateShard();
+    try {
+      createShardRequest.setShardName("invalid$shard@name");
+      fail();
+    } catch (IllegalArgumentException e) {
+      final String exceptionMessage = e.getMessage();
+      assertTrue(exceptionMessage.contains("Invalid shard"));
+      assertTrue(exceptionMessage.contains("invalid$shard@name"));
+      assertTrue(exceptionMessage.contains("must consist entirely of periods, underscores, and alphanumerics"));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
index 4e46373..8275ff1 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
@@ -16,17 +16,19 @@
  */
 package org.apache.solr.client.solrj.request;
 
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.core.Is.is;
+
 import java.io.File;
 import java.lang.invoke.MethodHandles;
 
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
-import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.SolrIgnoredThreadsFilter;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.embedded.AbstractEmbeddedSolrServerTestCase;
 import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
 import org.apache.solr.client.solrj.response.CoreAdminResponse;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -41,8 +43,8 @@ import org.junit.rules.TestRule;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.hamcrest.CoreMatchers.notNullValue;
-import static org.hamcrest.core.Is.is;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
+import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 
 @ThreadLeakFilters(defaultFilters = true, filters = {SolrIgnoredThreadsFilter.class})
 public class TestCoreAdmin extends AbstractEmbeddedSolrServerTestCase {
@@ -158,6 +160,34 @@ public class TestCoreAdmin extends AbstractEmbeddedSolrServerTestCase {
     assertTrue(gotExp);
   }
   
+  @Test
+  public void testInvalidCoreNamesAreRejectedWhenCreatingCore() {
+    final Create createRequest = new Create();
+    
+    try {
+      createRequest.setCoreName("invalid$core@name");
+      fail();
+    } catch (IllegalArgumentException e) {
+      final String exceptionMessage = e.getMessage();
+      assertTrue(exceptionMessage.contains("Invalid core"));
+      assertTrue(exceptionMessage.contains("invalid$core@name"));
+      assertTrue(exceptionMessage.contains("must consist entirely of periods, underscores and alphanumerics"));
+    }
+  }
+  
+  @Test
+  public void testInvalidCoreNamesAreRejectedWhenRenamingExistingCore() throws Exception {
+    try {
+      CoreAdminRequest.renameCore("validExistingCoreName", "invalid$core@name", null);
+      fail();
+    } catch (IllegalArgumentException e) {
+      final String exceptionMessage = e.getMessage();
+      assertTrue(exceptionMessage.contains("Invalid core"));
+      assertTrue(exceptionMessage.contains("invalid$core@name"));
+      assertTrue(exceptionMessage.contains("must consist entirely of periods, underscores and alphanumerics"));
+    }
+  }
+  
   @BeforeClass
   public static void before() {
     // wtf?


Re: lucene-solr git commit: SOLR-8677: Restrict creation of shards with invalid names

Posted by Anshum Gupta <an...@apache.org>.
Sorry, seems like I broke the build. I'll fix.

On Thu, Feb 18, 2016 at 3:19 PM, <an...@apache.org> wrote:

> Repository: lucene-solr
> Updated Branches:
>   refs/heads/master 8bcac81a2 -> d01230d63
>
>
> SOLR-8677: Restrict creation of shards with invalid names
>
>
> Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
> Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/d01230d6
> Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/d01230d6
> Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/d01230d6
>
> Branch: refs/heads/master
> Commit: d01230d6394b29fa6fd42377404c0c03d6e8a4d9
> Parents: 8bcac81
> Author: anshum <an...@apache.org>
> Authored: Thu Feb 18 14:00:10 2016 -0800
> Committer: anshum <an...@apache.org>
> Committed: Thu Feb 18 15:18:44 2016 -0800
>
> ----------------------------------------------------------------------
>  solr/CHANGES.txt                                |  6 +-
>  .../org/apache/solr/core/CoreContainer.java     | 12 ++-
>  .../solr/handler/admin/CollectionsHandler.java  | 31 ++++++-
>  .../solr/util/SolrIdentifierValidator.java      | 47 -----------
>  .../apache/solr/cloud/TestCollectionAPI.java    | 71 ++++++++++++++--
>  .../handler/admin/CoreAdminHandlerTest.java     |  4 +-
>  .../solrj/request/CollectionAdminRequest.java   | 79 ++++++++++++++++--
>  .../client/solrj/request/CoreAdminRequest.java  | 33 +++++++-
>  .../solrj/util/SolrIdentifierValidator.java     | 51 ++++++++++++
>  .../request/TestCollectionAdminRequest.java     | 85 ++++++++++++++++++++
>  .../client/solrj/request/TestCoreAdmin.java     | 38 ++++++++-
>  11 files changed, 380 insertions(+), 77 deletions(-)
> ----------------------------------------------------------------------
>
>
>
> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/CHANGES.txt
> ----------------------------------------------------------------------
> diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
> index e6635f1..3fba69d 100644
> --- a/solr/CHANGES.txt
> +++ b/solr/CHANGES.txt
> @@ -266,6 +266,11 @@ Other Changes
>  * SOLR-5730: Make Lucene's SortingMergePolicy and
> EarlyTerminatingSortingCollector configurable in Solr.
>    (Christine Poerschke, hossmann, Tomás Fernández Löbbe, Shai Erera)
>
> +Other Changes
> +----------------------
> +* SOLR-8677: Prevent shards containing invalid characters from being
> created.  Checks added server-side
> +  and in SolrJ.  (Shai Erera, Jason Gerlowski, Anshum Gupta)
> +
>  ======================= 5.5.0 =======================
>
>  Consult the LUCENE_CHANGES.txt file for additional, low level, changes in
> this release
> @@ -1018,7 +1023,6 @@ Bug Fixes
>
>  * SOLR-8355: update permissions were failing node recovery (noble ,
> Anshum Gupta)
>
> -
>  Optimizations
>  ----------------------
>
>
>
> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/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 426a493..7703721 100644
> --- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
> +++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
> @@ -36,6 +36,7 @@ import com.google.common.collect.ImmutableMap;
>  import com.google.common.collect.Maps;
>  import org.apache.solr.client.solrj.impl.HttpClientConfigurer;
>  import org.apache.solr.client.solrj.impl.HttpClientUtil;
> +import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
>  import org.apache.solr.cloud.Overseer;
>  import org.apache.solr.cloud.ZkController;
>  import org.apache.solr.common.SolrException;
> @@ -63,7 +64,6 @@ import org.apache.solr.security.PKIAuthenticationPlugin;
>  import org.apache.solr.security.SecurityPluginHolder;
>  import org.apache.solr.update.UpdateShardHandler;
>  import org.apache.solr.util.DefaultSolrThreadFactory;
> -import org.apache.solr.util.SolrIdentifierValidator;
>  import org.apache.zookeeper.KeeperException;
>  import org.slf4j.Logger;
>  import org.slf4j.LoggerFactory;
> @@ -804,7 +804,10 @@ public class CoreContainer {
>      SolrCore core = null;
>      try {
>        MDCLoggingContext.setCore(core);
> -      SolrIdentifierValidator.validateCoreName(dcore.getName());
> +      if (!SolrIdentifierValidator.validateCoreName(dcore.getName())) {
> +        throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid core: " +
> dcore.getName()
> +            + ". Core names must consist entirely of periods,
> underscores, and alphanumerics");
> +      }
>        if (zkSys.getZkController() != null) {
>          zkSys.getZkController().preRegister(dcore);
>        }
> @@ -1007,7 +1010,10 @@ public class CoreContainer {
>    }
>
>    public void rename(String name, String toName) {
> -    SolrIdentifierValidator.validateCoreName(toName);
> +    if(!SolrIdentifierValidator.validateCoreName(toName)) {
> +      throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid core: " +
> toName
> +          + ". Core names must consist entirely of periods, underscores,
> and alphanumerics");
> +    }
>      try (SolrCore core = getCore(name)) {
>        if (core != null) {
>          registerCore(toName, core, true);
>
>
> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
> b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
> index d04786f..68fac7d 100644
> ---
> a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
> +++
> b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
> @@ -69,6 +69,7 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient;
>  import org.apache.solr.client.solrj.request.CoreAdminRequest;
>  import
> org.apache.solr.client.solrj.request.CoreAdminRequest.RequestSyncShard;
>  import org.apache.solr.client.solrj.response.RequestStatusState;
> +import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
>  import org.apache.solr.cloud.DistributedMap;
>  import org.apache.solr.cloud.Overseer;
>  import org.apache.solr.cloud.OverseerCollectionMessageHandler;
> @@ -109,7 +110,6 @@ import org.apache.solr.handler.RequestHandlerBase;
>  import org.apache.solr.handler.component.ShardHandler;
>  import org.apache.solr.request.SolrQueryRequest;
>  import org.apache.solr.response.SolrQueryResponse;
> -import org.apache.solr.util.SolrIdentifierValidator;
>  import org.apache.zookeeper.CreateMode;
>  import org.apache.zookeeper.KeeperException;
>  import org.slf4j.Logger;
> @@ -347,7 +347,16 @@ public class CollectionsHandler extends
> RequestHandlerBase {
>          addMapObject(props, SNITCH);
>          verifyRuleParams(h.coreContainer, props);
>          final String collectionName = (String) props.get(NAME);
> -        SolrIdentifierValidator.validateCollectionName(collectionName);
> +        if
> (!SolrIdentifierValidator.validateCollectionName(collectionName)) {
> +          throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid
> collection: " + collectionName
> +          + ". Collection names must consist entirely of periods,
> underscores, and alphanumerics");
> +        }
> +        final String shardsParam = (String) props.get(SHARDS_PROP);
> +        if (StringUtils.isNotEmpty(shardsParam)) {
> +          log.info("Validating shards param!!!!!!!!" + shardsParam);
> +          verifyShardsParam(shardsParam);
> +          log.info("Validating shards param!!!!!!! done" + shardsParam);
> +        }
>          if (SYSTEM_COLL.equals(collectionName)) {
>            //We must always create a .system collection with only a single
> shard
>            props.put(NUM_SLICES, 1);
> @@ -419,7 +428,10 @@ public class CollectionsHandler extends
> RequestHandlerBase {
>        Map<String, Object> call(SolrQueryRequest req, SolrQueryResponse
> rsp, CollectionsHandler handler)
>            throws Exception {
>          final String aliasName = req.getParams().get(NAME);
> -        SolrIdentifierValidator.validateCollectionName(aliasName);
> +        if (!SolrIdentifierValidator.validateCollectionName(aliasName)) {
> +          throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid alias:
> " + aliasName
> +              + ". Aliases must consist entirely of periods, underscores,
> and alphanumerics");
> +        }
>          return req.getParams().required().getAll(null, NAME,
> "collections");
>        }
>      },
> @@ -483,6 +495,11 @@ public class CollectionsHandler extends
> RequestHandlerBase {
>              COLLECTION_PROP,
>              SHARD_ID_PROP);
>          ClusterState clusterState =
> handler.coreContainer.getZkController().getClusterState();
> +        final String newShardName = req.getParams().get(SHARD_ID_PROP);
> +        if (!SolrIdentifierValidator.validateShardName(newShardName)) {
> +          throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid shard:
> " + newShardName
> +              + ". Shard names must consist entirely of periods,
> underscores and alphanumerics");
> +        }
>          if (!ImplicitDocRouter.NAME.equals(((Map)
> clusterState.getCollection(req.getParams().get(COLLECTION_PROP)).get(DOC_ROUTER)).get(NAME)))
>            throw new SolrException(ErrorCode.BAD_REQUEST, "shards can be
> added only to 'implicit' collections");
>          req.getParams().getAll(map,
> @@ -963,6 +980,14 @@ public class CollectionsHandler extends
> RequestHandlerBase {
>      }
>      return props;
>    }
> +
> +  private static void verifyShardsParam(String shardsParam) {
> +    for (String shard : shardsParam.split(",")) {
> +      if (!SolrIdentifierValidator.validateShardName(shard))
> +        throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid shard: "
> + shard
> +            + ". Shard names must consist entirely of periods,
> underscores and alphanumerics");;
> +    }
> +  }
>
>    public static final List<String> MODIFIABLE_COLL_PROPS =
> ImmutableList.of(
>        RULE,
>
>
> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/core/src/java/org/apache/solr/util/SolrIdentifierValidator.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/core/src/java/org/apache/solr/util/SolrIdentifierValidator.java
> b/solr/core/src/java/org/apache/solr/util/SolrIdentifierValidator.java
> deleted file mode 100644
> index dd6133d..0000000
> --- a/solr/core/src/java/org/apache/solr/util/SolrIdentifierValidator.java
> +++ /dev/null
> @@ -1,47 +0,0 @@
> -/*
> - * Licensed to the Apache Software Foundation (ASF) under one or more
> - * contributor license agreements.  See the NOTICE file distributed with
> - * this work for additional information regarding copyright ownership.
> - * The ASF licenses this file to You under the Apache License, Version 2.0
> - * (the "License"); you may not use this file except in compliance with
> - * the License.  You may obtain a copy of the License at
> - *
> - *     http://www.apache.org/licenses/LICENSE-2.0
> - *
> - * Unless required by applicable law or agreed to in writing, software
> - * distributed under the License is distributed on an "AS IS" BASIS,
> - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> - * See the License for the specific language governing permissions and
> - * limitations under the License.
> - */
> -package org.apache.solr.util;
> -
> -import java.lang.invoke.MethodHandles;
> -import java.util.regex.Pattern;
> -
> -import org.apache.solr.common.SolrException;
> -import org.slf4j.Logger;
> -import org.slf4j.LoggerFactory;
> -
> -/**
> - * Ensures that provided identifiers align with Solr's
> recommendations/requirements for choosing
> - * collection, core, etc identifiers.
> - *
> - * Identifiers are allowed to contain underscores, periods, and
> alphanumeric characters.
> - */
> -public class SolrIdentifierValidator {
> -  private static final Logger log =
> LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
> -  final static Pattern identifierPattern =
> Pattern.compile("^[\\._A-Za-z0-9]*$");
> -
> -  public static void validateCollectionName(String collectionName) throws
> SolrException {
> -    validateCoreName(collectionName);
> -  }
> -
> -  public static void validateCoreName(String name) throws SolrException {
> -    if (name == null || !identifierPattern.matcher(name).matches()) {
> -      log.info("Validation failed on the invalid identifier [{}].
> Throwing SolrException to indicate a BAD REQUEST.", name);
> -      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
> -          "Invalid name: '" + name + "' Identifiers must consist entirely
> of periods, underscores and alphanumerics");
> -    }
> -  }
> -}
>
>
> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
> b/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
> index b19c1c1..353c708 100644
> --- a/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
> +++ b/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
> @@ -44,6 +44,7 @@ import org.apache.solr.common.util.NamedList;
>  import org.apache.zookeeper.KeeperException;
>  import org.junit.Test;
>
> +import static
> org.apache.solr.cloud.OverseerCollectionMessageHandler.ROUTER;
>  import static
> org.apache.solr.cloud.OverseerCollectionMessageHandler.SHARD_UNIQUE;
>
>  public class TestCollectionAPI extends ReplicaPropertiesBase {
> @@ -79,8 +80,10 @@ public class TestCollectionAPI extends
> ReplicaPropertiesBase {
>      replicaPropTest();
>      clusterStatusZNodeVersion();
>      testClusterStateMigration();
> -    testCollectionCreationNameValidation();
> +    testCollectionCreationCollectionNameValidation();
> +    testCollectionCreationShardNameValidation();
>      testAliasCreationNameValidation();
> +    testShardCreationNameValidation();
>    }
>
>    private void clusterStatusWithCollectionAndShard() throws IOException,
> SolrServerException {
> @@ -631,7 +634,7 @@ public class TestCollectionAPI extends
> ReplicaPropertiesBase {
>      }
>    }
>
> -  private void testCollectionCreationNameValidation() throws Exception {
> +  private void testCollectionCreationCollectionNameValidation() throws
> Exception {
>      try (CloudSolrClient client = createCloudClient(null)) {
>        ModifiableSolrParams params = new ModifiableSolrParams();
>        params.set("action",
> CollectionParams.CollectionAction.CREATE.toString());
> @@ -644,9 +647,32 @@ public class TestCollectionAPI extends
> ReplicaPropertiesBase {
>          fail();
>        } catch (RemoteSolrException e) {
>          final String errorMessage = e.getMessage();
> -        assertTrue(errorMessage.contains("Invalid name"));
> +        assertTrue(errorMessage.contains("Invalid collection"));
>          assertTrue(errorMessage.contains("invalid@name
> #with$weird%characters"));
> -        assertTrue(errorMessage.contains("Identifiers must consist
> entirely of"));
> +        assertTrue(errorMessage.contains("Collection names must consist
> entirely of"));
> +      }
> +    }
> +  }
> +
> +  private void testCollectionCreationShardNameValidation() throws
> Exception {
> +    try (CloudSolrClient client = createCloudClient(null)) {
> +      ModifiableSolrParams params = new ModifiableSolrParams();
> +      params.set("action",
> CollectionParams.CollectionAction.CREATE.toString());
> +      params.set("name", "valid_collection_name");
> +      params.set("router.name", "implicit");
> +      params.set("numShards", "1");
> +      params.set("shards", "invalid@name#with$weird%characters");
> +      SolrRequest request = new QueryRequest(params);
> +      request.setPath("/admin/collections");
> +
> +      try {
> +        client.request(request);
> +        fail();
> +      } catch (RemoteSolrException e) {
> +        final String errorMessage = e.getMessage();
> +        assertTrue(errorMessage.contains("Invalid shard"));
> +        assertTrue(errorMessage.contains("invalid@name
> #with$weird%characters"));
> +        assertTrue(errorMessage.contains("Shard names must consist
> entirely of"));
>        }
>      }
>    }
> @@ -665,9 +691,42 @@ public class TestCollectionAPI extends
> ReplicaPropertiesBase {
>          fail();
>        } catch (RemoteSolrException e) {
>          final String errorMessage = e.getMessage();
> -        assertTrue(errorMessage.contains("Invalid name"));
> +        assertTrue(errorMessage.contains("Invalid alias"));
> +        assertTrue(errorMessage.contains("invalid@name
> #with$weird%characters"));
> +        assertTrue(errorMessage.contains("Aliases must consist entirely
> of"));
> +      }
> +    }
> +  }
> +
> +  private void testShardCreationNameValidation() throws Exception {
> +    try (CloudSolrClient client = createCloudClient(null)) {
> +      client.connect();
> +      // Create a collection w/ implicit router
> +      ModifiableSolrParams params = new ModifiableSolrParams();
> +      params.set("action",
> CollectionParams.CollectionAction.CREATE.toString());
> +      params.set("name", "valid_collection_name");
> +      params.set("shards", "a");
> +      params.set("router.name", "implicit");
> +      SolrRequest request = new QueryRequest(params);
> +      request.setPath("/admin/collections");
> +      client.request(request);
> +
> +      params = new ModifiableSolrParams();
> +      params.set("action",
> CollectionParams.CollectionAction.CREATESHARD.toString());
> +      params.set("collection", "valid_collection_name");
> +      params.set("shard", "invalid@name#with$weird%characters");
> +
> +      request = new QueryRequest(params);
> +      request.setPath("/admin/collections");
> +
> +      try {
> +        client.request(request);
> +        fail();
> +      } catch (RemoteSolrException e) {
> +        final String errorMessage = e.getMessage();
> +        assertTrue(errorMessage.contains("Invalid shard"));
>          assertTrue(errorMessage.contains("invalid@name
> #with$weird%characters"));
> -        assertTrue(errorMessage.contains("Identifiers must consist
> entirely of"));
> +        assertTrue(errorMessage.contains("Shard names must consist
> entirely of"));
>        }
>      }
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
> b/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
> index 2159325..89f4b2b 100644
> ---
> a/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
> +++
> b/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
> @@ -139,7 +139,7 @@ public class CoreAdminHandlerTest extends
> SolrTestCaseJ4 {
>                resp);
>
>      } catch (SolrException se) {
> -      assertTrue("Expected error message for bad core name.",
> se.toString().contains("Invalid name"));
> +      assertTrue("Expected error message for bad core name.",
> se.toString().contains("Invalid core"));
>      }
>      CoreDescriptor cd = cores.getCoreDescriptor("ugly$core=name");
>      assertNull("Should NOT have added this core!", cd);
> @@ -228,7 +228,7 @@ public class CoreAdminHandlerTest extends
> SolrTestCaseJ4 {
>                CoreAdminParams.OTHER, "bad$name"),
>                resp);
>      } catch (SolrException e) { // why the heck does create return a
> SolrException (admittedly wrapping an IAE)
> -      assertTrue("Expected error message for bad core name.",
> e.getMessage().contains("Invalid name"));
> +      assertTrue("Expected error message for bad core name.",
> e.getMessage().contains("Invalid core"));
>      }
>
>      cd = cores.getCoreDescriptor("bad$name");
>
>
> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
> b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
> index e3446f7..700954f 100644
> ---
> a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
> +++
> b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
> @@ -16,9 +16,16 @@
>   */
>  package org.apache.solr.client.solrj.request;
>
> +import java.io.IOException;
> +import java.util.Collection;
> +import java.util.Iterator;
> +import java.util.Map;
> +import java.util.Properties;
> +
>  import org.apache.solr.client.solrj.SolrClient;
>  import org.apache.solr.client.solrj.SolrRequest;
>  import org.apache.solr.client.solrj.response.CollectionAdminResponse;
> +import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
>  import org.apache.solr.common.SolrException;
>  import org.apache.solr.common.cloud.DocCollection;
>  import org.apache.solr.common.cloud.ZkStateReader;
> @@ -31,12 +38,6 @@ import org.apache.solr.common.params.ShardParams;
>  import org.apache.solr.common.params.SolrParams;
>  import org.apache.solr.common.util.ContentStream;
>
> -import java.io.IOException;
> -import java.util.Collection;
> -import java.util.Iterator;
> -import java.util.Map;
> -import java.util.Properties;
> -
>  /**
>   * This class is experimental and subject to change.
>   *
> @@ -122,7 +123,7 @@ public abstract class CollectionAdminRequest <Q
> extends CollectionAdminRequest<Q
>    protected abstract static class CollectionSpecificAdminRequest <T
> extends CollectionAdminRequest<T>> extends CollectionAdminRequest<T> {
>      protected String collection = null;
>
> -    public final T setCollectionName(String collectionName) {
> +    public T setCollectionName(String collectionName) {
>        this.collection = collectionName;
>        return getThis();
>      }
> @@ -277,7 +278,6 @@ public abstract class CollectionAdminRequest <Q
> extends CollectionAdminRequest<Q
>      public Create setConfigName(String config) { this.configName =
> config; return this; }
>      public Create setCreateNodeSet(String nodeSet) { this.createNodeSet =
> nodeSet; return this; }
>      public Create setRouterName(String routerName) { this.routerName =
> routerName; return this; }
> -    public Create setShards(String shards) { this.shards = shards; return
> this; }
>      public Create setRouterField(String routerField) { this.routerField =
> routerField; return this; }
>      public Create setNumShards(Integer numShards) {this.numShards =
> numShards; return this; }
>      public Create setMaxShardsPerNode(Integer numShards) {
> this.maxShardsPerNode = numShards; return this; }
> @@ -296,6 +296,41 @@ public abstract class CollectionAdminRequest <Q
> extends CollectionAdminRequest<Q
>      public Integer getReplicationFactor() { return replicationFactor; }
>      public Boolean getAutoAddReplicas() { return autoAddReplicas; }
>      public Integer getStateFormat() { return stateFormat; }
> +
> +    /**
> +     * Provide the name of the shards to be created, separated by commas
> +     *
> +     * Shard names must consist entirely of periods, underscores and
> alphanumerics.  Other characters are not allowed.
> +     *
> +     * @throws IllegalArgumentException if any of the shard names contain
> invalid characters.
> +     */
> +    public Create setShards(String shards) {
> +      for (String shard : shards.split(",")) {
> +        if (!SolrIdentifierValidator.validateShardName(shard)) {
> +          throw new IllegalArgumentException("Invalid shard: " + shard
> +              + ". Shard names must consist entirely of periods,
> underscores and alphanumerics");
> +        }
> +      }
> +      this.shards = shards;
> +      return this;
> +    }
> +
> +    /**
> +     * Provide the name of the collection to be created.
> +     *
> +     * Collection names must consist entirely of periods, underscores and
> alphanumerics.  Other characters are not allowed.
> +     *
> +     * @throws IllegalArgumentException if the collection name contains
> invalid characters.
> +     */
> +    @Override
> +    public Create setCollectionName(String collectionName) throws
> SolrException {
> +      if
> (!SolrIdentifierValidator.validateCollectionName(collectionName)) {
> +        throw new IllegalArgumentException("Invalid collection: " +
> collectionName
> +            + ". Collection names must consist entirely of periods,
> underscores, and alphanumerics");
> +      }
> +      this.collection = collectionName;
> +      return this;
> +    }
>
>      public Properties getProperties() {
>        return properties;
> @@ -409,6 +444,23 @@ public abstract class CollectionAdminRequest <Q
> extends CollectionAdminRequest<Q
>      public CreateShard() {
>        action = CollectionAction.CREATESHARD;
>      }
> +
> +    /**
> +     * Provide the name of the shard to be created.
> +     *
> +     * Shard names must consist entirely of periods, underscores and
> alphanumerics.  Other characters are not allowed.
> +     *
> +     * @throws IllegalArgumentException if the shard name contains
> invalid characters.
> +     */
> +    @Override
> +    public CreateShard setShardName(String shardName) {
> +      if (!SolrIdentifierValidator.validateShardName(shardName)) {
> +        throw new IllegalArgumentException("Invalid shard: " + shardName
> +            + ". Shard names must consist entirely of periods,
> underscores and alphanumerics");
> +      }
> +      this.shardName = shardName;
> +      return this;
> +    }
>
>      @Override
>      public SolrParams getParams() {
> @@ -588,7 +640,18 @@ public abstract class CollectionAdminRequest <Q
> extends CollectionAdminRequest<Q
>        action = CollectionAction.CREATEALIAS;
>      }
>
> +    /**
> +     * Provide the name of the alias to be created.
> +     *
> +     * Alias names must consist entirely of periods, underscores and
> alphanumerics.  Other characters are not allowed.
> +     *
> +     * @throws IllegalArgumentException if the alias name contains
> invalid characters.
> +     */
>      public CreateAlias setAliasName(String aliasName) {
> +      if (!SolrIdentifierValidator.validateCollectionName(aliasName)) {
> +        throw new IllegalArgumentException("Invalid alias: " + aliasName
> +            + ". Aliases must consist entirely of periods, underscores,
> and alphanumerics");
> +      }
>        this.aliasName = aliasName;
>        return this;
>      }
>
>
> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
> b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
> index 188a9ea..2a2d4ae 100644
> ---
> a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
> +++
> b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
> @@ -20,6 +20,7 @@ import org.apache.solr.client.solrj.SolrClient;
>  import org.apache.solr.client.solrj.SolrRequest;
>  import org.apache.solr.client.solrj.SolrServerException;
>  import org.apache.solr.client.solrj.response.CoreAdminResponse;
> +import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
>  import org.apache.solr.common.cloud.Replica;
>  import org.apache.solr.common.cloud.ZkStateReader;
>  import org.apache.solr.common.params.CoreAdminParams;
> @@ -100,6 +101,22 @@ public class CoreAdminRequest extends
> SolrRequest<CoreAdminResponse> {
>      public Boolean getIsTransient() { return isTransient; }
>      public String getCollectionConfigName() { return
> collectionConfigName;}
>
> +    /**
> +     * Provide the name of the core to be created.
> +     *
> +     * Core names must consist entirely of periods, underscores and
> alphanumerics.  Other characters are not allowed.
> +     *
> +     * @throws IllegalArgumentException if the core name contains invalid
> characters.
> +     */
> +    @Override
> +    public void setCoreName(String coreName) {
> +      if (!SolrIdentifierValidator.validateCoreName(coreName)) {
> +        throw new IllegalArgumentException("Invalid collection: " +
> coreName
> +            + ". Core names must consist entirely of periods,
> underscores, and alphanumerics");
> +      }
> +      this.core = coreName;
> +    }
> +
>      @Override
>      public SolrParams getParams() {
>        if( action == null ) {
> @@ -450,7 +467,7 @@ public class CoreAdminRequest extends
> SolrRequest<CoreAdminResponse> {
>      super( METHOD.GET, path );
>    }
>
> -  public final void setCoreName( String coreName )
> +  public void setCoreName( String coreName )
>    {
>      this.core = coreName;
>    }
> @@ -535,8 +552,18 @@ public class CoreAdminRequest extends
> SolrRequest<CoreAdminResponse> {
>      return req.process(client);
>    }
>
> -  public static CoreAdminResponse renameCore(String coreName, String
> newName, SolrClient client ) throws SolrServerException, IOException
> -  {
> +  /**
> +   * Rename an existing core.
> +   *
> +   * @throws IllegalArgumentException if the new core name contains
> invalid characters.
> +   */
> +  public static CoreAdminResponse renameCore(String coreName, String
> newName, SolrClient client )
> +      throws SolrServerException, IOException {
> +    if (!SolrIdentifierValidator.validateCoreName(newName)) {
> +      throw new IllegalArgumentException("Invalid collection: " + newName
> +          + ". Core names must consist entirely of periods, underscores,
> and alphanumerics");
> +    }
> +
>      CoreAdminRequest req = new CoreAdminRequest();
>      req.setCoreName(coreName);
>      req.setOtherCoreName(newName);
>
>
> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/solrj/src/java/org/apache/solr/client/solrj/util/SolrIdentifierValidator.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/solrj/src/java/org/apache/solr/client/solrj/util/SolrIdentifierValidator.java
> b/solr/solrj/src/java/org/apache/solr/client/solrj/util/SolrIdentifierValidator.java
> new file mode 100644
> index 0000000..00d9b83
> --- /dev/null
> +++
> b/solr/solrj/src/java/org/apache/solr/client/solrj/util/SolrIdentifierValidator.java
> @@ -0,0 +1,51 @@
> +package org.apache.solr.client.solrj.util;
> +
> +import java.util.regex.Pattern;
> +
> +/*
> + * 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.
> + */
> +
> +/**
> + * Ensures that provided identifiers align with Solr's
> recommendations/requirements for choosing
> + * collection, core, etc identifiers.
> + *
> + * Identifiers are allowed to contain underscores, periods, and
> alphanumeric characters.
> + */
> +public class SolrIdentifierValidator {
> +  final static Pattern identifierPattern =
> Pattern.compile("^[\\._A-Za-z0-9]*$");
> +
> +  public static boolean validateShardName(String shardName) {
> +    return validateIdentifier(shardName);
> +  }
> +
> +  public static boolean validateCollectionName(String collectionName) {
> +    return validateIdentifier(collectionName);
> +  }
> +
> +  public static boolean validateCoreName(String name) {
> +    return validateIdentifier(name);
> +  }
> +
> +  private static boolean validateIdentifier(String identifier) {
> +    if (identifier == null || !
> identifierPattern.matcher(identifier).matches()) {
> +      return false;
> +    }
> +    return true;
> +  }
> +}
> +
> +
>
>
> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCollectionAdminRequest.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCollectionAdminRequest.java
> b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCollectionAdminRequest.java
> new file mode 100644
> index 0000000..6fef14c
> --- /dev/null
> +++
> b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCollectionAdminRequest.java
> @@ -0,0 +1,85 @@
> +/*
> + * Licensed to the Apache Software Foundation (ASF) under one or more
> + * contributor license agreements.  See the NOTICE file distributed with
> + * this work for additional information regarding copyright ownership.
> + * The ASF licenses this file to You under the Apache License, Version 2.0
> + * (the "License"); you may not use this file except in compliance with
> + * the License.  You may obtain a copy of the License at
> + *
> + *     http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + */
> +package org.apache.solr.client.solrj.request;
> +
> +import org.apache.lucene.util.LuceneTestCase;
> +import org.apache.solr.client.solrj.request.CollectionAdminRequest.Create;
> +import
> org.apache.solr.client.solrj.request.CollectionAdminRequest.CreateAlias;
> +import
> org.apache.solr.client.solrj.request.CollectionAdminRequest.CreateShard;
> +import org.junit.Test;
> +
> +/**
> + * Unit tests for {@link CollectionAdminRequest}.
> + */
> +public class TestCollectionAdminRequest extends LuceneTestCase {
> +
> +  @Test
> +  public void testInvalidCollectionNameRejectedWhenCreatingCollection() {
> +    final Create createRequest = new Create();
> +    try {
> +      createRequest.setCollectionName("invalid$collection@name");
> +      fail();
> +    } catch (IllegalArgumentException e) {
> +      final String exceptionMessage = e.getMessage();
> +      assertTrue(exceptionMessage.contains("Invalid collection"));
> +      assertTrue(exceptionMessage.contains("invalid$collection@name"));
> +      assertTrue(exceptionMessage.contains("must consist entirely of
> periods, underscores, and alphanumerics"));
> +    }
> +  }
> +
> +  @Test
> +  public void testInvalidShardNamesRejectedWhenCreatingCollection() {
> +    final Create createRequest = new Create();
> +    try {
> +      createRequest.setShards("invalid$shard@name");
> +      fail();
> +    } catch (IllegalArgumentException e) {
> +      final String exceptionMessage = e.getMessage();
> +      assertTrue(exceptionMessage.contains("Invalid shard"));
> +      assertTrue(exceptionMessage.contains("invalid$shard@name"));
> +      assertTrue(exceptionMessage.contains("must consist entirely of
> periods, underscores, and alphanumerics"));
> +    }
> +  }
> +
> +  @Test
> +  public void testInvalidAliasNameRejectedWhenCreatingAlias() {
> +    final CreateAlias createAliasRequest = new CreateAlias();
> +    try {
> +      createAliasRequest.setAliasName("invalid$alias@name");
> +      fail();
> +    } catch (IllegalArgumentException e) {
> +      final String exceptionMessage = e.getMessage();
> +      assertTrue(exceptionMessage.contains("Invalid collection"));
> +      assertTrue(exceptionMessage.contains("invalid$alias@name"));
> +      assertTrue(exceptionMessage.contains("must consist entirely of
> periods, underscores, and alphanumerics"));
> +    }
> +  }
> +
> +  @Test
> +  public void testInvalidShardNameRejectedWhenCreatingShard() {
> +    final CreateShard createShardRequest = new CreateShard();
> +    try {
> +      createShardRequest.setShardName("invalid$shard@name");
> +      fail();
> +    } catch (IllegalArgumentException e) {
> +      final String exceptionMessage = e.getMessage();
> +      assertTrue(exceptionMessage.contains("Invalid shard"));
> +      assertTrue(exceptionMessage.contains("invalid$shard@name"));
> +      assertTrue(exceptionMessage.contains("must consist entirely of
> periods, underscores, and alphanumerics"));
> +    }
> +  }
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d01230d6/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
> b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
> index 4e46373..8275ff1 100644
> ---
> a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
> +++
> b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
> @@ -16,17 +16,19 @@
>   */
>  package org.apache.solr.client.solrj.request;
>
> +import static org.hamcrest.CoreMatchers.notNullValue;
> +import static org.hamcrest.core.Is.is;
> +
>  import java.io.File;
>  import java.lang.invoke.MethodHandles;
>
> -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
> -import
> com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
>  import org.apache.commons.io.FileUtils;
>  import org.apache.lucene.util.LuceneTestCase;
>  import org.apache.solr.SolrIgnoredThreadsFilter;
>  import org.apache.solr.client.solrj.SolrClient;
>  import
> org.apache.solr.client.solrj.embedded.AbstractEmbeddedSolrServerTestCase;
>  import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
> +import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
>  import org.apache.solr.client.solrj.response.CoreAdminResponse;
>  import org.apache.solr.common.SolrException;
>  import org.apache.solr.common.params.ModifiableSolrParams;
> @@ -41,8 +43,8 @@ import org.junit.rules.TestRule;
>  import org.slf4j.Logger;
>  import org.slf4j.LoggerFactory;
>
> -import static org.hamcrest.CoreMatchers.notNullValue;
> -import static org.hamcrest.core.Is.is;
> +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
> +import
> com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
>
>  @ThreadLeakFilters(defaultFilters = true, filters =
> {SolrIgnoredThreadsFilter.class})
>  public class TestCoreAdmin extends AbstractEmbeddedSolrServerTestCase {
> @@ -158,6 +160,34 @@ public class TestCoreAdmin extends
> AbstractEmbeddedSolrServerTestCase {
>      assertTrue(gotExp);
>    }
>
> +  @Test
> +  public void testInvalidCoreNamesAreRejectedWhenCreatingCore() {
> +    final Create createRequest = new Create();
> +
> +    try {
> +      createRequest.setCoreName("invalid$core@name");
> +      fail();
> +    } catch (IllegalArgumentException e) {
> +      final String exceptionMessage = e.getMessage();
> +      assertTrue(exceptionMessage.contains("Invalid core"));
> +      assertTrue(exceptionMessage.contains("invalid$core@name"));
> +      assertTrue(exceptionMessage.contains("must consist entirely of
> periods, underscores and alphanumerics"));
> +    }
> +  }
> +
> +  @Test
> +  public void testInvalidCoreNamesAreRejectedWhenRenamingExistingCore()
> throws Exception {
> +    try {
> +      CoreAdminRequest.renameCore("validExistingCoreName",
> "invalid$core@name", null);
> +      fail();
> +    } catch (IllegalArgumentException e) {
> +      final String exceptionMessage = e.getMessage();
> +      assertTrue(exceptionMessage.contains("Invalid core"));
> +      assertTrue(exceptionMessage.contains("invalid$core@name"));
> +      assertTrue(exceptionMessage.contains("must consist entirely of
> periods, underscores and alphanumerics"));
> +    }
> +  }
> +
>    @BeforeClass
>    public static void before() {
>      // wtf?
>
>