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/02/09 04:16:22 UTC

[2/2] lucene-solr:master: SOLR-11925: Rename RoutedAliasCreateCollectionCmd as MaintainRoutedAliasCmd (internal Cmd)

SOLR-11925: Rename RoutedAliasCreateCollectionCmd as MaintainRoutedAliasCmd (internal Cmd)


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

Branch: refs/heads/master
Commit: 1527ce57d49721923ae43a81a10fe872ce94a2d8
Parents: 02b5172
Author: David Smiley <ds...@apache.org>
Authored: Thu Feb 8 23:16:12 2018 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Thu Feb 8 23:16:12 2018 -0500

----------------------------------------------------------------------
 .../cloud/api/collections/CreateAliasCmd.java   |   2 +-
 .../api/collections/MaintainRoutedAliasCmd.java | 300 ++++++++++++++++++
 .../OverseerCollectionMessageHandler.java       |   2 +-
 .../RoutedAliasCreateCollectionCmd.java         | 301 -------------------
 .../cloud/api/collections/TimeRoutedAlias.java  |   2 +-
 .../TimeRoutedAliasUpdateProcessor.java         |   4 +-
 .../solr/common/params/CollectionParams.java    |   2 +-
 7 files changed, 306 insertions(+), 307 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1527ce57/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 b0e6f44..b89d752 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
