You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by th...@apache.org on 2014/12/24 17:01:40 UTC

svn commit: r1647815 - in /lucene/dev/branches/branch_5x: ./ solr/ solr/CHANGES.txt solr/bin/ solr/bin/solr solr/core/ solr/core/src/java/org/apache/solr/util/SolrCLI.java

Author: thelabdude
Date: Wed Dec 24 16:01:39 2014
New Revision: 1647815

URL: http://svn.apache.org/r1647815
Log:
SOLR-6867: SolrCLI should check for existence before creating a new core/collection, more user-friendly error reporting (no stack trace), and the ability to pass a directory when using bin/solr to create a core or collection

Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/solr/bin/   (props changed)
    lucene/dev/branches/branch_5x/solr/bin/solr
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/util/SolrCLI.java

Modified: lucene/dev/branches/branch_5x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/CHANGES.txt?rev=1647815&r1=1647814&r2=1647815&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/solr/CHANGES.txt Wed Dec 24 16:01:39 2014
@@ -506,7 +506,9 @@ Other Changes
 * SOLR-6882: Misspelled collection API actions in ReplicaMutator exception messages.
   (Steve Rowe via shalin)
 
-* SOLR-6885: Add core name to RecoveryThread name. (Christine Poerschke via shalin)
+* SOLR-6867: SolrCLI should check for existence before creating a new core/collection, 
+  more user-friendly error reporting (no stack trace), and the ability to pass a 
+  directory when using bin/solr to create a core or collection (Timothy Potter)
 
 ==================  4.10.3 ==================
 

