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/03/07 20:17:58 UTC

lucene-solr:branch_7x: SOLR-11722: Merge CREATEDROUTEDALIAS command into CREATEALIAS. Only an API & docs change; the implementation was already there.

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x dc9b511f7 -> ae97500ec


SOLR-11722: Merge CREATEDROUTEDALIAS command into CREATEALIAS.  Only
an API & docs change; the implementation was already there.

(cherry picked from commit ee1c742)


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

Branch: refs/heads/branch_7x
Commit: ae97500ec2198280e4dc96bd8b5d64b65034cd5c
Parents: dc9b511
Author: David Smiley <ds...@apache.org>
Authored: Wed Mar 7 15:16:02 2018 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Wed Mar 7 15:17:34 2018 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   4 +-
 .../OverseerCollectionMessageHandler.java       |   1 -
 .../solr/handler/admin/CollectionsHandler.java  |  20 ++-
 .../solr/cloud/CreateRoutedAliasTest.java       |  39 ++++-
 solr/solr-ref-guide/src/collections-api.adoc    | 163 +++++++++----------
 .../solrj/request/CollectionAdminRequest.java   |   2 +-
 .../solrj/request/CollectionApiMapping.java     |   6 -
 .../solr/common/params/CollectionParams.java    |   1 -
 .../resources/apispec/collections.Commands.json |  29 +---
 9 files changed, 132 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae97500e/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 61fad8f..f36dba5 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -97,8 +97,8 @@ New Features
 
 * SOLR-11648: A new admin UI to display and execute suggestions (Apoorv Bhawsar , noble)
 
-* SOLR-11722: New CREATEROUTEDALIAS SolrCloud command to create a "time routed alias" over a series of collections
-  partitioned by time. (Gus Heck, David Smiley)
+* SOLR-11722: Added "time routed alias" creation support to the CREATEALIAS command.  It's for managing multiple
+  collections partitioned by time. (Gus Heck, David Smiley)
 
 * SOLR-11782: Refactor LatchWatcher.await to protect against spurious wakeup (Tomás Fernández Löbbe, David Smiley, Dawid Weiss)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae97500e/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 074e932..d1ef0bd 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,7 +226,6 @@ 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(MODIFYALIAS, new ModifyAliasCmd(this))
         .put(MAINTAINROUTEDALIAS, new MaintainRoutedAliasCmd(this))

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae97500e/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 70fb504..b7d7b28 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
@@ -519,21 +519,27 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     }),
 
     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);
+      String collections = req.getParams().get("collections");
+      Map<String, Object> result = req.getParams().getAll(null, REQUIRED_ROUTER_PARAMS);
       req.getParams().getAll(result, OPTIONAL_ROUTER_PARAMS);
+      if (collections != null) {
+        if (result.size() > 1) { // (NAME should be there, and if it's not we will fail below)
+          throw new SolrException(BAD_REQUEST, "Collections cannot be specified when creating a time routed alias.");
+        }
+        // regular alias creation...
+        return req.getParams().required().getAll(null, NAME, "collections");
+      }
+
+      // Ok so we are creating a time routed alias from here
 
+      // for validation....
+      req.getParams().required().getAll(null, REQUIRED_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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae97500e/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 2742a43..78dc476 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
@@ -109,7 +109,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
     //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" +
+        "  \"create-alias\" : {\n" +
         "    \"name\": \"" + aliasName + "\",\n" +
         "    \"router\" : {\n" +
         "      \"name\": \"time\",\n" +
@@ -184,7 +184,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
     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" +
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEALIAS" +
         "&wt=xml" +
         "&name=" + aliasName +
         "&router.field=evt_dt" +
@@ -241,9 +241,32 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
   }
 
   @Test
