You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2018/01/26 16:37:52 UTC

[1/4] lucene-solr:master: SOLR-11722: Improve the v2/v1 API mapping, including a bug. Wrapped getParameterNamesIterator failed to consider attrToParams. (committing this separately from rest)

Repository: lucene-solr
Updated Branches:
  refs/heads/master 09f903ef8 -> b0d244f65


SOLR-11722: Improve the v2/v1 API mapping, including a bug.
Wrapped getParameterNamesIterator failed to consider attrToParams.
(committing this separately from rest)


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

Branch: refs/heads/master
Commit: d8e9ab8785e638ecf07eed43055aa02332eb7862
Parents: 09f903e
Author: David Smiley <ds...@apache.org>
Authored: Thu Jan 25 14:23:44 2018 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Thu Jan 25 14:23:44 2018 -0500

----------------------------------------------------------------------
 .../src/java/org/apache/solr/api/ApiBag.java    |   7 +-
 .../handler/admin/BaseHandlerApiSupport.java    |  11 +-
 .../solrj/request/CollectionApiMapping.java     | 120 +++++++++++--------
 .../solr/common/util/JsonSchemaValidator.java   |  22 +++-
 4 files changed, 102 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8e9ab87/solr/core/src/java/org/apache/solr/api/ApiBag.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/ApiBag.java b/solr/core/src/java/org/apache/solr/api/ApiBag.java
index 2caaeb9..0c0d54b 100644
--- a/solr/core/src/java/org/apache/solr/api/ApiBag.java
+++ b/solr/core/src/java/org/apache/solr/api/ApiBag.java
@@ -305,7 +305,12 @@ public class ApiBag {
         continue;
       } else {
         List<String> errs = validator.validateJson(cmd.getCommandData());
-        if (errs != null) for (String err : errs) cmd.addError(err);
+        if (errs != null){
+          // otherwise swallowed in solrj tests, and just get "Error in command payload" in test log
+          // which is quite unhelpful.
+          log.error("Command errors for {}:{}", cmd.name, errs );
+          for (String err : errs) cmd.addError(err);
+        }
       }
 
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8e9ab87/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java b/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java
index 087c6f1..90a2dd2 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java
@@ -119,7 +119,7 @@ public abstract class BaseHandlerApiSupport implements ApiSupport {
         } catch (SolrException e) {
           throw e;
         } catch (Exception e) {
-          throw new SolrException(BAD_REQUEST, e);
+          throw new SolrException(BAD_REQUEST, e); //TODO BAD_REQUEST is a wild guess; should we flip the default?  fail here to investigate how this happens in tests
         } finally {
           req.setParams(params);
         }
@@ -129,6 +129,9 @@ public abstract class BaseHandlerApiSupport implements ApiSupport {
 
   }
 
