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:54 UTC

[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)

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),