@@ -121,7 +121,7 @@ public class CreateAliasCmd implements OverseerCollectionMessageHandler.Cmd {
     String initialCollectionName = TimeRoutedAlias.formatCollectionNameFromInstant(aliasName, startTime);
 
     // Create the collection
-    RoutedAliasCreateCollectionCmd.createCollectionAndWait(state, aliasName, aliasMetadata, initialCollectionName, ocmh);
+    MaintainRoutedAliasCmd.createCollectionAndWait(state, aliasName, aliasMetadata, initialCollectionName, ocmh);
     validateAllCollectionsExistAndNoDups(Collections.singletonList(initialCollectionName), zkStateReader);
 
     // Create/update the alias

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1527ce57/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java
new file mode 100644
index 0000000..d41a205
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.cloud.api.collections;
+
+import java.lang.invoke.MethodHandles;
+import java.text.ParseException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.Overseer;
+import org.apache.solr.cloud.OverseerSolrResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.Aliases;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.handler.admin.CollectionsHandler;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.DateMathParser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.COLL_CONF;
+import static org.apache.solr.cloud.api.collections.TimeRoutedAlias.CREATE_COLLECTION_PREFIX;
+import static org.apache.solr.cloud.api.collections.TimeRoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP;
+import static org.apache.solr.common.params.CommonParams.NAME;
+
+/**
+ * (Internal) For "time routed aliases", both deletes old collections and creates new collections
+ * associated with routed aliases.
+ *
+ * Note: this logic is within an Overseer because we want to leverage the mutual exclusion
+ * property afforded by the lock it obtains on the alias name.
+ *
+ * @since 7.3
+ * @lucene.internal
+ */
+public class MaintainRoutedAliasCmd implements OverseerCollectionMessageHandler.Cmd {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final String IF_MOST_RECENT_COLL_NAME = "ifMostRecentCollName"; //TODO rename to createAfter
+
+  private final OverseerCollectionMessageHandler ocmh;
+
+  public MaintainRoutedAliasCmd(OverseerCollectionMessageHandler ocmh) {
+    this.ocmh = ocmh;
+  }
+
+  /** Invokes this command from the client.  If there's a problem it will throw an exception. */
+  public static NamedList remoteInvoke(CollectionsHandler collHandler, String aliasName, String mostRecentCollName)
+      throws Exception {
+    final String operation = CollectionParams.CollectionAction.MAINTAINROUTEDALIAS.toLower();
+    Map<String, Object> msg = new HashMap<>();
+    msg.put(Overseer.QUEUE_OPERATION, operation);
+    msg.put(CollectionParams.NAME, aliasName);
+    msg.put(MaintainRoutedAliasCmd.IF_MOST_RECENT_COLL_NAME, mostRecentCollName);
+    final SolrResponse rsp = collHandler.sendToOCPQueue(new ZkNodeProps(msg));
+    if (rsp.getException() != null) {
+      throw rsp.getException();
+    }
+    return rsp.getResponse();
+  }
+
+  @Override
+  public void call(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception {
+    //---- PARSE PRIMARY MESSAGE PARAMS
+    // important that we use NAME for the alias as that is what the Overseer will get a lock on before calling us
+    final String aliasName = message.getStr(NAME);
+    // the client believes this is the mostRecent collection name.  We assert this if provided.
+    final String ifMostRecentCollName = message.getStr(IF_MOST_RECENT_COLL_NAME); // optional
+
+    // TODO collection param (or intervalDateMath override?), useful for data capped collections
+
+    //---- PARSE ALIAS INFO FROM ZK
+    final ZkStateReader.AliasesManager aliasesManager = ocmh.zkStateReader.aliasesManager;
+    final Aliases aliases = aliasesManager.getAliases();
+    final Map<String, String> aliasMetadata = aliases.getCollectionAliasMetadata(aliasName);
+    if (aliasMetadata == null) {
+      throw newAliasMustExistException(aliasName); // if it did exist, we'd have a non-null map
+    }
+    final TimeRoutedAlias timeRoutedAlias = new TimeRoutedAlias(aliasName, aliasMetadata);
+
+    final List<Map.Entry<Instant, String>> parsedCollections =
+        timeRoutedAlias.parseCollections(aliases, () -> newAliasMustExistException(aliasName));
+
+    //---- GET MOST RECENT COLL
+    final Map.Entry<Instant, String> mostRecentEntry = parsedCollections.get(0);
+    final Instant mostRecentCollTimestamp = mostRecentEntry.getKey();
+    final String mostRecentCollName = mostRecentEntry.getValue();
+    if (ifMostRecentCollName != null) {
+      if (!mostRecentCollName.equals(ifMostRecentCollName)) {
+        // Possibly due to race conditions in URPs on multiple leaders calling us at the same time
+        String msg = IF_MOST_RECENT_COLL_NAME + " expected " + ifMostRecentCollName + " but it's " + mostRecentCollName;
+        if (parsedCollections.stream().map(Map.Entry::getValue).noneMatch(ifMostRecentCollName::equals)) {
+          msg += ". Furthermore this collection isn't in the list of collections referenced by the alias.";
+        }
+        log.info(msg);
+        results.add("message", msg);
+        return;
+      }
+    } else if (mostRecentCollTimestamp.isAfter(Instant.now())) {
+      final String msg = "Most recent collection is in the future, so we won't create another.";
+      log.info(msg);
+      results.add("message", msg);
+      return;
+    }
+
+    //---- COMPUTE NEXT COLLECTION NAME
+    final Instant nextCollTimestamp = timeRoutedAlias.computeNextCollTimestamp(mostRecentCollTimestamp);
+    final String createCollName = TimeRoutedAlias.formatCollectionNameFromInstant(aliasName, nextCollTimestamp);
+
+    //---- DELETE OLDEST COLLECTIONS AND REMOVE FROM ALIAS (if configured)
+    NamedList deleteResults = deleteOldestCollectionsAndUpdateAlias(timeRoutedAlias, aliasesManager, nextCollTimestamp);
+    if (deleteResults != null) {
+      results.add("delete", deleteResults);
+    }
+
+    //---- CREATE THE COLLECTION
+    NamedList createResults = createCollectionAndWait(clusterState, aliasName, aliasMetadata,
+        createCollName, ocmh);
+    if (createResults != null) {
+      results.add("create", createResults);
+    }
+
+    //---- UPDATE THE ALIAS WITH NEW COLLECTION
+    aliasesManager.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, ','));
+      }
+    });
+
+  }
+
+  /**
+   * Deletes some of the oldest collection(s) based on {@link TimeRoutedAlias#getAutoDeleteAgeMath()}. If not present
+   * then does nothing.  Returns non-null results if something was deleted (or if we tried to).
+   * {@code now} is the date from which the math is relative to.
+   */
+  NamedList deleteOldestCollectionsAndUpdateAlias(TimeRoutedAlias timeRoutedAlias,
+                                                  ZkStateReader.AliasesManager aliasesManager,
+                                                  Instant now) throws Exception {
+    final String autoDeleteAgeMathStr = timeRoutedAlias.getAutoDeleteAgeMath();
+    if (autoDeleteAgeMathStr == null) {
+      return null;
+    }
+    final Instant delBefore;
+    try {
+      delBefore = new DateMathParser(Date.from(now), timeRoutedAlias.getTimeZone()).parseMath(autoDeleteAgeMathStr).toInstant();
+    } catch (ParseException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e); // note: should not happen by this point
+    }
+
+    String aliasName = timeRoutedAlias.getAliasName();
+
+    Collection<String> collectionsToDelete = new LinkedHashSet<>();
+
+    // First update the alias    (there may be no change to make!)
+    aliasesManager.applyModificationAndExportToZk(curAliases -> {
+      // note: we could re-parse the TimeRoutedAlias object from curAliases but I don't think there's a point to it.
+
+      final List<Map.Entry<Instant, String>> parsedCollections =
+          timeRoutedAlias.parseCollections(curAliases, () -> newAliasMustExistException(aliasName));
+
+      //iterating from newest to oldest, find the first collection that has a time <= "before".  We keep this collection
+      // (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;
+      for (Map.Entry<Instant, String> parsedCollection : parsedCollections) {
+        numToKeep++;
+        final Instant colInstant = parsedCollection.getKey();
+        if (colInstant.isBefore(delBefore) || colInstant.equals(delBefore)) {
+          break;
+        }
+      }
+      if (numToKeep == parsedCollections.size()) {
+        log.debug("No old time routed collections to delete.");
+        return curAliases;
+      }
+
+      final List<String> targetList = curAliases.getCollectionAliasListMap().get(aliasName);
+      // remember to delete these... (oldest to newest)
+      for (int i = targetList.size() - 1; i >= numToKeep; i--) {
+        collectionsToDelete.add(targetList.get(i));
+      }
+      // new alias list has only "numToKeep" first items
+      final List<String> collectionsToKeep = targetList.subList(0, numToKeep);
+      final String collectionsToKeepStr = StrUtils.join(collectionsToKeep, ',');
+      return curAliases.cloneWithCollectionAlias(aliasName, collectionsToKeepStr);
+    });
+
+    if (collectionsToDelete.isEmpty()) {
+      return null;
+    }
+
+    log.info("Removing old time routed collections: {}", collectionsToDelete);
+    // Should this be done asynchronously?  If we got "ASYNC" then probably.
+    //   It would shorten the time the Overseer holds a lock on the alias name
+    //   (deleting the collections will be done later and not use that lock).
+    //   Don't bother about parallel; it's unusual to have more than 1.
+    // Note we don't throw an exception here under most cases; instead the response will have information about
+    //   how each delete request went, possibly including a failure message.
+    final CollectionsHandler collHandler = ocmh.overseer.getCoreContainer().getCollectionsHandler();
+    NamedList results = new NamedList();
+    for (String collection : collectionsToDelete) {
+      final SolrParams reqParams = CollectionAdminRequest.deleteCollection(collection).getParams();
+      SolrQueryResponse rsp = new SolrQueryResponse();
+      collHandler.handleRequestBody(new LocalSolrQueryRequest(null, reqParams), rsp);
+      results.add(collection, rsp.getValues());
+    }
+    return results;
+  }
+
+  /**
+   * Creates a collection (for use in a routed alias), waiting for it to be ready before returning.
+   * If the collection already exists then this is not an error.
+   * IMPORTANT: Only call this from an {@link OverseerCollectionMessageHandler.Cmd}.
+   */
+  static NamedList createCollectionAndWait(ClusterState clusterState, 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(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.BAD_REQUEST,
+          "We require an explicit " + COLL_CONF );
+    }
+    createReqParams.set(NAME, createCollName);
+    createReqParams.set("property." + ROUTED_ALIAS_NAME_CORE_PROP, aliasName);
+    // a CollectionOperation reads params and produces a message (Map) that is supposed to be sent to the Overseer.
+    //   Although we could create the Map without it, there are a fair amount of rules we don't want to reproduce.
+    final Map<String, Object> createMsgMap = CollectionsHandler.CollectionOperation.CREATE_OP.execute(
+        new LocalSolrQueryRequest(null, createReqParams),
+        null,
+        ocmh.overseer.getCoreContainer().getCollectionsHandler());
+    createMsgMap.put(Overseer.QUEUE_OPERATION, "create");
+
+    NamedList results = new NamedList();
+    try {
+      // Since we are running in the Overseer here, send the message directly to the Overseer CreateCollectionCmd.
+      // note: there's doesn't seem to be any point in locking on the collection name, so we don't. We currently should
+      //   already have a lock on the alias name which should be sufficient.
+      ocmh.commandMap.get(CollectionParams.CollectionAction.CREATE).call(clusterState, new ZkNodeProps(createMsgMap), results);
+    } catch (SolrException e) {
+      // The collection might already exist, and that's okay -- we can adopt it.
+      if (!e.getMessage().contains("collection already exists")) {
+        throw e;
+      }
+    }
+
+    CollectionsHandler.waitForActiveCollection(createCollName, ocmh.overseer.getCoreContainer(),
+        new OverseerSolrResponse(results));
+    return results;
+  }
+
+  private SolrException newAliasMustExistException(String aliasName) {
+    return new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+        "Alias " + aliasName + " does not exist.");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1527ce57/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 056f108..6519a8e 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
@@ -230,7 +230,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
         .put(CREATEROUTEDALIAS, new CreateAliasCmd(this))
         .put(DELETEALIAS, new DeleteAliasCmd(this))
         .put(MODIFYALIAS, new ModifyAliasCmd(this))