Modified: lucene/dev/branches/branch_5x/solr/bin/solr
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/bin/solr?rev=1647815&r1=1647814&r2=1647815&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/bin/solr (original)
+++ lucene/dev/branches/branch_5x/solr/bin/solr Wed Dec 24 16:01:39 2014
@@ -215,29 +215,35 @@ function print_usage() {
     echo ""
     echo "  -n <name>       Name of core to create"
     echo ""
-    echo "  -c <configset>  Name of configuration directory to use, valid options are:"
+    echo "  -c <configset>  Name of configuration directory to use, built-in options are:"
     echo "      basic_configs: Minimal Solr configuration"
     echo "      data_driven_schema_configs: Managed schema with field-guessing support enabled"
     echo "      sample_techproducts_configs: Example configuration with many optional features enabled to"
     echo "         demonstrate the full power of Solr"
     echo "      If not specified, default is: data_driven_schema_configs"
     echo ""
+    echo "      Alternatively, you can pass the path to your own configuration directory instead of using"
+    echo "      one of the built-in configurations, such as: bin/solr create_core -n mycore -c /tmp/myconfig"
+    echo ""
   elif [ "$CMD" == "create_collection" ]; then
     echo ""
     echo "Usage: solr create_collection [-n name] [-c configset] [-shards #] [-replicationFactor #]"
     echo ""
     echo "  -n <name>               Name of collection to create"
     echo ""
-    echo "  -c <configset>          Name of configuration directory to use, valid options are:"
+    echo "  -c <configset>          Name of configuration directory to use, built-in options are:"
     echo "      basic_configs: Minimal Solr configuration"
     echo "      data_driven_schema_configs: Managed schema with field-guessing support enabled"
     echo "      sample_techproducts_configs: Example configuration with many optional features enabled to"
     echo "         demonstrate the full power of Solr"
     echo "      If not specified, default is: data_driven_schema_configs"
     echo ""
-    echo "  -shards <#>             Number of shards to split the collection into"
+    echo "      Alternatively, you can pass the path to your own configuration directory instead of using"
+    echo "      one of the built-in configurations, such as: bin/solr create_collection -n mycoll -c /tmp/myconfig"
+    echo ""
+    echo "  -shards <#>             Number of shards to split the collection into; default is 1"
     echo ""
-    echo "  -replicationFactor <#>  Number of copies of each document in the collection"
+    echo "  -replicationFactor <#>  Number of copies of each document in the collection, default is 1 (no replication)"
     echo ""
     echo ""
   fi
@@ -300,6 +306,7 @@ function run_tool() {
     -classpath "$DEFAULT_SERVER_DIR/solr-webapp/webapp/WEB-INF/lib/*:$DEFAULT_SERVER_DIR/lib/ext/*" \
     org.apache.solr.util.SolrCLI $*
 
+  return $?
 } # end run_tool function
 
 # get information about any Solr nodes running on this host
@@ -548,6 +555,12 @@ if [[ "$SCRIPT_CMD" == "create_core" ||
     CREATE_CONFIGSET=data_driven_schema_configs
   fi
 
+  # validate the configset arg
+  if [[ ! -d "$SOLR_TIP/server/solr/configsets/$CREATE_CONFIGSET" && ! -d "$CREATE_CONFIGSET" ]]; then
+    echo -e "\nSpecified configset $CREATE_CONFIGSET not found!\n"
+    exit 1
+  fi
+
   if [ -z "$CREATE_NAME" ]; then
     echo "$CREATE_TYPE name is required!"
     print_usage "$SCRIPT_CMD"
@@ -573,12 +586,12 @@ if [[ "$SCRIPT_CMD" == "create_core" ||
   if [ "$SCRIPT_CMD" == "create_collection" ]; then
     run_tool create_collection -name $CREATE_NAME -shards $CREATE_NUM_SHARDS -replicationFactor $CREATE_REPFACT \
       -config $CREATE_CONFIGSET -configsetsDir $SOLR_TIP/server/solr/configsets -solrUrl http://localhost:$CREATE_PORT/solr
+    exit $?
   else
     run_tool create_core -name $CREATE_NAME -solrUrl http://localhost:$CREATE_PORT/solr \
       -config $CREATE_CONFIGSET -configsetsDir $SOLR_TIP/server/solr/configsets
+    exit $?
   fi
-
-  exit $?
 fi
 
 
@@ -1184,9 +1197,15 @@ if [ "$EXAMPLE" != "cloud" ]; then
     if [ "$SOLR_MODE" == "solrcloud" ]; then
       run_tool create_collection -name $EXAMPLE -shards 1 -replicationFactor 1 \
         -config $EXAMPLE_CONFIGSET -configsetsDir $SOLR_TIP/server/solr/configsets -solrUrl http://localhost:$SOLR_PORT/solr
+      if [ $? -ne 0 ]; then
+        exit 1
+      fi
     else
       run_tool create_core -name $EXAMPLE -solrUrl http://localhost:$SOLR_PORT/solr \
         -config $EXAMPLE_CONFIGSET -configsetsDir $SOLR_TIP/server/solr/configsets
+      if [ $? -ne 0 ]; then
+        exit 1
+      fi
     fi
 
     if [ "$EXAMPLE" == "techproducts" ]; then
@@ -1283,13 +1302,26 @@ else
     USER_INPUT=
     echo "Please choose a configuration for the $CLOUD_COLLECTION collection, available options are:"
     read -e -p "basic_configs, data_driven_schema_configs, or sample_techproducts_configs [data_driven_schema_configs] " USER_INPUT
-    # trim whitespace out of the user input
-    CLOUD_CONFIG=`echo $USER_INPUT | tr -d ' '`
+    while true
+    do
+      # trim whitespace out of the user input
+      CLOUD_CONFIG=`echo $USER_INPUT | tr -d ' '`
+
+      # handle the default selection or empty input
+      if [ -z "$CLOUD_CONFIG" ]; then
+        CLOUD_CONFIG=data_driven_schema_configs
+      fi
+
+      # validate the configset arg
+      if [[ ! -d "$SOLR_TIP/server/solr/configsets/$CLOUD_CONFIG" && ! -d "$CLOUD_CONFIG" ]]; then
+        echo -e "\nOops! Specified configuration $CLOUD_CONFIG not found!"
+        read -e -p "Choose one of: basic_configs, data_driven_schema_configs, or sample_techproducts_configs [data_driven_schema_configs] " USER_INPUT
+        CLOUD_CONFIG=
+      else
+        break;
+      fi
+    done
 
-    # handle the default selection or empty input
-    if [ -z "$CLOUD_CONFIG" ]; then
-      CLOUD_CONFIG=data_driven_schema_configs
-    fi                    
   fi
    
   run_tool create_collection -name $CLOUD_COLLECTION -shards $CLOUD_NUM_SHARDS -replicationFactor $CLOUD_REPFACT \

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/util/SolrCLI.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/util/SolrCLI.java?rev=1647815&r1=1647814&r2=1647815&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/util/SolrCLI.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/util/SolrCLI.java Wed Dec 24 16:01:39 2014
@@ -111,6 +111,7 @@ public class SolrCLI {
       String zkHost = cli.getOptionValue("zkHost", ZK_HOST);
       
       log.debug("Connecting to Solr cluster: " + zkHost);
+      int exitStatus = 0;
       CloudSolrServer cloudSolrServer = null;
       try {
         cloudSolrServer = new CloudSolrServer(zkHost);
@@ -119,8 +120,17 @@ public class SolrCLI {
         if (collection != null)
           cloudSolrServer.setDefaultCollection(collection);
         
-        cloudSolrServer.connect();        
-        runCloudTool(cloudSolrServer, cli);
+        cloudSolrServer.connect();
+        exitStatus = runCloudTool(cloudSolrServer, cli);
+      } catch (Exception exc) {
+        // since this is a CLI, spare the user the stacktrace
+        String excMsg = exc.getMessage();
+        if (excMsg != null) {
+          System.err.println("\nERROR:"+excMsg+"\n");
+          exitStatus = 1;
+        } else {
+          throw exc;
+        }
       } finally {
         if (cloudSolrServer != null) {
           try {
@@ -129,13 +139,13 @@ public class SolrCLI {
         }
       }
       
-      return 0;
+      return exitStatus;
     }
     
     /**
      * Runs a SolrCloud tool with CloudSolrServer initialized
      */
-    protected abstract void runCloudTool(CloudSolrServer cloudSolrServer, CommandLine cli) 
+    protected abstract int runCloudTool(CloudSolrServer cloudSolrServer, CommandLine cli)
         throws Exception;
   }
   
@@ -182,9 +192,7 @@ public class SolrCLI {
         processCommandLineArgs(joinCommonAndToolOptions(tool.getOptions()), toolArgs);
 
     // run the tool
-    int exitCode = tool.runTool(cli);
-    
-    System.exit(exitCode);    
+    System.exit(tool.runTool(cli));
   }
   
   /**
@@ -327,9 +335,8 @@ public class SolrCLI {
       
       for (String classInPackage : classes) {
         Class<?> theClass = Class.forName(classInPackage);
-        if (Tool.class.isAssignableFrom(theClass)) {
+        if (Tool.class.isAssignableFrom(theClass))
           toolClasses.add((Class<Tool>) theClass);
-        }
       }
     } catch (Exception e) {
       // safe to squelch this as it's just looking for tools to run
@@ -350,9 +357,8 @@ public class SolrCLI {
         if (entry.getName().endsWith(".class")) {
           String className = entry.getName().replaceAll("[$].*", "")
               .replaceAll("[.]class", "").replace('/', '.');
-          if (className.startsWith(packageName)) {
+          if (className.startsWith(packageName))
             classes.add(className);
-          }
         }
       }
     }
@@ -367,9 +373,9 @@ public class SolrCLI {
     Throwable rootCause = SolrException.getRootCause(exc);
     boolean wasCommError =
         (rootCause instanceof ConnectException ||
-            rootCause instanceof ConnectTimeoutException ||
-            rootCause instanceof NoHttpResponseException ||
-            rootCause instanceof SocketException);
+         rootCause instanceof ConnectTimeoutException ||
+         rootCause instanceof NoHttpResponseException ||
+         rootCause instanceof SocketException);
     return wasCommError;
   }
   
@@ -470,7 +476,7 @@ public class SolrCLI {
       String errMsg = asString("/error/msg", json);
       if (errMsg == null)
         errMsg = String.valueOf(json);
-      throw new SolrServerException("Request to "+getUrl+" failed due to: "+errMsg);
+      throw new SolrServerException(errMsg);
     }
     return json;
   }  
@@ -623,7 +629,9 @@ public class SolrCLI {
         throws Exception
     {
       Map<String,Object> status = new LinkedHashMap<String,Object>();
-      
+
+      String solrHome = (String)info.get("solr_home");
+      status.put("solr_home", solrHome != null ? solrHome : "?");
       status.put("version", asString("/lucene/solr-impl-version", info));      
       status.put("startTime", asString("/jvm/jmx/startTime", info));
       status.put("uptime", uptime(asLong("/jvm/jmx/upTimeMS", info)));
@@ -706,198 +714,6 @@ public class SolrCLI {
 
   private static final String DEFAULT_CONFIG_SET = "data_driven_schema_configs";
 
-  /**
-   * Supports create_collection command in the bin/solr script.
-   */
-  public static class CreateCollectionTool implements Tool {
-
-    @Override
-    public String getName() {
-      return "create_collection";
-    }
-
-    @SuppressWarnings("static-access")
-    @Override
-    public Option[] getOptions() {
-      return new Option[] {
-          OptionBuilder
-              .withArgName("HOST")
-              .hasArg()
-              .isRequired(false)
-              .withDescription("Address of the Zookeeper ensemble; defaults to: "+ZK_HOST)
-              .create("zkHost"),
-          OptionBuilder
-              .withArgName("HOST")
-              .hasArg()
-              .isRequired(false)
-              .withDescription("Base Solr URL, which can be used to determine the zkHost if that's not known")
-              .create("solrUrl"),
-          OptionBuilder
-              .withArgName("NAME")
-              .hasArg()
-              .isRequired(true)
-              .withDescription("Name of collection to create.")
-              .create("name"),
-          OptionBuilder
-              .withArgName("#")
-              .hasArg()
-              .isRequired(false)
-              .withDescription("Number of shards; default is 1")
-              .create("shards"),
-          OptionBuilder
-              .withArgName("#")
-              .hasArg()
-              .isRequired(false)
-              .withDescription("Number of copies of each document across the collection (replicas per shard); default is 1")
-              .create("replicationFactor"),
-          OptionBuilder
-              .withArgName("#")
-              .hasArg()
-              .isRequired(false)
-              .withDescription("Maximum number of shards per Solr node; default is determined based on the number of shards, replication factor, and live nodes.")
-              .create("maxShardsPerNode"),
-          OptionBuilder
-              .withArgName("NAME")
-              .hasArg()
-              .isRequired(false)
-              .withDescription("Name of the configuration for this collection; default is "+DEFAULT_CONFIG_SET)
-              .create("config"),
-          OptionBuilder
-              .withArgName("DIR")
-              .hasArg()
-              .isRequired(true)
-              .withDescription("Path to configsets directory on the local system.")
-              .create("configsetsDir")
-      };
-    }
-
-    public int runTool(CommandLine cli) throws Exception {
-
-      // quiet down the ZK logging for cli tools
-      LogManager.getLogger("org.apache.zookeeper").setLevel(Level.ERROR);
-      LogManager.getLogger("org.apache.solr.common.cloud").setLevel(Level.WARN);
-
-      String zkHost = cli.getOptionValue("zkHost");
-      if (zkHost == null) {
-        // find it using the localPort
-        String solrUrl = cli.getOptionValue("solrUrl");
-        if (solrUrl == null)
-          throw new IllegalStateException(
-              "Must provide either the -zkHost or -solrUrl parameters to use the create_collection command!");
-
-        if (!solrUrl.endsWith("/"))
-          solrUrl += "/";
-
-        String systemInfoUrl = solrUrl+"admin/info/system";
-        HttpClient httpClient = getHttpClient();
-        try {
-          // hit Solr to get system info
-          Map<String,Object> systemInfo = getJson(httpClient, systemInfoUrl, 2);
-
-          // convert raw JSON into user-friendly output
-          StatusTool statusTool = new StatusTool();
-          Map<String,Object> status = statusTool.reportStatus(solrUrl, systemInfo, httpClient);
-
-          Map<String,Object> cloud = (Map<String, Object>)status.get("cloud");
-          if (cloud == null) {
-            System.err.println("\nERROR: Solr at "+solrUrl+
-                " is running in standalone server mode, please use the create_core command instead;\n" +
-                "create_collection can only be used when running in SolrCloud mode.\n");
-            return 1;
-          }
-
-          String zookeeper = (String) cloud.get("ZooKeeper");
-          if (zookeeper.endsWith("(embedded)")) {
-            zookeeper = zookeeper.substring(0,zookeeper.length()-"(embedded)".length());
-          }
-          zkHost = zookeeper;
-        } finally {
-          closeHttpClient(httpClient);
-        }
-      }
-
-      CloudSolrServer cloudSolrServer = null;
-      try {
-        cloudSolrServer = new CloudSolrServer(zkHost);
-        System.out.println("Connecting to ZooKeeper at "+zkHost);
-        cloudSolrServer.connect();
-        runCloudTool(cloudSolrServer, cli);
-      } finally {
-        if (cloudSolrServer != null) {
-          try {
-            cloudSolrServer.shutdown();
-          } catch (Exception ignore) {}
-        }
-      }
-
-      return 0;
-    }
-
-    protected void runCloudTool(CloudSolrServer cloudSolrServer, CommandLine cli) throws Exception {
-      Set<String> liveNodes = cloudSolrServer.getZkStateReader().getClusterState().getLiveNodes();
-      if (liveNodes.isEmpty())
-        throw new IllegalStateException("No live nodes found! Cannot create a collection until " +
-            "there is at least 1 live node in the cluster.");
-      String firstLiveNode = liveNodes.iterator().next();
-
-      // build a URL to create the collection
-      int numShards = optionAsInt(cli, "shards", 1);
-      int replicationFactor = optionAsInt(cli, "replicationFactor", 1);
-      int maxShardsPerNode = -1;
-
-      if (cli.hasOption("maxShardsPerNode")) {
-        maxShardsPerNode = Integer.parseInt(cli.getOptionValue("maxShardsPerNode"));
-      } else {
-        // need number of live nodes to determine maxShardsPerNode if it is not set
-        int numNodes = liveNodes.size();
-        maxShardsPerNode = ((numShards*replicationFactor)+numNodes-1)/numNodes;
-      }
-
-      String configSet = cli.getOptionValue("config", DEFAULT_CONFIG_SET);
-      // first, test to see if that config exists in ZK
-      if (!cloudSolrServer.getZkStateReader().getZkClient().exists("/configs/"+configSet, true)) {
-        File configsetsDir = new File(cli.getOptionValue("configsetsDir"));
-        if (!configsetsDir.isDirectory())
-          throw new FileNotFoundException(configsetsDir.getAbsolutePath()+" not found!");
-
-        // upload the configset if it exists
-        File configSetDir = new File(configsetsDir, configSet);
-        if (!configSetDir.isDirectory())
-          throw new FileNotFoundException("Specified config "+configSet+
-              " not found in "+configsetsDir.getAbsolutePath());
-
-        File confDir = new File(configSetDir,"conf");
-        System.out.println("Uploading "+confDir.getAbsolutePath()+
-            " for config "+configSet+" to ZooKeeper at "+cloudSolrServer.getZkHost());
-        ZkController.uploadConfigDir(cloudSolrServer.getZkStateReader().getZkClient(), confDir, configSet);
-      }
-
-      String baseUrl = cloudSolrServer.getZkStateReader().getBaseUrlForNodeName(firstLiveNode);
-      String collectionName = cli.getOptionValue("name");
-      String createCollectionUrl =
-          String.format(Locale.ROOT,
-              "%s/admin/collections?action=CREATE&name=%s&numShards=%d&replicationFactor=%d&maxShardsPerNode=%d&collection.configName=%s",
-              baseUrl,
-              collectionName,
-              numShards,
-              replicationFactor,
-              maxShardsPerNode,
-              configSet);
-
-      System.out.println("Creating new collection '"+collectionName+"' using command:\n\n"+createCollectionUrl+"\n");
-
-      Map<String,Object> json = getJson(createCollectionUrl);
-      CharArr arr = new CharArr();
-      new JSONWriter(arr, 2).write(json);
-      System.out.println(arr.toString());
-      System.out.println();
-    }
-
-    protected int optionAsInt(CommandLine cli, String option, int defaultVal) {
-      return Integer.parseInt(cli.getOptionValue(option, String.valueOf(defaultVal)));
-    }
-  } // end CreateCollectionTool class
-
   private static final long MS_IN_MIN = 60 * 1000L;
   private static final long MS_IN_HOUR = MS_IN_MIN * 60L;
   private static final long MS_IN_DAY = MS_IN_HOUR * 24L;
@@ -1056,7 +872,7 @@ public class SolrCLI {
     }
         
     @Override
-    protected void runCloudTool(CloudSolrServer cloudSolrServer, CommandLine cli) throws Exception {
+    protected int runCloudTool(CloudSolrServer cloudSolrServer, CommandLine cli) throws Exception {
       
       String collection = cli.getOptionValue("collection");
       if (collection == null)
@@ -1171,9 +987,273 @@ public class SolrCLI {
       CharArr arr = new CharArr();
       new JSONWriter(arr, 2).write(report);
       System.out.println(arr.toString());
+
+      return 0;
     }
   } // end HealthcheckTool
 
+  /**
+   * Supports create_collection command in the bin/solr script.
+   */
+  public static class CreateCollectionTool implements Tool {
+
+    @Override
+    public String getName() {
+      return "create_collection";
+    }
+
+    @SuppressWarnings("static-access")
+    @Override
+    public Option[] getOptions() {
+      return new Option[] {
+          OptionBuilder
+              .withArgName("HOST")
+              .hasArg()
+              .isRequired(false)
+              .withDescription("Address of the Zookeeper ensemble; defaults to: "+ZK_HOST)
+              .create("zkHost"),
+          OptionBuilder
+              .withArgName("HOST")
+              .hasArg()
+              .isRequired(false)
+              .withDescription("Base Solr URL, which can be used to determine the zkHost if that's not known")
+              .create("solrUrl"),
+          OptionBuilder
+              .withArgName("NAME")
+              .hasArg()
+              .isRequired(true)
+              .withDescription("Name of collection to create.")
+              .create("name"),
+          OptionBuilder
+              .withArgName("#")
+              .hasArg()
+              .isRequired(false)
+              .withDescription("Number of shards; default is 1")
+              .create("shards"),
+          OptionBuilder
+              .withArgName("#")
+              .hasArg()
+              .isRequired(false)
+              .withDescription("Number of copies of each document across the collection (replicas per shard); default is 1")
+              .create("replicationFactor"),
+          OptionBuilder
+              .withArgName("#")
+              .hasArg()
+              .isRequired(false)
+              .withDescription("Maximum number of shards per Solr node; default is determined based on the number of shards, replication factor, and live nodes.")
+              .create("maxShardsPerNode"),
+          OptionBuilder
+              .withArgName("NAME")
+              .hasArg()
+              .isRequired(false)
+              .withDescription("Name of the configuration for this collection; default is "+DEFAULT_CONFIG_SET)
+              .create("config"),
+          OptionBuilder
+              .withArgName("DIR")
+              .hasArg()
+              .isRequired(true)
+              .withDescription("Path to configsets directory on the local system.")
+              .create("configsetsDir")
+      };
+    }
+
+    public int runTool(CommandLine cli) throws Exception {
+
+      // quiet down the ZK logging for cli tools
+      LogManager.getLogger("org.apache.zookeeper").setLevel(Level.ERROR);
+      LogManager.getLogger("org.apache.solr.common.cloud").setLevel(Level.WARN);
+
+      String zkHost = cli.getOptionValue("zkHost");
+      if (zkHost == null) {
+        // find it using the localPort
+        String solrUrl = cli.getOptionValue("solrUrl");
+        if (solrUrl == null)
+          throw new IllegalStateException(
+              "Must provide either the -zkHost or -solrUrl parameters to use the create_collection command!");
+
+        if (!solrUrl.endsWith("/"))
+          solrUrl += "/";
+
+        String systemInfoUrl = solrUrl+"admin/info/system";
+        HttpClient httpClient = getHttpClient();
+        try {
+          // hit Solr to get system info
+          Map<String,Object> systemInfo = getJson(httpClient, systemInfoUrl, 2);
+
+          // convert raw JSON into user-friendly output
+          StatusTool statusTool = new StatusTool();
+          Map<String,Object> status = statusTool.reportStatus(solrUrl, systemInfo, httpClient);
+
+          Map<String,Object> cloud = (Map<String, Object>)status.get("cloud");
+          if (cloud == null) {
+            System.err.println("\nERROR: Solr at "+solrUrl+
+                " is running in standalone server mode, please use the create_core command instead;\n" +
+                "create_collection can only be used when running in SolrCloud mode.\n");
+            return 1;
+          }
+
+          String zookeeper = (String) cloud.get("ZooKeeper");
+          if (zookeeper.endsWith("(embedded)")) {
+            zookeeper = zookeeper.substring(0,zookeeper.length()-"(embedded)".length());
+          }
+          zkHost = zookeeper;
+        } finally {
+          closeHttpClient(httpClient);
+        }
+      }
+
+      int toolExitStatus = 0;
+      CloudSolrServer cloudSolrServer = null;
+      try {
+        cloudSolrServer = new CloudSolrServer(zkHost);
+        System.out.println("Connecting to ZooKeeper at " + zkHost);
+        cloudSolrServer.connect();
+        toolExitStatus = runCloudTool(cloudSolrServer, cli);
+      } catch (Exception exc) {
+        // since this is a CLI, spare the user the stacktrace
+        String excMsg = exc.getMessage();
+        if (excMsg != null) {
+          System.err.println("\nERROR: "+excMsg+"\n");
+          toolExitStatus = 1;
+        } else {
+          throw exc;
+        }
+      } finally {
+        if (cloudSolrServer != null) {
+          try {
+            cloudSolrServer.shutdown();
+          } catch (Exception ignore) {}
+        }
+      }
+
+      return toolExitStatus;
+    }
+
+    protected int runCloudTool(CloudSolrServer cloudSolrServer, CommandLine cli) throws Exception {
+      Set<String> liveNodes = cloudSolrServer.getZkStateReader().getClusterState().getLiveNodes();
+      if (liveNodes.isEmpty())
+        throw new IllegalStateException("No live nodes found! Cannot create a collection until " +
+            "there is at least 1 live node in the cluster.");
+      String firstLiveNode = liveNodes.iterator().next();
+
+      // build a URL to create the collection
+      int numShards = optionAsInt(cli, "shards", 1);
+      int replicationFactor = optionAsInt(cli, "replicationFactor", 1);
+      int maxShardsPerNode = -1;
+
+      if (cli.hasOption("maxShardsPerNode")) {
+        maxShardsPerNode = Integer.parseInt(cli.getOptionValue("maxShardsPerNode"));
+      } else {
+        // need number of live nodes to determine maxShardsPerNode if it is not set
+        int numNodes = liveNodes.size();
+        maxShardsPerNode = ((numShards*replicationFactor)+numNodes-1)/numNodes;
+      }
+
+      String configSet = cli.getOptionValue("config", DEFAULT_CONFIG_SET);
+      String configSetNameInZk = configSet;
+      File configSetDir = null;
+      // we try to be flexible and allow the user to specify a configuration directory instead of a configset name
+      File possibleConfigDir = new File(configSet);
+      if (possibleConfigDir.isDirectory()) {
+        configSetDir = possibleConfigDir;
+        configSetNameInZk = possibleConfigDir.getName();
+      } else {
+        File configsetsDir = new File(cli.getOptionValue("configsetsDir"));
+        if (!configsetsDir.isDirectory())
+          throw new FileNotFoundException(configsetsDir.getAbsolutePath()+" not found!");
+
+        // upload the configset if it exists
+        configSetDir = new File(configsetsDir, configSet);
+        if (!configSetDir.isDirectory()) {
+          throw new FileNotFoundException("Specified config " + configSet +
+              " not found in " + configsetsDir.getAbsolutePath());
+        }
+      }
+
+      File confDir = new File(configSetDir, "conf");
+      if (!confDir.isDirectory()) {
+        // config dir should contain a conf sub-directory but if not and there's a solrconfig.xml, then use it
+        if ((new File(configSetDir, "solrconfig.xml")).isFile()) {
+          confDir = configSetDir;
+        } else {
+          System.err.println("Specified configuration directory "+configSetDir.getAbsolutePath()+
+              " is invalid;\nit should contain either conf sub-directory or solrconfig.xml");
+          return 1;
+        }
+      }
+
+      // test to see if that config exists in ZK
+      if (!cloudSolrServer.getZkStateReader().getZkClient().exists("/configs/"+configSetNameInZk, true)) {
+        System.out.println("Uploading "+confDir.getAbsolutePath()+
+            " for config "+configSetNameInZk+" to ZooKeeper at "+cloudSolrServer.getZkHost());
+        ZkController.uploadConfigDir(cloudSolrServer.getZkStateReader().getZkClient(), confDir, configSetNameInZk);
+      }
+
+      String baseUrl = cloudSolrServer.getZkStateReader().getBaseUrlForNodeName(firstLiveNode);
+      String collectionName = cli.getOptionValue("name");
+
+      // since creating a collection is a heavy-weight operation, check for existence first
+      String collectionListUrl = baseUrl+"/admin/collections?action=list";
+      if (safeCheckCollectionExists(collectionListUrl, collectionName)) {
+        System.err.println("\nCollection '"+collectionName+"' already exists!");
+        System.err.println("\nChecked collection existence using Collections API command:\n"+collectionListUrl);
+        System.err.println();
+        return 1;
+      }
+
+      // doesn't seem to exist ... try to create
+      String createCollectionUrl =
+          String.format(Locale.ROOT,
+              "%s/admin/collections?action=CREATE&name=%s&numShards=%d&replicationFactor=%d&maxShardsPerNode=%d&collection.configName=%s",
+              baseUrl,
+              collectionName,
+              numShards,
+              replicationFactor,
+              maxShardsPerNode,
+              configSetNameInZk);
+
+      System.out.println("\nCreating new collection '"+collectionName+"' using command:\n"+createCollectionUrl+"\n");
+
+      Map<String,Object> json = null;
+      try {
+        json = getJson(createCollectionUrl);
+      } catch (SolrServerException sse) {
+        // check if already exists
+        if (safeCheckCollectionExists(collectionListUrl, collectionName)) {
+          System.err.println("Collection '"+collectionName+"' already exists!");
+          System.err.println("\nChecked collection existence using Collections API command:\n"+collectionListUrl);
+        } else {
+          System.err.println("Failed to create collection '"+collectionName+"' due to: "+sse.getMessage());
+        }
+        System.err.println();
+        return 1;
+      }
+
+      CharArr arr = new CharArr();
+      new JSONWriter(arr, 2).write(json);
+      System.out.println(arr.toString());
+      System.out.println();
+
+      return 0;
+    }
+
+    protected boolean safeCheckCollectionExists(String url, String collection) {
+      boolean exists = false;
+      try {
+        Map<String,Object> existsCheckResult = getJson(url);
+        List<String> collections = (List<String>) existsCheckResult.get("collections");
+        exists = collections != null && collections.contains(collection);
+      } catch (Exception exc) {
+        // just ignore it since we're only interested in a positive result here
+      }
+      return exists;
+    }
+
+    protected int optionAsInt(CommandLine cli, String option, int defaultVal) {
+      return Integer.parseInt(cli.getOptionValue(option, String.valueOf(defaultVal)));
+    }
+  } // end CreateCollectionTool class
+
   public static class CreateCoreTool implements Tool {
 
     @Override
@@ -1221,15 +1301,20 @@ public class SolrCLI {
 
       File configsetsDir = new File(cli.getOptionValue("configsetsDir"));
       if (!configsetsDir.isDirectory())
-        throw new FileNotFoundException(configsetsDir.getAbsolutePath()+" not found!");
+        throw new FileNotFoundException(configsetsDir.getAbsolutePath() + " not found!");
 
       String configSet = cli.getOptionValue("config", DEFAULT_CONFIG_SET);
       File configSetDir = new File(configsetsDir, configSet);
-      if (!configSetDir.isDirectory())
-        throw new FileNotFoundException("Specified config "+configSet+
-            " not found in "+configsetsDir.getAbsolutePath());
-
-      File confDir = new File(configSetDir,"conf");
+      if (!configSetDir.isDirectory()) {
+        // we allow them to pass a directory instead of a configset name
+        File possibleConfigDir = new File(configSet);
+        if (possibleConfigDir.isDirectory()) {
+          configSetDir = possibleConfigDir;
+        } else {
+          throw new FileNotFoundException("Specified config " + configSet +
+              " not found in " + configsetsDir.getAbsolutePath());
+        }
+      }
 
       String coreName = cli.getOptionValue("name");
 
@@ -1246,21 +1331,42 @@ public class SolrCLI {
 
         // convert raw JSON into user-friendly output
         solrHome = (String)systemInfo.get("solr_home");
-        if (solrHome == null) {
+        if (solrHome == null)
           solrHome = configsetsDir.getParentFile().getAbsolutePath();
-        }
+
       } finally {
         closeHttpClient(httpClient);
       }
 
+      String coreStatusUrl = solrUrl+"admin/cores?action=STATUS&core="+coreName;
+      if (safeCheckCoreExists(coreStatusUrl, coreName)) {
+        System.err.println("\nCore '"+coreName+"' already exists!");
+        System.err.println("\nChecked core existence using Core API command:\n"+coreStatusUrl);
+        System.err.println();
+        return 1;
+      }
+
       File coreInstanceDir = new File(solrHome, coreName);
+      File confDir = new File(configSetDir,"conf");
       if (!coreInstanceDir.isDirectory()) {
         coreInstanceDir.mkdirs();
         if (!coreInstanceDir.isDirectory())
           throw new IOException("Failed to create new core instance directory: "+coreInstanceDir.getAbsolutePath());
-      }
 
-      FileUtils.copyDirectoryToDirectory(confDir, coreInstanceDir);
+        if (confDir.isDirectory()) {
+          FileUtils.copyDirectoryToDirectory(confDir, coreInstanceDir);
+        } else {
+          // hmmm ... the configset we're cloning doesn't have a conf sub-directory,
+          // we'll just assume it is OK if it has solrconfig.xml
+          if ((new File(configSetDir, "solrconfig.xml")).isFile()) {
+            FileUtils.copyDirectory(configSetDir, new File(coreInstanceDir, "conf"));
+          } else {
+            System.err.println("\n"+configSetDir.getAbsolutePath()+" doesn't contain a conf subdirectory or solrconfig.xml\n");
+            return 1;
+          }
+        }
+        System.out.println("\nSetup new core instance directory:\n"+coreInstanceDir.getAbsolutePath());
+      }
 
       String createCoreUrl =
           String.format(Locale.ROOT,
@@ -1269,9 +1375,24 @@ public class SolrCLI {
               coreName,
               coreName);
 
-      System.out.println("Creating new core '"+coreName+"' using command:\n\n"+createCoreUrl+"\n");
+      System.out.println("\nCreating new core '"+coreName+"' using command:\n"+createCoreUrl+"\n");
+
+      Map<String,Object> json = null;
+      try {
+        json = getJson(createCoreUrl);
+      } catch (SolrServerException sse) {
+        // mostly likely the core already exists ...
+        if (safeCheckCoreExists(coreStatusUrl, coreName)) {
+          // core already exists
+          System.err.println("Core '"+coreName+"' already exists!");
+          System.err.println("\nChecked core existence using Core API command:\n"+coreStatusUrl);
+        } else {
+          System.err.println("Failed to create core '"+coreName+"' due to: "+sse.getMessage());
+        }
+        System.err.println();
+        return 1;
+      }
 
-      Map<String,Object> json = getJson(createCoreUrl);
       CharArr arr = new CharArr();
       new JSONWriter(arr, 2).write(json);
       System.out.println(arr.toString());
@@ -1279,5 +1400,18 @@ public class SolrCLI {
 
       return 0;
     }
+
+    protected boolean safeCheckCoreExists(String coreStatusUrl, String coreName) {
+      boolean exists = false;
+      try {
+        Map<String,Object> existsCheckResult = getJson(coreStatusUrl);
+        Map<String,Object> status = (Map<String, Object>)existsCheckResult.get("status");
+        Map<String,Object> coreStatus = (Map<String, Object>)status.get(coreName);
+        exists = coreStatus != null && coreStatus.containsKey("name");
+      } catch (Exception exc) {
+        // just ignore it since we're only interested in a positive result here
+      }
+      return exists;
+    }
   } // end CreateCoreTool class
 }