+  /**
+   * Wrapper for SolrParams that wraps V2 params and exposes them as V1 params.
+   */
   private static void wrapParams(final SolrQueryRequest req, final CommandOperation co, final ApiCommand cmd, final boolean useRequestParams) {
     final Map<String, String> pathValues = req.getPathTemplateValues();
     final Map<String, Object> map = co == null || !(co.getCommandData() instanceof Map) ?
@@ -148,7 +151,7 @@ public abstract class BaseHandlerApiSupport implements ApiSupport {
           }
 
           private Object getParams0(String param) {
-            param = cmd.meta().getParamSubstitute(param);
+            param = cmd.meta().getParamSubstitute(param); // v1 -> v2, possibly dotted path
             Object o = param.indexOf('.') > 0 ?
                 Utils.getObjectByPath(map, true, splitSmart(param, '.')) :
                 map.get(param);
@@ -172,11 +175,9 @@ public abstract class BaseHandlerApiSupport implements ApiSupport {
 
           @Override
           public Iterator<String> getParameterNamesIterator() {
-            return cmd.meta().getParamNames(co).iterator();
-
+            return cmd.meta().getParamNamesIterator(co);
           }
 
-
         });
 
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8e9ab87/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
index 701a45b..8f7689b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
@@ -21,8 +21,11 @@ package org.apache.solr.client.solrj.request;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Stream;
 
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.common.params.CollectionParams.CollectionAction;
@@ -77,7 +80,7 @@ public class CollectionApiMapping {
             "createNodeSet.shuffle", "shuffleNodes",
             "createNodeSet", "nodeSet"
         ),
-        Utils.makeMap("properties.", "property.")),
+        Utils.makeMap("property.", "properties.")),
 
     DELETE_COLL(EndPoint.PER_COLLECTION_DELETE,
         DELETE,
@@ -124,7 +127,7 @@ public class CollectionApiMapping {
         CREATESHARD,
         "create",
         Utils.makeMap("createNodeSet", "nodeSet"),
-        Utils.makeMap("coreProperties.", "property.")) {
+        Utils.makeMap("property.", "coreProperties.")) {
       @Override
       public String getParamSubstitute(String param) {
         return super.getParamSubstitute(param);
@@ -137,7 +140,7 @@ public class CollectionApiMapping {
         "split",
         Utils.makeMap(
             "split.key", "splitKey"),
-        Utils.makeMap("coreProperties.", "property.")),
+        Utils.makeMap("property.", "coreProperties.")),
     DELETE_SHARD(PER_COLLECTION_PER_SHARD_DELETE,
         DELETE, DELETESHARD),
 
@@ -146,7 +149,7 @@ public class CollectionApiMapping {
         ADDREPLICA,
         "add-replica",
         null,
-        Utils.makeMap("coreProperties.", "property.")),
+        Utils.makeMap("property.", "coreProperties.")),
 
     DELETE_REPLICA(PER_COLLECTION_PER_SHARD_PER_REPLICA_DELETE,
         DELETE, DELETEREPLICA),
@@ -203,12 +206,14 @@ public class CollectionApiMapping {
     public final String commandName;
     public final EndPoint endPoint;
     public final SolrRequest.METHOD method;
-    //mapping of http param name to json attribute
-    public final Map<String, String> paramstoAttr;
-    //mapping of old prefix to new for instance properties.a=val can be substituted with property:{a:val}
-    public final Map<String, String> prefixSubstitutes;
     public final CollectionAction action;
 
+    //bi-directional mapping of v1 http param name to v2 json attribute
+    public final Map<String, String> paramsToAttrs; // v1 -> v2
+    public final Map<String, String> attrsToParams; // v2 -> v1
+    //mapping of old prefix to new for instance properties.a=val can be substituted with property:{a:val}
+    public final Map<String, String> prefixParamsToAttrs; // v1 -> v2
+
     public SolrRequest.METHOD getMethod() {
       return method;
     }
@@ -219,20 +224,33 @@ public class CollectionApiMapping {
     }
 
     Meta(EndPoint endPoint, SolrRequest.METHOD method, CollectionAction action,
-         String commandName, Map paramstoAttr) {
-      this(endPoint, method, action, commandName, paramstoAttr, Collections.EMPTY_MAP);
-
+         String commandName, Map paramsToAttrs) {
+      this(endPoint, method, action, commandName, paramsToAttrs, Collections.emptyMap());
     }
 
+    // lame... the Maps aren't typed simply because callers want to use Utils.makeMap which yields object vals
+    @SuppressWarnings("unchecked")
     Meta(EndPoint endPoint, SolrRequest.METHOD method, CollectionAction action,
-         String commandName, Map paramstoAttr, Map prefixSubstitutes) {
+         String commandName, Map paramsToAttrs, Map prefixParamsToAttrs) {
       this.action = action;
       this.commandName = commandName;
       this.endPoint = endPoint;
       this.method = method;
-      this.paramstoAttr = paramstoAttr == null ? Collections.EMPTY_MAP : Collections.unmodifiableMap(paramstoAttr);
-      this.prefixSubstitutes = Collections.unmodifiableMap(prefixSubstitutes);
 
+      this.paramsToAttrs = paramsToAttrs == null ? Collections.emptyMap() : Collections.unmodifiableMap(paramsToAttrs);
+      this.attrsToParams = Collections.unmodifiableMap(reverseMap(this.paramsToAttrs));
+      this.prefixParamsToAttrs = prefixParamsToAttrs == null ? Collections.emptyMap() : Collections.unmodifiableMap(prefixParamsToAttrs);
+    }
+
+    private static Map<String, String> reverseMap(Map<String, String> input) { // swap keys and values
+      Map<String, String> attrToParams = new HashMap<>(input.size());
+      for (Map.Entry<String, String> entry :input.entrySet()) {
+        final String existing = attrToParams.put(entry.getValue(), entry.getKey());
+        if (existing != null) {
+          throw new IllegalArgumentException("keys and values must collectively be unique");
+        }
+      }
+      return attrToParams;
     }
 
     @Override
@@ -250,49 +268,52 @@ public class CollectionApiMapping {
       return endPoint;
     }
 
-
+    // Returns iterator of v1 "params".
     @Override
-    public Collection<String> getParamNames(CommandOperation op) {
+    public Iterator<String> getParamNamesIterator(CommandOperation op) {
       Collection<String> paramNames = getParamNames_(op, this);
-      if (!prefixSubstitutes.isEmpty()) {
-        Collection<String> result = new ArrayList<>(paramNames.size());
-        for (Map.Entry<String, String> e : prefixSubstitutes.entrySet()) {
-          for (String paramName : paramNames) {
-            if (paramName.startsWith(e.getKey())) {
-              result.add(paramName.replace(e.getKey(), e.getValue()));
-            } else {
-              result.add(paramName);
+      Stream<String> pStream = paramNames.stream();
+      if (!attrsToParams.isEmpty()) {
+        pStream = pStream.map(paramName -> attrsToParams.getOrDefault(paramName, paramName));
+      }
+      if (!prefixParamsToAttrs.isEmpty()) {
+        pStream = pStream.map(paramName -> {
+          for (Map.Entry<String, String> e : prefixParamsToAttrs.entrySet()) {
+            final String prefixV1 = e.getKey();
+            final String prefixV2 = e.getValue();
+            if (paramName.startsWith(prefixV2)) {
+              return prefixV1 + paramName.substring(prefixV2.length()); // replace
             }
           }
-          paramNames = result;
-        }
+          return paramName;
+        });
       }
-
-      return paramNames;
+      return pStream.iterator();
     }
 
+    // returns params (v1) from an underlying v2, with param (v1) input
     @Override
-    public String getParamSubstitute(String param) {
-      String s = paramstoAttr.containsKey(param) ? paramstoAttr.get(param) : param;
-      if (prefixSubstitutes != null) {
-        for (Map.Entry<String, String> e : prefixSubstitutes.entrySet()) {
-          if (s.startsWith(e.getValue())) return s.replace(e.getValue(), e.getKey());
+    public String getParamSubstitute(String param) {//input is v1
+      for (Map.Entry<String, String> e : prefixParamsToAttrs.entrySet()) {
+        final String prefixV1 = e.getKey();
+        final String prefixV2 = e.getValue();
+        if (param.startsWith(prefixV1)) {
+          return prefixV2 + param.substring(prefixV1.length()); // replace
         }
       }
-      return s;
+      return paramsToAttrs.getOrDefault(param, param);
     }
-    public Object getReverseParamSubstitute(String param) {
-      String s = paramstoAttr.containsKey(param) ? paramstoAttr.get(param) : param;
 
-      if (prefixSubstitutes != null) {
-        for (Map.Entry<String, String> e : prefixSubstitutes.entrySet()) {
-          if(param.startsWith(e.getValue())){
-            return new Pair<>(e.getKey().substring(0, e.getKey().length() - 1), param.substring(e.getValue().length()));
-          }
+    // TODO document!
+    public Object getReverseParamSubstitute(String param) {//input is v1
+      for (Map.Entry<String, String> e : prefixParamsToAttrs.entrySet()) {
+        final String prefixV1 = e.getKey();
+        final String prefixV2 = e.getValue();
+        if (param.startsWith(prefixV1)) {
+          return new Pair<>(prefixV2.substring(0, prefixV2.length() - 1), param.substring(prefixV1.length()));
         }
       }
-      return s;
-
+      return paramsToAttrs.getOrDefault(param, param);
     }
 
   }
@@ -386,14 +407,15 @@ public class CollectionApiMapping {
 
 
   private static Collection<String> getParamNames_(CommandOperation op, CommandMeta command) {
-    List<String> result = new ArrayList<>();
     Object o = op.getCommandData();
     if (o instanceof Map) {
       Map map = (Map) o;
+      List<String> result = new ArrayList<>();
       collectKeyNames(map, result, "");
+      return result;
+    } else {
+      return Collections.emptySet();
     }
-    return result;
-
   }
 
   public static void collectKeyNames(Map<String, Object> map, List<String> result, String prefix) {
@@ -415,11 +437,11 @@ public class CollectionApiMapping {
 
     V2EndPoint getEndPoint();
 
-    default Collection<String> getParamNames(CommandOperation op) {
-      return getParamNames_(op, CommandMeta.this);
+    default Iterator<String> getParamNamesIterator(CommandOperation op) {
+      return getParamNames_(op, CommandMeta.this).iterator();
     }
 
-
+    /** Given a v1 param, return the v2 attribute (possibly a dotted path). */
     default String getParamSubstitute(String name) {
       return name;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8e9ab87/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java b/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java
index e5a1d44..2a8d2d1 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java
@@ -18,6 +18,7 @@
 package org.apache.solr.common.util;
 
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -226,12 +227,27 @@ class RequiredValidator extends Validator<List<String>> {
 
   @Override
   boolean validate(Object o, List<String> errs) {
+    return validate(o,errs,requiredProps);
+  }
+
+  boolean validate( Object o, List<String> errs, Set<String> requiredProps) {
     if (o instanceof Map) {
       Set fnames = ((Map) o).keySet();
       for (String requiredProp : requiredProps) {
-        if (!fnames.contains(requiredProp)) {
-          errs.add("Missing required attribute '" + requiredProp + "' in object " + Utils.toJSONString(o));
-          return false;
+        if (requiredProp.contains(".")) {
+          if (requiredProp.endsWith(".")) {
+            errs.add("Illegal required attribute name (ends with '.': " + requiredProp + ").  This is a bug.");
+            return false;
+          }
+          String subprop = requiredProp.substring(requiredProp.indexOf(".") + 1);
+          if (!validate(((Map)o).get(requiredProp), errs, Collections.singleton(subprop))) {
+            return false;
+          }
+        } else {
+          if (!fnames.contains(requiredProp)) {
+            errs.add("Missing required attribute '" + requiredProp + "' in object " + Utils.toJSONString(o));
+            return false;
+          }
         }
       }
       return true;


[3/4] lucene-solr:master: SOLR-11722: New CREATEROUTEDALIAS cluster command for time routed aliases. (a refactoring of some of the related parts will follow in next commit)

Posted by ds...@apache.org.
SOLR-11722: New CREATEROUTEDALIAS cluster command for time routed aliases.
(a refactoring of some of the related parts will follow in next commit)


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

Branch: refs/heads/master
Commit: a1828a5664983b8e26177537d233a78d2c0c33cd
Parents: d8e9ab8
Author: David Smiley <ds...@apache.org>
Authored: Thu Jan 25 14:45:49 2018 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Thu Jan 25 14:45:49 2018 -0500

----------------------------------------------------------------------
 .../cloud/api/collections/CreateAliasCmd.java   | 163 +++++++-
 .../api/collections/CreateCollectionCmd.java    |   6 +-
 .../OverseerCollectionMessageHandler.java       |   1 +
 .../RoutedAliasCreateCollectionCmd.java         |  47 +--
 .../solr/handler/admin/CollectionsHandler.java  |  82 +++-
 .../apache/solr/request/SolrRequestInfo.java    |   2 +-
 .../TimeRoutedAliasUpdateProcessor.java         |   6 +-
 .../org/apache/solr/util/DateMathParser.java    |  17 +-
 .../org/apache/solr/util/TimeZoneUtils.java     |   2 +-
 solr/core/src/test-files/log4j.properties       |   2 +
 .../cloud/ConcurrentCreateRoutedAliasTest.java  | 221 +++++++++++
 .../solr/cloud/CreateRoutedAliasTest.java       | 370 +++++++++++++++++++
 .../TimeRoutedAliasUpdateProcessorTest.java     |   6 +-
 solr/solr-ref-guide/src/collections-api.adoc    | 163 ++++++++
 solr/solr-ref-guide/src/v2-api.adoc             |   2 +-
 .../solrj/impl/HttpClusterStateProvider.java    |   1 +
 .../solrj/request/CollectionAdminRequest.java   |  93 +++++
 .../solrj/request/CollectionApiMapping.java     |  15 +-
 .../org/apache/solr/common/cloud/Aliases.java   |  31 +-
 .../apache/solr/common/cloud/ZkStateReader.java |   3 +-
 .../solr/common/params/CollectionParams.java    |   1 +
 .../resources/apispec/collections.Commands.json |  61 ++-
 22 files changed, 1234 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
index c54d792..20f52f1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
@@ -17,25 +17,80 @@
  */
 package org.apache.solr.cloud.api.collections;
 
+import java.lang.invoke.MethodHandles;
+import java.text.ParseException;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
 import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Locale;
+import java.util.Map;
 import java.util.Set;
+import java.util.TimeZone;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.update.processor.TimeRoutedAliasUpdateProcessor;
+import org.apache.solr.util.DateMathParser;
+import org.apache.solr.util.TimeZoneUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.common.params.CommonParams.NAME;
-
+import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
+import static org.apache.solr.common.params.CommonParams.TZ;
 
 public class CreateAliasCmd implements OverseerCollectionMessageHandler.Cmd {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final String ROUTER_PREFIX = "router.";
+  public static final String ROUTER_START = ROUTER_PREFIX + "start"; //TODO, router related
+  public static final String ROUTER_TYPE = ROUTER_PREFIX + "name";
+  public static final String ROUTER_FIELD = ROUTER_PREFIX + "field";
+  public static final String ROUTER_INCREMENT = ROUTER_PREFIX + "interval";
+  public static final String ROUTER_MAX_FUTURE = ROUTER_PREFIX + "max-future-ms";
+
+  public static final String CREATE_COLLECTION_PREFIX = "create-collection.";
+
   private final OverseerCollectionMessageHandler ocmh;
 
+  /**
+   * Parameters required for creating a routed alias
+   */
+  public static final List<String> REQUIRED_ROUTER_PARAMS = Collections.unmodifiableList(Arrays.asList(
+      CommonParams.NAME,
+      ROUTER_TYPE,
+      ROUTER_FIELD,
+      ROUTER_START,
+      ROUTER_INCREMENT));
+
+  /**
+   * Optional parameters for creating a routed alias excluding parameters for collection creation.
+   */
+  public static final List<String> OPTIONAL_ROUTER_PARAMS = Collections.unmodifiableList(Arrays.asList(
+      ROUTER_MAX_FUTURE,
+      TZ)); // kinda special
+
+  private static Predicate<String> PARAM_IS_METADATA =
+      key -> key.equals(TZ) ||
+          (key.startsWith(ROUTER_PREFIX) && !key.equals(ROUTER_START)) || //nocommit reconsider START special case
+          key.startsWith(CREATE_COLLECTION_PREFIX);
+
+  private static boolean anyRoutingParams(ZkNodeProps message) {
+    return message.keySet().stream().anyMatch(k -> k.startsWith(ROUTER_PREFIX));
+  }
+
   public CreateAliasCmd(OverseerCollectionMessageHandler ocmh) {
     this.ocmh = ocmh;
   }
@@ -43,14 +98,84 @@ public class CreateAliasCmd implements OverseerCollectionMessageHandler.Cmd {
   @Override
   public void call(ClusterState state, ZkNodeProps message, NamedList results)
       throws Exception {
-    final String aliasName = message.getStr(NAME);
-    final List<String> canonicalCollectionList = parseCollectionsParameter(message.get("collections"));
-    final String canonicalCollectionsString = StrUtils.join(canonicalCollectionList, ',');
-
+    final String aliasName = message.getStr(CommonParams.NAME);
     ZkStateReader zkStateReader = ocmh.zkStateReader;
-    validateAllCollectionsExistAndNoDups(canonicalCollectionList, zkStateReader);
+    ZkStateReader.AliasesManager holder = zkStateReader.aliasesHolder;
+
+    //TODO refactor callCreatePlainAlias
+    if (!anyRoutingParams(message)) {
+
+      final List<String> canonicalCollectionList = parseCollectionsParameter(message.get("collections"));
+      final String canonicalCollectionsString = StrUtils.join(canonicalCollectionList, ',');
+      validateAllCollectionsExistAndNoDups(canonicalCollectionList, zkStateReader);
+      holder.applyModificationAndExportToZk(aliases -> aliases.cloneWithCollectionAlias(aliasName, canonicalCollectionsString));
+
+    } else { //TODO refactor callCreateRoutedAlias
+
+      // Validate we got everything we need
+      if (!message.getProperties().keySet().containsAll(REQUIRED_ROUTER_PARAMS)) {
+        throw new SolrException(BAD_REQUEST, "A routed alias requires these params: " + REQUIRED_ROUTER_PARAMS
+        + " plus some create-collection prefixed ones.");
+      }
+
+      Map<String, String> aliasMetadata = new LinkedHashMap<>();
+      message.getProperties().entrySet().stream()
+          .filter(entry -> PARAM_IS_METADATA.test(entry.getKey()))
+          .forEach(entry -> aliasMetadata.put(entry.getKey(), (String) entry.getValue()));
 
-    zkStateReader.aliasesHolder.applyModificationAndExportToZk(aliases -> aliases.cloneWithCollectionAlias(aliasName, canonicalCollectionsString));
+      //TODO read these from metadata where appropriate. This leads to consistent logic between initial routed alias
+      //  collection creation, and subsequent collections to be created.
+
+      final String routingType = message.getStr(ROUTER_TYPE);
+      final String tz = message.getStr(TZ);
+      final String start = message.getStr(ROUTER_START);
+      final String increment = message.getStr(ROUTER_INCREMENT);
+      final String maxFutureMs = message.getStr(ROUTER_MAX_FUTURE);
+
+      try {
+        if (maxFutureMs != null && 0 > Long.parseLong(maxFutureMs)) {
+          throw new NumberFormatException("Negative value not allowed here");
+        }
+      } catch (NumberFormatException e) {
+        throw new SolrException(BAD_REQUEST, ROUTER_MAX_FUTURE + " must be a valid long integer representing a number " +
+            "of milliseconds greater than or equal to zero");
+      }
+
+      if (!"time".equals(routingType)) {
+        throw new SolrException(BAD_REQUEST, "Only time based routing is supported at this time");
+      }
+
+      // Check for invalid timezone
+      TimeZone zone = TimeZoneUtils.parseTimezone(tz);
+
+      // check that the increment is valid date math
+      try {
+        new DateMathParser(zone).parseMath(increment);
+      } catch (ParseException e) {
+        throw new SolrException(BAD_REQUEST,e.getMessage(),e);
+      }
+
+      Instant startTime = parseStart(start, zone);
+
+      // It's too much work to check the routed field against the schema, there seems to be no good way to get
+      // a copy of the schema aside from loading it directly from zookeeper based on the config name, but that
+      // also requires I load solrconfig.xml to check what the value for managedSchemaResourceName is too, (or
+      // discover that managed schema is not turned on and read schema.xml instead... and check for dynamic
+      // field patterns too. As much as it would be nice to validate all inputs it's not worth the effort.
+
+      String initialCollectionName = TimeRoutedAliasUpdateProcessor
+          .formatCollectionNameFromInstant(aliasName, startTime);
+
+      // Create the collection
+      NamedList createResults = new NamedList();
+      RoutedAliasCreateCollectionCmd.createCollectionAndWait(state, createResults, aliasName, aliasMetadata, initialCollectionName, ocmh);
+      validateAllCollectionsExistAndNoDups(Collections.singletonList(initialCollectionName), zkStateReader);
+
+      // Create/update the alias
+      holder.applyModificationAndExportToZk(aliases -> aliases
+          .cloneWithCollectionAlias(aliasName, initialCollectionName)
+          .cloneWithCollectionAliasMetadata(aliasName, aliasMetadata));
+    }
 
     // Sleep a bit to allow ZooKeeper state propagation.
     //
@@ -67,30 +192,44 @@ public class CreateAliasCmd implements OverseerCollectionMessageHandler.Cmd {
     Thread.sleep(100);
   }
 
+  private Instant parseStart(String str, TimeZone zone) {
+    Instant start = DateMathParser.parseMath(new Date(), str, zone).toInstant();
+    checkMilis(start);
+    return start;
+  }
+
+  private void checkMilis(Instant date) {
+    if (!date.truncatedTo(ChronoUnit.SECONDS).equals(date)) {
+      throw new SolrException(BAD_REQUEST,
+          "Date or date math for start time includes milliseconds, which is not supported. " +
+              "(Hint: 'NOW' used without rounding always has this problem)");
+    }
+  }
+
   private void validateAllCollectionsExistAndNoDups(List<String> collectionList, ZkStateReader zkStateReader) {
     final String collectionStr = StrUtils.join(collectionList, ',');
 
     if (new HashSet<>(collectionList).size() != collectionList.size()) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+      throw new SolrException(BAD_REQUEST,
           String.format(Locale.ROOT,  "Can't create collection alias for collections='%s', since it contains duplicates", collectionStr));
     }
     ClusterState clusterState = zkStateReader.getClusterState();
     Set<String> aliasNames = zkStateReader.getAliases().getCollectionAliasListMap().keySet();
     for (String collection : collectionList) {
       if (clusterState.getCollectionOrNull(collection) == null && !aliasNames.contains(collection)) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+        throw new SolrException(BAD_REQUEST,
             String.format(Locale.ROOT,  "Can't create collection alias for collections='%s', '%s' is not an existing collection or alias", collectionStr, collection));
       }
     }
   }
-  
+
   /**
    * The v2 API directs that the 'collections' parameter be provided as a JSON array (e.g. ["a", "b"]).  We also
    * maintain support for the legacy format, a comma-separated list (e.g. a,b).
    */
   @SuppressWarnings("unchecked")
   private List<String> parseCollectionsParameter(Object colls) {
-    if (colls == null) throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "missing collections param");
+    if (colls == null) throw new SolrException(BAD_REQUEST, "missing collections param");
     if (colls instanceof List) return (List<String>) colls;
     return StrUtils.splitSmart(colls.toString(), ",", true).stream()
         .map(String::trim)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
index d5ceb6a..4c6ce47 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
@@ -400,9 +400,6 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
         try {
           Map<String,Object> collectionProps = new HashMap<>();
 
-          // TODO: if collection.configName isn't set, and there isn't already a conf in zk, just use that?
-          String defaultConfigName = System.getProperty(ZkController.COLLECTION_PARAM_PREFIX + ZkController.CONFIGNAME_PROP, collection);
-
           if (params.size() > 0) {
             collectionProps.putAll(params);
             // if the config name wasn't passed in, use the default
@@ -412,6 +409,8 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
             }
 
           } else if (System.getProperty("bootstrap_confdir") != null) {
+            String defaultConfigName = System.getProperty(ZkController.COLLECTION_PARAM_PREFIX + ZkController.CONFIGNAME_PROP, collection);
+
             // if we are bootstrapping a collection, default the config for
             // a new collection to the collection we are bootstrapping
             log.info("Setting config for collection:" + collection + " to " + defaultConfigName);
@@ -440,6 +439,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
           stateManager.makePath(collectionPath, Utils.toJSON(zkProps), CreateMode.PERSISTENT, false);
 
         } catch (KeeperException e) {
+          //TODO shouldn't the stateManager ensure this does not happen; should throw AlreadyExistsException
           // it's okay if the node already exists
           if (e.code() != KeeperException.Code.NODEEXISTS) {
             throw e;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
index 9529ee1..3df8cde 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
@@ -226,6 +226,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
         .put(RELOAD, this::reloadCollection)
         .put(DELETE, new DeleteCollectionCmd(this))
         .put(CREATEALIAS, new CreateAliasCmd(this))
+        .put(CREATEROUTEDALIAS, new CreateAliasCmd(this))
         .put(DELETEALIAS, new DeleteAliasCmd(this))
         .put(ROUTEDALIAS_CREATECOLL, new RoutedAliasCreateCollectionCmd(this))
         .put(OVERSEERSTATUS, new OverseerStatusCmd(this))

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAliasCreateCollectionCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAliasCreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAliasCreateCollectionCmd.java
index 8cfd0bd..caf0b01 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAliasCreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAliasCreateCollectionCmd.java
@@ -43,6 +43,7 @@ import org.apache.solr.util.TimeZoneUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.solr.cloud.api.collections.CreateAliasCmd.CREATE_COLLECTION_PREFIX;
 import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.COLL_CONF;
 import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.update.processor.TimeRoutedAliasUpdateProcessor.ROUTER_FIELD_METADATA;
@@ -62,8 +63,6 @@ public class RoutedAliasCreateCollectionCmd implements OverseerCollectionMessage
 
   public static final String IF_MOST_RECENT_COLL_NAME = "ifMostRecentCollName";
 
-  public static final String COLL_METAPREFIX = "collection-create.";
-
   private final OverseerCollectionMessageHandler ocmh;
 
   public RoutedAliasCreateCollectionCmd(OverseerCollectionMessageHandler ocmh) {
@@ -133,15 +132,36 @@ public class RoutedAliasCreateCollectionCmd implements OverseerCollectionMessage
     final String createCollName = TimeRoutedAliasUpdateProcessor.formatCollectionNameFromInstant(aliasName, nextCollTimestamp);
 
     //---- CREATE THE COLLECTION
+    createCollectionAndWait(clusterState, results, aliasName, aliasMetadata, createCollName, ocmh);
+
+    //TODO delete some of the oldest collection(s) ?
+
+    //---- UPDATE THE ALIAS
+    aliasesHolder.applyModificationAndExportToZk(curAliases -> {
+      final List<String> curTargetCollections = curAliases.getCollectionAliasListMap().get(aliasName);
+      if (curTargetCollections.contains(createCollName)) {
+        return curAliases;
+      } else {
+        List<String> newTargetCollections = new ArrayList<>(curTargetCollections.size() + 1);
+        // prepend it on purpose (thus reverse sorted). Solr alias resolution defaults to the first collection in a list
+        newTargetCollections.add(createCollName);
+        newTargetCollections.addAll(curTargetCollections);
+        return curAliases.cloneWithCollectionAlias(aliasName, StrUtils.join(newTargetCollections, ','));
+      }
+    });
+
+  }
+
+  static void createCollectionAndWait(ClusterState clusterState, NamedList results, String aliasName, Map<String, String> aliasMetadata, String createCollName, OverseerCollectionMessageHandler ocmh) throws Exception {
     // Map alias metadata starting with a prefix to a create-collection API request
     final ModifiableSolrParams createReqParams = new ModifiableSolrParams();
     for (Map.Entry<String, String> e : aliasMetadata.entrySet()) {
-      if (e.getKey().startsWith(COLL_METAPREFIX)) {
-        createReqParams.set(e.getKey().substring(COLL_METAPREFIX.length()), e.getValue());
+      if (e.getKey().startsWith(CREATE_COLLECTION_PREFIX)) {
+        createReqParams.set(e.getKey().substring(CREATE_COLLECTION_PREFIX.length()), e.getValue());
       }
     }
     if (createReqParams.get(COLL_CONF) == null) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
           "We require an explicit " + COLL_CONF );
     }
     createReqParams.set(NAME, createCollName);
@@ -157,23 +177,6 @@ public class RoutedAliasCreateCollectionCmd implements OverseerCollectionMessage
     ocmh.commandMap.get(CollectionParams.CollectionAction.CREATE).call(clusterState, new ZkNodeProps(createMsgMap), results);
 
     CollectionsHandler.waitForActiveCollection(createCollName, null, ocmh.overseer.getCoreContainer(), new OverseerSolrResponse(results));
-
-    //TODO delete some of the oldest collection(s) ?
-
-    //---- UPDATE THE ALIAS
-    aliasesHolder.applyModificationAndExportToZk(curAliases -> {
-      final List<String> curTargetCollections = curAliases.getCollectionAliasListMap().get(aliasName);
-      if (curTargetCollections.contains(createCollName)) {
-        return curAliases;
-      } else {
-        List<String> newTargetCollections = new ArrayList<>(curTargetCollections.size() + 1);
-        // prepend it on purpose (thus reverse sorted). Solr alias resolution defaults to the first collection in a list
-        newTargetCollections.add(createCollName);
-        newTargetCollections.addAll(curTargetCollections);
-        return curAliases.cloneWithCollectionAlias(aliasName, StrUtils.join(newTargetCollections, ','));
-      }
-    });
-
   }
 
   private SolrException newAliasMustExistException(String aliasName) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/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 56f979d..2e05e07 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
@@ -30,6 +30,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
 
 import com.google.common.collect.ImmutableSet;
 import org.apache.commons.io.IOUtils;
@@ -42,11 +43,11 @@ 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.Overseer;
-import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
 import org.apache.solr.cloud.OverseerSolrResponse;
 import org.apache.solr.cloud.OverseerTaskQueue;
 import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent;
 import org.apache.solr.cloud.ZkController;
+import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
 import org.apache.solr.cloud.overseer.SliceMutator;
 import org.apache.solr.cloud.rule.ReplicaAssigner;
 import org.apache.solr.cloud.rule.Rule;
@@ -84,6 +85,7 @@ import org.apache.solr.core.snapshots.CollectionSnapshotMetaData;
 import org.apache.solr.core.snapshots.SolrSnapshotManager;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.component.ShardHandler;
+import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.security.AuthorizationContext;
@@ -100,6 +102,9 @@ import static org.apache.solr.client.solrj.response.RequestStatusState.NOT_FOUND
 import static org.apache.solr.client.solrj.response.RequestStatusState.RUNNING;
 import static org.apache.solr.client.solrj.response.RequestStatusState.SUBMITTED;
 import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
+import static org.apache.solr.cloud.api.collections.CreateAliasCmd.CREATE_COLLECTION_PREFIX;
+import static org.apache.solr.cloud.api.collections.CreateAliasCmd.OPTIONAL_ROUTER_PARAMS;
+import static org.apache.solr.cloud.api.collections.CreateAliasCmd.REQUIRED_ROUTER_PARAMS;
 import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.COLL_CONF;
 import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.COLL_PROP_PREFIX;
 import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.CREATE_NODE_SET;
@@ -335,12 +340,24 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     return collectionQueue.containsTaskWithRequestId(ASYNC, asyncId);
   }
 
+  /**
+   * Copy prefixed params into a map.  There must only be one value for these parameters.
+   *
+   * @param params The source of params from which copies should be made
+   * @param props The map into which param names and values should be copied as keys and values respectively
+   * @param prefix The prefix to select.
+   * @return the map supplied in the props parameter, modified to contain the prefixed params.
+   */
   private static Map<String, Object> copyPropertiesWithPrefix(SolrParams params, Map<String, Object> props, String prefix) {
     Iterator<String> iter =  params.getParameterNamesIterator();
     while (iter.hasNext()) {
       String param = iter.next();
       if (param.startsWith(prefix)) {
-        props.put(param, params.get(param));
+        final String[] values = params.getParams(param);
+        if (values.length != 1) {
+          throw new SolrException(BAD_REQUEST, "Only one value can be present for parameter " + param);
+        }
+        props.put(param, values[0]);
       }
     }
     return props;
@@ -472,10 +489,51 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       }
       return null;
     }),
+
     CREATEALIAS_OP(CREATEALIAS, (req, rsp, h) -> {
       SolrIdentifierValidator.validateAliasName(req.getParams().get(NAME));
       return req.getParams().required().getAll(null, NAME, "collections");
     }),
+
+    CREATEROUTEDALIAS_OP(CREATEROUTEDALIAS, (req, rsp, h) -> {
+      String alias = req.getParams().get(NAME);
+      SolrIdentifierValidator.validateAliasName(alias);
+      Map<String, Object> result = req.getParams().required().getAll(null, REQUIRED_ROUTER_PARAMS);
+      req.getParams().getAll(result, OPTIONAL_ROUTER_PARAMS);
+
+      ModifiableSolrParams createCollParams = new ModifiableSolrParams(); // without prefix
+
+      // add to result params that start with "create-collection.".
+      //   Additionally, save these without the prefix to createCollParams
+
+      forEach(req.getParams(), (p, v) -> {
+          if (p.startsWith(CREATE_COLLECTION_PREFIX)) {
+            // This is what SolrParams#getAll(Map, Collection)} does
+            if (v.length == 1) {
+              result.put(p, v[0]);
+            } else {
+              result.put(p, v);
+            }
+            createCollParams.set(p.substring(CREATE_COLLECTION_PREFIX.length()), v);
+          }
+        });
+
+      // Verify that the create-collection prefix'ed params appear to be valid.
+      if (createCollParams.get(NAME) != null) {
+        throw new SolrException(BAD_REQUEST, "routed aliases calculate names for their " +
+            "dependent collections, you cannot specify the name.");
+      }
+      if (createCollParams.get(COLL_CONF) == null) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            "We require an explicit " + COLL_CONF );
+      }
+      // note: could insist on a config name here as well.... or wait to throw at overseer
+      createCollParams.add(NAME, "TMP_name_TMP_name_TMP"); // just to pass validation
+      CREATE_OP.execute(new LocalSolrQueryRequest(null, createCollParams), rsp, h); // ignore results
+
+      return result;
+    }),
+
     DELETEALIAS_OP(DELETEALIAS, (req, rsp, h) -> req.getParams().required().getAll(null, NAME)),
 
     /**
@@ -931,6 +989,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
           "shard");
     }),
     DELETENODE_OP(DELETENODE, (req, rsp, h) -> req.getParams().required().getAll(null, "node"));
+
     public final CollectionOp fun;
     CollectionAction action;
     long timeOut;
@@ -1053,7 +1112,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     String nodeNotLive = null;
 
     CloudConfig ccfg = cc.getConfig().getCloudConfig();
-    Integer numRetries = ccfg.getCreateCollectionWaitTimeTillActive();
+    Integer numRetries = ccfg.getCreateCollectionWaitTimeTillActive(); // this config is actually # seconds, not # tries
     Boolean checkLeaderOnly = ccfg.isCreateCollectionCheckLeaderActive();
     log.info("Wait for new collection to be active for at most " + numRetries + " seconds. Check all shard "
         + (checkLeaderOnly ? "leaders" : "replicas"));
@@ -1090,7 +1149,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
 
         if (replicaNotAlive == null) return;
       }
-      Thread.sleep(1000);
+      Thread.sleep(1000); // thus numRetries is roughly number of seconds
     }
     if (nodeNotLive != null && replicaState != null) {
       log.error("Timed out waiting for new collection's replicas to become ACTIVE "
@@ -1166,4 +1225,19 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
   public Boolean registerV2() {
     return Boolean.TRUE;
   }
+
+  /**
+   * Calls the consumer for each parameter and with all values.
+   * This may be more convenient than using the iterator.
+   */
+  //TODO put on SolrParams, or maybe SolrParams should implement Iterable<Map.Entry<String,String[]>
+  private static void forEach(SolrParams params, BiConsumer<String, String[]> consumer) {
+    //TODO do we add a predicate for the parameter as a filter? It would avoid calling getParams
+    final Iterator<String> iterator = params.getParameterNamesIterator();
+    while (iterator.hasNext()) {
+      String param = iterator.next();
+      String[] values = params.getParams(param);
+      consumer.accept(param, values);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
index f1a718d..7f02b24 100644
--- a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
+++ b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
@@ -100,7 +100,7 @@ public class SolrRequestInfo {
     return now;
   }
 
-  /** The TimeZone specified by the request, or null if none was specified */
+  /** The TimeZone specified by the request, or UTC if none was specified. */
   public TimeZone getClientTimeZone() {
     if (tz == null)  {
       tz = TimeZoneUtils.parseTimezone(req.getParams().get(CommonParams.TZ));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
index 6f71acc..cbd9eec 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
@@ -42,8 +42,8 @@ import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
 import org.apache.solr.cloud.Overseer;
-import org.apache.solr.cloud.api.collections.RoutedAliasCreateCollectionCmd;
 import org.apache.solr.cloud.ZkController;
+import org.apache.solr.cloud.api.collections.RoutedAliasCreateCollectionCmd;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Aliases;
 import org.apache.solr.common.cloud.Replica;
@@ -79,7 +79,9 @@ import static org.apache.solr.update.processor.DistributingUpdateProcessorFactor
  * requests to create new collections on-demand.
  *
  * Depends on this core having a special core property that points to the alias name that this collection is a part of.
- * And further requires certain metadata on the Alias.
+ * And further requires certain metadata on the Alias. Collections pointed to by the alias must be named for the alias
+ * plus underscored ('_') and a time stamp of ISO_DATE plus optionally _HH_mm_ss. These collections should not be
+ * created by the user, but are created automatically by the time partitioning system.
  *
  * @since 7.2.0
  */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/core/src/java/org/apache/solr/util/DateMathParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/DateMathParser.java b/solr/core/src/java/org/apache/solr/util/DateMathParser.java
index 2124d1d..f5af734 100644
--- a/solr/core/src/java/org/apache/solr/util/DateMathParser.java
+++ b/solr/core/src/java/org/apache/solr/util/DateMathParser.java
@@ -217,12 +217,25 @@ public class DateMathParser  {
   /**
    * Parses a String which may be a date (in the standard ISO-8601 format)
    * followed by an optional math expression.
-   * @param now an optional fixed date to use as "NOW"
+   * The TimeZone is taken from the {@code TZ} param retrieved via {@link SolrRequestInfo}, defaulting to UTC.
+   * @param now an optional fixed date to use as "NOW". {@link SolrRequestInfo} is consulted if unspecified.
    * @param val the string to parse
    */
+  //TODO this API is a bit clumsy.  "now" is rarely used.
   public static Date parseMath(Date now, String val) {
+    return parseMath(now, val, null);
+  }
+
+  /**
+   * Parses a String which may be a date (in the standard ISO-8601 format)
+   * followed by an optional math expression.
+   * @param now an optional fixed date to use as "NOW"
+   * @param val the string to parse
+   * @param zone the timezone to use
+   */
+  public static Date parseMath(Date now, String val, TimeZone zone) {
     String math;
-    final DateMathParser p = new DateMathParser();
+    final DateMathParser p = new DateMathParser(zone);
 
     if (null != now) p.setNow(now);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/core/src/java/org/apache/solr/util/TimeZoneUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/TimeZoneUtils.java b/solr/core/src/java/org/apache/solr/util/TimeZoneUtils.java
index 0600a83..9ab4911 100644
--- a/solr/core/src/java/org/apache/solr/util/TimeZoneUtils.java
+++ b/solr/core/src/java/org/apache/solr/util/TimeZoneUtils.java
@@ -86,7 +86,7 @@ public final class TimeZoneUtils {
 
   /**
    * Parse the specified timezone ID. If null input then return UTC. If we can't resolve it then
-   * throw an exception.
+   * throw an exception.  Does not return null.
    */
   public static TimeZone parseTimezone(String tzStr) {
     if (tzStr != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/core/src/test-files/log4j.properties
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/log4j.properties b/solr/core/src/test-files/log4j.properties
index 2697203..969439a 100644
--- a/solr/core/src/test-files/log4j.properties
+++ b/solr/core/src/test-files/log4j.properties
@@ -29,6 +29,8 @@ log4j.logger.org.apache.solr.hadoop=INFO
 #log4j.logger.org.apache.solr.common.cloud.ClusterStateUtil=DEBUG
 #log4j.logger.org.apache.solr.cloud.OverseerAutoReplicaFailoverThread=DEBUG
 
+#log4j.logger.org.apache.http.wire=DEBUG
+#log4j.logger.org.apache.http.headers=DEBUG
 #log4j.logger.org.apache.http.impl.conn.PoolingHttpClientConnectionManager=DEBUG
 #log4j.logger.org.apache.http.impl.conn.BasicClientConnectionManager=DEBUG
 #log4j.logger.org.apache.http=DEBUG

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/core/src/test/org/apache/solr/cloud/ConcurrentCreateRoutedAliasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ConcurrentCreateRoutedAliasTest.java b/solr/core/src/test/org/apache/solr/cloud/ConcurrentCreateRoutedAliasTest.java
new file mode 100644
index 0000000..a76a6ff
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/ConcurrentCreateRoutedAliasTest.java
@@ -0,0 +1,221 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.common.util.IOUtils;
+import org.apache.zookeeper.KeeperException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@LuceneTestCase.Slow
+public class ConcurrentCreateRoutedAliasTest extends SolrTestCaseJ4 {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private MiniSolrCloudCluster solrCluster;
+
+  // to avoid having to delete stuff...
+  volatile int num = 0;
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    solrCluster = new MiniSolrCloudCluster(4, createTempDir(), buildJettyConfig("/solr"));
+  }
+
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    solrCluster.shutdown();
+    super.tearDown();
+  }
+
+  @Test
+  public void testConcurrentCreateRoutedAliasMinimal() throws IOException, KeeperException.NoNodeException {
+    // this is the test where be blow out a bunch of create commands all out at once.
+    // other tests are more functionality based, and just use a single thread.
+
+    // Failure of this test very occasionally due to overseer overload would not be worrisome (just bothersome).
+    // Any use case creating large numbers of time routed aliases concurrently would be an EXTREMELY odd
+    // if not fundamentally broken use case. This test method is just here to guard against any race
+    // conditions in the code that could crop up rarely in lower volume usage.
+
+    // That said any failures involving about NPE's or missing parameters or oddities other than overwhelming
+    // the overseer queue with retry races emanating from this test should be investigated. Also if it fails
+    // frequently that needs to be investigated of course.
+
+
+    final AtomicReference<Exception> failure = new AtomicReference<>();
+
+    // Note: this number of threads seems to work regularly with the up-tweaked number of retries (50) in
+    // org.apache.solr.common.cloud.ZkStateReader.AliasesManager.applyModificationAndExportToZk()
+    // with the original 5 retries this wouldn't reliably pass with 10 threads, but with 50 retries it seems
+    // to handle 50 threads about a dozen times without any failure (on a 32 thread processor)
+    // it also passed 3/3 at 150 threads and 2/3 with 250 threads on both 1 node and 4 nodes...
+    // the failure mode seems to be overseer tasks that are not found. I suspect this happens when enough
+    // threads get into retry races and the spam overwhelms the overseer. (that this can happen might imply
+    // an issue over there, but I'm not sure, since there is an intentional hard limit on the overseer queue
+    // and I haven't tried to count the retries up and figure out if the requests are actually exceeding that
+    // limit or not, but the speed of retries might indicate an effectively hot loop, but again, a separate issue.
+
+    // The hope is that the level of concurrency supported by create routed alias and the code it uses is such
+    // that this test wouldn't spuriously fail more than once a year. If that's true users should never see
+    // an issue in the wild unless they are doing something we probably don't want to support anyway
+
+    final CreateRoutedAliasThread[] threads = new CreateRoutedAliasThread[50];
+    int numStart = num;
+    for (; num < threads.length + numStart; num++) {
+      final String aliasName = "testAlias" + num;
+      final String baseUrl = solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString();
+      final SolrClient solrClient = getHttpSolrClient(baseUrl);
+
+
+      int i = num - numStart;
+      threads[i] = new CreateRoutedAliasThread("create-delete-search-" + i, aliasName, "NOW/HOUR",
+          solrClient, failure, false);
+    }
+
+    startAll(threads);
+    joinAll(threads);
+
+    assertNull("concurrent alias creation failed " + failure.get(), failure.get());
+  }
+
+
+  @Test
+  public void testConcurrentCreateRoutedAliasComplex() {
+    final AtomicReference<Exception> failure = new AtomicReference<>();
+
+    final CreateRoutedAliasThread[] threads = new CreateRoutedAliasThread[1];
+    int numStart = num;
+    System.out.println("NUM ==> " +num);
+    for (; num < threads.length + numStart; num++) {
+      final String aliasName = "testAliasCplx" + num;
+      final String baseUrl = solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString();
+      final SolrClient solrClient = getHttpSolrClient(baseUrl);
+
+      int i = num - numStart;
+      threads[i] = new CreateRoutedAliasThread("create-routed-alias-cplx-" + i,
+          aliasName, "2017-12-25T23:24:25Z",
+          solrClient, failure, true);
+    }
+
+    startAll(threads);
+    joinAll(threads);
+
+    assertNull("concurrent alias creation failed " + failure.get(), failure.get());
+  }
+
+  private void joinAll(final CreateRoutedAliasThread[] threads) {
+    for (CreateRoutedAliasThread t : threads) {
+      try {
+        t.joinAndClose();
+      } catch (InterruptedException e) {
+        Thread.interrupted();
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  private void startAll(final Thread[] threads) {
+    for (Thread t : threads) {
+      t.start();
+    }
+  }
+
+  private static class CreateRoutedAliasThread extends Thread {
+    final String aliasName;
+    protected final String start;
+    protected final SolrClient solrClient;
+    protected final AtomicReference<Exception> failure;
+
+    CreateRoutedAliasThread(
+        String name, String aliasName, String start, SolrClient solrClient,
+        AtomicReference<Exception> failure, boolean v2) {
+      super(name);
+      this.aliasName = aliasName;
+      this.start = start;
+      this.solrClient = solrClient;
+      this.failure = failure;
+    }
+
+    @Override
+    public void run() {
+        doWork();
+    }
+
+    void doWork() {
+      createAlias();
+    }
+
+    void addFailure(Exception e) {
+      log.error("Add Failure", e);
+      synchronized (failure) {
+        if (failure.get() != null) {
+          failure.get().addSuppressed(e);
+        } else {
+          failure.set(e);
+        }
+      }
+    }
+
+    private void createAlias() {
+      try {
+        CollectionAdminRequest.CreateTimeRoutedAlias rq = CollectionAdminRequest
+            .createTimeRoutedAlias(
+                aliasName,
+                start,
+                "+12HOUR",
+                "routedFoo_dt",
+                CollectionAdminRequest.createCollection("_ignored_", "_default", 1, 1)
+            );
+
+        final CollectionAdminResponse response = rq.process(solrClient);
+        if (response.getStatus() != 0) {
+          addFailure(new RuntimeException("failed to create collection " + aliasName));
+        }
+      } catch (Exception e) {
+        addFailure(e);
+      }
+
+    }
+
+
+    void joinAndClose() throws InterruptedException {
+      try {
+        super.join(60000);
+      } finally {
+        IOUtils.closeQuietly(solrClient);
+      }
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
new file mode 100644
index 0000000..3dce4aa
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
@@ -0,0 +1,370 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.Date;
+import java.util.Map;
+import java.util.TimeZone;
+
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.methods.HttpUriRequest;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.util.EntityUtils;
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.common.cloud.Aliases;
+import org.apache.solr.common.cloud.CompositeIdRouter;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.ImplicitDocRouter;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.update.processor.TimeRoutedAliasUpdateProcessor;
+import org.apache.solr.util.DateMathParser;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Direct http tests of the CreateRoutedAlias functionality.
+ */
+@SolrTestCaseJ4.SuppressSSL
+public class CreateRoutedAliasTest extends SolrCloudTestCase {
+
+  private CloudSolrClient solrClient;
+  private CloseableHttpClient httpClient;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2).configure();
+
+//    final Properties properties = new Properties();
+//    properties.setProperty("immutable", "true"); // we won't modify it in this test
+//    new ConfigSetAdminRequest.Create()
+//        .setConfigSetName(configName)
+//        .setBaseConfigSetName("_default")
+//        .setNewConfigSetProperties(properties)
+//        .process(cluster.getSolrClient());
+  }
+
+  @After
+  public void finish() throws Exception {
+    IOUtils.close(solrClient, httpClient);
+  }
+
+  @Before
+  public void doBefore() throws Exception {
+    solrClient = getCloudSolrClient(cluster);
+    httpClient = (CloseableHttpClient) solrClient.getHttpClient();
+    // delete aliases first since they refer to the collections
+    ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
+    //TODO create an API to delete collections attached to the routed alias when the alias is removed
+    zkStateReader.aliasesHolder.applyModificationAndExportToZk(aliases -> {
+      Aliases a = zkStateReader.getAliases();
+      for (String alias : a.getCollectionAliasMap().keySet()) {
+        a = a.cloneWithCollectionAlias(alias,null); // remove
+      }
+      return a;
+    });
+    cluster.deleteAllCollections();
+  }
+
+  // This is a fairly complete test where we set many options and see that it both affected the created
+  //  collection and that the alias metadata was saved accordingly
+  @Test
+  public void testV2() throws Exception {
+    // note we don't use TZ in this test, thus it's UTC
+    final String aliasName = getTestName();
+
+    String createNode = cluster.getRandomJetty(random()).getNodeName();
+
+    final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+    //TODO fix Solr test infra so that this /____v2/ becomes /api/
+    HttpPost post = new HttpPost(baseUrl + "/____v2/c");
+    post.setEntity(new StringEntity("{\n" +
+        "  \"create-routed-alias\" : {\n" +
+        "    \"name\": \"" + aliasName + "\",\n" +
+        "    \"router\" : {\n" +
+        "      \"name\": \"time\",\n" +
+        "      \"field\": \"evt_dt\",\n" +
+        "      \"start\":\"NOW/DAY\",\n" + // small window for test failure once a day.
+        "      \"interval\":\"+2HOUR\",\n" +
+        "      \"max-future-ms\":\"14400000\"\n" +
+        "    },\n" +
+        //TODO should we use "NOW=" param?  Won't work with v2 and is kinda a hack any way since intended for distrib
+        "    \"create-collection\" : {\n" +
+        "      \"router\": {\n" +
+        "        \"name\":\"implicit\",\n" +
+        "        \"field\":\"foo_s\"\n" +
+        "      },\n" +
+        "      \"shards\":\"foo,bar\",\n" +
+        "      \"config\":\"_default\",\n" +
+        "      \"tlogReplicas\":1,\n" +
+        "      \"pullReplicas\":1,\n" +
+        "      \"maxShardsPerNode\":4,\n" + // note: we also expect the 'policy' to work fine
+        "      \"nodeSet\": ['" + createNode + "'],\n" +
+        "      \"properties\" : {\n" +
+        "        \"foobar\":\"bazbam\",\n" +
+        "        \"foobar2\":\"bazbam2\"\n" +
+        "      }\n" +
+        "    }\n" +
+        "  }\n" +
+        "}", ContentType.APPLICATION_JSON));
+    assertSuccess(post);
+
+    Date startDate = DateMathParser.parseMath(new Date(), "NOW/DAY");
+    String initialCollectionName = TimeRoutedAliasUpdateProcessor
+        .formatCollectionNameFromInstant(aliasName, startDate.toInstant()
+        );
+    // small chance could fail due to "NOW"; see above
+    assertCollectionExists(initialCollectionName);
+
+    // Test created collection:
+    final DocCollection coll = solrClient.getClusterStateProvider().getState(initialCollectionName).get();
+    //System.err.println(coll);
+    //TODO how do we assert the configSet ?
+    assertEquals(ImplicitDocRouter.class, coll.getRouter().getClass());
+    assertEquals("foo_s", ((Map)coll.get("router")).get("field"));
+    assertEquals(2, coll.getSlices().size()); // numShards
+    assertEquals(4, coll.getSlices().stream()
+        .mapToInt(s -> s.getReplicas().size()).sum()); // num replicas
+    // we didn't ask for any NRT replicas
+    assertEquals(0, coll.getSlices().stream()
+        .mapToInt(s -> s.getReplicas(r -> r.getType() == Replica.Type.NRT).size()).sum());
+    //assertEquals(1, coll.getNumNrtReplicas().intValue()); // TODO seems to be erroneous; I figured 'null'
+    assertEquals(1, coll.getNumTlogReplicas().intValue()); // per-shard
+    assertEquals(1, coll.getNumPullReplicas().intValue()); // per-shard
+    assertEquals(4, coll.getMaxShardsPerNode());
+    //TODO SOLR-11877 assertEquals(2, coll.getStateFormat());
+    assertTrue("nodeSet didn't work?",
+        coll.getSlices().stream().flatMap(s -> s.getReplicas().stream())
+            .map(Replica::getNodeName).allMatch(createNode::equals));
+
+    // Test Alias metadata:
+    Aliases aliases = cluster.getSolrClient().getZkStateReader().getAliases();
+    Map<String, String> collectionAliasMap = aliases.getCollectionAliasMap();
+    assertEquals(initialCollectionName, collectionAliasMap.get(aliasName));
+    Map<String, String> meta = aliases.getCollectionAliasMetadata(aliasName);
+    //System.err.println(new TreeMap(meta));
+    assertEquals("evt_dt",meta.get("router.field"));
+    assertEquals("_default",meta.get("create-collection.collection.configName"));
+    assertEquals("foo_s",meta.get("create-collection.router.field"));
+    assertEquals("bazbam",meta.get("create-collection.property.foobar"));
+    assertEquals("bazbam2",meta.get("create-collection.property.foobar2"));
+    assertEquals(createNode,meta.get("create-collection.createNodeSet"));
+  }
+
+  @Test
+  public void testV1() throws Exception {
+    final String aliasName = getTestName();
+    final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+    Instant start = Instant.now().truncatedTo(ChronoUnit.HOURS); // mostly make sure no millis
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEROUTEDALIAS" +
+        "&wt=xml" +
+        "&name=" + aliasName +
+        "&router.field=evt_dt" +
+        "&router.name=time" +
+        "&router.start=" + start +
+        "&router.interval=%2B30MINUTE" +
+        "&create-collection.collection.configName=_default" +
+        "&create-collection.router.field=foo_s" +
+        "&create-collection.numShards=1" +
+        "&create-collection.replicationFactor=2");
+    assertSuccess(get);
+
+    String initialCollectionName = TimeRoutedAliasUpdateProcessor
+        .formatCollectionNameFromInstant(aliasName, start
+        );
+    assertCollectionExists(initialCollectionName);
+
+    // Test created collection:
+    final DocCollection coll = solrClient.getClusterStateProvider().getState(initialCollectionName).get();
+    //TODO how do we assert the configSet ?
+    assertEquals(CompositeIdRouter.class, coll.getRouter().getClass());
+    assertEquals("foo_s", ((Map)coll.get("router")).get("field"));
+    assertEquals(1, coll.getSlices().size()); // numShards
+    assertEquals(2, coll.getReplicationFactor().intValue()); // num replicas
+    //TODO SOLR-11877 assertEquals(2, coll.getStateFormat());
+
+    // Test Alias metadata
+    Aliases aliases = cluster.getSolrClient().getZkStateReader().getAliases();
+    Map<String, String> collectionAliasMap = aliases.getCollectionAliasMap();
+    String alias = collectionAliasMap.get(aliasName);
+    assertNotNull(alias);
+    Map<String, String> meta = aliases.getCollectionAliasMetadata(aliasName);
+    assertNotNull(meta);
+    assertEquals("evt_dt",meta.get("router.field"));
+    assertEquals("_default",meta.get("create-collection.collection.configName"));
+    assertEquals(null,meta.get("start"));
+  }
+
+  // TZ should not affect the first collection name if absolute date given for start
+  @Test
+  public void testTimezoneAbsoluteDate() throws Exception {
+    final String aliasName = getTestName();
+    try (SolrClient client = getCloudSolrClient(cluster)) {
+      CollectionAdminRequest.createTimeRoutedAlias(
+          aliasName,
+          "2018-01-15T00:00:00Z",
+          "+30MINUTE",
+          "evt_dt",
+          CollectionAdminRequest.createCollection("_ignored_", "_default", 1, 1)
+          )
+          .setTimeZone(TimeZone.getTimeZone("GMT-10"))
+          .process(client);
+    }
+
+    assertCollectionExists(aliasName + "_2018-01-15");
+  }
+
+  @Test
+  public void testAliasNameMustBeValid() throws Exception {
+    final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEROUTEDALIAS" +
+        "&wt=json" +
+        "&name=735741!45" +  // ! not allowed
+        "&router.field=evt_dt" +
+        "&router.name=time" +
+        "&router.start=2018-01-15T00:00:00Z" +
+        "&router.interval=%2B30MINUTE" +
+        "&create-collection.collection.configName=_default" +
+        "&create-collection.numShards=1");
+    assertFailure(get, "Invalid alias");
+  }
+
+  @Test
+  public void testRandomRouterNameFails() throws Exception {
+    final String aliasName = getTestName();
+    final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEROUTEDALIAS" +
+        "&wt=json" +
+        "&name=" + aliasName +
+        "&router.field=evt_dt" +
+        "&router.name=tiafasme" + //bad
+        "&router.start=2018-01-15T00:00:00Z" +
+        "&router.interval=%2B30MINUTE" +
+        "&create-collection.collection.configName=_default" +
+        "&create-collection.numShards=1");
+    assertFailure(get, "Only time based routing is supported");
+  }
+
+  @Test
+  public void testTimeStampWithMsFails() throws Exception {
+    final String aliasName = getTestName();
+    final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEROUTEDALIAS" +
+        "&wt=json" +
+        "&name=" + aliasName +
+        "&router.field=evt_dt" +
+        "&router.name=time" +
+        "&router.start=2018-01-15T00:00:00.001Z" + // bad: no milliseconds permitted
+        "&router.interval=%2B30MINUTE" +
+        "&create-collection.collection.configName=_default" +
+        "&create-collection.numShards=1");
+    assertFailure(get, "Date or date math for start time includes milliseconds");
+  }
+
+  @Test
+  public void testBadDateMathIntervalFails() throws Exception {
+    final String aliasName = getTestName();
+    final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEROUTEDALIAS" +
+        "&wt=json" +
+        "&name=" + aliasName +
+        "&router.field=evt_dt" +
+        "&router.name=time" +
+        "&router.start=2018-01-15T00:00:00Z" +
+        "&router.interval=%2B30MINUTEx" + // bad; trailing 'x'
+        "&router.max-future-ms=60000" +
+        "&create-collection.collection.configName=_default" +
+        "&create-collection.numShards=1");
+    assertFailure(get, "Unit not recognized");
+  }
+  @Test
+  public void testNegativeFutureFails() throws Exception {
+    final String aliasName = getTestName();
+    final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEROUTEDALIAS" +
+        "&wt=json" +
+        "&name=" + aliasName +
+        "&router.field=evt_dt" +
+        "&router.name=time" +
+        "&router.start=2018-01-15T00:00:00Z" +
+        "&router.interval=%2B30MINUTE" +
+        "&router.max-future-ms=-60000" + // bad: negative
+        "&create-collection.collection.configName=_default" +
+        "&create-collection.numShards=1");
+    assertFailure(get, "router.max-future-ms must be a valid long integer");
+  }
+  @Test
+  public void testUnParseableFutureFails() throws Exception {
+    final String aliasName = "testAlias";
+    final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEROUTEDALIAS" +
+        "&wt=json" +
+        "&name=" + aliasName +
+        "&router.field=evt_dt" +
+        "&router.name=time" +
+        "&router.start=2018-01-15T00:00:00Z" +
+        "&router.interval=%2B30MINUTE" +
+        "&router.max-future-ms=SixtyThousandMiliseconds" + // bad
+        "&create-collection.collection.configName=_default" +
+        "&create-collection.numShards=1");
+    assertFailure(get, "router.max-future-ms must be a valid long integer");
+  }
+
+  private void assertSuccess(HttpUriRequest msg) throws IOException {
+    try (CloseableHttpResponse response = httpClient.execute(msg)) {
+      if (200 != response.getStatusLine().getStatusCode()) {
+        System.err.println(EntityUtils.toString(response.getEntity()));
+        fail("Unexpected status: " + response.getStatusLine());
+      }
+    }
+  }
+
+  private void assertFailure(HttpUriRequest msg, String expectedErrorSubstring) throws IOException {
+    try (CloseableHttpResponse response = httpClient.execute(msg)) {
+      assertEquals(400, response.getStatusLine().getStatusCode());
+      String entity = EntityUtils.toString(response.getEntity());
+      assertTrue("Didn't find expected error string within response: " + entity,
+          entity.contains(expectedErrorSubstring));
+    }
+  }
+
+  private void assertCollectionExists(String name) throws IOException, SolrServerException {
+    solrClient.getClusterStateProvider().connect(); // TODO get rid of this
+    //  https://issues.apache.org/jira/browse/SOLR-9784?focusedCommentId=16332729
+
+    assertNotNull(name + " not found", solrClient.getClusterStateProvider().getState(name));
+    // note: could also do:
+    //List collections = CollectionAdminRequest.listCollections(solrClient);
+  }
+
+  // not testing collection parameters, those should inherit error checking from the collection creation code.
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
index f524fda..64f5555 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
@@ -141,9 +141,9 @@ public class TimeRoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
 
     zkStateReader.aliasesHolder.applyModificationAndExportToZk(a ->
         a.cloneWithCollectionAliasMetadata(alias, TimeRoutedAliasUpdateProcessor.ROUTER_FIELD_METADATA, timeField)
-        .cloneWithCollectionAliasMetadata(alias, "collection-create.collection.configName", configName)
-        .cloneWithCollectionAliasMetadata(alias, "collection-create.numShards", "1")
-        .cloneWithCollectionAliasMetadata(alias, "collection-create.replicationFactor", "1")
+        .cloneWithCollectionAliasMetadata(alias, "create-collection.collection.configName", configName)
+        .cloneWithCollectionAliasMetadata(alias, "create-collection.numShards", "1")
+        .cloneWithCollectionAliasMetadata(alias, "create-collection.replicationFactor", "1")
         .cloneWithCollectionAliasMetadata(alias, "router.interval", "+1DAY"));
 
     // now we index a document

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/solr-ref-guide/src/collections-api.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/collections-api.adoc b/solr/solr-ref-guide/src/collections-api.adoc
index 8a48ba0..b840e97 100644
--- a/solr/solr-ref-guide/src/collections-api.adoc
+++ b/solr/solr-ref-guide/src/collections-api.adoc
@@ -518,6 +518,169 @@ http://localhost:8983/solr/admin/collections?action=CREATEALIAS&name=testalias&c
 </response>
 ----
 
+[[createroutedalias]]
+== CREATEROUTEDALIAS: Create an alias that partitions data
+
+CREATEROUTEDALIAS will create a special type of alias that automates the partitioning of data across a series of
+collections. This feature allows for indefinite indexing of data without degradation of performance otherwise
+experienced due to the continuous growth of an index. As new data arrives, a field on the document is inspected and
+the document is then potentially re-routed to another collection. The underlying collections can
+can be queried independently but more likely the alias created by this command will be used. These collections are created
+automatically on the fly as new data arrives based on the parameters supplied in this command.
+
+*NOTE* Presently only partitioning of time based data is available, though other schemes may become available in
+the future.
+[source,text]
+----
+localhost:8983/solr/admin/collections?action=CREATEROUTEDALIAS&name=timedata&router.start=NOW/DAY&router.field=evt_dt&router.name=time&router.interval=%2B1DAY&router.max-future-ms=3600000&create-collection.collection.configName=myConfig&create-collection.numShards=2
+----
+
+If run on Jan 15, 2018 The above will create an alias named timedata, that contains collections with names such as
+`timedata` and an initial collection named `timedata_2018_01_15`. Updates sent to this alias with a (required) value
+in `evt_dt` that is before or after 2018-01-15 will be rejected, until the last 60 minutes of 2018-01-15. After
+2018-01-15T23:00:00 documents for either 2018-01-15 or 2018-01-16 will be accepted. As soon as the system receives a
+document for an allowable time window for which there is no collection it will automatically create the next required
+collection (and potentially any intervening collections if router.interval is smaller than router.max-future-ms). Both
+the initial collection and any subsequent collections will be created using the specified configset. All Collection
+creation parameters other than `name` are allowed, prefixed by `create-collection.`
+
+This means that one could (for example) partition their collections by day, and within each daily collection route
+the data to shards based on customer id. Such shards can be of any type (NRT, PULL or TLOG), and rule based replica
+placement strategies may also be used. The values supplied in this command for collection creation will be retained
+in alias metadata, and can be verified by inspecting aliases.json in zookeeper.
+
+=== CREATEROUTEDALIAS Parameters
+
+`name`::
+The alias name to be created. This parameter is required, and also functions as a prefix for the names of the
+dependent collections that will be created. It must therefore adhere to normal requirements for alias and collection
+naming.
+
+`router.start`::
+The start date/time of data for this time routed alias in Solr's standard date/time format (ISO-8601 or "NOW"
+optionally with "date math").
+The first collection created for the alias will be internally named after this value.
+If a document is submitted with an earlier value for router.field then the earliest collection the alias points to then
+it will yield an error since it can't be routed.
+This date/time MUST NOT have a milliseconds component other than 0.
+Particularly, this means `NOW` will fail 999 times out of 1000, though `NOW/SECOND`, `NOW/MINUTE`, etc. will work just fine.
+This param is required.
+
+`TZ`::
+The timezone to be used when evaluating any date math in router.start or router.interval.  This is equivalent to the
+same parameter supplied to search queries, but understand in this case it's persisted with most of the other parameters
+as alias metadata.
+If GMT-4 is supplied for this value then a document dated 2018-01-14T21:00:00:01.2345Z would be stored in the
+myAlias_2018-01-15_01 collection (assumming an interval of +1HOUR). The default timezone is UTC.
+
+`router.field`::
+The date field to inspect to determine which underlying collection an incoming document should be routed to.
+This field is required on all incoming documents.
+
+`router.name`::
+The type of routing to use. Presently only `time` is valid.  This param is required.
+
+`router.interval`::
+A fragment of a date math expression that will be appended to a timestamp to determine the next collection in the series.
+Any date math expression that can be evaluated if appended to a timestamp of the form 2018-01-15T16:17:18 will
+work here. This param is required.
+
+`router.max-future-ms`::
+The maximum milliseconds into the future that a document is allowed to have in `router.field` for it to be accepted
+without error.  If there was no limit, than an erroneous value could trigger many collections to be created.
+The default is 10 minutes worth.
+
+`create-collection.*`::
+The * can be replaced with any parameter from the <<create,CREATE>> command except `name`. All other fields
+are identical in requirements and naming except that we insist that the configset be explicitly specified.
+The configset must be created beforehand, either uploaded or copied and modified.
+It's probably a bad idea to use "data driven" mode as schema mutations might happen concurrently leading to errors.
+
+`async`::
+Request ID to track this action which will be <<Asynchronous Calls,processed asynchronously>>.
+
+=== CREATEROUTEDALIAS Response
+
+The output will simply be a responseHeader with details of the time it took to process the request. To confirm the
+creation of the alias and the values of the associated metadata, you can look in the Solr Admin UI, under the Cloud
+section and find the `aliases.json` file. The initial collection should also be visible in various parts
+of the admin UI.
+
+=== Examples using CREATEROUTEDALIAS
+
+Create an alias named "myTimeData" for data begining on `2018-01-15` in the UTC time zone and partitioning daily
+based on the `evt_dt` field in the incomming documents. Data more than an hour beyond the latest (most recent)
+partiton is to be rejected and collections are created using a config set named myConfig and
+
+
+*Input*
+
+[source,text]
+----
+localhost:8983/solr/admin/collections?action=CREATEROUTEDALIAS&name=myTimeData&router.start=NOW/DAY&router.field=evt_dt&router.name=time&router.interval=%2B1DAY&router.max-future-ms=3600000&create-collection.collection.configName=myConfig&create-collection.numShards=2
+----
+
+*Output*
+
+[source,xml]
+----
+<response>
+  <lst name="responseHeader">
+    <int name="status">0</int>
+    <int name="QTime">1234</int>
+  </lst>
+</response>
+----
+
+A somewhat contrived example demonstrating the <<v2-api.adoc#top-v2-api,V2 API>> usage and additional collection creation options.
+Notice that the collection creation fields follow the v2 api naming convention, not the v1 naming conventions.
+
+*Input*
+
+[source,json]
+----
+POST /api/c
+{
+  "create-routed-alias" : {
+    "name": "somethingTemporalThisWayComes",
+    "router" : {
+      "name": "time",
+      "field": "evt_dt",
+      "start":"NOW/MINUTE",
+      "interval":"+2HOUR",
+      "max-future-ms":"14400000"
+    },
+    "create-collection" : {
+      "config":"_default",
+      "router": {
+        "name":"implicit",
+        "field":"foo_s"
+      },
+      "shards":"foo,bar,baz",
+      "numShards": 3,
+      "tlogReplicas":1,
+      "pullReplicas":1,
+      "maxShardsPerNode":2,
+      "properties" : {
+        "foobar":"bazbam"
+      }
+    }
+  }
+}
+----
+
+*Output*
+
+[source,xml]
+----
+{
+    "responseHeader": {
+        "status": 0,
+        "QTime": 1234
+    }
+}
+----
+
 [[listaliases]]
 == LISTALIASES: List of all aliases in the cluster
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/solr-ref-guide/src/v2-api.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/v2-api.adoc b/solr/solr-ref-guide/src/v2-api.adoc
index c7ecfee..e2da0bf 100644
--- a/solr/solr-ref-guide/src/v2-api.adoc
+++ b/solr/solr-ref-guide/src/v2-api.adoc
@@ -16,7 +16,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
+[[top-v2-api]]
 The v2 API is a modernized self-documenting API interface covering most current Solr APIs. It is anticipated that once the v2 API reaches full coverage, and Solr-internal API usages like SolrJ and the Admin UI have been converted from the old API to the v2 API, the old API will eventually be retired.
 
 For now the two API styles will coexist, and all the old APIs will continue to work without any change. You can disable all v2 API endpoints by starting your servers with this system property: `-Ddisable.v2.api=true`.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
index b623157..deb8fbc 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
@@ -139,6 +139,7 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
     Set<String> liveNodes = new HashSet((List<String>)(cluster.get("live_nodes")));
     this.liveNodes = liveNodes;
     liveNodesTimestamp = System.nanoTime();
+    //TODO SOLR-11877 we don't know the znode path; CLUSTER_STATE is probably wrong leading to bad stateFormat
     ClusterState cs = ClusterState.load(znodeVersion, collectionsMap, liveNodes, ZkStateReader.CLUSTER_STATE);
     if (clusterProperties != null) {
       Map<String, Object> properties = (Map<String, Object>) cluster.get("properties");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/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 edc5a8b..44eeef5 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
@@ -21,6 +21,7 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Properties;
+import java.util.TimeZone;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
@@ -41,6 +42,7 @@ import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.CollectionParams.CollectionAction;
 import org.apache.solr.common.params.CommonAdminParams;
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.ShardParams;
@@ -1357,6 +1359,97 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
   }
 
   /**
+   * Returns a SolrRequest to create a routed alias. Only time based routing is supported presently,
+   * For time based routing, the start a standard Solr timestamp string (possibly with "date math").
+   *
+   * @param aliasName the name of the alias to create.
+   * @param start the start of the routing.  A standard Solr date: ISO-8601 or NOW with date math.
+   * @param interval date math representing the time duration of each collection (e.g. {@code +1DAY})
+   * @param routerField the document field to contain the timestamp to route on
+   * @param createCollTemplate Holds options to create a collection.  The "name" is ignored.
+   */
+  public static CreateTimeRoutedAlias createTimeRoutedAlias(String aliasName, String start,
+                                                            String interval,
+                                                            String routerField,
+                                                            Create createCollTemplate) {
+
+    return new CreateTimeRoutedAlias(aliasName, routerField, start, interval, createCollTemplate);
+  }
+
+  public static class CreateTimeRoutedAlias extends AsyncCollectionAdminRequest {
+    // TODO: This and other commands in this file seem to need to share some sort of constants class with core
+    // to allow this stuff not to be duplicated. (this is pasted from CreateAliasCmd.java), however I think
+    // a comprehensive cleanup of this for all the requests in this class should be done as a separate ticket.
+
+    public static final String ROUTING_TYPE = "router.name";
+    public static final String ROUTING_FIELD = "router.field";
+    public static final String ROUTING_START = "router.start";
+    public static final String ROUTING_INCREMENT = "router.interval";
+    public static final String ROUTING_MAX_FUTURE = "router.max-future-ms";
+
+    private final String aliasName;
+    private final String routerField;
+    private final String start;
+    private final String interval;
+    //Optional:
+    private TimeZone tz;
+    private Integer maxFutureMs;
+
+    private final Create createCollTemplate;
+
+    public CreateTimeRoutedAlias(String aliasName, String routerField, String start, String interval, Create createCollTemplate) {
+      super(CollectionAction.CREATEROUTEDALIAS);
+      this.aliasName = aliasName;
+      this.start = start;
+      this.interval = interval;
+      this.routerField = routerField;
+      this.createCollTemplate = createCollTemplate;
+    }
+
+    /** Sets the timezone for interpreting any Solr "date math. */
+    public CreateTimeRoutedAlias setTimeZone(TimeZone tz) {
+      this.tz = tz;
+      return this;
+    }
+
+    /** Sets how long into the future (millis) that we will allow a document to pass. */
+    public CreateTimeRoutedAlias setMaxFutureMs(Integer maxFutureMs) {
+      this.maxFutureMs = maxFutureMs;
+      return this;
+    }
+
+    @Override
+    public SolrParams getParams() {
+      ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
+      params.add(CommonParams.NAME, aliasName);
+      params.add(ROUTING_TYPE, "time");
+      params.add(ROUTING_FIELD, routerField);
+      params.add(ROUTING_START, start);
+      params.add(ROUTING_INCREMENT, interval);
+      if (tz != null) {
+        params.add(CommonParams.TZ, tz.getID());
+      }
+      if (maxFutureMs != null) {
+        params.add(ROUTING_MAX_FUTURE, ""+maxFutureMs);
+      }
+
+      // merge the above with collectionParams.  Above takes precedence.
+      ModifiableSolrParams createCollParams = new ModifiableSolrParams(); // output target
+      final SolrParams collParams = createCollTemplate.getParams();
+      final Iterator<String> pIter = collParams.getParameterNamesIterator();
+      while (pIter.hasNext()) {
+        String key = pIter.next();
+        if (key.equals(CollectionParams.ACTION) || key.equals("name")) {
+          continue;
+        }
+        createCollParams.set("create-collection." + key, collParams.getParams(key));
+      }
+      return SolrParams.wrapDefaults(params, createCollParams);
+    }
+
+  }
+
+  /**
    * Returns a SolrRequest to delete an alias
    */
   public static DeleteAlias deleteAlias(String aliasName) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
index 8f7689b..dc00119 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
@@ -25,6 +25,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 import org.apache.solr.client.solrj.SolrRequest;
@@ -116,7 +117,19 @@ public class CollectionApiMapping {
         CREATEALIAS,
         "create-alias",
         null),
-
+    CREATE_ROUTED_ALIAS(COLLECTIONS_COMMANDS,
+        POST,
+        CREATEROUTEDALIAS,
+        "create-routed-alias",
+        // same as the CREATE_COLLECTION but with "create-collection" prefix
+        CREATE_COLLECTION.paramsToAttrs.entrySet().stream().collect(Collectors.toMap(
+            entry -> "create-collection." + entry.getKey(),
+            entry -> "create-collection." + entry.getValue()
+        )),
+        CREATE_COLLECTION.prefixParamsToAttrs.entrySet().stream().collect(Collectors.toMap(
+            entry -> "create-collection." + entry.getKey(),
+            entry -> "create-collection." + entry.getValue()
+        ))),
     DELETE_ALIAS(COLLECTIONS_COMMANDS,
         POST,
         DELETEALIAS,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/solrj/src/java/org/apache/solr/common/cloud/Aliases.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/Aliases.java b/solr/solrj/src/java/org/apache/solr/common/cloud/Aliases.java
index d77b544..14ccacc 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/Aliases.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/Aliases.java
@@ -238,19 +238,36 @@ public class Aliases {
    * @param metadataValue the metadata to add/replace, null to remove the key.
    *                      @return An immutable copy of the aliases with the new metadata.
    */
-  public Aliases cloneWithCollectionAliasMetadata(String alias, String metadataKey, String metadataValue){
+  public Aliases cloneWithCollectionAliasMetadata(String alias, String metadataKey, String metadataValue) {
+    return cloneWithCollectionAliasMetadata(alias, Collections.singletonMap(metadataKey,metadataValue));
+  }
+
+  /**
+   * Set the values for some metadata keys on a collection alias. This is done by creating a new Aliases instance
+   * with the same data as the current one but with a modification based on the parameters.
+   * <p>
+   * Note that the state in zookeeper is unaffected by this method and the change must still be persisted via
+   * {@link ZkStateReader.AliasesManager#applyModificationAndExportToZk(UnaryOperator)}
+   *
+   * @param alias the alias to update
+   * @param metadata the metadata to add/replace, null values in the map will remove the key.
+   * @return An immutable copy of the aliases with the new metadata.
+   */
+  public Aliases cloneWithCollectionAliasMetadata(String alias, Map<String,String> metadata) {
     if (!collectionAliases.containsKey(alias)) {
       throw new IllegalArgumentException(alias + " is not a valid alias");
     }
-    if (metadataKey == null) {
-      throw new IllegalArgumentException("Null is not a valid metadata key");
+    if (metadata == null) {
+      throw new IllegalArgumentException("Null is not a valid metadata map");
     }
     Map<String,Map<String,String>> newColMetadata = new LinkedHashMap<>(this.collectionAliasMetadata);//clone to modify
     Map<String, String> newMetaMap = new LinkedHashMap<>(newColMetadata.getOrDefault(alias, Collections.emptyMap()));
-    if (metadataValue != null) {
-      newMetaMap.put(metadataKey, metadataValue);
-    } else {
-      newMetaMap.remove(metadataKey);
+    for (Map.Entry<String, String> metaEntry : metadata.entrySet()) {
+      if (metaEntry.getValue() != null) {
+        newMetaMap.put(metaEntry.getKey(), metaEntry.getValue());
+      } else {
+        newMetaMap.remove(metaEntry.getKey());
+      }
     }
     newColMetadata.put(alias, Collections.unmodifiableMap(newMetaMap));
     return new Aliases(collectionAliases, newColMetadata, zNodeVersion);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 8ab7ecb..5a1abd9 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -1458,7 +1458,8 @@ public class ZkStateReader implements Closeable {
      */
     public void applyModificationAndExportToZk(UnaryOperator<Aliases> op) {
       final long deadlineNanos = System.nanoTime() + TimeUnit.SECONDS.toNanos(30);
-      int triesLeft = 5;
+      // note: triesLeft tuning is based on ConcurrentCreateRoutedAliasTest
+      int triesLeft = 30;
       while (triesLeft > 0) {
         triesLeft--;
         // we could synchronize on "this" but there doesn't seem to be a point; we have a retry loop.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
index 9d5fc36..9f6a254 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
@@ -76,6 +76,7 @@ public interface CollectionParams {
     RELOAD(true, LockLevel.COLLECTION),
     SYNCSHARD(true, LockLevel.SHARD),
     CREATEALIAS(true, LockLevel.COLLECTION),
+    CREATEROUTEDALIAS(true, LockLevel.COLLECTION),
     DELETEALIAS(true, LockLevel.COLLECTION),
     LISTALIASES(false, LockLevel.NONE),
     ROUTEDALIAS_CREATECOLL(true, LockLevel.COLLECTION),


[4/4] lucene-solr:master: SOLR-11722: Refactor out a TimeRoutedAlias class from various parts. Also allowed TRA's to be tolerant of pre-existing collections.

Posted by ds...@apache.org.
SOLR-11722: Refactor out a TimeRoutedAlias class from various parts.
Also allowed TRA's to be tolerant of pre-existing collections.


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

Branch: refs/heads/master
Commit: b0d244f656b5f0030bfda97aaf5e6e7ad085325c
Parents: a1828a5
Author: David Smiley <ds...@apache.org>
Authored: Fri Jan 26 11:37:00 2018 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Fri Jan 26 11:37:00 2018 -0500

----------------------------------------------------------------------
 .../cloud/api/collections/CreateAliasCmd.java   | 184 +++++-----------
 .../RoutedAliasCreateCollectionCmd.java         |  51 +++--
 .../cloud/api/collections/TimeRoutedAlias.java  | 211 +++++++++++++++++++
 .../solr/handler/admin/CollectionsHandler.java  |   6 +-
 .../TimeRoutedAliasUpdateProcessor.java         | 129 +++---------
 .../solr/cloud/CreateRoutedAliasTest.java       |  18 +-
 .../TimeRoutedAliasUpdateProcessorTest.java     |  38 ++--
 .../solrj/request/CollectionAdminRequest.java   |  20 +-
 8 files changed, 352 insertions(+), 305 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b0d244f6/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
index 20f52f1..9a5b1a7 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
@@ -17,11 +17,8 @@
  */
 package org.apache.solr.cloud.api.collections;
 
-import java.lang.invoke.MethodHandles;
-import java.text.ParseException;
 import java.time.Instant;
 import java.time.temporal.ChronoUnit;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.Date;
 import java.util.HashSet;
@@ -31,7 +28,6 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 import java.util.TimeZone;
-import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
 import org.apache.solr.common.SolrException;
@@ -41,54 +37,16 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.update.processor.TimeRoutedAliasUpdateProcessor;
 import org.apache.solr.util.DateMathParser;
-import org.apache.solr.util.TimeZoneUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
-import static org.apache.solr.common.params.CommonParams.TZ;
 
 public class CreateAliasCmd implements OverseerCollectionMessageHandler.Cmd {
 
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  public static final String ROUTER_PREFIX = "router.";
-  public static final String ROUTER_START = ROUTER_PREFIX + "start"; //TODO, router related
-  public static final String ROUTER_TYPE = ROUTER_PREFIX + "name";
-  public static final String ROUTER_FIELD = ROUTER_PREFIX + "field";
-  public static final String ROUTER_INCREMENT = ROUTER_PREFIX + "interval";
-  public static final String ROUTER_MAX_FUTURE = ROUTER_PREFIX + "max-future-ms";
-
-  public static final String CREATE_COLLECTION_PREFIX = "create-collection.";
-
   private final OverseerCollectionMessageHandler ocmh;
 
-  /**
-   * Parameters required for creating a routed alias
-   */
-  public static final List<String> REQUIRED_ROUTER_PARAMS = Collections.unmodifiableList(Arrays.asList(
-      CommonParams.NAME,
-      ROUTER_TYPE,
-      ROUTER_FIELD,
-      ROUTER_START,
-      ROUTER_INCREMENT));
-
-  /**
-   * Optional parameters for creating a routed alias excluding parameters for collection creation.
-   */
-  public static final List<String> OPTIONAL_ROUTER_PARAMS = Collections.unmodifiableList(Arrays.asList(
-      ROUTER_MAX_FUTURE,
-      TZ)); // kinda special
-
-  private static Predicate<String> PARAM_IS_METADATA =
-      key -> key.equals(TZ) ||
-          (key.startsWith(ROUTER_PREFIX) && !key.equals(ROUTER_START)) || //nocommit reconsider START special case
-          key.startsWith(CREATE_COLLECTION_PREFIX);
-
   private static boolean anyRoutingParams(ZkNodeProps message) {
-    return message.keySet().stream().anyMatch(k -> k.startsWith(ROUTER_PREFIX));
+    return message.keySet().stream().anyMatch(k -> k.startsWith(TimeRoutedAlias.ROUTER_PREFIX));
   }
 
   public CreateAliasCmd(OverseerCollectionMessageHandler ocmh) {
@@ -100,81 +58,11 @@ public class CreateAliasCmd implements OverseerCollectionMessageHandler.Cmd {
       throws Exception {
     final String aliasName = message.getStr(CommonParams.NAME);
     ZkStateReader zkStateReader = ocmh.zkStateReader;
-    ZkStateReader.AliasesManager holder = zkStateReader.aliasesHolder;
 
-    //TODO refactor callCreatePlainAlias
     if (!anyRoutingParams(message)) {
-
-      final List<String> canonicalCollectionList = parseCollectionsParameter(message.get("collections"));
-      final String canonicalCollectionsString = StrUtils.join(canonicalCollectionList, ',');
-      validateAllCollectionsExistAndNoDups(canonicalCollectionList, zkStateReader);
-      holder.applyModificationAndExportToZk(aliases -> aliases.cloneWithCollectionAlias(aliasName, canonicalCollectionsString));
-
-    } else { //TODO refactor callCreateRoutedAlias
-
-      // Validate we got everything we need
-      if (!message.getProperties().keySet().containsAll(REQUIRED_ROUTER_PARAMS)) {
-        throw new SolrException(BAD_REQUEST, "A routed alias requires these params: " + REQUIRED_ROUTER_PARAMS
-        + " plus some create-collection prefixed ones.");
-      }
-
-      Map<String, String> aliasMetadata = new LinkedHashMap<>();
-      message.getProperties().entrySet().stream()
-          .filter(entry -> PARAM_IS_METADATA.test(entry.getKey()))
-          .forEach(entry -> aliasMetadata.put(entry.getKey(), (String) entry.getValue()));
-
-      //TODO read these from metadata where appropriate. This leads to consistent logic between initial routed alias
-      //  collection creation, and subsequent collections to be created.
-
-      final String routingType = message.getStr(ROUTER_TYPE);
-      final String tz = message.getStr(TZ);
-      final String start = message.getStr(ROUTER_START);
-      final String increment = message.getStr(ROUTER_INCREMENT);
-      final String maxFutureMs = message.getStr(ROUTER_MAX_FUTURE);
-
-      try {
-        if (maxFutureMs != null && 0 > Long.parseLong(maxFutureMs)) {
-          throw new NumberFormatException("Negative value not allowed here");
-        }
-      } catch (NumberFormatException e) {
-        throw new SolrException(BAD_REQUEST, ROUTER_MAX_FUTURE + " must be a valid long integer representing a number " +
-            "of milliseconds greater than or equal to zero");
-      }
-
-      if (!"time".equals(routingType)) {
-        throw new SolrException(BAD_REQUEST, "Only time based routing is supported at this time");
-      }
-
-      // Check for invalid timezone
-      TimeZone zone = TimeZoneUtils.parseTimezone(tz);
-
-      // check that the increment is valid date math
-      try {
-        new DateMathParser(zone).parseMath(increment);
-      } catch (ParseException e) {
-        throw new SolrException(BAD_REQUEST,e.getMessage(),e);
-      }
-
-      Instant startTime = parseStart(start, zone);
-
-      // It's too much work to check the routed field against the schema, there seems to be no good way to get
-      // a copy of the schema aside from loading it directly from zookeeper based on the config name, but that
-      // also requires I load solrconfig.xml to check what the value for managedSchemaResourceName is too, (or
-      // discover that managed schema is not turned on and read schema.xml instead... and check for dynamic
-      // field patterns too. As much as it would be nice to validate all inputs it's not worth the effort.
-
-      String initialCollectionName = TimeRoutedAliasUpdateProcessor
-          .formatCollectionNameFromInstant(aliasName, startTime);
-
-      // Create the collection
-      NamedList createResults = new NamedList();
-      RoutedAliasCreateCollectionCmd.createCollectionAndWait(state, createResults, aliasName, aliasMetadata, initialCollectionName, ocmh);
-      validateAllCollectionsExistAndNoDups(Collections.singletonList(initialCollectionName), zkStateReader);
-
-      // Create/update the alias
-      holder.applyModificationAndExportToZk(aliases -> aliases
-          .cloneWithCollectionAlias(aliasName, initialCollectionName)
-          .cloneWithCollectionAliasMetadata(aliasName, aliasMetadata));
+      callCreatePlainAlias(message, aliasName, zkStateReader);
+    } else {
+      callCreateRoutedAlias(message, aliasName, zkStateReader, state);
     }
 
     // Sleep a bit to allow ZooKeeper state propagation.
@@ -192,6 +80,57 @@ public class CreateAliasCmd implements OverseerCollectionMessageHandler.Cmd {
     Thread.sleep(100);
   }
 
+  private void callCreatePlainAlias(ZkNodeProps message, String aliasName, ZkStateReader zkStateReader) {
+    final List<String> canonicalCollectionList = parseCollectionsParameter(message.get("collections"));
+    final String canonicalCollectionsString = StrUtils.join(canonicalCollectionList, ',');
+    validateAllCollectionsExistAndNoDups(canonicalCollectionList, zkStateReader);
+    zkStateReader.aliasesHolder
+        .applyModificationAndExportToZk(aliases -> aliases.cloneWithCollectionAlias(aliasName, canonicalCollectionsString));
+  }
+
+  /**
+   * The v2 API directs that the 'collections' parameter be provided as a JSON array (e.g. ["a", "b"]).  We also
+   * maintain support for the legacy format, a comma-separated list (e.g. a,b).
+   */
+  @SuppressWarnings("unchecked")
+  private List<String> parseCollectionsParameter(Object colls) {
+    if (colls == null) throw new SolrException(BAD_REQUEST, "missing collections param");
+    if (colls instanceof List) return (List<String>) colls;
+    return StrUtils.splitSmart(colls.toString(), ",", true).stream()
+        .map(String::trim)
+        .collect(Collectors.toList());
+  }
+
+  private void callCreateRoutedAlias(ZkNodeProps message, String aliasName, ZkStateReader zkStateReader, ClusterState state) throws Exception {
+    // Validate we got everything we need
+    if (!message.getProperties().keySet().containsAll(TimeRoutedAlias.REQUIRED_ROUTER_PARAMS)) {
+      throw new SolrException(BAD_REQUEST, "A routed alias requires these params: " + TimeRoutedAlias.REQUIRED_ROUTER_PARAMS
+      + " plus some create-collection prefixed ones.");
+    }
+
+    Map<String, String> aliasMetadata = new LinkedHashMap<>();
+    message.getProperties().entrySet().stream()
+        .filter(entry -> TimeRoutedAlias.PARAM_IS_METADATA.test(entry.getKey()))
+        .forEach(entry -> aliasMetadata.put(entry.getKey(), (String) entry.getValue())); // way easier than .collect
+
+    TimeRoutedAlias timeRoutedAlias = new TimeRoutedAlias(aliasName, aliasMetadata); // validates as well
+
+    String start = message.getStr(TimeRoutedAlias.ROUTER_START);
+    Instant startTime = parseStart(start, timeRoutedAlias.getTimeZone());
+
+    String initialCollectionName = TimeRoutedAlias.formatCollectionNameFromInstant(aliasName, startTime);
+
+    // Create the collection
+    NamedList createResults = new NamedList();
+    RoutedAliasCreateCollectionCmd.createCollectionAndWait(state, createResults, aliasName, aliasMetadata, initialCollectionName, ocmh);
+    validateAllCollectionsExistAndNoDups(Collections.singletonList(initialCollectionName), zkStateReader);
+
+    // Create/update the alias
+    zkStateReader.aliasesHolder.applyModificationAndExportToZk(aliases -> aliases
+        .cloneWithCollectionAlias(aliasName, initialCollectionName)
+        .cloneWithCollectionAliasMetadata(aliasName, aliasMetadata));
+  }
+
   private Instant parseStart(String str, TimeZone zone) {
     Instant start = DateMathParser.parseMath(new Date(), str, zone).toInstant();
     checkMilis(start);
@@ -223,17 +162,4 @@ public class CreateAliasCmd implements OverseerCollectionMessageHandler.Cmd {
     }
   }
 
-  /**
-   * The v2 API directs that the 'collections' parameter be provided as a JSON array (e.g. ["a", "b"]).  We also
-   * maintain support for the legacy format, a comma-separated list (e.g. a,b).
-   */
-  @SuppressWarnings("unchecked")
-  private List<String> parseCollectionsParameter(Object colls) {
-    if (colls == null) throw new SolrException(BAD_REQUEST, "missing collections param");
-    if (colls instanceof List) return (List<String>) colls;
-    return StrUtils.splitSmart(colls.toString(), ",", true).stream()
-        .map(String::trim)
-        .collect(Collectors.toList());
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b0d244f6/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAliasCreateCollectionCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAliasCreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAliasCreateCollectionCmd.java
index caf0b01..938ac8b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAliasCreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAliasCreateCollectionCmd.java
@@ -22,7 +22,6 @@ import java.time.Instant;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import java.util.TimeZone;
 
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.OverseerSolrResponse;
@@ -32,22 +31,18 @@ import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.request.LocalSolrQueryRequest;
-import org.apache.solr.update.processor.TimeRoutedAliasUpdateProcessor;
-import org.apache.solr.util.TimeZoneUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.cloud.api.collections.CreateAliasCmd.CREATE_COLLECTION_PREFIX;
 import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.COLL_CONF;
+import static org.apache.solr.cloud.api.collections.TimeRoutedAlias.CREATE_COLLECTION_PREFIX;
+import static org.apache.solr.cloud.api.collections.TimeRoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP;
 import static org.apache.solr.common.params.CommonParams.NAME;
-import static org.apache.solr.update.processor.TimeRoutedAliasUpdateProcessor.ROUTER_FIELD_METADATA;
-import static org.apache.solr.update.processor.TimeRoutedAliasUpdateProcessor.ROUTER_INTERVAL_METADATA;
 
 /**
  * For "routed aliases", creates another collection and adds it to the alias. In some cases it will not
@@ -69,11 +64,6 @@ public class RoutedAliasCreateCollectionCmd implements OverseerCollectionMessage
     this.ocmh = ocmh;
   }
 
-  /* TODO:
-  There are a few classes related to time routed alias processing.  We need to share some logic better.
-   */
-
-
   @Override
   public void call(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception {
     //---- PARSE PRIMARY MESSAGE PARAMS
@@ -92,17 +82,10 @@ public class RoutedAliasCreateCollectionCmd implements OverseerCollectionMessage
       throw newAliasMustExistException(aliasName); // if it did exist, we'd have a non-null map
     }
 
-    String routeField = aliasMetadata.get(ROUTER_FIELD_METADATA);
-    if (routeField == null) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-          "This command only works on time routed aliases.  Expected alias metadata not found.");
-    }
-    String intervalDateMath = aliasMetadata.getOrDefault(ROUTER_INTERVAL_METADATA, "+1DAY");
-    TimeZone intervalTimeZone = TimeZoneUtils.parseTimezone(aliasMetadata.get(CommonParams.TZ));
+    final TimeRoutedAlias timeRoutedAlias = new TimeRoutedAlias(aliasName, aliasMetadata);
 
-    //TODO this is ugly; how can we organize the code related to this feature better?
     final List<Map.Entry<Instant, String>> parsedCollections =
-        TimeRoutedAliasUpdateProcessor.parseCollections(aliasName, aliases, () -> newAliasMustExistException(aliasName));
+        timeRoutedAlias.parseCollections(aliases, () -> newAliasMustExistException(aliasName));
 
     //---- GET MOST RECENT COLL
     final Map.Entry<Instant, String> mostRecentEntry = parsedCollections.get(0);
@@ -127,9 +110,8 @@ public class RoutedAliasCreateCollectionCmd implements OverseerCollectionMessage
     }
 
     //---- COMPUTE NEXT COLLECTION NAME
-    final Instant nextCollTimestamp = TimeRoutedAliasUpdateProcessor.computeNextCollTimestamp(mostRecentCollTimestamp, intervalDateMath, intervalTimeZone);
-    assert nextCollTimestamp.isAfter(mostRecentCollTimestamp);
-    final String createCollName = TimeRoutedAliasUpdateProcessor.formatCollectionNameFromInstant(aliasName, nextCollTimestamp);
+    final Instant nextCollTimestamp = timeRoutedAlias.computeNextCollTimestamp(mostRecentCollTimestamp);
+    final String createCollName = TimeRoutedAlias.formatCollectionNameFromInstant(aliasName, nextCollTimestamp);
 
     //---- CREATE THE COLLECTION
     createCollectionAndWait(clusterState, results, aliasName, aliasMetadata, createCollName, ocmh);
@@ -152,6 +134,11 @@ public class RoutedAliasCreateCollectionCmd implements OverseerCollectionMessage
 
   }
 
+  /**
+   * Creates a collection (for use in a routed alias), waiting for it to be ready before returning.
+   * If the collection already exists then this is not an error.
+   * IMPORTANT: Only call this from an {@link OverseerCollectionMessageHandler.Cmd}.
+   */
   static void createCollectionAndWait(ClusterState clusterState, NamedList results, String aliasName, Map<String, String> aliasMetadata, String createCollName, OverseerCollectionMessageHandler ocmh) throws Exception {
     // Map alias metadata starting with a prefix to a create-collection API request
     final ModifiableSolrParams createReqParams = new ModifiableSolrParams();
@@ -165,7 +152,7 @@ public class RoutedAliasCreateCollectionCmd implements OverseerCollectionMessage
           "We require an explicit " + COLL_CONF );
     }
     createReqParams.set(NAME, createCollName);
-    createReqParams.set("property." + TimeRoutedAliasUpdateProcessor.TIME_PARTITION_ALIAS_NAME_CORE_PROP, aliasName);
+    createReqParams.set("property." + ROUTED_ALIAS_NAME_CORE_PROP, aliasName);
     // a CollectionOperation reads params and produces a message (Map) that is supposed to be sent to the Overseer.
     //   Although we could create the Map without it, there are a fair amount of rules we don't want to reproduce.
     final Map<String, Object> createMsgMap = CollectionsHandler.CollectionOperation.CREATE_OP.execute(
@@ -173,8 +160,18 @@ public class RoutedAliasCreateCollectionCmd implements OverseerCollectionMessage
         null,
         ocmh.overseer.getCoreContainer().getCollectionsHandler());
     createMsgMap.put(Overseer.QUEUE_OPERATION, "create");
-    // Since we are running in the Overseer here, send the message directly to the Overseer CreateCollectionCmd
-    ocmh.commandMap.get(CollectionParams.CollectionAction.CREATE).call(clusterState, new ZkNodeProps(createMsgMap), results);
+
+    try {
+      // Since we are running in the Overseer here, send the message directly to the Overseer CreateCollectionCmd.
+      // note: there's doesn't seem to be any point in locking on the collection name, so we don't. We currently should
+      //   already have a lock on the alias name which should be sufficient.
+      ocmh.commandMap.get(CollectionParams.CollectionAction.CREATE).call(clusterState, new ZkNodeProps(createMsgMap), results);
+    } catch (SolrException e) {
+      // The collection might already exist, and that's okay -- we can adopt it.
+      if (!e.getMessage().contains("collection already exists")) {
+        throw e;
+      }
+    }
 
     CollectionsHandler.waitForActiveCollection(createCollName, null, ocmh.overseer.getCoreContainer(), new OverseerSolrResponse(results));
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b0d244f6/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java
new file mode 100644
index 0000000..c3407aa
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java
@@ -0,0 +1,211 @@
+/*
+ * 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.cloud.api.collections;
+
+import java.text.ParseException;
+import java.time.Instant;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeFormatterBuilder;
+import java.time.temporal.ChronoField;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import com.google.common.base.Objects;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.Aliases;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.params.RequiredSolrParams;
+import org.apache.solr.util.DateMathParser;
+import org.apache.solr.util.TimeZoneUtils;
+
+import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
+import static org.apache.solr.common.params.CommonParams.TZ;
+
+/**
+ * Holds configuration for a routed alias, and some common code and constants.
+ *
+ * @see CreateAliasCmd
+ * @see RoutedAliasCreateCollectionCmd
+ * @see org.apache.solr.update.processor.TimeRoutedAliasUpdateProcessor
+ */
+public class TimeRoutedAlias {
+
+  // These are parameter names to routed alias creation, AND are stored as metadata with the alias.
+  public static final String ROUTER_PREFIX = "router.";
+  public static final String ROUTER_TYPE_NAME = ROUTER_PREFIX + "name";
+  public static final String ROUTER_FIELD = ROUTER_PREFIX + "field";
+  public static final String ROUTER_START = ROUTER_PREFIX + "start";
+  public static final String ROUTER_INTERVAL = ROUTER_PREFIX + "interval";
+  public static final String ROUTER_MAX_FUTURE = ROUTER_PREFIX + "max-future-ms";
+  public static final String CREATE_COLLECTION_PREFIX = "create-collection.";
+  // plus TZ and NAME
+
+  /**
+   * Parameters required for creating a routed alias
+   */
+  public static final List<String> REQUIRED_ROUTER_PARAMS = Collections.unmodifiableList(Arrays.asList(
+      CommonParams.NAME,
+      ROUTER_TYPE_NAME,
+      ROUTER_FIELD,
+      ROUTER_START,
+      ROUTER_INTERVAL));
+
+  /**
+   * Optional parameters for creating a routed alias excluding parameters for collection creation.
+   */
+  public static final List<String> OPTIONAL_ROUTER_PARAMS = Collections.unmodifiableList(Arrays.asList(
+      ROUTER_MAX_FUTURE,
+      TZ)); // kinda special
+
+  static Predicate<String> PARAM_IS_METADATA =
+      key -> key.equals(TZ) ||
+          (key.startsWith(ROUTER_PREFIX) && !key.equals(ROUTER_START)) || //TODO reconsider START special case
+          key.startsWith(CREATE_COLLECTION_PREFIX);
+
+  public static final String ROUTED_ALIAS_NAME_CORE_PROP = "routedAliasName"; // core prop
+
+  // This format must be compatible with collection name limitations
+  private static final DateTimeFormatter DATE_TIME_FORMATTER = new DateTimeFormatterBuilder()
+      .append(DateTimeFormatter.ISO_LOCAL_DATE).appendPattern("[_HH[_mm[_ss]]]") //brackets mean optional
+      .parseDefaulting(ChronoField.HOUR_OF_DAY, 0)
+      .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0)
+      .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0)
+      .toFormatter(Locale.ROOT).withZone(ZoneOffset.UTC); // deliberate -- collection names disregard TZ
+
+  public static Instant parseInstantFromCollectionName(String aliasName, String collection) {
+    final String dateTimePart = collection.substring(aliasName.length() + 1);
+    return DATE_TIME_FORMATTER.parse(dateTimePart, Instant::from);
+  }
+
+  public static String formatCollectionNameFromInstant(String aliasName, Instant timestamp) {
+    String nextCollName = DATE_TIME_FORMATTER.format(timestamp);
+    for (int i = 0; i < 3; i++) { // chop off seconds, minutes, hours
+      if (nextCollName.endsWith("_00")) {
+        nextCollName = nextCollName.substring(0, nextCollName.length()-3);
+      }
+    }
+    assert DATE_TIME_FORMATTER.parse(nextCollName, Instant::from).equals(timestamp);
+    return aliasName + "_" + nextCollName;
+  }
+
+
+  //
+  // Instance data and methods
+  //
+
+  private final String aliasName;
+  private final String routeField;
+  private final long maxFutureMs;
+  private final String intervalDateMath; // ex: +1DAY
+  private final TimeZone timeZone;
+
+  public TimeRoutedAlias(String aliasName, Map<String, String> aliasMetadata) {
+    this.aliasName = aliasName;
+    final MapSolrParams params = new MapSolrParams(aliasMetadata); // for convenience
+    final RequiredSolrParams required = params.required();
+    if (!"time".equals(required.get(ROUTER_TYPE_NAME))) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Only 'time' routed aliases is supported right now.");
+    }
+    routeField = required.get(ROUTER_FIELD);
+    intervalDateMath = required.get(ROUTER_INTERVAL);
+
+    //optional:
+    maxFutureMs = params.getLong(ROUTER_MAX_FUTURE, TimeUnit.MINUTES.toMillis(10));
+    timeZone = TimeZoneUtils.parseTimezone(aliasMetadata.get(CommonParams.TZ));
+
+    // More validation:
+
+    // check that the interval is valid date math
+    try {
+      new DateMathParser(timeZone).parseMath(intervalDateMath);
+    } catch (ParseException e) {
+      throw new SolrException(BAD_REQUEST, "bad " + TimeRoutedAlias.ROUTER_INTERVAL + ", " + e, e);
+    }
+
+    if (maxFutureMs < 0) {
+      throw new SolrException(BAD_REQUEST, ROUTER_MAX_FUTURE + " must be >= 0");
+    }
+  }
+
+  public String getAliasName() {
+    return aliasName;
+  }
+
+  public String getRouteField() {
+    return routeField;
+  }
+
+  public long getMaxFutureMs() {
+    return maxFutureMs;
+  }
+
+  public String getIntervalDateMath() {
+    return intervalDateMath;
+  }
+
+  public TimeZone getTimeZone() {
+    return timeZone;
+  }
+
+  @Override
+  public String toString() {
+    return Objects.toStringHelper(this)
+        .add("aliasName", aliasName)
+        .add("routeField", routeField)
+        .add("maxFutureMs", maxFutureMs)
+        .add("intervalDateMath", intervalDateMath)
+        .add("timeZone", timeZone)
+        .toString();
+  }
+
+  /** Parses the timestamp from the collection list and returns them in reverse sorted order (most recent 1st) */
+  public List<Map.Entry<Instant,String>> parseCollections(Aliases aliases, Supplier<SolrException> aliasNotExist) {
+    final List<String> collections = aliases.getCollectionAliasListMap().get(aliasName);
+    if (collections == null) {
+      throw aliasNotExist.get();
+    }
+    // note: I considered TreeMap but didn't like the log(N) just to grab the most recent when we use it later
+    List<Map.Entry<Instant,String>> result = new ArrayList<>(collections.size());
+    for (String collection : collections) {
+      Instant colStartTime = parseInstantFromCollectionName(aliasName, collection);
+      result.add(new AbstractMap.SimpleImmutableEntry<>(colStartTime, collection));
+    }
+    result.sort((e1, e2) -> e2.getKey().compareTo(e1.getKey())); // reverse sort by key
+    return result;
+  }
+
+  /** Computes the timestamp of the next collection given the timestamp of the one before. */
+  public Instant computeNextCollTimestamp(Instant fromTimestamp) {
+    final Instant nextCollTimestamp =
+        DateMathParser.parseMath(Date.from(fromTimestamp), "NOW" + intervalDateMath, timeZone).toInstant();
+    assert nextCollTimestamp.isAfter(fromTimestamp);
+    return nextCollTimestamp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b0d244f6/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 2e05e07..d4598e9 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
@@ -102,9 +102,9 @@ import static org.apache.solr.client.solrj.response.RequestStatusState.NOT_FOUND
 import static org.apache.solr.client.solrj.response.RequestStatusState.RUNNING;
 import static org.apache.solr.client.solrj.response.RequestStatusState.SUBMITTED;
 import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
-import static org.apache.solr.cloud.api.collections.CreateAliasCmd.CREATE_COLLECTION_PREFIX;
-import static org.apache.solr.cloud.api.collections.CreateAliasCmd.OPTIONAL_ROUTER_PARAMS;
-import static org.apache.solr.cloud.api.collections.CreateAliasCmd.REQUIRED_ROUTER_PARAMS;
+import static org.apache.solr.cloud.api.collections.TimeRoutedAlias.CREATE_COLLECTION_PREFIX;
+import static org.apache.solr.cloud.api.collections.TimeRoutedAlias.OPTIONAL_ROUTER_PARAMS;
+import static org.apache.solr.cloud.api.collections.TimeRoutedAlias.REQUIRED_ROUTER_PARAMS;
 import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.COLL_CONF;
 import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.COLL_PROP_PREFIX;
 import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.CREATE_NODE_SET;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b0d244f6/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
index cbd9eec..1235842 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
@@ -19,31 +19,22 @@ package org.apache.solr.update.processor;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
-import java.text.ParseException;
 import java.time.Instant;
-import java.time.ZoneOffset;
-import java.time.format.DateTimeFormatter;
-import java.time.format.DateTimeFormatterBuilder;
-import java.time.temporal.ChronoField;
-import java.util.AbstractMap;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
-import java.util.Locale;
 import java.util.Map;
-import java.util.TimeZone;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
-import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.api.collections.RoutedAliasCreateCollectionCmd;
+import org.apache.solr.cloud.api.collections.TimeRoutedAlias;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Aliases;
 import org.apache.solr.common.cloud.Replica;
@@ -51,7 +42,6 @@ import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.UpdateParams;
@@ -65,8 +55,6 @@ import org.apache.solr.update.CommitUpdateCommand;
 import org.apache.solr.update.DeleteUpdateCommand;
 import org.apache.solr.update.SolrCmdDistributor;
 import org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
-import org.apache.solr.util.DateMathParser;
-import org.apache.solr.util.TimeZoneUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -89,19 +77,6 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
   //TODO do we make this more generic to others who want to partition collections using something else?
 
   public static final String ALIAS_DISTRIB_UPDATE_PARAM = "alias." + DISTRIB_UPDATE_PARAM; // param
-  public static final String TIME_PARTITION_ALIAS_NAME_CORE_PROP = "timePartitionAliasName"; // core prop
-  // alias metadata:
-  public static final String ROUTER_FIELD_METADATA = "router.field";
-  public static final String ROUTER_MAX_FUTURE_TIME_METADATA = "router.maxFutureMs";
-  public static final String ROUTER_INTERVAL_METADATA = "router.interval";
-
-  // This format must be compatible with collection name limitations
-  public static final DateTimeFormatter DATE_TIME_FORMATTER = new DateTimeFormatterBuilder()
-      .append(DateTimeFormatter.ISO_LOCAL_DATE).appendPattern("[_HH[_mm[_ss]]]") //brackets mean optional
-      .parseDefaulting(ChronoField.HOUR_OF_DAY, 0)
-      .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0)
-      .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0)
-      .toFormatter(Locale.ROOT).withZone(ZoneOffset.UTC);
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
@@ -109,11 +84,8 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
   private static ConcurrentHashMap<String, Semaphore> aliasToSemaphoreMap = new ConcurrentHashMap<>(4);
 
   private final String thisCollection;
-  private final String aliasName;
-  private final String routeField;
-  private final long maxFutureMs;
-  private final String intervalDateMath;
-  private final TimeZone intervalTimeZone;
+
+  private final TimeRoutedAlias timeRoutedAlias;
 
   private final ZkController zkController;
   private final SolrCmdDistributor cmdDistrib;
@@ -125,19 +97,19 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
 
   public static UpdateRequestProcessor wrap(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next) {
     //TODO get from "Collection property"
-    final String timePartitionAliasName = req.getCore().getCoreDescriptor()
-        .getCoreProperty(TIME_PARTITION_ALIAS_NAME_CORE_PROP, null);
+    final String aliasName = req.getCore().getCoreDescriptor()
+        .getCoreProperty(TimeRoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP, null);
     final DistribPhase shardDistribPhase =
         DistribPhase.parseParam(req.getParams().get(DISTRIB_UPDATE_PARAM));
     final DistribPhase aliasDistribPhase =
         DistribPhase.parseParam(req.getParams().get(ALIAS_DISTRIB_UPDATE_PARAM));
-    if (timePartitionAliasName == null || aliasDistribPhase != DistribPhase.NONE || shardDistribPhase != DistribPhase.NONE) {
+    if (aliasName == null || aliasDistribPhase != DistribPhase.NONE || shardDistribPhase != DistribPhase.NONE) {
       // if aliasDistribPhase is not NONE, then there is no further collection routing to be done here.
       //    TODO this may eventually not be true but at the moment it is
       // if shardDistribPhase is not NONE, then the phase is after the scope of this URP
       return next;
     } else {
-      return new TimeRoutedAliasUpdateProcessor(req, rsp, next, timePartitionAliasName, aliasDistribPhase);
+      return new TimeRoutedAliasUpdateProcessor(req, rsp, next, aliasName, aliasDistribPhase);
     }
   }
 
@@ -148,7 +120,6 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
     assert aliasDistribPhase == DistribPhase.NONE;
     final SolrCore core = req.getCore();
     this.thisCollection = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
-    this.aliasName = aliasName;
     CoreContainer cc = core.getCoreContainer();
     zkController = cc.getZkController();
     cmdDistrib = new SolrCmdDistributor(cc.getUpdateShardHandler());
@@ -158,15 +129,11 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
     if (aliasMetadata == null) {
       throw newAliasMustExistException(); // if it did exist, we'd have a non-null map
     }
-    routeField = aliasMetadata.get(ROUTER_FIELD_METADATA);
-    intervalDateMath = aliasMetadata.getOrDefault(ROUTER_INTERVAL_METADATA, "+1DAY");
-    String futureTimeStr = aliasMetadata.get(ROUTER_MAX_FUTURE_TIME_METADATA);
-    if (futureTimeStr != null) {
-      maxFutureMs = Long.parseLong(futureTimeStr);
-    } else {
-      maxFutureMs = TimeUnit.MINUTES.toMillis(10);
+    try {
+      this.timeRoutedAlias = new TimeRoutedAlias(aliasName, aliasMetadata);
+    } catch (Exception e) { // ensure we throw SERVER_ERROR not BAD_REQUEST at this stage
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Routed alias has invalid metadata: " + e, e);
     }
-    intervalTimeZone = TimeZoneUtils.parseTimezone(aliasMetadata.get(CommonParams.TZ));
 
     ModifiableSolrParams outParams = new ModifiableSolrParams(req.getParams());
     // Don't distribute these params; they will be distributed from the local processCommit separately.
@@ -185,9 +152,13 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
     outParamsToLeader = outParams;
   }
 
+  private String getAliasName() {
+    return timeRoutedAlias.getAliasName();
+  }
+
   @Override
   public void processAdd(AddUpdateCommand cmd) throws IOException {
-    final Object routeValue = cmd.getSolrInputDocument().getFieldValue(routeField);
+    final Object routeValue = cmd.getSolrInputDocument().getFieldValue(timeRoutedAlias.getRouteField());
     final Instant routeTimestamp = parseRouteKey(routeValue);
 
     updateParsedCollectionAliases();
@@ -197,7 +168,7 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
 
       if (targetCollection == null) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-            "Doc " + cmd.getPrintableId() + " couldn't be routed with " + routeField + "=" + routeTimestamp);
+            "Doc " + cmd.getPrintableId() + " couldn't be routed with " + timeRoutedAlias.getRouteField() + "=" + routeTimestamp);
       }
 
       // Note: the following rule is tempting but not necessary and is not compatible with
@@ -217,15 +188,15 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
       }
 
       // Check the doc isn't too far in the future
-      final Instant maxFutureTime = Instant.now().plusMillis(maxFutureMs);
+      final Instant maxFutureTime = Instant.now().plusMillis(timeRoutedAlias.getMaxFutureMs());
       if (routeTimestamp.isAfter(maxFutureTime)) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
             "The document's time routed key of " + routeValue + " is too far in the future given " +
-                ROUTER_MAX_FUTURE_TIME_METADATA + "=" + maxFutureMs);
+                TimeRoutedAlias.ROUTER_MAX_FUTURE + "=" + timeRoutedAlias.getMaxFutureMs());
       }
 
       // Create a new collection?
-      final Instant nextCollTimestamp = computeNextCollTimestamp(mostRecentCollTimestamp, intervalDateMath, intervalTimeZone);
+      final Instant nextCollTimestamp = timeRoutedAlias.computeNextCollTimestamp(mostRecentCollTimestamp);
       if (routeTimestamp.isBefore(nextCollTimestamp)) {
         break; // thus we don't need another collection
       }
@@ -251,22 +222,6 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
     }
   }
 
-  /** Computes the timestamp of the next collection given the timestamp of the one before. */
-  public static Instant computeNextCollTimestamp(Instant fromTimestamp, String intervalDateMath, TimeZone intervalTimeZone) {
-    //TODO overload DateMathParser.parseMath to take tz and "now"
-    final DateMathParser dateMathParser = new DateMathParser(intervalTimeZone);
-    dateMathParser.setNow(Date.from(fromTimestamp));
-    final Instant nextCollTimestamp;
-    try {
-      nextCollTimestamp = dateMathParser.parseMath(intervalDateMath).toInstant();
-    } catch (ParseException e) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-          "Invalid Date Math String:'" + intervalDateMath +'\'', e);
-    }
-    assert nextCollTimestamp.isAfter(fromTimestamp);
-    return nextCollTimestamp;
-  }
-
   private Instant parseRouteKey(Object routeKey) {
     final Instant docTimestamp;
     if (routeKey instanceof Instant) {
@@ -290,9 +245,9 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
     final Aliases aliases = zkController.getZkStateReader().getAliases(); // note: might be different from last request
     if (this.parsedCollectionsAliases != aliases) {
       if (this.parsedCollectionsAliases != null) {
-        log.debug("Observing possibly updated alias: {}", aliasName);
+        log.debug("Observing possibly updated alias: {}", getAliasName());
       }
-      this.parsedCollectionsDesc = parseCollections(aliasName, aliases, this::newAliasMustExistException);
+      this.parsedCollectionsDesc = timeRoutedAlias.parseCollections(aliases, this::newAliasMustExistException);
       this.parsedCollectionsAliases = aliases;
       return true;
     }
@@ -321,13 +276,13 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
     //   Alternatively a Lock or CountDownLatch could have been used but they didn't seem
     //   to make it any easier.
 
-    final Semaphore semaphore = aliasToSemaphoreMap.computeIfAbsent(aliasName, n -> new Semaphore(1));
+    final Semaphore semaphore = aliasToSemaphoreMap.computeIfAbsent(getAliasName(), n -> new Semaphore(1));
     if (semaphore.tryAcquire()) {
       try {
         final String operation = CollectionParams.CollectionAction.ROUTEDALIAS_CREATECOLL.toLower();
         Map<String, Object> msg = new HashMap<>();
         msg.put(Overseer.QUEUE_OPERATION, operation);
-        msg.put(CollectionParams.NAME, aliasName);
+        msg.put(CollectionParams.NAME, getAliasName());
         msg.put(RoutedAliasCreateCollectionCmd.IF_MOST_RECENT_COLL_NAME, mostRecentCollName);
         SolrQueryResponse rsp = new SolrQueryResponse();
         try {
@@ -373,44 +328,12 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
     }
   }
 
-  /** Parses the timestamp from the collection list and returns them in reverse sorted order (most recent 1st) */
-  public static List<Map.Entry<Instant,String>> parseCollections(String aliasName, Aliases aliases, Supplier<SolrException> aliasNotExist) {
-    final List<String> collections = aliases.getCollectionAliasListMap().get(aliasName);
-    if (collections == null) {
-      throw aliasNotExist.get();
-    }
-    // note: I considered TreeMap but didn't like the log(N) just to grab the most recent when we use it later
-    List<Map.Entry<Instant,String>> result = new ArrayList<>(collections.size());
-    for (String collection : collections) {
-      Instant colStartTime = parseInstantFromCollectionName(aliasName, collection);
-      result.add(new AbstractMap.SimpleImmutableEntry<>(colStartTime, collection));
-    }
-    result.sort((e1, e2) -> e2.getKey().compareTo(e1.getKey())); // reverse sort by key
-    return result;
-  }
-
   private SolrException newAliasMustExistException() {
     throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE,
-        "Collection " + thisCollection + " created for use with alias " + aliasName + " which doesn't exist anymore." +
+        "Collection " + thisCollection + " created for use with alias " + getAliasName() + " which doesn't exist anymore." +
             " You cannot write to this unless the alias exists.");
   }
 
-  static Instant parseInstantFromCollectionName(String aliasName, String collection) {
-    final String dateTimePart = collection.substring(aliasName.length() + 1);
-    return DATE_TIME_FORMATTER.parse(dateTimePart, Instant::from);
-  }
-
-  public static String formatCollectionNameFromInstant(String aliasName, Instant timestamp) {
-    String nextCollName = TimeRoutedAliasUpdateProcessor.DATE_TIME_FORMATTER.format(timestamp);
-    for (int i = 0; i < 3; i++) { // chop off seconds, minutes, hours
-      if (nextCollName.endsWith("_00")) {
-        nextCollName = nextCollName.substring(0, nextCollName.length()-3);
-      }
-    }
-    assert TimeRoutedAliasUpdateProcessor.DATE_TIME_FORMATTER.parse(nextCollName, Instant::from).equals(timestamp);
-    return aliasName + "_" + nextCollName;
-  }
-
   @Override
   public void processDelete(DeleteUpdateCommand cmd) throws IOException {
     final List<SolrCmdDistributor.Node> nodes = lookupShardLeadersOfCollections();
@@ -453,7 +376,7 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
 
   private List<SolrCmdDistributor.Node> lookupShardLeadersOfCollections() {
     final Aliases aliases = zkController.getZkStateReader().getAliases();
-    List<String> collections = aliases.getCollectionAliasListMap().get(aliasName);
+    List<String> collections = aliases.getCollectionAliasListMap().get(getAliasName());
     if (collections == null) {
       throw newAliasMustExistException();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b0d244f6/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
index 3dce4aa..8384e29 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
@@ -38,13 +38,13 @@ import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.api.collections.TimeRoutedAlias;
 import org.apache.solr.common.cloud.Aliases;
 import org.apache.solr.common.cloud.CompositeIdRouter;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.ImplicitDocRouter;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.update.processor.TimeRoutedAliasUpdateProcessor;
 import org.apache.solr.util.DateMathParser;
 import org.junit.After;
 import org.junit.Before;
@@ -139,9 +139,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
     assertSuccess(post);
 
     Date startDate = DateMathParser.parseMath(new Date(), "NOW/DAY");
-    String initialCollectionName = TimeRoutedAliasUpdateProcessor
-        .formatCollectionNameFromInstant(aliasName, startDate.toInstant()
-        );
+    String initialCollectionName = TimeRoutedAlias.formatCollectionNameFromInstant(aliasName, startDate.toInstant());
     // small chance could fail due to "NOW"; see above
     assertCollectionExists(initialCollectionName);
 
@@ -198,9 +196,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
         "&create-collection.replicationFactor=2");
     assertSuccess(get);
 
-    String initialCollectionName = TimeRoutedAliasUpdateProcessor
-        .formatCollectionNameFromInstant(aliasName, start
-        );
+    String initialCollectionName = TimeRoutedAlias.formatCollectionNameFromInstant(aliasName, start);
     assertCollectionExists(initialCollectionName);
 
     // Test created collection:
@@ -271,7 +267,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
         "&router.interval=%2B30MINUTE" +
         "&create-collection.collection.configName=_default" +
         "&create-collection.numShards=1");
-    assertFailure(get, "Only time based routing is supported");
+    assertFailure(get, "Only 'time' routed aliases is supported right now");
   }
 
   @Test
@@ -320,7 +316,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
         "&router.max-future-ms=-60000" + // bad: negative
         "&create-collection.collection.configName=_default" +
         "&create-collection.numShards=1");
-    assertFailure(get, "router.max-future-ms must be a valid long integer");
+    assertFailure(get, "must be >= 0");
   }
   @Test
   public void testUnParseableFutureFails() throws Exception {
@@ -333,10 +329,10 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
         "&router.name=time" +
         "&router.start=2018-01-15T00:00:00Z" +
         "&router.interval=%2B30MINUTE" +
-        "&router.max-future-ms=SixtyThousandMiliseconds" + // bad
+        "&router.max-future-ms=SixtyThousandMilliseconds" + // bad
         "&create-collection.collection.configName=_default" +
         "&create-collection.numShards=1");
-    assertFailure(get, "router.max-future-ms must be a valid long integer");
+    assertFailure(get, "SixtyThousandMilliseconds"); //TODO improve SolrParams.getLong
   }
 
   private void assertSuccess(HttpUriRequest msg) throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b0d244f6/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
index 64f5555..7774c70 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
@@ -41,10 +41,10 @@ import org.apache.solr.client.solrj.response.FieldStatsInfo;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.cloud.api.collections.TimeRoutedAlias;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.request.SolrQueryRequest;
@@ -94,7 +94,6 @@ public class TimeRoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
     // manipulate the config...
 
         String conf = "{" +
-            "  'set-user-property' : {'timePartitionAliasName':'" + alias + "'}," + // no data driven
             "  'set-user-property' : {'update.autoCreateFields':false}," + // no data driven
             "  'add-updateprocessor' : {" +
             "    'name':'tolerant', 'class':'solr.TolerantUpdateProcessorFactory'" +
@@ -122,11 +121,12 @@ public class TimeRoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
             .contains(configName)
     );
 
-    // start with one collection and an alias for it
+    // Start with one collection manually created (and use higher numShards & replicas than we'll use for others)
+    //  This tests we may pre-create the collection and it's acceptable.
     final String col23rd = alias + "_2017-10-23";
     CollectionAdminRequest.createCollection(col23rd, configName, 2, 2)
         .setMaxShardsPerNode(2)
-        .withProperty(TimeRoutedAliasUpdateProcessor.TIME_PARTITION_ALIAS_NAME_CORE_PROP, alias)
+        .withProperty(TimeRoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP, alias)
         .process(solrClient);
 
     List<String> retrievedConfigSetNames = new ConfigSetAdminRequest.List().process(solrClient).getConfigSets();
@@ -135,16 +135,10 @@ public class TimeRoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
         expectedConfigSetNames.size() == retrievedConfigSetNames.size());
     assertTrue("ConfigNames should be :" + expectedConfigSetNames, expectedConfigSetNames.containsAll(retrievedConfigSetNames) && retrievedConfigSetNames.containsAll(expectedConfigSetNames));
 
-    CollectionAdminRequest.createAlias(alias, col23rd).process(solrClient);
-    //TODO use SOLR-11617 client API to set alias metadata
-    final ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
-
-    zkStateReader.aliasesHolder.applyModificationAndExportToZk(a ->
-        a.cloneWithCollectionAliasMetadata(alias, TimeRoutedAliasUpdateProcessor.ROUTER_FIELD_METADATA, timeField)
-        .cloneWithCollectionAliasMetadata(alias, "create-collection.collection.configName", configName)
-        .cloneWithCollectionAliasMetadata(alias, "create-collection.numShards", "1")
-        .cloneWithCollectionAliasMetadata(alias, "create-collection.replicationFactor", "1")
-        .cloneWithCollectionAliasMetadata(alias, "router.interval", "+1DAY"));
+    CollectionAdminRequest.createTimeRoutedAlias(alias, "2017-10-23T00:00:00Z", "+1DAY", timeField,
+        CollectionAdminRequest.createCollection("_unused_", configName, 1, 1)
+            .setMaxShardsPerNode(2))
+        .process(solrClient);
 
     // now we index a document
     assertUpdateResponse(solrClient.add(alias, newDoc(Instant.parse("2017-10-23T00:00:00Z"))));
@@ -158,12 +152,12 @@ public class TimeRoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
     // a document which is too far into the future
     testFailedDocument(Instant.now().plus(30, ChronoUnit.MINUTES), "too far in the future");
 
-    // add another collection, add to alias  (soonest comes first)
+    // add another collection with the precise name we expect, but don't add to alias explicitly.  When we add a document
+    //   destined for this collection, Solr will see it already exists and add it to the alias.
     final String col24th = alias + "_2017-10-24";
     CollectionAdminRequest.createCollection(col24th, configName,  1, 1) // more shards and replicas now
-        .withProperty("timePartitionAliasName", alias)
+        .withProperty(TimeRoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP, alias)
         .process(solrClient);
-    CollectionAdminRequest.createAlias(alias, col24th + "," + col23rd).process(solrClient);
 
     // index 3 documents in a random fashion
     addDocsAndCommit(
@@ -305,7 +299,7 @@ public class TimeRoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
     int totalNumFound = 0;
     Instant colEndInstant = null; // exclusive end
     for (String col : cols) { // ASSUMPTION: reverse sorted order
-      final Instant colStartInstant = TimeRoutedAliasUpdateProcessor.parseInstantFromCollectionName(alias, col);
+      final Instant colStartInstant = TimeRoutedAlias.parseInstantFromCollectionName(alias, col);
       final QueryResponse colStatsResp = solrClient.query(col, params(
           "q", "*:*",
           "rows", "0",
@@ -336,13 +330,13 @@ public class TimeRoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
   @Test
   public void testParse() {
     assertEquals(Instant.parse("2017-10-02T03:04:05Z"),
-      TimeRoutedAliasUpdateProcessor.parseInstantFromCollectionName(alias, alias + "_2017-10-02_03_04_05"));
+      TimeRoutedAlias.parseInstantFromCollectionName(alias, alias + "_2017-10-02_03_04_05"));
     assertEquals(Instant.parse("2017-10-02T03:04:00Z"),
-      TimeRoutedAliasUpdateProcessor.parseInstantFromCollectionName(alias, alias + "_2017-10-02_03_04"));
+      TimeRoutedAlias.parseInstantFromCollectionName(alias, alias + "_2017-10-02_03_04"));
     assertEquals(Instant.parse("2017-10-02T03:00:00Z"),
-      TimeRoutedAliasUpdateProcessor.parseInstantFromCollectionName(alias, alias + "_2017-10-02_03"));
+      TimeRoutedAlias.parseInstantFromCollectionName(alias, alias + "_2017-10-02_03"));
     assertEquals(Instant.parse("2017-10-02T00:00:00Z"),
-      TimeRoutedAliasUpdateProcessor.parseInstantFromCollectionName(alias, alias + "_2017-10-02"));
+      TimeRoutedAlias.parseInstantFromCollectionName(alias, alias + "_2017-10-02"));
   }
 
   public static class IncrementURPFactory extends FieldMutatingUpdateProcessorFactory {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b0d244f6/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 44eeef5..e2e3540 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
@@ -1381,11 +1381,11 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     // to allow this stuff not to be duplicated. (this is pasted from CreateAliasCmd.java), however I think
     // a comprehensive cleanup of this for all the requests in this class should be done as a separate ticket.
 
-    public static final String ROUTING_TYPE = "router.name";
-    public static final String ROUTING_FIELD = "router.field";
-    public static final String ROUTING_START = "router.start";
-    public static final String ROUTING_INCREMENT = "router.interval";
-    public static final String ROUTING_MAX_FUTURE = "router.max-future-ms";
+    public static final String ROUTER_TYPE_NAME = "router.name";
+    public static final String ROUTER_FIELD = "router.field";
+    public static final String ROUTER_START = "router.start";
+    public static final String ROUTER_INTERVAL = "router.interval";
+    public static final String ROUTER_MAX_FUTURE = "router.max-future-ms";
 
     private final String aliasName;
     private final String routerField;
@@ -1422,15 +1422,15 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     public SolrParams getParams() {
       ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
       params.add(CommonParams.NAME, aliasName);
-      params.add(ROUTING_TYPE, "time");
-      params.add(ROUTING_FIELD, routerField);
-      params.add(ROUTING_START, start);
-      params.add(ROUTING_INCREMENT, interval);
+      params.add(ROUTER_TYPE_NAME, "time");
+      params.add(ROUTER_FIELD, routerField);
+      params.add(ROUTER_START, start);
+      params.add(ROUTER_INTERVAL, interval);
       if (tz != null) {
         params.add(CommonParams.TZ, tz.getID());
       }
       if (maxFutureMs != null) {
-        params.add(ROUTING_MAX_FUTURE, ""+maxFutureMs);
+        params.add(ROUTER_MAX_FUTURE, ""+maxFutureMs);
       }
 
       // merge the above with collectionParams.  Above takes precedence.


[2/4] lucene-solr:master: SOLR-11722: New CREATEROUTEDALIAS cluster command for time routed aliases. (a refactoring of some of the related parts will follow in next commit)

Posted by ds...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1828a56/solr/solrj/src/resources/apispec/collections.Commands.json
----------------------------------------------------------------------
diff --git a/solr/solrj/src/resources/apispec/collections.Commands.json b/solr/solrj/src/resources/apispec/collections.Commands.json
index 294b163..17172cc 100644
--- a/solr/solrj/src/resources/apispec/collections.Commands.json
+++ b/solr/solrj/src/resources/apispec/collections.Commands.json
@@ -152,6 +152,63 @@
         "collections"
       ]
     },
+    "create-routed-alias": {
+      "type": "object",
+      "documentation": "https://lucene.apache.org/solr/guide/collections-api.html#createalias",
+      "description": "Creates an alias that manages a series of time-partitioned collections.",
+      "properties": {
+        "name": {
+          "type": "string",
+          "description": "The alias name to be created."
+        },
+        "router" : {
+          "type":"object",
+          "documentation": "https://lucene.apache.org/solr/guide/collections-api.html#createalias",
+          "description":"routing specific attributes",
+          "properties" : {
+            "name" : {
+              "type" : "string",
+              "description": "The type of routing to perform. Currently only 'time' is supported, and it's required."
+            },
+            "field" : {
+              "type": "string",
+              "description": "The date field name in incoming documents that is consulted to decide which collection the document should be routed to."
+            },
+            "start": {
+              "type": "string",
+              "description": "The earliest date/time in a document that may be indexed into this alias. Documents with values less than this will return an error. For time based routing this may be a date math expression."
+            },
+            "interval" : {
+              "type": "string",
+              "description": "A specification of the width of the interval for each partition collection. For time based routing this should be a date math expression fragment starting with the + character."
+            },
+            "max-future-ms": {
+              "type": "integer",
+              "description":"How many milliseconds into the future to accept document. Documents with a value in router.field that is greater than now() + max-future-ms will be rejected to avoid provisioning too much resources."
+            }
+          }
+        },
+        "TZ": {
+          "type": "string",
+          "description": "Optional timezone for use with any date math that may exist in other parameters.  Defaults to UTC."
+        },
+        "create-collection": {
+          "type": "object",
+          "documentation": "https://lucene.apache.org/solr/guide/collections-api.html#create",
+          "description": "The settings to use to create a collection for each new time partition. Most options from the collection create command are available, except for 'name', 'async' and 'waitForFinalState'.",
+          "additionalProperties": true
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously."
+        }
+      },
+      "required": [
+        "name",
+        "router",
+        "create-collection"
+      ]
+    },
     "delete-alias": {
       "documentation": "https://lucene.apache.org/solr/guide/collections-api.html#deletealias",
       "description": "Deletes a collection alias",
@@ -166,7 +223,9 @@
           "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously."
         }
       },
-      "required":["name"]
+      "required": [
+        "name"
+      ]
     },
     "backup-collection": {
       "documentation": "https://lucene.apache.org/solr/guide/collections-api.html#backup",