You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by gu...@apache.org on 2019/06/20 18:32:39 UTC

[lucene-solr] branch branch_8x updated: SOLR-13419 - Better infix for TRA collection names

This is an automated email from the ASF dual-hosted git repository.

gus pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 45b000e  SOLR-13419 - Better infix for TRA collection names
45b000e is described below

commit 45b000e7ae08b04bdf369f8f70acd6698809bbde
Author: Gus Heck <gu...@apache.org>
AuthorDate: Thu Jun 20 14:32:14 2019 -0400

    SOLR-13419 - Better infix for TRA collection names
---
 solr/CHANGES.txt                                   |   5 +
 .../solr/cloud/api/collections/AliasCmd.java       |   4 +
 .../collections/MaintainTimeRoutedAliasCmd.java    |  27 +-
 .../solr/cloud/api/collections/RoutedAlias.java    |  19 +-
 .../cloud/api/collections/TimeRoutedAlias.java     |  14 +-
 .../apache/solr/cloud/CreateRoutedAliasTest.java   |   8 +-
 .../processor/RoutedAliasUpdateProcessorTest.java  |   5 +-
 .../TimeRoutedAliasUpdateProcessorTest.java        | 346 ++++++++++++++++++---
 .../solrj/impl/BaseHttpClusterStateProvider.java   |   6 +-
 .../solrj/request/CollectionAdminRequest.java      |  32 +-
 10 files changed, 395 insertions(+), 71 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 073eb7b..ae09090 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -49,6 +49,11 @@ Upgrade Notes
   not add a property to core.properties anymoore so any external code that inspected core.properties will not find the
   'routedAliasName' key in new cores belonging to routed aliases.
 
+* SOLR-13419: Time Routed Aliases now have a '__TRA__' infix preceding the timestamp portion of the collection names.
+  collections with the old format will not be altered and will continue to work, but any new TRA's or new collections
+  for existing TRA's will use the new format. Solr will handle this invisibly, but any external code that attempted to
+  predict collection names in TRA's will probably need adjustment.
+
 New Features
 ----------------------
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java
index 752bc5b..03cc404 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java
@@ -17,6 +17,7 @@
 
 package org.apache.solr.cloud.api.collections;
 
+import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -34,6 +35,8 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.request.LocalSolrQueryRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.cloud.api.collections.RoutedAlias.CREATE_COLLECTION_PREFIX;
 import static org.apache.solr.cloud.api.collections.RoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP;