-        .put(ROUTEDALIAS_CREATECOLL, new RoutedAliasCreateCollectionCmd(this))
+        .put(MAINTAINROUTEDALIAS, new MaintainRoutedAliasCmd(this))
         .put(OVERSEERSTATUS, new OverseerStatusCmd(this))
         .put(DELETESHARD, new DeleteShardCmd(this))
         .put(DELETEREPLICA, new DeleteReplicaCmd(this))

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1527ce57/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
deleted file mode 100644
index 2bba70d..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/RoutedAliasCreateCollectionCmd.java
+++ /dev/null
@@ -1,301 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cloud.api.collections;
-
-import java.lang.invoke.MethodHandles;
-import java.text.ParseException;
-import java.time.Instant;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.solr.client.solrj.SolrResponse;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.cloud.Overseer;
-import org.apache.solr.cloud.OverseerSolrResponse;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.Aliases;
-import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.ZkNodeProps;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.handler.admin.CollectionsHandler;
-import org.apache.solr.request.LocalSolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.DateMathParser;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.COLL_CONF;
-import static org.apache.solr.cloud.api.collections.TimeRoutedAlias.CREATE_COLLECTION_PREFIX;
-import static org.apache.solr.cloud.api.collections.TimeRoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP;
-import static org.apache.solr.common.params.CommonParams.NAME;
-
-/**
- * (Internal) For "time routed aliases", both deletes old collections and creates new collections
- * associated with routed aliases.
- *
- * Note: this logic is within an Overseer because we want to leverage the mutual exclusion
- * property afforded by the lock it obtains on the alias name.
- *
- * @since 7.3
- * @lucene.internal
- */
-// TODO rename class to MaintainRoutedAliasCmd
-public class RoutedAliasCreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  public static final String IF_MOST_RECENT_COLL_NAME = "ifMostRecentCollName"; //TODO rename to createAfter
-
-  private final OverseerCollectionMessageHandler ocmh;
-
-  public RoutedAliasCreateCollectionCmd(OverseerCollectionMessageHandler ocmh) {
-    this.ocmh = ocmh;
-  }
-
-  /** Invokes this command from the client.  If there's a problem it will throw an exception. */
-  public static NamedList remoteInvoke(CollectionsHandler collHandler, String aliasName, String mostRecentCollName)
-      throws Exception {
-    final String operation = CollectionParams.CollectionAction.ROUTEDALIAS_CREATECOLL.toLower();
-    Map<String, Object> msg = new HashMap<>();
-    msg.put(Overseer.QUEUE_OPERATION, operation);
-    msg.put(CollectionParams.NAME, aliasName);
-    msg.put(RoutedAliasCreateCollectionCmd.IF_MOST_RECENT_COLL_NAME, mostRecentCollName);
-    final SolrResponse rsp = collHandler.sendToOCPQueue(new ZkNodeProps(msg));
-    if (rsp.getException() != null) {
-      throw rsp.getException();
-    }
-    return rsp.getResponse();
-  }
-
-  @Override
-  public void call(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception {
-    //---- PARSE PRIMARY MESSAGE PARAMS
-    // important that we use NAME for the alias as that is what the Overseer will get a lock on before calling us
-    final String aliasName = message.getStr(NAME);
-    // the client believes this is the mostRecent collection name.  We assert this if provided.
-    final String ifMostRecentCollName = message.getStr(IF_MOST_RECENT_COLL_NAME); // optional
-
-    // TODO collection param (or intervalDateMath override?), useful for data capped collections
-
-    //---- PARSE ALIAS INFO FROM ZK
-    final ZkStateReader.AliasesManager aliasesManager = ocmh.zkStateReader.aliasesManager;
-    final Aliases aliases = aliasesManager.getAliases();
-    final Map<String, String> aliasMetadata = aliases.getCollectionAliasMetadata(aliasName);
-    if (aliasMetadata == null) {
-      throw newAliasMustExistException(aliasName); // if it did exist, we'd have a non-null map
-    }
-    final TimeRoutedAlias timeRoutedAlias = new TimeRoutedAlias(aliasName, aliasMetadata);
-
-    final List<Map.Entry<Instant, String>> parsedCollections =
-        timeRoutedAlias.parseCollections(aliases, () -> newAliasMustExistException(aliasName));
-
-    //---- GET MOST RECENT COLL
-    final Map.Entry<Instant, String> mostRecentEntry = parsedCollections.get(0);
-    final Instant mostRecentCollTimestamp = mostRecentEntry.getKey();
-    final String mostRecentCollName = mostRecentEntry.getValue();
-    if (ifMostRecentCollName != null) {
-      if (!mostRecentCollName.equals(ifMostRecentCollName)) {
-        // Possibly due to race conditions in URPs on multiple leaders calling us at the same time
-        String msg = IF_MOST_RECENT_COLL_NAME + " expected " + ifMostRecentCollName + " but it's " + mostRecentCollName;
-        if (parsedCollections.stream().map(Map.Entry::getValue).noneMatch(ifMostRecentCollName::equals)) {
-          msg += ". Furthermore this collection isn't in the list of collections referenced by the alias.";
-        }
-        log.info(msg);
-        results.add("message", msg);
-        return;
-      }
-    } else if (mostRecentCollTimestamp.isAfter(Instant.now())) {
-      final String msg = "Most recent collection is in the future, so we won't create another.";
-      log.info(msg);
-      results.add("message", msg);
-      return;
-    }
-
-    //---- COMPUTE NEXT COLLECTION NAME
-    final Instant nextCollTimestamp = timeRoutedAlias.computeNextCollTimestamp(mostRecentCollTimestamp);
-    final String createCollName = TimeRoutedAlias.formatCollectionNameFromInstant(aliasName, nextCollTimestamp);
-
-    //---- DELETE OLDEST COLLECTIONS AND REMOVE FROM ALIAS (if configured)
-    NamedList deleteResults = deleteOldestCollectionsAndUpdateAlias(timeRoutedAlias, aliasesManager, nextCollTimestamp);
-    if (deleteResults != null) {
-      results.add("delete", deleteResults);
-    }
-
-    //---- CREATE THE COLLECTION
-    NamedList createResults = createCollectionAndWait(clusterState, aliasName, aliasMetadata,
-        createCollName, ocmh);
-    if (createResults != null) {
-      results.add("create", createResults);
-    }
-
-    //---- UPDATE THE ALIAS WITH NEW COLLECTION
-    aliasesManager.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, ','));
-      }
-    });
-
-  }
-
-  /**
-   * Deletes some of the oldest collection(s) based on {@link TimeRoutedAlias#getAutoDeleteAgeMath()}. If not present
-   * then does nothing.  Returns non-null results if something was deleted (or if we tried to).
-   * {@code now} is the date from which the math is relative to.
-   */
-  NamedList deleteOldestCollectionsAndUpdateAlias(TimeRoutedAlias timeRoutedAlias,
-                                                  ZkStateReader.AliasesManager aliasesManager,
-                                                  Instant now) throws Exception {
-    final String autoDeleteAgeMathStr = timeRoutedAlias.getAutoDeleteAgeMath();
-    if (autoDeleteAgeMathStr == null) {
-      return null;
-    }
-    final Instant delBefore;
-    try {
-      delBefore = new DateMathParser(Date.from(now), timeRoutedAlias.getTimeZone()).parseMath(autoDeleteAgeMathStr).toInstant();
-    } catch (ParseException e) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e); // note: should not happen by this point
-    }
-
-    String aliasName = timeRoutedAlias.getAliasName();
-
-    Collection<String> collectionsToDelete = new LinkedHashSet<>();
-
-    // First update the alias    (there may be no change to make!)
-    aliasesManager.applyModificationAndExportToZk(curAliases -> {
-      // note: we could re-parse the TimeRoutedAlias object from curAliases but I don't think there's a point to it.
-
-      final List<Map.Entry<Instant, String>> parsedCollections =
-          timeRoutedAlias.parseCollections(curAliases, () -> newAliasMustExistException(aliasName));
-
-      //iterating from newest to oldest, find the first collection that has a time <= "before".  We keep this collection
-      // (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;
-      for (Map.Entry<Instant, String> parsedCollection : parsedCollections) {
-        numToKeep++;
-        final Instant colInstant = parsedCollection.getKey();
-        if (colInstant.isBefore(delBefore) || colInstant.equals(delBefore)) {
-          break;
-        }
-      }
-      if (numToKeep == parsedCollections.size()) {
-        log.debug("No old time routed collections to delete.");
-        return curAliases;
-      }
-
-      final List<String> targetList = curAliases.getCollectionAliasListMap().get(aliasName);
-      // remember to delete these... (oldest to newest)
-      for (int i = targetList.size() - 1; i >= numToKeep; i--) {
-        collectionsToDelete.add(targetList.get(i));
-      }
-      // new alias list has only "numToKeep" first items
-      final List<String> collectionsToKeep = targetList.subList(0, numToKeep);
-      final String collectionsToKeepStr = StrUtils.join(collectionsToKeep, ',');
-      return curAliases.cloneWithCollectionAlias(aliasName, collectionsToKeepStr);
-    });
-
-    if (collectionsToDelete.isEmpty()) {
-      return null;
-    }
-
-    log.info("Removing old time routed collections: {}", collectionsToDelete);
-    // Should this be done asynchronously?  If we got "ASYNC" then probably.
-    //   It would shorten the time the Overseer holds a lock on the alias name
-    //   (deleting the collections will be done later and not use that lock).
-    //   Don't bother about parallel; it's unusual to have more than 1.
-    // Note we don't throw an exception here under most cases; instead the response will have information about
-    //   how each delete request went, possibly including a failure message.
-    final CollectionsHandler collHandler = ocmh.overseer.getCoreContainer().getCollectionsHandler();
-    NamedList results = new NamedList();
-    for (String collection : collectionsToDelete) {
-      final SolrParams reqParams = CollectionAdminRequest.deleteCollection(collection).getParams();
-      SolrQueryResponse rsp = new SolrQueryResponse();
-      collHandler.handleRequestBody(new LocalSolrQueryRequest(null, reqParams), rsp);
-      results.add(collection, rsp.getValues());
-    }
-    return results;
-  }
-
-  /**
-   * Creates a collection (for use in a routed alias), waiting for it to be ready before returning.
-   * If the collection already exists then this is not an error.
-   * IMPORTANT: Only call this from an {@link OverseerCollectionMessageHandler.Cmd}.
-   */
-  static NamedList createCollectionAndWait(ClusterState clusterState, 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(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.BAD_REQUEST,
-          "We require an explicit " + COLL_CONF );
-    }
-    createReqParams.set(NAME, createCollName);
-    createReqParams.set("property." + ROUTED_ALIAS_NAME_CORE_PROP, aliasName);
-    // a CollectionOperation reads params and produces a message (Map) that is supposed to be sent to the Overseer.
-    //   Although we could create the Map without it, there are a fair amount of rules we don't want to reproduce.
-    final Map<String, Object> createMsgMap = CollectionsHandler.CollectionOperation.CREATE_OP.execute(
-        new LocalSolrQueryRequest(null, createReqParams),
-        null,
-        ocmh.overseer.getCoreContainer().getCollectionsHandler());
-    createMsgMap.put(Overseer.QUEUE_OPERATION, "create");
-
-    NamedList results = new NamedList();
-    try {
-      // Since we are running in the Overseer here, send the message directly to the Overseer CreateCollectionCmd.
-      // note: there's doesn't seem to be any point in locking on the collection name, so we don't. We currently should
-      //   already have a lock on the alias name which should be sufficient.
-      ocmh.commandMap.get(CollectionParams.CollectionAction.CREATE).call(clusterState, new ZkNodeProps(createMsgMap), results);
-    } catch (SolrException e) {
-      // The collection might already exist, and that's okay -- we can adopt it.
-      if (!e.getMessage().contains("collection already exists")) {
-        throw e;
-      }
-    }
-
-    CollectionsHandler.waitForActiveCollection(createCollName, ocmh.overseer.getCoreContainer(),
-        new OverseerSolrResponse(results));
-    return results;
-  }
-
-  private SolrException newAliasMustExistException(String aliasName) {
-    return new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-        "Alias " + aliasName + " does not exist.");
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1527ce57/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java
index 0ffaaa9..16529c8 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
@@ -51,7 +51,7 @@ import static org.apache.solr.common.params.CommonParams.TZ;
  * Holds configuration for a routed alias, and some common code and constants.
  *
  * @see CreateAliasCmd