+  public void testCollectionNamesMustBeAbsent() throws Exception {
+    CollectionAdminRequest.createCollection("collection1meta", "_default", 2, 1).process(cluster.getSolrClient());
+    CollectionAdminRequest.createCollection("collection2meta", "_default", 1, 1).process(cluster.getSolrClient());
+    waitForState("Expected collection1 to be created with 2 shards and 1 replica", "collection1meta", clusterShape(2, 1));
+    waitForState("Expected collection2 to be created with 1 shard and 1 replica", "collection2meta", clusterShape(1, 1));
+    ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
+    zkStateReader.createClusterStateWatchersAndUpdate();
+
+    final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEALIAS" +
+        "&wt=json" +
+        "&name=" + getTestName() +
+        "&collections=collection1meta,collection2meta" +
+        "&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, "Collections cannot be specified");
+  }
+
+  @Test
   public void testAliasNameMustBeValid() throws Exception {
     final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
-    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEROUTEDALIAS" +
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEALIAS" +
         "&wt=json" +
         "&name=735741!45" +  // ! not allowed
         "&router.field=evt_dt" +
@@ -259,7 +282,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
   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" +
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEALIAS" +
         "&wt=json" +
         "&name=" + aliasName +
         "&router.field=evt_dt" +
@@ -275,7 +298,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
   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" +
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEALIAS" +
         "&wt=json" +
         "&name=" + aliasName +
         "&router.field=evt_dt" +
@@ -291,7 +314,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
   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" +
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEALIAS" +
         "&wt=json" +
         "&name=" + aliasName +
         "&router.field=evt_dt" +
@@ -307,7 +330,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
   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" +
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEALIAS" +
         "&wt=json" +
         "&name=" + aliasName +
         "&router.field=evt_dt" +
@@ -323,7 +346,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
   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" +
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEALIAS" +
         "&wt=json" +
         "&name=" + aliasName +
         "&router.field=evt_dt" +

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae97500e/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 77f054a..022efe4 100644
--- a/solr/solr-ref-guide/src/collections-api.adoc
+++ b/solr/solr-ref-guide/src/collections-api.adoc
@@ -476,76 +476,45 @@ http://localhost:8983/solr/admin/collections?action=DELETESHARD&collection=anoth
 [[createalias]]
 == CREATEALIAS: Create or Modify an Alias for a Collection
 
-The `CREATEALIAS` action will create a new alias pointing to one or more collections. If an alias by the same name already exists, this action will replace the existing alias, effectively acting like an atomic "MOVE" command.
+The `CREATEALIAS` action will create a new alias pointing to one or more collections. If an alias by the same name
+already exists, this action will replace the existing alias, effectively acting like an atomic "MOVE" command. Aliases
+come in 2 flavors: routed and non-routed.
 
-`/admin/collections?action=CREATEALIAS&name=_name_&collections=_collectionlist_`
-
-=== CREATEALIAS Parameters
-
-`name`::
-The alias name to be created. This parameter is required.
-
-`collections`::
-A comma-separated list of collections to be aliased. The collections must already exist in the cluster. This parameter is required.
-
-`async`::
-Request ID to track this action which will be <<Asynchronous Calls,processed asynchronously>>.
-
-=== CREATEALIAS 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, you can look in the Solr Admin UI, under the Cloud section and find the `aliases.json` file.
-
-=== Examples using CREATEALIAS
-
-*Input*
-
-Create an alias named "testalias" and link it to the collections named "anotherCollection" and "testCollection".
+Non-routed aliases are simpler and can serve to rename a collection or to distribute queries across several collections.
+While it is possible to send updates to an alias spanning multiple collections, non-routed alias have no logic for
+distributing documents among the referenced collections so all updates will go to the first collection in the list.
 
-[source,text]
-----
-http://localhost:8983/solr/admin/collections?action=CREATEALIAS&name=testalias&collections=anotherCollection,testCollection&wt=xml
-----
-
-*Output*
-
-[source,xml]
-----
-<response>
-  <lst name="responseHeader">
-    <int name="status">0</int>
-    <int name="QTime">122</int>
-  </lst>
-</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.
+`/admin/collections?action=CREATEALIAS&name=_name_&collections=_collectionlist_`
 
-NOTE: Presently only partitioning of time-based data is available, though other schemes may become available in
-the future.
+Routed aliases are more complex to configure, but include logic for automatically inspecting a field on the document
+and using the value in that field to select a destination collection. Additionally the routed alias automates the
+partitioning of data across a series of collections by creating new collections periodically. 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 usually the alias will be used.
+These collections are created automatically on the fly as new data arrives based on the parameters supplied in this
+command. Also for very high volume use cases or for use cases requiring only a limited span of data to be retained,
+collections older than a given age can be deleted. This delete of old collections only occurs if new documents are
+sent to the alias.
+
+*NOTE:* Presently this is only supported for temporal fields stored as a
+<<field-types-included-with-solr.adoc#field-types-included-with-solr,DatePointField or TrieDateField>> type. Other
+well ordered field types may be added in future versions.
 
 [source,text]
 ----
-admin/collections?action=CREATEROUTEDALIAS&name=timedata&router.start=NOW/DAY&router.field=evt_dt&router.name=time&router.interval=%2B1DAY&router.maxFutureMs=3600000&create-collection.collection.configName=myConfig&create-collection.numShards=2
+localhost:8983/solr/admin/collections?action=CREATEALIAS&name=timedata&router.start=NOW/DAY&router.field=evt_dt&router.name=time&router.interval=%2B1DAY&router.maxFutureMs=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.maxFutureMs`). 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.`
+If run on Jan 15, 2018 The above will create an alias named timedata, that contains collections with names prefixed with
+`timedata` and an initial collection named `timedata_2018_01_15` will be created immediately. 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.maxFutureMs). 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
@@ -554,25 +523,38 @@ 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
+*NOTE:* Presently only updates are routed and queries are distributed to all collections in the alias, but future
+features may enable routing of the query to the single appropriate collection based on a special parameter or perhaps
+a filter on the routed field.
+
+=== CREATEALIAS 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.
+The alias name to be created. This parameter is required. If the alias is to be routed it also functions
+as a prefix for the names of the dependent collections that will be created. It must therefore adhere to normal
+requirements for collection naming.
+
+`async`::
+Request ID to track this action which will be <<Asynchronous Calls,processed asynchronously>>.
+
+==== Non-Routed Alias Parameters
+
+`collections`::
+A comma-separated list of collections to be aliased. The collections must already exist in the cluster.
+This parameter signals the creation of a simple (non-routed) alias. If it is present all routing parameters are
+prohibited. If routing parameters are present this parameter is prohibited.
+
+==== Routed Alias Parameters
 
 `router.start`::
 The start date/time of data for this time routed alias in Solr's standard date/time format (i.e., ISO-8601 or "NOW"
 optionally with <<working-with-dates.adoc#date-math,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` than 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 parameter is required.