@@ -47,6 +50,7 @@ import static org.apache.solr.common.params.CommonParams.NAME;
  */
 abstract class AliasCmd implements OverseerCollectionMessageHandler.Cmd {
 
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   /**
    * Creates a collection (for use in a routed alias), waiting for it to be ready before returning.
    * If the collection already exists then this is not an error.<p>
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainTimeRoutedAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainTimeRoutedAliasCmd.java
index c1a015e..8fdf769 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainTimeRoutedAliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainTimeRoutedAliasCmd.java
@@ -20,11 +20,14 @@ package org.apache.solr.cloud.api.collections;
 import java.lang.invoke.MethodHandles;
 import java.text.ParseException;
 import java.time.Instant;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
 import java.util.Collection;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 
 import org.apache.solr.client.solrj.SolrResponse;
@@ -190,22 +193,38 @@ public class MaintainTimeRoutedAliasCmd extends AliasCmd {
       // (and all newer to left) but we delete older collections, which are the ones that follow.
       // This logic will always keep the first collection, which we can't delete.
       int numToKeep = 0;
+      DateTimeFormatter dtf = null;
+      if (log.isDebugEnabled()) {
+        dtf = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.n", Locale.ROOT);
+        dtf = dtf.withZone(ZoneId.of("UTC"));
+      }
       for (Map.Entry<Instant, String> parsedCollection : parsedCollections) {
         numToKeep++;
         final Instant colInstant = parsedCollection.getKey();
         if (colInstant.isBefore(delBefore) || colInstant.equals(delBefore)) {
+          if (log.isDebugEnabled()) { // don't perform formatting unless debugging
+            log.debug("{} is equal to or before {} deletions may be required", dtf.format(colInstant),dtf.format(delBefore));
+          }
           break;
+        } else {
+          if (log.isDebugEnabled()) { // don't perform formatting unless debugging
+            log.debug("{} is not before {} and will be retained", dtf.format(colInstant),dtf.format(delBefore));
+          }
         }
       }
       if (numToKeep == parsedCollections.size()) {
-        log.debug("No old time routed collections to delete.");
+        log.debug("No old time routed collections to delete... parsed collections={}", parsedCollections);
         return curAliases;
       }
-
-      final List<String> targetList = curAliases.getCollectionAliasListMap().get(aliasName);
+      log.debug("Collections will be deleted... parsed collections={}", parsedCollections);
+      Map<String, List<String>> collectionAliasListMap = curAliases.getCollectionAliasListMap();
+      final List<String> targetList = collectionAliasListMap.get(aliasName);
       // remember to delete these... (oldest to newest)
+      log.debug("Iterating backwards on collection list to find deletions: {}", targetList);
       for (int i = targetList.size() - 1; i >= numToKeep; i--) {
-        collectionsToDelete.add(targetList.get(i));
+        String toDelete = targetList.get(i);
+        log.debug("Adding to TRA delete list:{}",toDelete);
+        collectionsToDelete.add(toDelete);
       }
       // new alias list has only "numToKeep" first items
       final List<String> collectionsToKeep = targetList.subList(0, numToKeep);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAlias.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAlias.java
index 027100f..fb658cd 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAlias.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAlias.java
@@ -35,10 +35,25 @@ public interface RoutedAlias {
 
   /**
    * Types supported. Every entry here must have a case in the switch statement in {@link #fromProps(String, Map)}
+   *
+   * Routed Alias collections have a naming pattern of XYZ where X is the alias name, Y is the separator prefix and
+   * Z is the data driven value distinguishing the bucket.
    */
   enum SupportedRouterTypes {
-    TIME,
-    CATEGORY
+    TIME {
+      @Override
+      public String getSeparatorPrefix() {
+        return "__TRA__";
+      }
+    },
+    CATEGORY {
+      @Override
+      public String getSeparatorPrefix() {
+        return "__CRA__";
+      }
+    };
+    public abstract String getSeparatorPrefix();
+
   }
 
   String ROUTER_TYPE_NAME = ROUTER_PREFIX + "name";
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java
index 8685961..c11e6ed 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java
@@ -73,6 +73,7 @@ import static org.apache.solr.common.params.CommonParams.TZ;
  */
 public class TimeRoutedAlias implements RoutedAlias {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  public static final SupportedRouterTypes TYPE = SupportedRouterTypes.TIME;
 
   // This class is created once per request and the overseer methods prevent duplicate create requests
   // from creating extra copies. All we need to track here is that we don't spam preemptive creates to
@@ -149,7 +150,7 @@ public class TimeRoutedAlias implements RoutedAlias {
     final MapSolrParams params = new MapSolrParams(this.aliasMetadata); // for convenience
     final RequiredSolrParams required = params.required();
     if (!"time".equals(required.get(ROUTER_TYPE_NAME))) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Only 'time' routed aliases is supported right now.");
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Only 'time' routed aliases is supported by TimeRoutedAlias.");
     }
     routeField = required.get(ROUTER_FIELD);
     intervalMath = required.get(ROUTER_INTERVAL);
@@ -204,7 +205,13 @@ public class TimeRoutedAlias implements RoutedAlias {
   }
 
   public static Instant parseInstantFromCollectionName(String aliasName, String collection) {
-    final String dateTimePart = collection.substring(aliasName.length() + 1);
+    String separatorPrefix = TYPE.getSeparatorPrefix();
+    final String dateTimePart;
+    if (collection.contains(separatorPrefix)) {
+      dateTimePart = collection.substring(collection.lastIndexOf(separatorPrefix) + separatorPrefix.length());
+    } else {
+      dateTimePart = collection.substring(aliasName.length() + 1);
+    }
     return DATE_TIME_FORMATTER.parse(dateTimePart, Instant::from);
   }
 
@@ -216,7 +223,7 @@ public class TimeRoutedAlias implements RoutedAlias {
       }
     }
     assert DATE_TIME_FORMATTER.parse(nextCollName, Instant::from).equals(timestamp);
-    return aliasName + "_" + nextCollName;
+    return aliasName + TYPE.getSeparatorPrefix() + nextCollName;
   }
 
   Instant parseStringAsInstant(String str, TimeZone zone) {
@@ -363,6 +370,7 @@ public class TimeRoutedAlias implements RoutedAlias {
             // parsedCollectionsDesc since candidateCollectionDesc was chosen, we could create collection n+2
             // instead of collection n+1.
             String mostRecentCollName = this.parsedCollectionsDesc.get(0).getValue();
+            log.debug("Most recent at preemptive: {}", mostRecentCollName);
 
             // This line does not block and the document can be added immediately
             preemptiveAsync(() -> createNextCollection(mostRecentCollName, collectionsHandler), core);
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 85a755a..3cf9e2f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
@@ -50,6 +50,8 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import static org.apache.solr.cloud.api.collections.RoutedAlias.SupportedRouterTypes.TIME;
+
 /**
  * Direct http tests of the CreateRoutedAlias functionality.
  */
@@ -224,17 +226,17 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
           .process(client);
     }
 
-    assertCollectionExists(aliasName + "_2018-01-15");
+    assertCollectionExists(aliasName + TIME.getSeparatorPrefix() +"2018-01-15");
   }
 
   @Test
   public void testCollectionNamesMustBeAbsent() throws Exception {
     CollectionAdminRequest.createCollection("collection1meta", "_default", 2, 1).process(cluster.getSolrClient());
     CollectionAdminRequest.createCollection("collection2meta", "_default", 1, 1).process(cluster.getSolrClient());
-    
+
     cluster.waitForActiveCollection("collection1meta", 2, 2);
     cluster.waitForActiveCollection("collection2meta", 1, 1);
-    
+
     waitForState("Expected collection1 to be created with 2 shards and 1 replica", "collection1meta", clusterShape(2, 2));
     waitForState("Expected collection2 to be created with 1 shard and 1 replica", "collection2meta", clusterShape(1, 1));
     ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
diff --git a/solr/core/src/test/org/apache/solr/update/processor/RoutedAliasUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/RoutedAliasUpdateProcessorTest.java
index a15f8c0..e3192d7 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/RoutedAliasUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/RoutedAliasUpdateProcessorTest.java
@@ -97,7 +97,7 @@ public abstract class RoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
     return getSaferTestName();
   }
 