- * @see RoutedAliasCreateCollectionCmd
+ * @see MaintainRoutedAliasCmd
  * @see org.apache.solr.update.processor.TimeRoutedAliasUpdateProcessor
  */
 public class TimeRoutedAlias {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1527ce57/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 cd47c5c..b8ccbd3 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
@@ -31,7 +31,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
 import org.apache.solr.cloud.ZkController;
-import org.apache.solr.cloud.api.collections.RoutedAliasCreateCollectionCmd;
+import org.apache.solr.cloud.api.collections.MaintainRoutedAliasCmd;
 import org.apache.solr.cloud.api.collections.TimeRoutedAlias;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Aliases;
@@ -275,7 +275,7 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
     final Semaphore semaphore = aliasToSemaphoreMap.computeIfAbsent(getAliasName(), n -> new Semaphore(1));
     if (semaphore.tryAcquire()) {
       try {
-        RoutedAliasCreateCollectionCmd.remoteInvoke(collHandler, getAliasName(), mostRecentCollName);
+        MaintainRoutedAliasCmd.remoteInvoke(collHandler, getAliasName(), mostRecentCollName);
         // we don't care about the response.  It's possible no collection was created because
         //  of a race and that's okay... we'll ultimately retry any way.
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1527ce57/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 74fbb95..d404d60 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
@@ -80,7 +80,7 @@ public interface CollectionParams {
     DELETEALIAS(true, LockLevel.COLLECTION),
     MODIFYALIAS(true, LockLevel.COLLECTION),
     LISTALIASES(false, LockLevel.NONE),
-    ROUTEDALIAS_CREATECOLL(true, LockLevel.COLLECTION),
+    MAINTAINROUTEDALIAS(true, LockLevel.COLLECTION),
     DELETEROUTEDALIASCOLLECTIONS(true, LockLevel.COLLECTION),
     SPLITSHARD(true, LockLevel.SHARD),
     DELETESHARD(true, LockLevel.SHARD),