+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 parameter is required.
 
 `TZ`::
 The timezone to be used when evaluating any date math in router.start or router.interval. This is equivalent to the
@@ -621,17 +603,34 @@ are identical in requirements and naming except that we insist that the configse
 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>>.
+=== CREATEALIAS 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, you can look in the Solr Admin UI, under the Cloud section and find the
+`aliases.json` file. The initial collection for routed aliases should also be visible in various parts of the admin UI.
 
-=== CREATEROUTEDALIAS Response
+=== Examples using CREATEALIAS
 
-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.
+*Input*
 
-=== Examples using CREATEROUTEDALIAS
+Create an alias named "testalias" and link it to the collections named "anotherCollection" and "testCollection".
+
+[source,text]
+----
+http://localhost:8983/solr/admin/collections?action=CREATEALIAS&name=testalias&collections=anotherCollection,testCollection&wt=xml
+----
+
+*Output*
+
+[source,xml]
+----
+<response>
+  <lst name="responseHeader">
+    <int name="status">0</int>
+    <int name="QTime">122</int>
+  </lst>
+</response>
+----
 
 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)
@@ -642,7 +641,7 @@ partiton is to be rejected and collections are created using a config set named
 
 [source,text]
 ----
-http://localhost:8983/solr/admin/collections?action=CREATEROUTEDALIAS&name=myTimeData&router.start=NOW/DAY&router.field=evt_dt&router.name=time&router.interval=%2B1DAY&router.maxFutureMs=3600000&create-collection.collection.configName=myConfig&create-collection.numShards=2
+http://localhost:8983/solr/admin/collections?action=CREATEALIAS&name=myTimeData&router.start=NOW/DAY&router.field=evt_dt&router.name=time&router.interval=%2B1DAY&router.maxFutureMs=3600000&create-collection.collection.configName=myConfig&create-collection.numShards=2
 ----
 
 *Output*

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae97500e/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 b7bbf2f..1c4750e 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
@@ -1434,7 +1434,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     private final Create createCollTemplate;
 
     public CreateTimeRoutedAlias(String aliasName, String routerField, String start, String interval, Create createCollTemplate) {
-      super(CollectionAction.CREATEROUTEDALIAS);
+      super(CollectionAction.CREATEALIAS);
       this.aliasName = aliasName;
       this.start = start;
       this.interval = interval;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae97500e/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 d950f11..f3d5ca5 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
@@ -116,12 +116,6 @@ public class CollectionApiMapping {
         POST,
         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()

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae97500e/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 cbf14cc..f9f3df8 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,7 +76,6 @@ public interface CollectionParams {
     RELOAD(true, LockLevel.COLLECTION),
     SYNCSHARD(true, LockLevel.SHARD),
     CREATEALIAS(true, LockLevel.COLLECTION),
-    CREATEROUTEDALIAS(true, LockLevel.COLLECTION),
     DELETEALIAS(true, LockLevel.COLLECTION),
     MODIFYALIAS(true, LockLevel.COLLECTION),
     LISTALIASES(false, LockLevel.NONE),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae97500e/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 dfdfd52..245a224 100644
--- a/solr/solrj/src/resources/apispec/collections.Commands.json
+++ b/solr/solrj/src/resources/apispec/collections.Commands.json
@@ -128,7 +128,7 @@
     },
     "create-alias": {
       "documentation": "https://lucene.apache.org/solr/guide/collections-api.html#createalias",
-      "description": "Allows one or more collections to be known by another name. If this command is used on an existing alias, the existing alias will be replaced with the new collection details.",
+      "description": "Allows one or more collections to be known by another name, or creates an alias that manages a series of time-partitioned collections.. If this command is used on an existing alias, the existing alias will be replaced with the new collection details.",
       "type": "object",
       "properties": {
         "name": {
@@ -137,30 +137,11 @@
         },
         "collections": {
           "type": "array",
-          "description": "The list of collections to be known as this alias.",
+          "description": "The list of collections to be known as this alias. Incompatible with any of the routing parameters. Either this parameter or a complete set of routing parameters is required",
           "items": {
             "type": "string"
           }
         },
-        "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",
-        "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",
@@ -185,7 +166,7 @@
             "maxFutureMs": {
               "type": "integer",
               "description":"How many milliseconds into the future to accept document. Documents with a value in router.field that is greater than now() + maxFutureMs will be rejected to avoid provisioning too much resources."
-            }
+            },
             "autoDeleteAge": {
               "type": "string",
               "description": "A date math expressions yielding a time in the past. Collections covering a period of time entirely before this age will be automatically deleted."
@@ -208,9 +189,7 @@
         }
       },
       "required": [
-        "name",
-        "router",
-        "create-collection"
+        "name"
       ]
     },
     "delete-alias": {