-  void createConfigSet(String configName) throws SolrServerException, IOException {
+  void createConfigSet(String configName) throws SolrServerException, IOException, InterruptedException {
     // First create a configSet
     // Then we create a collection with the name of the eventual config.
     // We configure it, and ultimately delete the collection, leaving a modified config-set behind.
@@ -109,8 +109,9 @@ public abstract class RoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
 
     CollectionAdminRequest.createCollection(configName, configName, 1, 1).process(getSolrClient());
 
-    // TODO: fix SOLR-13059, a where this wait isn't working ~0.3% of the time.
+    // TODO: fix SOLR-13059, a where this wait isn't working ~0.3% of the time without the sleep.
     waitCol(1,configName);
+    Thread.sleep(500); // because YUCK but works (beasts 2500x20 ok vs failing in ~500x20 every time)
     // manipulate the config...
     checkNoError(getSolrClient().request(new V2Request.Builder("/collections/" + configName + "/config")
         .withMethod(SolrRequest.METHOD.POST)
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 6636bb6..1e2afff 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
@@ -31,20 +31,25 @@ import java.util.concurrent.ExecutorService;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.BaseHttpClusterStateProvider;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.ClusterStateProvider;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.ConfigSetAdminRequest;
 import org.apache.solr.client.solrj.response.FieldStatsInfo;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.RoutedAlias;
 import org.apache.solr.cloud.api.collections.TimeRoutedAlias;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.Aliases;
+import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.update.UpdateCommand;
 import org.apache.solr.util.LogLevel;
@@ -56,6 +61,7 @@ import org.slf4j.LoggerFactory;
 
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static org.apache.solr.cloud.api.collections.RoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP;
+import static org.apache.solr.cloud.api.collections.RoutedAlias.SupportedRouterTypes.TIME;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTIONS_ZKNODE;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROPS_ZKNODE;
 
@@ -66,6 +72,7 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
   private static final String alias = "myalias";
   private static final String alias2 = "myalias2";
   private static final String timeField = "timestamp_dt";
+  public static final String TRA = TIME.getSeparatorPrefix();
 
   private  CloudSolrClient solrClient;
 
@@ -99,7 +106,7 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
 
     // Start with one collection manually created (and use higher numShards & replicas than we'll use for others)
     //  This tests we may pre-create the collection and it's acceptable.
-    final String col23rd = alias + "_2017-10-23";
+    final String col23rd = alias + TRA + "2017-10-23";
     CollectionAdminRequest.createCollection(col23rd, configName, 2, 2)
         .setMaxShardsPerNode(2)
         .withProperty(ROUTED_ALIAS_NAME_CORE_PROP, alias)
@@ -134,7 +141,7 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
 
     // add another collection with the precise name we expect, but don't add to alias explicitly.  When we add a document
     //   destined for this collection, Solr will see it already exists and add it to the alias.
-    final String col24th = alias + "_2017-10-24";
+    final String col24th = alias + TRA + "2017-10-24";
     CollectionAdminRequest.createCollection(col24th, configName,  1, 1) // more shards and replicas now
         .withProperty(ROUTED_ALIAS_NAME_CORE_PROP, alias)
         .process(solrClient);
@@ -177,16 +184,17 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
         newDoc(Instant.parse("2017-10-26T05:00:00Z")),
         newDoc(Instant.parse("2017-10-26T06:00:00Z"))
     );
-    assertInvariants(alias + "_2017-10-26", alias + "_2017-10-25", col24th);
+    assertInvariants(alias + TRA + "2017-10-26", alias + TRA + "2017-10-25", col24th);
 
     // verify that collection properties are set when the collections are created. Note: first 2 collections in
     // this test have a core property instead, of a collection property but that MUST continue to work as well
     // for back compatibility's reasons.
     Thread.sleep(1000);
     byte[] data = cluster.getZkClient()
-        .getData(COLLECTIONS_ZKNODE + "/" + alias + "_2017-10-26" + "/" + COLLECTION_PROPS_ZKNODE,null, null, true);
+        .getData(COLLECTIONS_ZKNODE + "/" + alias + TRA + "2017-10-26" + "/" + COLLECTION_PROPS_ZKNODE,null, null, true);
     assertNotNull(data);
     assertTrue(data.length > 0);
+    @SuppressWarnings("unchecked")
     Map<String,String> props = (Map<String, String>) Utils.fromJSON(data);
     assertTrue(props.containsKey(ROUTED_ALIAS_NAME_CORE_PROP));
     assertEquals(alias,props.get(ROUTED_ALIAS_NAME_CORE_PROP));
@@ -203,7 +211,7 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
         newDoc(Instant.parse("2017-10-27T08:00:00Z")) // new
     );
     numDocsDeletedOrFailed += numDocsToBeAutoDeleted;
-    assertInvariants(alias + "_2017-10-27", alias + "_2017-10-26");
+    assertInvariants(alias + TRA + "2017-10-27", alias + TRA + "2017-10-26");
   }
 
   /**
@@ -233,9 +241,9 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
     assertUpdateResponse(solrClient.commit(alias));
 
     // wait for all the collections to exist...
-    waitColAndAlias(alias, "_", "2017-10-23", numShards);
-    waitColAndAlias(alias, "_", "2017-10-24", numShards);
-    waitColAndAlias(alias, "_", "2017-10-25", numShards);
+    waitColAndAlias(alias, TRA, "2017-10-23", numShards);
+    waitColAndAlias(alias, TRA, "2017-10-24", numShards);
+    waitColAndAlias(alias, TRA, "2017-10-25", numShards);
 
     // at this point we now have 3 collections with 4 shards each, and 3 replicas per shard for a total of
     // 36 total replicas, 1/3 of which are leaders. We will add 3 docs and each has a 33% chance of hitting a
@@ -326,7 +334,6 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
     for (Exception threadException : threadExceptions) {
       if (threadException != null) {
         Thread.sleep(5000); // avoid spurious fails due to TRA thread not done yet
-        //noinspection ThrowFromFinallyBlock
         throw threadException;
       }
     }
@@ -338,8 +345,8 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
     assertTrue(threadFinished[1]);
 
     // if one of these times out then the test has failed due to interference between aliases
-    waitColAndAlias(alias, "_", "2017-10-26", numShards);
-    waitColAndAlias(alias2, "_", "2017-10-26", numShards);
+    waitColAndAlias(alias, TRA, "2017-10-26", numShards);
+    waitColAndAlias(alias2, TRA, "2017-10-26", numShards);
 
     // after this we can ignore alias2
     checkPreemptiveCase1(alias);
@@ -359,8 +366,8 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
             .setMaxShardsPerNode(numReplicas)).setPreemptiveCreateWindow("3HOUR")
         .process(solrClient);
 
-    waitColAndAlias("foo", "_", "2017-10-23",2);
-    waitCoreCount("foo_2017-10-23", 1); // prove this works, for confidence in deletion checking below.
+    waitColAndAlias("foo", TRA, "2017-10-23",2);
+    waitCoreCount("foo" + TRA + "2017-10-23", 4); // prove this works, for confidence in deletion checking below.
     assertUpdateResponse(solrClient.add("foo",
         sdoc("id","1","timestamp_dt", "2017-10-23T00:00:00Z") // no extra collections should be created
     ));
@@ -386,7 +393,7 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
         sdoc("id", "7", "timestamp_dt", "2017-10-25T23:01:00Z")), // should cause preemptive creation of 10-27 now
         params));
     assertUpdateResponse(solrClient.commit(alias));
-    waitColAndAlias(alias, "_", "2017-10-27", numShards);
+    waitColAndAlias(alias, TRA, "2017-10-27", numShards);
 
     cols = new CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
     assertEquals(5,cols.size()); // only one created in async case
@@ -400,8 +407,8 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
         sdoc("id", "8", "timestamp_dt", "2017-10-25T23:01:00Z")), // should cause preemptive creation of 10-28 now
         params));
     assertUpdateResponse(solrClient.commit(alias));
-    waitColAndAlias(alias, "_", "2017-10-27", numShards);
-    waitColAndAlias(alias, "_", "2017-10-28", numShards);
+    waitColAndAlias(alias, TRA, "2017-10-27", numShards);
+    waitColAndAlias(alias, TRA, "2017-10-28", numShards);
 
     cols = new CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
     assertEquals(6,cols.size()); // Subsequent documents continue to create up to limit
@@ -433,7 +440,7 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
         sdoc("id", "12", "timestamp_dt", "2017-10-28T23:03:00Z")), // should be ignored due to in progress creation
         params));
     assertUpdateResponse(solrClient.commit(alias));
-    waitColAndAlias(alias, "_", "2017-10-29", numShards);
+    waitColAndAlias(alias, TRA, "2017-10-29", numShards);
 
     cols = new CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
     assertEquals(7,cols.size());
@@ -455,8 +462,8 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
         sdoc("id", "13", "timestamp_dt", "2017-10-30T23:03:00Z")), // lucky?
         params));
     assertUpdateResponse(solrClient.commit(alias));
-    waitColAndAlias(alias, "_", "2017-10-30", numShards);
-    waitColAndAlias(alias, "_", "2017-10-31", numShards); // spooky! async case arising in middle of sync creation!!
+    waitColAndAlias(alias, TRA, "2017-10-30", numShards);
+    waitColAndAlias(alias, TRA, "2017-10-31", numShards); // spooky! async case arising in middle of sync creation!!
 
     cols = new CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
     assertEquals(9,cols.size());
@@ -478,48 +485,53 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
     assertUpdateResponse(add(alias, Collections.singletonList(
         sdoc("id", "14", "timestamp_dt", "2017-10-31T23:01:00Z")), // should cause preemptive creation 11-01
         params));
-    waitColAndAlias(alias, "_", "2017-11-01", numShards);
+    waitColAndAlias(alias, TRA, "2017-11-01", numShards);
 
     assertUpdateResponse(add(alias, Collections.singletonList(
         sdoc("id", "15", "timestamp_dt", "2017-10-31T23:01:00Z")), // should cause preemptive creation 11-02
         params));
-    waitColAndAlias(alias, "_", "2017-11-02", numShards);
+    waitColAndAlias(alias, TRA, "2017-11-02", numShards);
 
     assertUpdateResponse(add(alias, Collections.singletonList(
         sdoc("id", "16", "timestamp_dt", "2017-10-31T23:01:00Z")), // should cause preemptive creation 11-03
         params));
-    waitColAndAlias(alias, "_", "2017-11-03", numShards);
+    waitColAndAlias(alias, TRA, "2017-11-03", numShards);
 
     assertUpdateResponse(add(alias, Collections.singletonList(
         sdoc("id", "17", "timestamp_dt", "2017-10-31T23:01:00Z")), // should NOT cause preemptive creation 11-04
         params));
 
     cols = new CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
-    assertTrue("Preemptive creation beyond ROUTER_PREEMPTIVE_CREATE_MATH setting of 3DAY!",!cols.contains("myalias_2017-11-04"));
+    assertTrue("Preemptive creation beyond ROUTER_PREEMPTIVE_CREATE_MATH setting of 3DAY!",!cols.contains("myalias" + TRA + "2017-11-04"));
 
     assertUpdateResponse(add(alias, Collections.singletonList(
         sdoc("id", "18", "timestamp_dt", "2017-11-01T23:01:00Z")), // should cause preemptive creation 11-04
         params));
-    waitColAndAlias(alias, "_", "2017-11-04",numShards);
+    waitColAndAlias(alias, TRA, "2017-11-04",numShards);
 
   }
 
-  // used to verify a core has been deleted (count = 0)
   private void waitCoreCount(String collection, int count) {
     long start = System.nanoTime();
-    CoreContainer coreContainer = cluster.getRandomJetty(random()).getCoreContainer();
     int coreFooCount;
+    List<JettySolrRunner> jsrs = cluster.getJettySolrRunners();
     do {
       coreFooCount = 0;
-      List<CoreDescriptor> coreDescriptors = coreContainer.getCoreDescriptors();
-      for (CoreDescriptor coreDescriptor : coreDescriptors) {
-        String collectionName = coreDescriptor.getCollectionName();
-        if (collection.equals(collectionName)) {
-          coreFooCount ++;
+      // have to check all jetties... there was a very confusing bug where we only checked one and
+      // thus might pick a jetty without a core for the collection and succeed if count = 0 when we
+      // should have failed, or at least waited longer
+      for (JettySolrRunner jsr : jsrs) {
+        List<CoreDescriptor> coreDescriptors = jsr.getCoreContainer().getCoreDescriptors();
+        for (CoreDescriptor coreDescriptor : coreDescriptors) {
+          String collectionName = coreDescriptor.getCollectionName();
+          if (collection.equals(collectionName)) {
+            System.out.println("found:" + collectionName);
+            coreFooCount ++;
+          }
         }
       }
-      if (NANOSECONDS.toSeconds(System.nanoTime() - start) > 10) {
-        fail("took over 10 seconds after collection creation to update aliases");
+      if (NANOSECONDS.toSeconds(System.nanoTime() - start) > 60) {
+        fail("took over 60 seconds after collection creation to update aliases:"+collection + " core count=" + coreFooCount + " was looking for " + count);
       } else {
         try {
           Thread.sleep(500);
@@ -528,7 +540,6 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
           fail(e.getMessage());
         }
       }
-
     } while(coreFooCount != count);
   }
 
@@ -551,7 +562,7 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
     List<String> colsT1;
     colsT1 = new CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
     assertEquals(3, colsT1.size());
-    assertTrue("Preemptive creation appears to not be asynchronous anymore", !colsT1.contains("myalias_2017-10-26"));
+    assertTrue("Preemptive creation appears to not be asynchronous anymore", !colsT1.contains("myalias" + TRA + "2017-10-26"));
     assertNumDocs("2017-10-23", 1, alias);
     assertNumDocs("2017-10-24", 1, alias);
     assertNumDocs("2017-10-25", 3, alias);
@@ -569,7 +580,7 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
     List<String> cols;
     cols = new CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
     assertTrue("Preemptive creation happened twice and created a collection " +
-        "further in the future than the configured time slice!",!cols.contains("myalias_2017-10-27"));
+        "further in the future than the configured time slice!",!cols.contains("myalias" + TRA + "2017-10-27"));
 
     assertEquals(4, cols.size());
     assertNumDocs("2017-10-23", 1, alias);
@@ -587,9 +598,9 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
     assertUpdateResponse(solrClient.commit(alias));
 
     // wait for all the collections to exist...
-    waitColAndAlias(alias, "_", "2017-10-23", numShards); // This one should have already existed from the alias creation
-    waitColAndAlias(alias, "_", "2017-10-24", numShards); // Create 1
-    waitColAndAlias(alias, "_", "2017-10-25", numShards); // Create 2nd synchronously (ensure this is not broken)
+    waitColAndAlias(alias, TRA, "2017-10-23", numShards); // This one should have already existed from the alias creation
+    waitColAndAlias(alias, TRA, "2017-10-24", numShards); // Create 1
+    waitColAndAlias(alias, TRA, "2017-10-25", numShards); // Create 2nd synchronously (ensure this is not broken)
 
     // normal update, nothing special, no collection creation required.
     List<String> cols = new CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
@@ -601,7 +612,7 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
   }
 
   private void assertNumDocs(final String datePart, int expected, String alias) throws SolrServerException, IOException {
-    QueryResponse resp = solrClient.query(alias + "_" + datePart, params(
+    QueryResponse resp = solrClient.query(alias + TRA + datePart, params(
         "q", "*:*",
         "rows", "10"));
     assertEquals(expected, resp.getResults().getNumFound());
@@ -683,13 +694,258 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
   @Test
   public void testParse() {
     assertEquals(Instant.parse("2017-10-02T03:04:05Z"),
-      TimeRoutedAlias.parseInstantFromCollectionName(alias, alias + "_2017-10-02_03_04_05"));
+      TimeRoutedAlias.parseInstantFromCollectionName(alias, alias + TRA + "2017-10-02_03_04_05"));
     assertEquals(Instant.parse("2017-10-02T03:04:00Z"),
-      TimeRoutedAlias.parseInstantFromCollectionName(alias, alias + "_2017-10-02_03_04"));
+      TimeRoutedAlias.parseInstantFromCollectionName(alias, alias + TRA + "2017-10-02_03_04"));
     assertEquals(Instant.parse("2017-10-02T03:00:00Z"),
-      TimeRoutedAlias.parseInstantFromCollectionName(alias, alias + "_2017-10-02_03"));
+      TimeRoutedAlias.parseInstantFromCollectionName(alias, alias + TRA + "2017-10-02_03"));
     assertEquals(Instant.parse("2017-10-02T00:00:00Z"),
-      TimeRoutedAlias.parseInstantFromCollectionName(alias, alias + "_2017-10-02"));
+      TimeRoutedAlias.parseInstantFromCollectionName(alias, alias + TRA + "2017-10-02"));
+  }
+
+  /**
+   * Need to ensure that the existing TRA's gracefully handle, old, new and mixtures thereof. TRA's with
+   * an autoDeleteAge setting will gracefully convert to the new format over time.
+   */
+  @Test
+  public void handleLegacyCollectionNames() throws Exception {
+    manuallyConstructLegacyTRA();
+
+    // OK we now have an alias with legacy names and 2 documents. Let's try to query it to ensure query time back compat
+
+    QueryResponse resp = solrClient.query(alias, params(
+        "q", "*:*",
+        "rows", "10"
+    ));
+
+    assertEquals(2,resp.getResults().getNumFound());
+
+    // verify that we can still add documents to it.
+    assertUpdateResponse(solrClient.add(alias,
+        sdoc("id","3","timestamp_dt", "2017-10-23T00:00:01Z")
+    ));
+    solrClient.commit(alias);
+    resp = solrClient.query(alias, params(
+        "q", "*:*",
+        "rows", "10"
+    ));
+    assertEquals(3,resp.getResults().getNumFound());
+
+
+    // verify that it can create new collections
+    assertUpdateResponse(solrClient.add(alias,
+        sdoc("id","4","timestamp_dt", "2017-10-24T23:00:01Z") // preemptive
+    ));
+    solrClient.commit(alias);
+    waitColAndAlias(alias, TRA, "2017-10-25",1);
+
+    // verify that mixed old/new collections works for update/query
+    resp = solrClient.query(alias, params(
+        "q", "*:*",
+        "rows", "10"
+    ));
+    assertEquals(4,resp.getResults().getNumFound());
+
+    // verify that documents go to the right collections
+
+    assertUpdateResponse(solrClient.add(alias,
+        sdoc("id","5","timestamp_dt", "2017-10-25T12:00:01Z") // preemptive
+    ));
+    solrClient.commit(alias);
+
+    resp = solrClient.query("myalias_2017-10-23", params(
+        "q", "*:*",
+        "rows", "10",
+        "sort", "id asc"
+    ));
+    assertEquals(2,resp.getResults().getNumFound());
+    assertEquals(resp.getResults().get(0).getFirstValue("id"), "1");
+    assertEquals(resp.getResults().get(1).getFirstValue("id"), "3");
+
+    resp = solrClient.query("myalias_2017-10-24", params(
+        "q", "*:*",
+        "rows", "10",
+        "sort", "id asc"
+    ));
+    assertEquals(2,resp.getResults().getNumFound());
+    assertEquals(resp.getResults().get(0).getFirstValue("id"), "2");
+    assertEquals(resp.getResults().get(1).getFirstValue("id"), "4");
+
+    resp = solrClient.query("myalias" + TRA + "2017-10-25", params(
+        "q", "*:*",
+        "rows", "10",
+        "sort", "id asc"
+    ));
+    assertEquals(1,resp.getResults().getNumFound());
+    assertEquals(resp.getResults().get(0).getFirstValue("id"), "5");
+
+    // verify that auto-delete will age out old collections
+    checkCollectionCountIs(3);
+
+    assertUpdateResponse(solrClient.add(alias,
+        sdoc("id","6","timestamp_dt", "2017-10-26T12:00:01Z") // preemptive
+    ));
+    waitColAndAlias(alias, TRA,"2017-10-26",1);
+    checkCollectionCountIs(4)
+        .containsAll(Arrays.asList(
+            "myalias_2017-10-23",
+            "myalias_2017-10-24",
+            "myalias" + TRA + "2017-10-25",
+            "myalias" + TRA + "2017-10-26"));
+
+    assertUpdateResponse(solrClient.add(alias,
+        sdoc("id","7","timestamp_dt", "2017-10-27T12:00:01Z") // preemptive
+    ));
+    waitColAndAlias(alias, TRA,"2017-10-27",1);
+    waitCoreCount("myalias_2017-10-23",0);
+    checkCollectionCountIs(4)
+        .containsAll(Arrays.asList(
+            "myalias_2017-10-24",
+            "myalias" + TRA + "2017-10-25",
+            "myalias" + TRA + "2017-10-26",
+            "myalias" + TRA + "2017-10-27"));
+
+    assertUpdateResponse(solrClient.add(alias,
+        sdoc("id","8","timestamp_dt", "2017-10-28T12:00:01Z") // preemptive
+    ));
+    waitColAndAlias(alias, TRA,"2017-10-28",1);
+    waitCoreCount("myalias_2017-10-24",0);
+    checkCollectionCountIs(4)
+        .containsAll(Arrays.asList(
+            "myalias" + TRA + "2017-10-25",
+            "myalias" + TRA + "2017-10-26",
+            "myalias" + TRA + "2017-10-27",
+            "myalias" + TRA + "2017-10-28"));
+
+    // verify that auto-delete works on new collections.
+    assertUpdateResponse(solrClient.add(alias,
+        sdoc("id","9","timestamp_dt", "2017-10-29T12:00:01Z") // preemptive
+    ));
+    waitColAndAlias(alias, TRA,"2017-10-29",1);
+    waitCoreCount("myalias" + TRA + "2017-10-25",0);
+    checkCollectionCountIs(4)
+        .containsAll(Arrays.asList(
+            "myalias" + TRA + "2017-10-26",
+            "myalias" + TRA + "2017-10-27",
+            "myalias" + TRA + "2017-10-28",
+            "myalias" + TRA + "2017-10-29"));
+
+    solrClient.commit(alias);
+
+    resp = solrClient.query(alias, params(
+        "q", "*:*",
+        "rows", "10"
+    ));
+    assertEquals(4,resp.getResults().getNumFound());
+
+  }
+
+  private List<String> checkCollectionCountIs(int num) {
+    ClusterStateProvider clusterStateProvider = solrClient.getClusterStateProvider();
+    List<String> collections = clusterStateProvider.resolveAlias(alias);
+    if (clusterStateProvider instanceof BaseHttpClusterStateProvider) {
+      collections = ((BaseHttpClusterStateProvider)clusterStateProvider).resolveAlias(alias,true);
+    }
+    System.out.println();
+    System.out.println(clusterStateProvider.getClass());
+    System.out.println(collections);
+    System.out.println();
+    assertEquals(num, collections.size()); // starting point
+    return collections;
+  }
+
+  // here we do things not to be emulated elsewhere to create a legacy condition and ensure that we can
+  // work with both old and new formats.
+  private void manuallyConstructLegacyTRA() throws Exception {
+    // first create a "modern" alias
+    String configName = getSaferTestName();
+    createConfigSet(configName);
+
+    final int numShards = 1 ;
+    final int numReplicas = 1 ;
+    CollectionAdminRequest.createTimeRoutedAlias(alias, "2017-10-23T00:00:00Z", "+1DAY", getTimeField(),
+        CollectionAdminRequest.createCollection("_unused_", configName, numShards, numReplicas)
+            .setMaxShardsPerNode(numReplicas)).setPreemptiveCreateWindow("3HOUR").setAutoDeleteAge("/DAY-3DAYS")
+        .process(solrClient);
+
+    // now create collections that look like the legacy (pre __TRA__) names...
+    String legacy23 = alias + "_" + "2017-10-23";
+    CollectionAdminRequest.createCollection(legacy23, configName, numShards,numReplicas).process(solrClient);
+    String legacy24 = alias + "_" + "2017-10-24";
+    CollectionAdminRequest.createCollection(legacy24, configName, numShards,numReplicas).process(solrClient);
+
+    waitCol(1,legacy23);
+    waitCol(1,legacy24);
+    // put some data in the legacy collections:
+    assertUpdateResponse(solrClient.add(legacy23,
+        sdoc("id","1","timestamp_dt", "2017-10-23T00:00:01Z")
+    ));
+    assertUpdateResponse(solrClient.add(legacy24,
+        sdoc("id","2","timestamp_dt", "2017-10-24T00:00:01Z")
+    ));
+
+    solrClient.commit(legacy23);
+    solrClient.commit(legacy24);
+
+    QueryResponse resp = solrClient.query(legacy23, params(
+        "q", "*:*",
+        "rows", "10"));
+    assertEquals(1,resp.getResults().getNumFound());
+
+    resp = solrClient.query(legacy24, params(
+        "q", "*:*",
+        "rows", "10"));
+    assertEquals(1,resp.getResults().getNumFound());
+
+    // now knock out the collection backing our alias
+    ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
+    Aliases aliases = zkStateReader.getAliases();
+    List<String> collections = aliases.getCollectionAliasListMap().get(alias);
+    for (String collection : collections) {
+      CollectionAdminRequest.deleteCollection(collection);
+    }
+
+    // now grab the zk data so we can hack in our legacy collections..
+    byte[] data = zkStateReader.getZkClient().getData("/aliases.json", null, null, true);
+
+    // some tidbits for handling zk data here are swiped from Aliases.json
+    Map<String, Map> aliasMap;
+    if (data == null || data.length == 0) {
+      aliasMap = Collections.emptyMap();
+    } else {
+      //noinspection unchecked
+      aliasMap = (Map<String, Map>) Utils.fromJSON(data);
+    }
+    assertNotEquals(0, aliasMap.size());
+
+    Map colAliases = aliasMap.getOrDefault("collection", Collections.emptyMap());
+    assertNotEquals(0,colAliases.size());
+
+    String singleInitialCollection = (String) colAliases.get(alias);
+    assertFalse(singleInitialCollection.contains(","));
+
+    // replace with our two new collections... in asc order!
+    colAliases.put(alias,String.join(",",legacy24,legacy23));
+
+    data = Utils.toJSON(aliasMap);
+    zkStateReader.getZkClient().setData("/aliases.json",data,true);
+
+    zkStateReader.aliasesManager.update(); // make sure we've updated with the data we just sent
+
+    aliases = zkStateReader.aliasesManager.getAliases();
+    assertEquals(2,aliases.getCollectionAliasListMap().get(alias).size());
+
+    CollectionAdminRequest.deleteCollection(singleInitialCollection).process(solrClient);
+
+    waitCoreCount(singleInitialCollection,0);
+
+    // now make the legacy collections part of the alias
+    CollectionAdminRequest.setCollectionProperty(legacy23,RoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP, alias).process(solrClient);
+    CollectionAdminRequest.setCollectionProperty(legacy24,RoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP, alias).process(solrClient);
+    CollectionAdminRequest.reloadCollection(legacy23).process(solrClient);
+    CollectionAdminRequest.reloadCollection(legacy24).process(solrClient);
+
+    cluster.getOpenOverseer().getCoreContainer().getZkController().getZkStateReader().aliasesManager.update();
   }
 
 }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseHttpClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseHttpClusterStateProvider.java
index 87f4a2f..6c6eb25 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseHttpClusterStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseHttpClusterStateProvider.java
@@ -191,7 +191,11 @@ public abstract class BaseHttpClusterStateProvider implements ClusterStateProvid
 
   @Override
   public List<String> resolveAlias(String aliasName) {
-    return Aliases.resolveAliasesGivenAliasMap(getAliases(false), aliasName);
+    return resolveAlias(aliasName, false);
+  }
+
+  public List<String> resolveAlias(String aliasName, boolean forceFetch) {
+    return Aliases.resolveAliasesGivenAliasMap(getAliases(forceFetch), aliasName);
   }
 
   @Override
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 68f828d..85c9088 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
@@ -358,7 +358,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
   public static Create createCollection(String collection, String config, Integer numShards, Integer numNrtReplicas, Integer numTlogReplicas, Integer numPullReplicas) {
     return new Create(collection, config, numShards, numNrtReplicas, numTlogReplicas, numPullReplicas);
   }
-  
+
   /**
    * Returns a SolrRequest for creating a collection
    * @param collection the collection name
@@ -394,7 +394,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
   public static Create createCollectionWithImplicitRouter(String collection, String config, String shards, int numReplicas) {
     return new Create(collection, config, shards, numReplicas);
   }
-  
+
   /**
    * Returns a SolrRequest for creating a collection with the implicit router and specific types of replicas
    * @param collection  the collection name
@@ -448,7 +448,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     protected Create(String collection, String config, String shards, int numNrtReplicas) {
       this(collection, config, ImplicitDocRouter.NAME, null, checkNotNull("shards",shards), numNrtReplicas, null, null);
     }
-    
+
     private Create(String collection, String config, String routerName, Integer numShards, String shards, Integer numNrtReplicas, Integer  numTlogReplicas, Integer numPullReplicas) {
       super(CollectionAction.CREATE, SolrIdentifierValidator.validateCollectionName(collection));
       // NOTE: there's very little we can assert about the args because nothing but "collection" is required by the server
@@ -489,7 +489,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     public String getShards() { return  shards; }
     public Integer getNumShards() { return numShards; }
     public Integer getMaxShardsPerNode() { return maxShardsPerNode; }
-    
+
     public Integer getReplicationFactor() { return getNumNrtReplicas(); }
     public Integer getNumNrtReplicas() { return nrtReplicas; }
     public Boolean getAutoAddReplicas() { return autoAddReplicas; }
@@ -497,12 +497,12 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     public Integer getNumPullReplicas() {return pullReplicas;}
 
     public Integer getStateFormat() { return stateFormat; }
-    
+
     /**
      * Provide the name of the shards to be created, separated by commas
-     * 
+     *
      * Shard names must consist entirely of periods, underscores, hyphens, and alphanumerics.  Other characters are not allowed.
-     * 
+     *
      * @throws IllegalArgumentException if any of the shard names contain invalid characters.
      */
     public Create setShards(String shards) {
@@ -514,7 +514,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       this.shards = shards;
       return this;
     }
-    
+
     public Properties getProperties() {
       return properties;
     }
@@ -1604,7 +1604,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
         throw new IllegalArgumentException("Either requestid or flush parameter must be specified.");
       if (requestId != null && flush != null)
         throw new IllegalArgumentException("Both requestid and flush parameters can not be specified together.");
-      
+
       this.requestId = requestId;
       this.flush = flush;
     }
@@ -1737,6 +1737,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     public static final String ROUTER_INTERVAL = "router.interval";
     public static final String ROUTER_MAX_FUTURE = "router.maxFutureMs";
     public static final String ROUTER_PREEMPTIVE_CREATE_WINDOW = "router.preemptiveCreateMath";
+    public static final String ROUTER_AUTO_DELETE_AGE = "router.autoDeleteAge";
 
     private final String aliasName;
     private final String routerField;
@@ -1746,6 +1747,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     private TimeZone tz;
     private Integer maxFutureMs;
     private String preemptiveCreateMath;
+    private String autoDeleteAge;
 
     private final Create createCollTemplate;
 
@@ -1775,6 +1777,11 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       return this;
     }
 
+    public CreateTimeRoutedAlias setAutoDeleteAge(String autoDeleteAge) {
+      this.autoDeleteAge = autoDeleteAge;
+      return this;
+    }
+
     @Override
     public SolrParams getParams() {
       ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
@@ -1792,6 +1799,9 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       if (preemptiveCreateMath != null) {
         params.add(ROUTER_PREEMPTIVE_CREATE_WINDOW, preemptiveCreateMath);
       }
+      if (autoDeleteAge != null) {
+        params.add(ROUTER_AUTO_DELETE_AGE, autoDeleteAge);
+      }
 
       // merge the above with collectionParams.  Above takes precedence.
       ModifiableSolrParams createCollParams = new ModifiableSolrParams(); // output target
@@ -1903,7 +1913,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       return params;
     }
   }
-  
+
   /**
    * Returns a SolrRequest to add a replica of type {@link org.apache.solr.common.cloud.Replica.Type#NRT} to a shard in a collection
    *
@@ -1913,7 +1923,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
   }
 
   /**
-   * Returns a SolrRequest to add a replica of the specified type to a shard in a collection.  
+   * Returns a SolrRequest to add a replica of the specified type to a shard in a collection.
    * If the replica type is null, the server default will be used.
    *
    */