From c97551cc36c2b89320428a821e45c79951260673 Mon Sep 17 00:00:00 2001 From: Gus Heck Date: Mon, 15 Jul 2019 09:57:13 -0400 Subject: [PATCH] SOLR-13375 - 2 dimensional routed aliases --- .../apache/solr/cloud/OverseerTaskQueue.java | 59 +- .../solr/cloud/api/collections/AliasCmd.java | 29 +- .../api/collections/CategoryRoutedAlias.java | 142 ++-- .../cloud/api/collections/CreateAliasCmd.java | 3 +- .../api/collections/DeleteCollectionCmd.java | 18 +- .../collections/DimensionalRoutedAlias.java | 365 +++++++++ .../MaintainCategoryRoutedAliasCmd.java | 183 ----- .../collections/MaintainRoutedAliasCmd.java | 182 +++++ .../MaintainTimeRoutedAliasCmd.java | 257 ------- .../OverseerCollectionMessageHandler.java | 37 +- .../cloud/api/collections/RoutedAlias.java | 394 ++++++++-- .../api/collections/TimeRoutedAlias.java | 472 ++++++------ .../org/apache/solr/core/CoreContainer.java | 7 + .../handler/admin/BaseHandlerApiSupport.java | 29 + .../handler/admin/CollectionsHandler.java | 27 +- .../processor/RoutedAliasUpdateProcessor.java | 6 +- .../solr/cloud/AliasIntegrationTest.java | 30 +- .../solr/cloud/CreateRoutedAliasTest.java | 5 +- ...ategoryRoutedAliasUpdateProcessorTest.java | 6 + ...nsionalRoutedAliasUpdateProcessorTest.java | 726 ++++++++++++++++++ .../RoutedAliasUpdateProcessorTest.java | 68 +- .../TimeRoutedAliasUpdateProcessorTest.java | 177 ++--- solr/solr-ref-guide/src/aliases.adoc | 126 ++- .../src/collection-aliasing.adoc | 231 +++++- .../solr/client/solrj/RoutedAliasTypes.java | 47 ++ .../solrj/request/CollectionAdminRequest.java | 174 ++++- .../solr/common/cloud/ConnectionManager.java | 41 +- .../cloud/ZkClientConnectionStrategy.java | 36 +- .../solr/common/params/CollectionParams.java | 3 +- .../apispec/collections.Commands.json | 8 + 30 files changed, 2804 insertions(+), 1084 deletions(-) create mode 100644 solr/core/src/java/org/apache/solr/cloud/api/collections/DimensionalRoutedAlias.java delete mode 100644 solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainCategoryRoutedAliasCmd.java create mode 100644 solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java delete mode 100644 solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainTimeRoutedAliasCmd.java create mode 100644 solr/core/src/test/org/apache/solr/update/processor/DimensionalRoutedAliasUpdateProcessorTest.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/RoutedAliasTypes.java diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java index 62d788c31ed..bbfeadba9a0 100644 --- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java +++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java @@ -16,16 +16,20 @@ */ package org.apache.solr.cloud; -import com.codahale.metrics.Timer; import java.lang.invoke.MethodHandles; import java.util.ArrayList; import java.util.List; import java.util.TreeSet; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Predicate; + +import com.codahale.metrics.Timer; +import org.apache.solr.common.SolrException; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.util.Pair; @@ -44,9 +48,12 @@ import org.slf4j.LoggerFactory; */ public class OverseerTaskQueue extends ZkDistributedQueue { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - + private static final String RESPONSE_PREFIX = "qnr-" ; + private final AtomicBoolean shuttingDown = new AtomicBoolean(false); + private final AtomicInteger pendingResponses = new AtomicInteger(0); + public OverseerTaskQueue(SolrZkClient zookeeper, String dir) { this(zookeeper, dir, new Stats()); } @@ -54,7 +61,18 @@ public class OverseerTaskQueue extends ZkDistributedQueue { public OverseerTaskQueue(SolrZkClient zookeeper, String dir, Stats stats) { super(zookeeper, dir, stats); } - + + public void allowOverseerPendingTasksToComplete() { + shuttingDown.set(true); + while (pendingResponses.get() > 0) { + try { + Thread.sleep(50); + } catch (InterruptedException e) { + log.error("Interrupted while waiting for overseer queue to drain before shutdown!"); + } + } + } + /** * Returns true if the queue contains a task with the specified async id. */ @@ -119,11 +137,11 @@ public class OverseerTaskQueue extends ZkDistributedQueue { private final Condition eventReceived; private WatchedEvent event; private Event.EventType latchEventType; - + LatchWatcher() { this(null); } - + LatchWatcher(Event.EventType eventType) { this.lock = new ReentrantLock(); this.eventReceived = lock.newCondition(); @@ -170,7 +188,7 @@ public class OverseerTaskQueue extends ZkDistributedQueue { /** * Inserts data into zookeeper. - * + * * @return true if data was successfully added */ private String createData(String path, byte[] data, CreateMode mode) @@ -187,13 +205,16 @@ public class OverseerTaskQueue extends ZkDistributedQueue { } } } - + /** * Offer the data and wait for the response - * + * */ public QueueEvent offer(byte[] data, long timeout) throws KeeperException, InterruptedException { + if (shuttingDown.get()) { + throw new SolrException(SolrException.ErrorCode.CONFLICT,"Solr is shutting down, no more overseer tasks may be offered"); + } Timer.Context time = stats.time(dir + "_offer"); try { // Create and watch the response node before creating the request node; @@ -207,6 +228,7 @@ public class OverseerTaskQueue extends ZkDistributedQueue { createRequestNode(data, watchID); if (stat != null) { + pendingResponses.incrementAndGet(); watcher.await(timeout); } byte[] bytes = zookeeper.getData(watchID, null, null, true); @@ -217,6 +239,7 @@ public class OverseerTaskQueue extends ZkDistributedQueue { return event; } finally { time.stop(); + pendingResponses.decrementAndGet(); } } @@ -285,7 +308,7 @@ public class OverseerTaskQueue extends ZkDistributedQueue { } return null; } - + public static class QueueEvent { @Override public int hashCode() { @@ -294,7 +317,7 @@ public class OverseerTaskQueue extends ZkDistributedQueue { result = prime * result + ((id == null) ? 0 : id.hashCode()); return result; } - + @Override public boolean equals(Object obj) { if (this == obj) return true; @@ -306,36 +329,36 @@ public class OverseerTaskQueue extends ZkDistributedQueue { } else if (!id.equals(other.id)) return false; return true; } - + private WatchedEvent event = null; private String id; private byte[] bytes; - + QueueEvent(String id, byte[] bytes, WatchedEvent event) { this.id = id; this.bytes = bytes; this.event = event; } - + public void setId(String id) { this.id = id; } - + public String getId() { return id; } - + public void setBytes(byte[] bytes) { this.bytes = bytes; } - + public byte[] getBytes() { return bytes; } - + public WatchedEvent getWatchedEvent() { return event; } - + } } 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 03cc4046de9..3afc805d53a 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,9 +17,6 @@ package org.apache.solr.cloud.api.collections; -import java.lang.invoke.MethodHandles; -import java.util.ArrayList; -import java.util.List; import java.util.Map; import org.apache.solr.cloud.Overseer; @@ -28,15 +25,11 @@ import org.apache.solr.common.SolrException; import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.CollectionProperties; 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.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; @@ -50,7 +43,12 @@ 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()); + final OverseerCollectionMessageHandler ocmh; + + AliasCmd(OverseerCollectionMessageHandler ocmh) { + this.ocmh = ocmh; + } + /** * 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.

@@ -100,18 +98,5 @@ abstract class AliasCmd implements OverseerCollectionMessageHandler.Cmd { return results; } - void updateAlias(String aliasName, ZkStateReader.AliasesManager aliasesManager, String createCollName) { - aliasesManager.applyModificationAndExportToZk(curAliases -> { - final List curTargetCollections = curAliases.getCollectionAliasListMap().get(aliasName); - if (curTargetCollections.contains(createCollName)) { - return curAliases; - } else { - List 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, ',')); - } - }); - } + } diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CategoryRoutedAlias.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CategoryRoutedAlias.java index 7d731494a6f..c2f4ec8baf2 100644 --- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CategoryRoutedAlias.java +++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CategoryRoutedAlias.java @@ -18,6 +18,7 @@ package org.apache.solr.cloud.api.collections; import java.lang.invoke.MethodHandles; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; @@ -27,22 +28,19 @@ import java.util.Set; import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; -import org.apache.solr.cloud.ZkController; +import org.apache.solr.client.solrj.RoutedAliasTypes; import org.apache.solr.common.SolrException; -import org.apache.solr.common.SolrException.ErrorCode; +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.CommonParams; -import org.apache.solr.core.CoreContainer; -import org.apache.solr.core.SolrCore; -import org.apache.solr.handler.admin.CollectionsHandler; -import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.update.AddUpdateCommand; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST; -public class CategoryRoutedAlias implements RoutedAlias { +public class CategoryRoutedAlias extends RoutedAlias { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private static final String COLLECTION_INFIX = "__CRA__"; @@ -51,7 +49,7 @@ public class CategoryRoutedAlias implements RoutedAlias { // expects a collection but also works with an alias to handle or error out on empty alias. The // collection with this constant as a suffix is automatically removed after the alias begins to // receive data. - public static final String UNINITIALIZED = "NEW_CATEGORY_ROUTED_ALIAS_WAITING_FOR_DATA__TEMP"; + public static final String UNINITIALIZED = "NEW_CATEGORY_ROUTED_ALIAS_WAITING_FOR_DATA_TEMP"; @SuppressWarnings("WeakerAccess") public static final String ROUTER_MAX_CARDINALITY = "router.maxCardinality"; @@ -67,7 +65,6 @@ public class CategoryRoutedAlias implements RoutedAlias { ROUTER_MAX_CARDINALITY ))); - @SuppressWarnings("WeakerAccess") public static final String ROUTER_MUST_MATCH = "router.mustMatch"; /** @@ -78,7 +75,7 @@ public class CategoryRoutedAlias implements RoutedAlias { ROUTER_MAX_CARDINALITY, ROUTER_MUST_MATCH))); - private Aliases parsedAliases; // a cached reference to the source of what we parse into parsedCollectionsDesc + private Aliases aliases; private final String aliasName; private final Map aliasMetadata; private final Integer maxCardinality; @@ -89,18 +86,18 @@ public class CategoryRoutedAlias implements RoutedAlias { this.aliasMetadata = aliasMetadata; this.maxCardinality = parseMaxCardinality(aliasMetadata.get(ROUTER_MAX_CARDINALITY)); final String mustMatch = this.aliasMetadata.get(ROUTER_MUST_MATCH); - this.mustMatch = mustMatch == null? null: compileMustMatch(mustMatch); + this.mustMatch = mustMatch == null ? null : compileMustMatch(mustMatch); } @Override - public boolean updateParsedCollectionAliases(ZkController zkController) { - final Aliases aliases = zkController.getZkStateReader().getAliases(); // note: might be different from last request - if (this.parsedAliases != aliases) { - if (this.parsedAliases != null) { + public boolean updateParsedCollectionAliases(ZkStateReader zkStateReader, boolean contextualize) { + final Aliases aliases = zkStateReader.getAliases(); // note: might be different from last request + if (this.aliases != aliases) { + if (this.aliases != null) { log.debug("Observing possibly updated alias: {}", getAliasName()); } // slightly inefficient, but not easy to make changes to the return value of parseCollections - this.parsedAliases = aliases; + this.aliases = aliases; return true; } return false; @@ -116,23 +113,28 @@ public class CategoryRoutedAlias implements RoutedAlias { return aliasMetadata.get(ROUTER_FIELD); } + @Override + public RoutedAliasTypes getRoutedAliasType() { + return RoutedAliasTypes.CATEGORY; + } + @Override public void validateRouteValue(AddUpdateCommand cmd) throws SolrException { - if (this.parsedAliases == null) { - updateParsedCollectionAliases(cmd.getReq().getCore().getCoreContainer().getZkController()); + if (this.aliases == null) { + updateParsedCollectionAliases(cmd.getReq().getCore().getCoreContainer().getZkController().zkStateReader, false); } Object fieldValue = cmd.getSolrInputDocument().getFieldValue(getRouteField()); // possible future enhancement: allow specification of an "unknown" category name to where we can send // docs that are uncategorized. if (fieldValue == null) { - throw new SolrException(BAD_REQUEST,"Route value is null"); + throw new SolrException(BAD_REQUEST, "Route value is null"); } String dataValue = String.valueOf(fieldValue); String candidateCollectionName = buildCollectionNameFromValue(dataValue); - List cols = getCollectionList(this.parsedAliases); + List cols = getCollectionList(this.aliases); if (cols.contains(candidateCollectionName)) { return; @@ -173,53 +175,6 @@ public class CategoryRoutedAlias implements RoutedAlias { return aliasName + COLLECTION_INFIX + safeKeyValue(value); } - /** - * Method to possibly create a collection. It's possible that the collection will already have been created - * either by a prior invocation in this thread or another thread. This method is idempotent, multiple invocations - * are harmless. - * - * @param cmd The command that might cause collection creation - * @return the collection to which the the update should be directed, possibly a newly created collection. - */ - @Override - public String createCollectionsIfRequired(AddUpdateCommand cmd) { - SolrQueryRequest req = cmd.getReq(); - SolrCore core = req.getCore(); - CoreContainer coreContainer = core.getCoreContainer(); - CollectionsHandler collectionsHandler = coreContainer.getCollectionsHandler(); - String dataValue = String.valueOf(cmd.getSolrInputDocument().getFieldValue(getRouteField())); - String candidateCollectionName = buildCollectionNameFromValue(dataValue); - - try { - // Note: CRA's have no way to predict values that determine collection so preemptive async creation - // is not possible. We have no choice but to block and wait (to do otherwise would imperil the overseer). - do { - if (getCollectionList(this.parsedAliases).contains(candidateCollectionName)) { - return candidateCollectionName; - } else { - // this could time out in which case we simply let it throw an error - MaintainCategoryRoutedAliasCmd.remoteInvoke(collectionsHandler, getAliasName(), candidateCollectionName); - // It's possible no collection was created because of a race and that's okay... we'll retry. - - // Ensure our view of the aliases has updated. If we didn't do this, our zkStateReader might - // not yet know about the new alias (thus won't see the newly added collection to it), and we might think - // we failed. - collectionsHandler.getCoreContainer().getZkController().getZkStateReader().aliasesManager.update(); - - // we should see some sort of update to our aliases - if (!updateParsedCollectionAliases(coreContainer.getZkController())) { // thus we didn't make progress... - // this is not expected, even in known failure cases, but we check just in case - throw new SolrException(ErrorCode.SERVER_ERROR, - "We need to create a new category routed collection but for unknown reasons were unable to do so."); - } - } - } while (true); - } catch (SolrException e) { - throw e; - } catch (Exception e) { - throw new SolrException(ErrorCode.SERVER_ERROR, e); - } - } private Integer parseMaxCardinality(String maxCardinality) { try { @@ -239,15 +194,18 @@ public class CategoryRoutedAlias implements RoutedAlias { } } - private List getCollectionList(Aliases p) { - return p.getCollectionAliasListMap().get(this.aliasName); - } - @Override public String computeInitialCollectionName() { return buildCollectionNameFromValue(UNINITIALIZED); } + @Override + String[] formattedRouteValues(SolrInputDocument doc) { + String routeField = getRouteField(); + String fieldValue = (String) doc.getFieldValue(routeField); + return new String[] {safeKeyValue(fieldValue)}; + } + @Override public Map getAliasMetadata() { return aliasMetadata; @@ -262,4 +220,44 @@ public class CategoryRoutedAlias implements RoutedAlias { public Set getOptionalParams() { return OPTIONAL_ROUTER_PARAMS; } -} + + @Override + public CandidateCollection findCandidateGivenValue(AddUpdateCommand cmd) { + Object value = cmd.getSolrInputDocument().getFieldValue(getRouteField()); + String targetColName = buildCollectionNameFromValue(String.valueOf(value)); + ZkStateReader zkStateReader = cmd.getReq().getCore().getCoreContainer().getZkController().zkStateReader; + updateParsedCollectionAliases(zkStateReader, true); + List collectionList = getCollectionList(this.aliases); + if (collectionList.contains(targetColName)) { + return new CandidateCollection(CreationType.NONE, targetColName); + } else { + return new CandidateCollection(CreationType.SYNCHRONOUS, targetColName); + } + } + + @Override + protected String getHeadCollectionIfOrdered(AddUpdateCommand cmd) { + return buildCollectionNameFromValue(String.valueOf(cmd.getSolrInputDocument().getFieldValue(getRouteField()))); + } + + @Override + protected List calculateActions(String targetCol) { + List collectionList = getCollectionList(aliases); + if (!collectionList.contains(targetCol)) { + ArrayList actionList = new ArrayList<>(); + actionList.add(new Action(this,ActionType.ENSURE_EXISTS, targetCol)); + for (String s : collectionList) { + // can't remove the uninitialized on the first pass otherwise there is a risk of momentarily having + // an empty alias if thread scheduling plays tricks on us. + if (s.contains(UNINITIALIZED) && collectionList.size() > 1) { + actionList.add(new Action(this,ActionType.ENSURE_REMOVED, s)); + } + } + return actionList; + } else { + return Collections.emptyList(); + } + } + + +} \ No newline at end of file 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 8d0d45cd5a2..fdadf77aedb 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 @@ -41,7 +41,6 @@ import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR; public class CreateAliasCmd extends AliasCmd { - private final OverseerCollectionMessageHandler ocmh; private static boolean anyRoutingParams(ZkNodeProps message) { return message.keySet().stream().anyMatch(k -> k.startsWith(CollectionAdminParams.ROUTER_PREFIX)); @@ -49,7 +48,7 @@ public class CreateAliasCmd extends AliasCmd { @SuppressWarnings("WeakerAccess") public CreateAliasCmd(OverseerCollectionMessageHandler ocmh) { - this.ocmh = ocmh; + super(ocmh); } @Override diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteCollectionCmd.java index 6fe02ee84c1..648f5ba3910 100644 --- a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteCollectionCmd.java +++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteCollectionCmd.java @@ -18,13 +18,6 @@ package org.apache.solr.cloud.api.collections; -import static org.apache.solr.common.params.CollectionAdminParams.COLOCATED_WITH; -import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES; -import static org.apache.solr.common.params.CollectionAdminParams.WITH_COLLECTION; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETE; -import static org.apache.solr.common.params.CommonAdminParams.ASYNC; -import static org.apache.solr.common.params.CommonParams.NAME; - import java.lang.invoke.MethodHandles; import java.util.ArrayList; import java.util.HashSet; @@ -58,6 +51,13 @@ import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CollectionAdminParams.COLOCATED_WITH; +import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES; +import static org.apache.solr.common.params.CollectionAdminParams.WITH_COLLECTION; +import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETE; +import static org.apache.solr.common.params.CommonAdminParams.ASYNC; +import static org.apache.solr.common.params.CommonParams.NAME; + public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private final OverseerCollectionMessageHandler ocmh; @@ -70,6 +70,10 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd @Override public void call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception { + Object o = message.get(MaintainRoutedAliasCmd.INVOKED_BY_ROUTED_ALIAS); + if (o != null) { + ((Runnable)o).run(); // this will ensure the collection is removed from the alias before it disappears. + } final String extCollection = message.getStr(NAME); ZkStateReader zkStateReader = ocmh.zkStateReader; diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DimensionalRoutedAlias.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DimensionalRoutedAlias.java new file mode 100644 index 00000000000..499642ae1e6 --- /dev/null +++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DimensionalRoutedAlias.java @@ -0,0 +1,365 @@ +/* + * 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.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import org.apache.solr.client.solrj.RoutedAliasTypes; +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.update.AddUpdateCommand; + +import static org.apache.solr.client.solrj.request.CollectionAdminRequest.DimensionalRoutedAlias.addDimensionIndexIfRequired; +import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR; + +public class DimensionalRoutedAlias extends RoutedAlias { + + private final String name; + private List dimensions; + + // things we don't need to calc twice... + private Set reqParams = new HashSet<>(); + private Set optParams = new HashSet<>(); + private Map aliasMetadata; + + private static final Pattern SEP_MATCHER = Pattern.compile("("+ + Arrays.stream(RoutedAliasTypes.values()) + .filter(v -> v != RoutedAliasTypes.DIMENSIONAL) + .map(RoutedAliasTypes::getSeparatorPrefix) + .collect(Collectors.joining("|")) + + ")"); + + + DimensionalRoutedAlias(List dimensions, String name, Map props) { + this.dimensions = dimensions; + this.name = name; + this.aliasMetadata = props; + } + + interface Deffered { + T get(); + } + + static RoutedAlias dimensionForType(Map props, RoutedAliasTypes type, + int index, Deffered dra) { + // this switch must have a case for every element of the RoutedAliasTypes enum EXCEPT DIMENSIONAL + switch (type) { + case TIME: + return new TimeRoutedAliasDimension(props, index, dra); + case CATEGORY: + return new CategoryRoutedAliasDimension(props, index, dra); + default: + // if we got a type not handled by the switch there's been a bogus implementation. + throw new SolrException(SERVER_ERROR, "Router " + type + " is not fully implemented. If you see this" + + "error in an official release please file a bug report. Available types were:" + + Arrays.asList(RoutedAliasTypes.values())); + } + } + + + @Override + public boolean updateParsedCollectionAliases(ZkStateReader zkStateReader, boolean contextualize) { + boolean result = false; + for (RoutedAlias dimension : dimensions) { + result |= dimension.updateParsedCollectionAliases(zkStateReader, contextualize); + } + return result; + } + + @Override + public String computeInitialCollectionName() { + StringBuilder sb = new StringBuilder(getAliasName()); + for (RoutedAlias dimension : dimensions) { + // N. B. getAliasName is generally safe as a regex because it must conform to collection naming rules + // and those rules exclude regex special characters. A malicious request might do something expensive, but + // if you have malicious users able to run admin commands and create aliases, it is very likely that you have + // much bigger problems than an expensive regex. + String routeString = dimension.computeInitialCollectionName().replaceAll(dimension.getAliasName() , ""); + sb.append(routeString); + } + return sb.toString(); + } + + @Override + String[] formattedRouteValues(SolrInputDocument doc) { + String[] result = new String[dimensions.size()]; + for (int i = 0; i < dimensions.size(); i++) { + RoutedAlias dimension = dimensions.get(i); + result[i] = dimension.formattedRouteValues(doc)[0]; + } + return result; + } + + @Override + public String getAliasName() { + return name; + } + + @Override + public String getRouteField() { + throw new UnsupportedOperationException("DRA's route via their dimensions, this method should not be called"); + } + + @Override + public RoutedAliasTypes getRoutedAliasType() { + return RoutedAliasTypes.DIMENSIONAL; + } + + @Override + public void validateRouteValue(AddUpdateCommand cmd) throws SolrException { + for (RoutedAlias dimension : dimensions) { + dimension.validateRouteValue(cmd); + } + } + + + @Override + public Map getAliasMetadata() { + return aliasMetadata; + } + + @Override + public Set getRequiredParams() { + if (reqParams.size() == 0) { + indexParams(reqParams, dimensions, RoutedAlias::getRequiredParams); + // the top level Dimensional[foo,bar] designation needs to be retained + reqParams.add(ROUTER_TYPE_NAME); + reqParams.add(ROUTER_FIELD); + } + return reqParams; + } + + @Override + public Set getOptionalParams() { + + if (optParams.size() == 0) { + indexParams(optParams, dimensions, RoutedAlias::getOptionalParams); + } + return optParams; + + } + + @Override + public CandidateCollection findCandidateGivenValue(AddUpdateCommand cmd) { + contextualizeDimensions(formattedRouteValues(cmd.solrDoc)); + List subPartCandidates = new ArrayList<>(); + for (RoutedAlias dimension : dimensions) { + subPartCandidates.add(dimension.findCandidateGivenValue(cmd)); + } + + StringBuilder col2Create = new StringBuilder(getAliasName()); + StringBuilder destCol = new StringBuilder(getAliasName()); + CreationType max = CreationType.NONE; + for (CandidateCollection subCol : subPartCandidates) { + col2Create.append(subCol.getCreationCollection()); + destCol.append(subCol.getDestinationCollection()); + if (subCol.getCreationType().ordinal() > max.ordinal()) { + max = subCol.getCreationType(); + } + } + return new CandidateCollection(max,destCol.toString(),col2Create.toString()); + } + + @Override + protected String getHeadCollectionIfOrdered(AddUpdateCommand cmd) { + StringBuilder head = new StringBuilder(getAliasName()); + for (RoutedAlias dimension : dimensions) { + head.append(dimension.getHeadCollectionIfOrdered(cmd).substring(getAliasName().length())); + } + return head.toString(); + } + + + /** + * Determine the combination of adds/deletes implied by the arrival of a document destined for the + * specified collection. + * + * @param targetCol the collection for which a document is destined. + * @return A list of actions across the DRA. + */ + @Override + protected List calculateActions(String targetCol) { + String[] routeValues = SEP_MATCHER.split(targetCol); + // remove the alias name to avoid all manner of off by one errors... + routeValues = Arrays.copyOfRange(routeValues,1,routeValues.length); + List> dimActs = new ArrayList<>(routeValues.length); + contextualizeDimensions(routeValues); + for (int i = 0; i < routeValues.length; i++) { + String routeValue = routeValues[i]; + RoutedAlias dim = dimensions.get(i); + dimActs.add(dim.calculateActions(dim.getAliasName() + getSeparatorPrefix(dim)+ routeValue) ); + } + Set result = new LinkedHashSet<>(); + StringBuilder currentSuffix = new StringBuilder(); + for (int i = routeValues.length -1; i >=0 ; i--) { // also lowest up to match + String routeValue = routeValues[i]; + RoutedAlias dim = dimensions.get(i); + String dimStr = dim.getRoutedAliasType().getSeparatorPrefix() + routeValue; + List actions = dimActs.get(i); + for (Iterator iterator = actions.iterator(); iterator.hasNext(); ) { + Action action = iterator.next(); + iterator.remove(); + result.add(new Action(action.sourceAlias, action.actionType, action.targetCollection + currentSuffix)); + } + result.addAll(actions); + Set revisedResult = new LinkedHashSet<>(); + + for (Action action : result) { + if (action.sourceAlias == dim) { + revisedResult.add(action); // should already have the present value + continue; + } + // the rest are from lower dimensions and thus require a prefix. + revisedResult.add(new Action(action.sourceAlias, action.actionType,dimStr + action.targetCollection)); + } + result = revisedResult; + currentSuffix.append(dimStr); + } + Set revisedResult = new LinkedHashSet<>(); + for (Action action : result) { + revisedResult.add(new Action(action.sourceAlias, action.actionType,getAliasName() + action.targetCollection)); + } + return new ArrayList<>(revisedResult); + } + + private void contextualizeDimensions(String[] routeValues) { + for (RoutedAlias dimension : dimensions) { + ((DraContextualized)dimension).setContext(routeValues); + } + } + + + private static String getSeparatorPrefix(RoutedAlias dim) { + return dim.getRoutedAliasType().getSeparatorPrefix(); + } + + private static void indexParams(Set result, List dimensions, Function> supplier) { + for (int i = 0; i < dimensions.size(); i++) { + RoutedAlias dimension = dimensions.get(i); + Set params = supplier.apply(dimension); + for (String param : params) { + addDimensionIndexIfRequired(result, i, param); + } + } + } + + private interface DraContextualized { + + static List dimensionCollectionListView(int index, Aliases aliases, Deffered dra, String[] context, boolean ordered) { + List cols = aliases.getCollectionAliasListMap().get(dra.get().name); + LinkedHashSet view = new LinkedHashSet<>(cols.size()); + List dimensions = dra.get().dimensions; + for (String col : cols) { + Matcher m = SEP_MATCHER.matcher(col); + if (!m.find()) { + throw new IllegalStateException("Invalid Dimensional Routed Alias name:" + col); + } + String[] split = SEP_MATCHER.split(col); + if (split.length != dimensions.size() + 1) { + throw new IllegalStateException("Dimension Routed Alias collection with wrong number of dimensions. (" + + col + ") expecting " + dimensions.stream().map(d -> + d.getRoutedAliasType().toString()).collect(Collectors.toList())); + } + boolean matchesAllHigherDims = index == 0; + boolean matchesAllLowerDims = context == null || index == context.length - 1; + if (context != null) { + for (int i = 0; i < context.length; i++) { + if (i == index) { + continue; + } + String s = split[i+1]; + String ctx = context[i]; + if (i <= index) { + matchesAllHigherDims |= s.equals(ctx); + } else { + matchesAllLowerDims |= s.equals(ctx); + } + } + } else { + matchesAllHigherDims = true; + matchesAllLowerDims = true; + } + // dimensions with an implicit order need to start from their initial configuration + // and count up to maintain order in the alias collection list with respect to that dimension + if (matchesAllHigherDims && !ordered || matchesAllHigherDims && matchesAllLowerDims) { + view.add("" + getSeparatorPrefix(dimensions.get(index)) + split[index + 1]); + } + } + return new ArrayList<>(view); + } + + void setContext(String[] context); + } + + private static class TimeRoutedAliasDimension extends TimeRoutedAlias implements DraContextualized { + private final int index; + private final Deffered dra; + private String[] context; + + TimeRoutedAliasDimension(Map props, int index, Deffered dra) throws SolrException { + super("", props); + this.index = index; + this.dra = dra; + } + + @Override + List getCollectionList(Aliases aliases) { + return DraContextualized.dimensionCollectionListView(index, aliases, dra, context, true); + } + + @Override + public void setContext(String[] context) { + this.context = context; + } + } + + private static class CategoryRoutedAliasDimension extends CategoryRoutedAlias implements DraContextualized { + private final int index; + private final Deffered dra; + private String[] context; + + CategoryRoutedAliasDimension(Map props, int index, Deffered dra) { + super("", props); + this.index = index; + this.dra = dra; + } + + @Override + List getCollectionList(Aliases aliases) { + return DraContextualized.dimensionCollectionListView(index, aliases, dra, context, false); + } + + @Override + public void setContext(String[] context) { + this.context = context; + } + } +} diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainCategoryRoutedAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainCategoryRoutedAliasCmd.java deleted file mode 100644 index 28787355acd..00000000000 --- a/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainCategoryRoutedAliasCmd.java +++ /dev/null @@ -1,183 +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.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.Semaphore; - -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.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.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.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import static org.apache.solr.cloud.api.collections.CategoryRoutedAlias.UNINITIALIZED; -import static org.apache.solr.common.params.CommonParams.NAME; - -public class MaintainCategoryRoutedAliasCmd extends AliasCmd { - private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - @SuppressWarnings("WeakerAccess") - public static final String IF_CATEGORY_COLLECTION_NOT_FOUND = "ifCategoryCollectionNotFound"; - - private static NamedSimpleSemaphore DELETE_LOCK = new NamedSimpleSemaphore(); - - private final OverseerCollectionMessageHandler ocmh; - - MaintainCategoryRoutedAliasCmd(OverseerCollectionMessageHandler ocmh) { - this.ocmh = ocmh; - } - - /** - * Invokes this command from the client. If there's a problem it will throw an exception. - * Please note that is important to never add async to this invocation. This method must - * block (up to the standard OCP timeout) to prevent large batches of add's from sending a message - * to the overseer for every document added in RoutedAliasUpdateProcessor. - */ - @SuppressWarnings("WeakerAccess") - public static void remoteInvoke(CollectionsHandler collHandler, String aliasName, String categoryCollection) - throws Exception { - final String operation = CollectionParams.CollectionAction.MAINTAINCATEGORYROUTEDALIAS.toLower(); - Map msg = new HashMap<>(); - msg.put(Overseer.QUEUE_OPERATION, operation); - msg.put(CollectionParams.NAME, aliasName); - msg.put(IF_CATEGORY_COLLECTION_NOT_FOUND, categoryCollection); - final SolrResponse rsp = collHandler.sendToOCPQueue(new ZkNodeProps(msg)); - if (rsp.getException() != null) { - throw rsp.getException(); - } - } - - @Override - public void call(ClusterState state, 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 collection name should exist. Our goal is to ensure it does. - final String categoryRequired = message.getStr(IF_CATEGORY_COLLECTION_NOT_FOUND); // optional - - - //---- PARSE ALIAS INFO FROM ZK - final ZkStateReader.AliasesManager aliasesManager = ocmh.zkStateReader.aliasesManager; - final Aliases aliases = aliasesManager.getAliases(); - final Map aliasMetadata = aliases.getCollectionAliasProperties(aliasName); - if (aliasMetadata.isEmpty()) { - throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, - "Alias " + aliasName + " does not exist or is not a routed alias."); // if it did exist, we'd have a non-null map - } - final CategoryRoutedAlias categoryRoutedAlias = (CategoryRoutedAlias) RoutedAlias.fromProps(aliasName, aliasMetadata); - - if (categoryRoutedAlias == null) { - throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, getClass() + " got alias metadata with an " + - "invalid routing type and produced null"); - } - - - //---- SEARCH FOR REQUESTED COLL - Map> collectionAliasListMap = aliases.getCollectionAliasListMap(); - - // if we found it the collection already exists and we're done (concurrent creation on another request) - // so this if does not need an else. - if (!collectionAliasListMap.get(aliasName).contains(categoryRequired)) { - //---- DETECT and REMOVE the initial place holder collection if it still exists: - - String initialCollection = categoryRoutedAlias.buildCollectionNameFromValue(UNINITIALIZED); - - // important not to delete the place holder collection it until after a second collection exists, - // otherwise we have a situation where the alias has no collections briefly and concurrent - // requests to the alias will fail with internal errors (incl. queries etc). - - List colList = new ArrayList<>(collectionAliasListMap.get(aliasName)); - if (colList.contains(initialCollection) && colList.size() > 1 ) { - - // need to run the delete async, otherwise we may deadlock with incoming updates that are attempting - // to create collections (they will have called getCore() but may be waiting on the overseer alias lock - // we hold and we will be waiting for the Core reference count to reach zero). By deleting asynchronously - // we allow this request to complete and the alias lock to be released, which allows the update to complete - // so that we can do the delete. Additionally we don't want to cause multiple delete operations during - // the time the delete is in progress, since that just wastes overseer cycles. - // TODO: check TRA's are protected against this - - if (DELETE_LOCK.tryAcquire(aliasName)) { - // note that the overseer might not have any cores (and the unit test occasionally catches this) - ocmh.overseer.getCoreContainer().runAsync(() -> { - aliasesManager.applyModificationAndExportToZk(curAliases -> { - colList.remove(initialCollection); - final String collectionsToKeepStr = StrUtils.join(colList, ','); - return curAliases.cloneWithCollectionAlias(aliasName, collectionsToKeepStr); - }); - final CollectionsHandler collHandler = ocmh.overseer.getCoreContainer().getCollectionsHandler(); - final SolrParams reqParams = CollectionAdminRequest - .deleteCollection(initialCollection).getParams(); - SolrQueryResponse rsp = new SolrQueryResponse(); - try { - collHandler.handleRequestBody(new LocalSolrQueryRequest(null, reqParams), rsp); - } catch (Exception e) { - log.error("Could not delete initial collection from CRA", e); - } - //noinspection unchecked - results.add(UNINITIALIZED, rsp.getValues()); - DELETE_LOCK.release(aliasName); - }); - } - } - - //---- CREATE THE COLLECTION - NamedList createResults = createCollectionAndWait(state, aliasName, aliasMetadata, - categoryRequired, ocmh); - if (createResults != null) { - //noinspection unchecked - results.add("create", createResults); - } - //---- UPDATE THE ALIAS WITH NEW COLLECTION - updateAlias(aliasName, aliasesManager, categoryRequired); - } - } - - private static class NamedSimpleSemaphore { - - private final HashMap semaphores = new HashMap<>(); - - NamedSimpleSemaphore() { - } - - boolean tryAcquire(String name) { - return semaphores.computeIfAbsent(name, s -> new Semaphore(1)).tryAcquire(); - } - - public void release(String name) { - semaphores.get(name).release(); - } - } -} 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 00000000000..693e6c1278d --- /dev/null +++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java @@ -0,0 +1,182 @@ +/* + * 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.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.solr.client.solrj.SolrResponse; +import org.apache.solr.cloud.Overseer; +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.CollectionProperties; +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.util.NamedList; +import org.apache.solr.common.util.StrUtils; +import org.apache.solr.handler.admin.CollectionsHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.solr.common.params.CommonParams.NAME; + +public class MaintainRoutedAliasCmd extends AliasCmd { + + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + static final String INVOKED_BY_ROUTED_ALIAS = "invokedByRoutedAlias"; + static final String ROUTED_ALIAS_TARGET_COL = "routedAliasTargetCol"; + + MaintainRoutedAliasCmd(OverseerCollectionMessageHandler ocmh) { + super(ocmh); + } + + /** + * Invokes this command from the client. If there's a problem it will throw an exception. + * Please note that is important to never add async to this invocation. This method must + * block (up to the standard OCP timeout) to prevent large batches of add's from sending a message + * to the overseer for every document added in RoutedAliasUpdateProcessor. + */ + static void remoteInvoke(CollectionsHandler collHandler, String aliasName, String targetCol) + throws Exception { + final String operation = CollectionParams.CollectionAction.MAINTAINROUTEDALIAS.toLower(); + Map msg = new HashMap<>(); + msg.put(Overseer.QUEUE_OPERATION, operation); + msg.put(CollectionParams.NAME, aliasName); + msg.put(MaintainRoutedAliasCmd.ROUTED_ALIAS_TARGET_COL, targetCol); + final SolrResponse rsp = collHandler.sendToOCPQueue(new ZkNodeProps(msg)); + if (rsp.getException() != null) { + throw rsp.getException(); + } + } + + void addCollectionToAlias(String aliasName, ZkStateReader.AliasesManager aliasesManager, String createCollName) { + aliasesManager.applyModificationAndExportToZk(curAliases -> { + final List curTargetCollections = curAliases.getCollectionAliasListMap().get(aliasName); + if (curTargetCollections.contains(createCollName)) { + return curAliases; + } else { + List newTargetCollections = new ArrayList<>(curTargetCollections.size() + 1); + // prepend it on purpose (thus reverse sorted). Solr alias resolution defaults to the first collection in a list + newTargetCollections.add(createCollName); + newTargetCollections.addAll(curTargetCollections); + return curAliases.cloneWithCollectionAlias(aliasName, StrUtils.join(newTargetCollections, ',')); + } + }); + } + + private void removeCollectionFromAlias(String aliasName, ZkStateReader.AliasesManager aliasesManager, String createCollName) { + aliasesManager.applyModificationAndExportToZk(curAliases -> { + final List curTargetCollections = curAliases.getCollectionAliasListMap().get(aliasName); + if (curTargetCollections.contains(createCollName)) { + List newTargetCollections = new ArrayList<>(curTargetCollections.size()); + newTargetCollections.addAll(curTargetCollections); + newTargetCollections.remove(createCollName); + return curAliases.cloneWithCollectionAlias(aliasName, StrUtils.join(newTargetCollections, ',')); + } else { + return curAliases; + } + }); + } + + @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); + final String routeValue = message.getStr(ROUTED_ALIAS_TARGET_COL); + + final ZkStateReader.AliasesManager aliasesManager = ocmh.zkStateReader.aliasesManager; + final Aliases aliases = aliasesManager.getAliases(); + final Map aliasMetadata = aliases.getCollectionAliasProperties(aliasName); + if (aliasMetadata.isEmpty()) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, + "Alias " + aliasName + " does not exist or is not a routed alias."); // if it did exist, we'd have a non-null map + } + final RoutedAlias ra = RoutedAlias.fromProps(aliasName, aliasMetadata); + if (ra == null) { + throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "MaintainRoutedAlias called on non-routed alias"); + } + + ra.updateParsedCollectionAliases(ocmh.zkStateReader, true); + List actions = ra.calculateActions(routeValue); + for (RoutedAlias.Action action : actions) { + boolean exists = ocmh.zkStateReader.getClusterState().getCollectionOrNull(action.targetCollection) != null; + switch (action.actionType) { + case ENSURE_REMOVED: + if (exists) { + ocmh.tpe.submit(() -> { + try { + deleteTargetCollection(clusterState, results, aliasName, aliasesManager, action); + } catch (Exception e) { + log.warn("Deletion of {} by {} failed (this might be ok if two clients were " + + "writing to a routed alias at the same time and both caused a deletion)", + action.targetCollection, ra.getAliasName()); + log.debug("Exception for last message:", e); + } + }); + } + break; + case ENSURE_EXISTS: + if (!exists) { + addTargetCollection(clusterState, results, aliasName, aliasesManager, aliasMetadata, action); + } else { + // check that the collection is properly integrated into the alias (see + // TimeRoutedAliasUpdateProcessorTest.java:141). Presently we need to ensure inclusion in the alias + // and the presence of the appropriate collection property. Note that this only works if the collection + // happens to fall where we would have created one already. Support for un-even collection sizes will + // take additional work (though presently they might work if the below book keeping is done by hand) + if (!ra.getCollectionList(aliases).contains(action.targetCollection)) { + addCollectionToAlias(aliasName, aliasesManager, action.targetCollection); + Map collectionProperties = ocmh.zkStateReader + .getCollectionProperties(action.targetCollection, 1000); + if (!collectionProperties.containsKey(RoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP)) { + CollectionProperties props = new CollectionProperties(ocmh.zkStateReader.getZkClient()); + props.setCollectionProperty(action.targetCollection, RoutedAlias.ROUTED_ALIAS_NAME_CORE_PROP, aliasName); + } + } + } + break; + default: + throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown action type!"); + } + } + } + + public void addTargetCollection(ClusterState clusterState, NamedList results, String aliasName, ZkStateReader.AliasesManager aliasesManager, Map aliasMetadata, RoutedAlias.Action action) throws Exception { + NamedList createResults = createCollectionAndWait(clusterState, aliasName, aliasMetadata, + action.targetCollection, ocmh); + if (createResults != null) { + results.add("create", createResults); + } + addCollectionToAlias(aliasName, aliasesManager, action.targetCollection); + } + + public void deleteTargetCollection(ClusterState clusterState, NamedList results, String aliasName, ZkStateReader.AliasesManager aliasesManager, RoutedAlias.Action action) throws Exception { + Map delProps = new HashMap<>(); + delProps.put(INVOKED_BY_ROUTED_ALIAS, + (Runnable) () -> removeCollectionFromAlias(aliasName, aliasesManager, action.targetCollection)); + delProps.put(NAME, action.targetCollection); + ZkNodeProps messageDelete = new ZkNodeProps(delProps); + new DeleteCollectionCmd(ocmh).call(clusterState, messageDelete, results); + } +} 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 deleted file mode 100644 index 8fdf7697f4c..00000000000 --- a/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainTimeRoutedAliasCmd.java +++ /dev/null @@ -1,257 +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.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; -import org.apache.solr.client.solrj.request.CollectionAdminRequest; -import org.apache.solr.cloud.Overseer; -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.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.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 MaintainTimeRoutedAliasCmd extends AliasCmd { - 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 MaintainTimeRoutedAliasCmd(OverseerCollectionMessageHandler ocmh) { - this.ocmh = ocmh; - } - - /** - * Invokes this command from the client. If there's a problem it will throw an exception. - * Please note that is important to never add async to this invocation. This method must - * block (up to the standard OCP timeout) to prevent large batches of add's from sending a message - * to the overseer for every document added in RoutedAliasUpdateProcessor. - */ - public static NamedList remoteInvoke(CollectionsHandler collHandler, String aliasName, String mostRecentCollName) - throws Exception { - final String operation = CollectionParams.CollectionAction.MAINTAINTIMEROUTEDALIAS.toLower(); - Map msg = new HashMap<>(); - msg.put(Overseer.QUEUE_OPERATION, operation); - msg.put(CollectionParams.NAME, aliasName); - msg.put(MaintainTimeRoutedAliasCmd.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 aliasMetadata = aliases.getCollectionAliasProperties(aliasName); - if (aliasMetadata.isEmpty()) { - throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, - "Alias " + aliasName + " does not exist or is not a routed alias."); // if it did exist, we'd have a non-null map - } - final TimeRoutedAlias timeRoutedAlias = new TimeRoutedAlias(aliasName, aliasMetadata); - - final List> parsedCollections = - timeRoutedAlias.parseCollections(aliases); - - //---- GET MOST RECENT COLL - final Map.Entry 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 - updateAlias(aliasName, aliasesManager, createCollName); - - } - - /** - * 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 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> parsedCollections = - timeRoutedAlias.parseCollections(curAliases); - - //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; - 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 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... parsed collections={}", parsedCollections); - return curAliases; - } - log.debug("Collections will be deleted... parsed collections={}", parsedCollections); - Map> collectionAliasListMap = curAliases.getCollectionAliasListMap(); - final List 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--) { - 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 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; - } - -} 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 d8be1f2ccf9..6fbab131a5f 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 @@ -234,8 +234,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler, .put(CREATEALIAS, new CreateAliasCmd(this)) .put(DELETEALIAS, new DeleteAliasCmd(this)) .put(ALIASPROP, new SetAliasPropCmd(this)) - .put(MAINTAINTIMEROUTEDALIAS, new MaintainTimeRoutedAliasCmd(this)) - .put(MAINTAINCATEGORYROUTEDALIAS, new MaintainCategoryRoutedAliasCmd(this)) + .put(MAINTAINROUTEDALIAS, new MaintainRoutedAliasCmd(this)) .put(OVERSEERSTATUS, new OverseerStatusCmd(this)) .put(DELETESHARD, new DeleteShardCmd(this)) .put(DELETEREPLICA, new DeleteReplicaCmd(this)) @@ -428,7 +427,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler, } catch (TimeoutException e) { return false; } - + return true; } @@ -613,20 +612,20 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler, private void modifyCollection(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception { - + final String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP); //the rest of the processing is based on writing cluster state properties //remove the property here to avoid any errors down the pipeline due to this property appearing String configName = (String) message.getProperties().remove(CollectionAdminParams.COLL_CONF); - + if(configName != null) { validateConfigOrThrowSolrException(configName); - + boolean isLegacyCloud = Overseer.isLegacy(zkStateReader); createConfNode(cloudManager.getDistribStateManager(), configName, collectionName, isLegacyCloud); reloadCollection(null, new ZkNodeProps(NAME, collectionName), results); } - + overseer.offerStateUpdate(Utils.toJSON(message)); TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource); @@ -688,7 +687,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler, } } } - + if (result.size() == coreNames.size()) { return result; } else { @@ -697,7 +696,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler, if (timeout.hasTimedOut()) { throw new SolrException(ErrorCode.SERVER_ERROR, "Timed out waiting to see all replicas: " + coreNames + " in cluster state. Last state: " + coll); } - + Thread.sleep(100); } } @@ -720,7 +719,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler, * That check should be done before the config node is created. */ public static void createConfNode(DistribStateManager stateManager, String configName, String coll, boolean isLegacyCloud) throws IOException, AlreadyExistsException, BadVersionException, KeeperException, InterruptedException { - + if (configName != null) { String collDir = ZkStateReader.COLLECTIONS_ZKNODE + "/" + coll; log.debug("creating collections conf node {} ", collDir); @@ -738,7 +737,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler, } } } - + private List collectionCmd(ZkNodeProps message, ModifiableSolrParams params, NamedList results, Replica.State stateMatcher, String asyncId) { return collectionCmd( message, params, results, stateMatcher, asyncId, Collections.emptySet()); @@ -800,7 +799,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler, } failure.add(key, value); } - + @SuppressWarnings("unchecked") private static void addSuccess(NamedList results, String key, Object value) { SimpleOrderedMap success = (SimpleOrderedMap) results.get("success"); @@ -840,7 +839,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler, response.add("STATUS", "failed"); return response; } - + String r = (String) srsp.getSolrResponse().getResponse().get("STATUS"); if (r.equals("running")) { log.debug("The task is still RUNNING, continuing to wait."); @@ -943,15 +942,15 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler, */ @Deprecated static boolean INCLUDE_TOP_LEVEL_RESPONSE = true; - + public ShardRequestTracker syncRequestTracker() { return new ShardRequestTracker(null); } - + public ShardRequestTracker asyncRequestTracker(String asyncId) { return new ShardRequestTracker(asyncId); } - + public class ShardRequestTracker{ private final String asyncId; private final NamedList shardAsyncIdByNode = new NamedList(); @@ -959,7 +958,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler, private ShardRequestTracker(String asyncId) { this.asyncId = asyncId; } - + /** * Send request to all replicas of a slice * @return List of replicas which is not live for receiving the request @@ -983,7 +982,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler, } return notLiveReplicas; } - + public void sendShardRequest(String nodeName, ModifiableSolrParams params, ShardHandler shardHandler) { sendShardRequest(nodeName, params, shardHandler, adminPath, zkStateReader); @@ -1008,7 +1007,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler, shardHandler.submit(sreq, replica, sreq.params); } - + void processResponses(NamedList results, ShardHandler shardHandler, boolean abortOnError, String msgOnError) { processResponses(results, shardHandler, abortOnError, msgOnError, Collections.emptySet()); } 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 fb658cdfdb8..4aac46aa1cb 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 @@ -17,52 +17,58 @@ package org.apache.solr.cloud.api.collections; +import java.lang.invoke.MethodHandles; +import java.util.AbstractMap.SimpleEntry; +import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; +import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Objects; import java.util.Set; +import java.util.stream.Collectors; import com.google.common.collect.Sets; +import org.apache.solr.client.solrj.RoutedAliasTypes; import org.apache.solr.cloud.ZkController; 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.CommonParams; +import org.apache.solr.core.CoreContainer; +import org.apache.solr.core.SolrCore; +import org.apache.solr.handler.admin.CollectionsHandler; +import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.update.AddUpdateCommand; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST; import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR; import static org.apache.solr.common.params.CollectionAdminParams.ROUTER_PREFIX; -public interface RoutedAlias { +public abstract class RoutedAlias { + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - /** - * 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 { - @Override - public String getSeparatorPrefix() { - return "__TRA__"; - } - }, - CATEGORY { - @Override - public String getSeparatorPrefix() { - return "__CRA__"; - } - }; - public abstract String getSeparatorPrefix(); - } + @SuppressWarnings("WeakerAccess") + public static final String ROUTER_TYPE_NAME = ROUTER_PREFIX + "name"; + @SuppressWarnings("WeakerAccess") + public static final String ROUTER_FIELD = ROUTER_PREFIX + "field"; + public static final String CREATE_COLLECTION_PREFIX = "create-collection."; + @SuppressWarnings("WeakerAccess") + public static final Set MINIMAL_REQUIRED_PARAMS = Sets.newHashSet(ROUTER_TYPE_NAME, ROUTER_FIELD); + public static final String ROUTED_ALIAS_NAME_CORE_PROP = "routedAliasName"; // core prop + private static final String DIMENSIONAL = "Dimensional["; - String ROUTER_TYPE_NAME = ROUTER_PREFIX + "name"; - String ROUTER_FIELD = ROUTER_PREFIX + "field"; - String CREATE_COLLECTION_PREFIX = "create-collection."; - Set MINIMAL_REQUIRED_PARAMS = Sets.newHashSet(ROUTER_TYPE_NAME, ROUTER_FIELD); - String ROUTED_ALIAS_NAME_CORE_PROP = "routedAliasName"; // core prop + // This class is created once per request and the overseer methods prevent duplicate create requests + // from creating extra copies via locking on the alias name. All we need to track here is that we don't + // spam preemptive creates to the overseer multiple times from *this* request. + boolean preemptiveCreateOnceAlready = false; - static SolrException newAliasMustExistException(String aliasName) { + public static SolrException newAliasMustExistException(String aliasName) { throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, "Routed alias " + aliasName + " appears to have been removed during request processing."); } @@ -76,19 +82,102 @@ public interface RoutedAlias { * @return An implementation appropriate for the supplied properties, or null if no type is specified. * @throws SolrException If the properties are invalid or the router type is unknown. */ - static RoutedAlias fromProps(String aliasName, Map props) throws SolrException { + public static RoutedAlias fromProps(String aliasName, Map props) throws SolrException { String typeStr = props.get(ROUTER_TYPE_NAME); if (typeStr == null) { return null; // non-routed aliases are being created } - SupportedRouterTypes routerType; - try { - routerType = SupportedRouterTypes.valueOf(typeStr.toUpperCase(Locale.ENGLISH)); - } catch (IllegalArgumentException e) { - throw new SolrException(BAD_REQUEST, "Router name: " + typeStr + " is not in supported types, " - + Arrays.asList(SupportedRouterTypes.values())); + List routerTypes = new ArrayList<>(); + // check for Dimensional[foo,bar,baz] + if (typeStr.startsWith(DIMENSIONAL)) { + // multi-dimensional routed alias + typeStr = typeStr.substring(DIMENSIONAL.length(), typeStr.length() - 1); + String[] types = typeStr.split(","); + java.util.List fields = new ArrayList<>(); + if (types.length > 2) { + throw new SolrException(BAD_REQUEST,"More than 2 dimensions is not supported yet. " + + "Please monitor SOLR-13628 for progress"); + } + for (int i = 0; i < types.length; i++) { + String type = types[i]; + addRouterTypeOf(type, routerTypes); + + // v2 api case - the v2 -> v1 mapping mechanisms can't handle this conversion because they expect + // strings or arrays of strings, not lists of objects. + if (props.containsKey("router.routerList")) { + @SuppressWarnings("unchecked") // working around solrparams inability to express lists of objects + HashMap tmp = new HashMap(props); + @SuppressWarnings("unchecked") // working around solrparams inability to express lists of objects + List> v2RouterList = (List>) tmp.get("router.routerList"); + Map o = v2RouterList.get(i); + for (Map.Entry entry : o.entrySet()) { + props.put(ROUTER_PREFIX + i + "." + entry.getKey(), String.valueOf(entry.getValue())); + } + } + // Here we need to push the type into each dimension's params. We could have eschewed the + // "Dimensional[dim1,dim2]" style notation, to simplify this case but I think it's nice + // to be able to understand the dimensionality at a glance without having to hunt for name properties + // in the list of properties for each dimension. + String typeName = ROUTER_PREFIX + i + ".name"; + // can't use computeIfAbsent because the non-dimensional case where typeName is present + // happens to be an unmodifiable map and will fail. + if (!props.containsKey(typeName)) { + props.put(typeName, type); + } + fields.add(props.get(ROUTER_PREFIX + i + ".field")); + } + // this next remove is checked for key because when we build from aliases.json's data it we get an + // immutable map which would cause UnsupportedOperationException to be thrown. This remove is here + // to prevent this property from making it into aliases.json + //noinspection RedundantCollectionOperation + if (props.containsKey("router.routerList")) { + props.remove("router.routerList"); + } + // Keep code that handles single dimensions happy by providing this value, otherwise ignored. + if (!props.containsKey(ROUTER_FIELD)) { + props.put(ROUTER_FIELD, String.join(",", fields)); + } + } else { + // non-dimensional case + addRouterTypeOf(typeStr, routerTypes); } + if (routerTypes.size() == 1) { + RoutedAliasTypes routerType = routerTypes.get(0); + return routedAliasForType(aliasName, props, routerType); + } else { + List dimensions = new ArrayList<>(); + // this array allows us to get past the chicken/egg problem of needing access to the + // DRA inside the dimensions, but needing the dimensions to create the DRA + DimensionalRoutedAlias[] dra = new DimensionalRoutedAlias[1]; + for (int i = 0; i < routerTypes.size(); i++) { + RoutedAliasTypes routerType = routerTypes.get(i); + // NOTE setting the name to empty string is very important here, as that allows us to simply + // concatenate the "names" of the parts to get the correct collection name for the DRA + dimensions.add(DimensionalRoutedAlias.dimensionForType( selectForIndex(i, props), routerType, i, () -> dra[0])); + } + return dra[0] = new DimensionalRoutedAlias(dimensions, props.get(CommonParams.NAME), props); + } + } + + private static void addRouterTypeOf(String type, List routerTypes) { + try { + routerTypes.add(RoutedAliasTypes.valueOf(type.toUpperCase(Locale.ENGLISH))); + } catch (IllegalArgumentException iae) { + throw new SolrException(BAD_REQUEST, "Router name: " + type + " is not in supported types, " + + Arrays.asList(RoutedAliasTypes.values())); + } + } + + private static Map selectForIndex(int i, Map original) { + return original.entrySet().stream() + .filter(e -> e.getKey().matches("(((?!^router\\.).)*$|(^router\\." + i + ".*$))")) + .map(e -> new SimpleEntry<>(e.getKey().replaceAll("(.*\\.)" + i + "\\.(.*)", "$1$2"), e.getValue())) + .collect(Collectors.toMap(SimpleEntry::getKey, SimpleEntry::getValue)); + } + + private static RoutedAlias routedAliasForType(String aliasName, Map props, RoutedAliasTypes routerType) { + // this switch must have a case for every element of the RoutedAliasTypes enum EXCEPT DIMENSIONAL switch (routerType) { case TIME: return new TimeRoutedAlias(aliasName, props); @@ -98,7 +187,7 @@ public interface RoutedAlias { // if we got a type not handled by the switch there's been a bogus implementation. throw new SolrException(SERVER_ERROR, "Router " + routerType + " is not fully implemented. If you see this" + "error in an official release please file a bug report. Available types were:" - + Arrays.asList(SupportedRouterTypes.values())); + + Arrays.asList(RoutedAliasTypes.values())); } } @@ -108,36 +197,41 @@ public interface RoutedAlias { * Note that this will return true if some other alias was modified or if properties were modified. These * are spurious and the caller should be written to be tolerant of no material changes. */ - boolean updateParsedCollectionAliases(ZkController zkController); + public abstract boolean updateParsedCollectionAliases(ZkStateReader zkStateReader, boolean conextualize); + + List getCollectionList(Aliases aliases) { + return aliases.getCollectionAliasListMap().get(getAliasName()); + } /** * Create the initial collection for this RoutedAlias if applicable. - * + *

* Routed Aliases do not aggregate existing collections, instead they create collections on the fly. If the initial * collection can be determined from initialization parameters it should be calculated here. * * @return optional string of initial collection name */ - String computeInitialCollectionName(); + abstract String computeInitialCollectionName(); + abstract String[] formattedRouteValues(SolrInputDocument doc) ; /** * The name of the alias. This name is used in place of a collection name for both queries and updates. * * @return The name of the Alias. */ - String getAliasName(); - - String getRouteField(); + public abstract String getAliasName(); + abstract String getRouteField(); + abstract RoutedAliasTypes getRoutedAliasType(); /** * Check that the value we will be routing on is legal for this type of routed alias. * * @param cmd the command containing the document */ - void validateRouteValue(AddUpdateCommand cmd) throws SolrException; + public abstract void validateRouteValue(AddUpdateCommand cmd) throws SolrException; /** * Create any required collections and return the name of the collection to which the current document should be sent. @@ -146,15 +240,221 @@ public interface RoutedAlias { * @return The name of the proper destination collection for the document which may or may not be a * newly created collection */ - String createCollectionsIfRequired(AddUpdateCommand cmd); + public String createCollectionsIfRequired(AddUpdateCommand cmd) { + + // Even though it is possible that multiple requests hit this code in the 1-2 sec that + // it takes to create a collection, it's an established anti-pattern to feed data with a very large number + // of client connections. This in mind, we only guard against spamming the overseer within a batch of + // updates. We are intentionally tolerating a low level of redundant requests in favor of simpler code. Most + // super-sized installations with many update clients will likely be multi-tenant and multiple tenants + // probably don't write to the same alias. As such, we have deferred any solution to the "many clients causing + // collection creation simultaneously" problem until such time as someone actually has that problem in a + // real world use case that isn't just an anti-pattern. + CandidateCollection candidateCollectionDesc = findCandidateGivenValue(cmd); + + try { + // It's important not to add code between here and the prior call to findCandidateGivenValue() + // in processAdd() that invokes updateParsedCollectionAliases(). Doing so would update parsedCollectionsDesc + // and create a race condition. When Routed aliases have an implicit sort for their collections we + // are relying on the fact that collectionList.get(0) is returning the head of the parsed collections that + // existed when the collection list was consulted for the candidate value. If this class updates it's notion + // of the list of collections since candidateCollectionDesc was chosen, we could create collection n+2 + // instead of collection n+1. + return createAllRequiredCollections( cmd, candidateCollectionDesc); + } catch (SolrException e) { + throw e; + } catch (Exception e) { + throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e); + } + } /** * @return get alias related metadata */ - Map getAliasMetadata(); + abstract Map getAliasMetadata(); - Set getRequiredParams(); + public abstract Set getRequiredParams(); - Set getOptionalParams(); + public abstract Set getOptionalParams(); + abstract CandidateCollection findCandidateGivenValue(AddUpdateCommand cmd); + + class CandidateCollection { + private final CreationType creationType; + private final String destinationCollection; + private final String creationCollection; + + CandidateCollection(CreationType creationType, String destinationCollection, String creationCollection) { + this.creationType = creationType; + this.destinationCollection = destinationCollection; + this.creationCollection = creationCollection; + } + + CandidateCollection(CreationType creationType, String collection) { + this.creationType = creationType; + this.destinationCollection = collection; + this.creationCollection = collection; + } + + CreationType getCreationType() { + return creationType; + } + + String getDestinationCollection() { + return destinationCollection; + } + + String getCreationCollection() { + return creationCollection; + } + } + + /** + * Create as many collections as required. This method loops to allow for the possibility that the route value + * requires more than one collection to be created. Since multiple threads may be invoking maintain on separate + * requests to the same alias, we must pass in a descriptor that details what collection is to be created. + * This assumption is checked when the command is executed in the overseer. When this method + * finds that all collections required have been created it returns the (possibly new) destination collection + * for the document that caused the creation cycle. + * + * @param cmd the update command being processed + * @param targetCollectionDesc the descriptor for the presently selected collection . + * @return The destination collection, possibly created during this method's execution + */ + private String createAllRequiredCollections(AddUpdateCommand cmd, CandidateCollection targetCollectionDesc) { + + SolrQueryRequest req = cmd.getReq(); + SolrCore core = req.getCore(); + CoreContainer coreContainer = core.getCoreContainer(); + do { + switch (targetCollectionDesc.getCreationType()) { + case NONE: + return targetCollectionDesc.destinationCollection; // we don't need another collection + case SYNCHRONOUS: + targetCollectionDesc = doSynchronous( cmd, targetCollectionDesc, coreContainer); + break; + case ASYNC_PREEMPTIVE: + return doPreemptive(targetCollectionDesc, core, coreContainer); + default: + throw unknownCreateType(); + } + } while (true); + } + + private CandidateCollection doSynchronous(AddUpdateCommand cmd, CandidateCollection targetCollectionDesc, CoreContainer coreContainer) { + ensureCollection(targetCollectionDesc.getCreationCollection(), coreContainer); // *should* throw if fails for some reason but... + ZkController zkController = coreContainer.getZkController(); + updateParsedCollectionAliases(zkController.zkStateReader, true); + List observedCols = zkController.zkStateReader.aliasesManager.getAliases().getCollectionAliasListMap().get(getAliasName()); + if (!observedCols.contains(targetCollectionDesc.creationCollection)) { + // if collection creation did not occur we've failed. Bail out. + throw new SolrException(SERVER_ERROR, "After we attempted to create " + targetCollectionDesc.creationCollection + " it did not exist"); + } + // then recalculate the candiate, which may result in continuation or termination the loop calling this method + targetCollectionDesc = findCandidateGivenValue(cmd); + return targetCollectionDesc; + } + + private String doPreemptive(CandidateCollection targetCollectionDesc, SolrCore core, CoreContainer coreContainer) { + + if (!this.preemptiveCreateOnceAlready) { + preemptiveAsync(() -> { + try { + ensureCollection(targetCollectionDesc.creationCollection, coreContainer); + } catch (Exception e) { + log.error("Async creation of a collection for routed Alias " + this.getAliasName() + " failed!", e); + } + }, core); + } + return targetCollectionDesc.destinationCollection; + } + + /** + * Calculate the head collection (i.e. the most recent one for a TRA) if this routed alias has an + * implicit order, or if the collection is unordered return the appropriate collection name + * for the value in the current document. This method should never return null. + */ + abstract protected String getHeadCollectionIfOrdered(AddUpdateCommand cmd); + + private void preemptiveAsync(Runnable r, SolrCore core) { + preemptiveCreateOnceAlready = true; + core.runAsync(r); + } + + private SolrException unknownCreateType() { + return new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown creation type while adding " + + "document to a Time Routed Alias! This is a bug caused when a creation type has been added but " + + "not all code has been updated to handle it."); + } + + void ensureCollection(String targetCollection, CoreContainer coreContainer) { + CollectionsHandler collectionsHandler = coreContainer.getCollectionsHandler(); + + // Invoke MANINTAIN_ROUTED_ALIAS (in the Overseer, locked by alias name). It will create the collection + // and update the alias contingent on the requested collection name not already existing. + // otherwise it will return (without error). + try { + MaintainRoutedAliasCmd.remoteInvoke(collectionsHandler, getAliasName(), targetCollection); + // 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. + + // Ensure our view of the aliases has updated. If we didn't do this, our zkStateReader might + // not yet know about the new alias (thus won't see the newly added collection to it), and we might think + // we failed. + coreContainer.getZkController().getZkStateReader().aliasesManager.update(); + updateParsedCollectionAliases(coreContainer.getZkController().getZkStateReader(),false); + } catch (RuntimeException e) { + throw e; + } catch (Exception e) { + throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e); + } + } + + /** + * Determine the combination of adds/deletes implied by the arrival of a document destined for the + * specified collection. + * + * @param targetCol the collection for which a document is destined. + * @return A list of actions across the DRA. + */ + protected abstract List calculateActions(String targetCol); + + protected static class Action { + final RoutedAlias sourceAlias; + final ActionType actionType; + final String targetCollection; // dra's need to edit this so not final + + public Action(RoutedAlias sourceAlias, ActionType actionType, String targetCollection) { + this.sourceAlias = sourceAlias; + this.actionType = actionType; + this.targetCollection = targetCollection; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Action action = (Action) o; + return Objects.equals(sourceAlias, action.sourceAlias) && + actionType == action.actionType && + Objects.equals(targetCollection, action.targetCollection); + } + + @Override + public int hashCode() { + return Objects.hash(sourceAlias, actionType, targetCollection); + } + + } + + enum ActionType { + ENSURE_REMOVED, + ENSURE_EXISTS + } + + enum CreationType { + NONE, + ASYNC_PREEMPTIVE, + SYNCHRONOUS, + } } 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 c11e6edeac3..96fb0ef2520 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 @@ -20,6 +20,7 @@ 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.ZoneOffset; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatterBuilder; @@ -27,28 +28,25 @@ import java.time.temporal.ChronoField; import java.time.temporal.ChronoUnit; import java.util.AbstractMap; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.Date; -import java.util.HashSet; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.TimeZone; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import com.google.common.base.MoreObjects; -import org.apache.solr.cloud.ZkController; +import org.apache.solr.client.solrj.RoutedAliasTypes; 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.CommonParams; import org.apache.solr.common.params.MapSolrParams; import org.apache.solr.common.params.RequiredSolrParams; -import org.apache.solr.core.CoreContainer; -import org.apache.solr.core.SolrCore; -import org.apache.solr.handler.admin.CollectionsHandler; -import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.update.AddUpdateCommand; import org.apache.solr.update.processor.RoutedAliasUpdateProcessor; import org.apache.solr.util.DateMathParser; @@ -57,9 +55,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.apache.commons.lang3.StringUtils.isNotBlank; -import static org.apache.solr.cloud.api.collections.TimeRoutedAlias.CreationType.ASYNC_PREEMPTIVE; -import static org.apache.solr.cloud.api.collections.TimeRoutedAlias.CreationType.NONE; -import static org.apache.solr.cloud.api.collections.TimeRoutedAlias.CreationType.SYNCHRONOUS; +import static org.apache.solr.cloud.api.collections.RoutedAlias.CreationType.ASYNC_PREEMPTIVE; +import static org.apache.solr.cloud.api.collections.RoutedAlias.CreationType.NONE; +import static org.apache.solr.cloud.api.collections.RoutedAlias.CreationType.SYNCHRONOUS; import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST; import static org.apache.solr.common.params.CollectionAdminParams.ROUTER_PREFIX; import static org.apache.solr.common.params.CommonParams.TZ; @@ -68,17 +66,12 @@ import static org.apache.solr.common.params.CommonParams.TZ; * Holds configuration for a routed alias, and some common code and constants. * * @see CreateAliasCmd - * @see MaintainTimeRoutedAliasCmd + * @see MaintainRoutedAliasCmd * @see RoutedAliasUpdateProcessor */ -public class TimeRoutedAlias implements RoutedAlias { +public class TimeRoutedAlias extends 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 - // the overseer multiple times from *this* request. - private volatile boolean preemptiveCreateOnceAlready = false; + public static final RoutedAliasTypes TYPE = RoutedAliasTypes.TIME; // These two fields may be updated within the calling thread during processing but should // never be updated by any async creation thread. @@ -86,8 +79,11 @@ public class TimeRoutedAlias implements RoutedAlias { private Aliases parsedCollectionsAliases; // a cached reference to the source of what we parse into parsedCollectionsDesc // These are parameter names to routed alias creation, AND are stored as metadata with the alias. + @SuppressWarnings("WeakerAccess") public static final String ROUTER_START = ROUTER_PREFIX + "start"; + @SuppressWarnings("WeakerAccess") public static final String ROUTER_INTERVAL = ROUTER_PREFIX + "interval"; + @SuppressWarnings("WeakerAccess") public static final String ROUTER_MAX_FUTURE = ROUTER_PREFIX + "maxFutureMs"; public static final String ROUTER_AUTO_DELETE_AGE = ROUTER_PREFIX + "autoDeleteAge"; public static final String ROUTER_PREEMPTIVE_CREATE_MATH = ROUTER_PREFIX + "preemptiveCreateMath"; @@ -96,26 +92,21 @@ public class TimeRoutedAlias implements RoutedAlias { /** * Parameters required for creating a routed alias */ - public static final Set REQUIRED_ROUTER_PARAMS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList( - CommonParams.NAME, - ROUTER_TYPE_NAME, - ROUTER_FIELD, - ROUTER_START, - ROUTER_INTERVAL))); + @SuppressWarnings("WeakerAccess") + public static final Set REQUIRED_ROUTER_PARAMS = Set.of( + CommonParams.NAME, ROUTER_TYPE_NAME, ROUTER_FIELD, ROUTER_START, ROUTER_INTERVAL); /** * Optional parameters for creating a routed alias excluding parameters for collection creation. */ //TODO lets find a way to remove this as it's harder to maintain than required list - public static final Set OPTIONAL_ROUTER_PARAMS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList( - ROUTER_MAX_FUTURE, - ROUTER_AUTO_DELETE_AGE, - ROUTER_PREEMPTIVE_CREATE_MATH, - TZ))); // kinda special + @SuppressWarnings("WeakerAccess") + public static final Set OPTIONAL_ROUTER_PARAMS = Set.of( + ROUTER_MAX_FUTURE, ROUTER_AUTO_DELETE_AGE, ROUTER_PREEMPTIVE_CREATE_MATH, TZ); // kinda special // This format must be compatible with collection name limitations - private static final DateTimeFormatter DATE_TIME_FORMATTER = new DateTimeFormatterBuilder() + static final DateTimeFormatter DATE_TIME_FORMATTER = new DateTimeFormatterBuilder() .append(DateTimeFormatter.ISO_LOCAL_DATE).appendPattern("[_HH[_mm[_ss]]]") //brackets mean optional .parseDefaulting(ChronoField.HOUR_OF_DAY, 0) .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0) @@ -149,8 +140,9 @@ public class TimeRoutedAlias implements RoutedAlias { this.aliasName = aliasName; 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 by TimeRoutedAlias."); + String type = required.get(ROUTER_TYPE_NAME).toLowerCase(Locale.ROOT); + if (!"time".equals(type)) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Only 'time' routed aliases is supported by TimeRoutedAlias, found:" + type); } routeField = required.get(ROUTER_FIELD); intervalMath = required.get(ROUTER_INTERVAL); @@ -168,7 +160,7 @@ public class TimeRoutedAlias implements RoutedAlias { // check that the date math is valid final Date now = new Date(); try { - final Date after = new DateMathParser(now, timeZone).parseMath(intervalMath); + final Date after = new DateMathParser(now, timeZone).parseMath(getIntervalMath()); if (!after.after(now)) { throw new SolrException(BAD_REQUEST, "duration must add to produce a time in the future"); } @@ -178,7 +170,7 @@ public class TimeRoutedAlias implements RoutedAlias { if (autoDeleteAgeMath != null) { try { - final Date before = new DateMathParser(now, timeZone).parseMath(autoDeleteAgeMath); + final Date before = new DateMathParser(now, timeZone).parseMath(autoDeleteAgeMath); if (now.before(before)) { throw new SolrException(BAD_REQUEST, "duration must round or subtract to produce a time in the past"); } @@ -204,6 +196,15 @@ public class TimeRoutedAlias implements RoutedAlias { return formatCollectionNameFromInstant(aliasName, parseStringAsInstant(this.start, timeZone)); } + @Override + String[] formattedRouteValues(SolrInputDocument doc) { + String routeField = getRouteField(); + Date fieldValue = (Date) doc.getFieldValue(routeField); + String dest = calcCandidateCollection(fieldValue.toInstant()).getDestinationCollection(); + int nonValuePrefix = getAliasName().length() + getRoutedAliasType().getSeparatorPrefix().length(); + return new String[]{dest.substring(nonValuePrefix)}; + } + public static Instant parseInstantFromCollectionName(String aliasName, String collection) { String separatorPrefix = TYPE.getSeparatorPrefix(); final String dateTimePart; @@ -219,20 +220,20 @@ public class TimeRoutedAlias implements RoutedAlias { String nextCollName = DATE_TIME_FORMATTER.format(timestamp); for (int i = 0; i < 3; i++) { // chop off seconds, minutes, hours if (nextCollName.endsWith("_00")) { - nextCollName = nextCollName.substring(0, nextCollName.length()-3); + nextCollName = nextCollName.substring(0, nextCollName.length() - 3); } } assert DATE_TIME_FORMATTER.parse(nextCollName, Instant::from).equals(timestamp); return aliasName + TYPE.getSeparatorPrefix() + nextCollName; } - Instant parseStringAsInstant(String str, TimeZone zone) { + private Instant parseStringAsInstant(String str, TimeZone zone) { Instant start = DateMathParser.parseMath(new Date(), str, zone).toInstant(); - checkMilis(start); + checkMillis(start); return start; } - private void checkMilis(Instant date) { + private void checkMillis(Instant date) { if (!date.truncatedTo(ChronoUnit.SECONDS).equals(date)) { throw new SolrException(BAD_REQUEST, "Date or date math for start time includes milliseconds, which is not supported. " + @@ -241,16 +242,19 @@ public class TimeRoutedAlias implements RoutedAlias { } @Override - public boolean updateParsedCollectionAliases(ZkController zkController) { - final Aliases aliases = zkController.getZkStateReader().getAliases(); // note: might be different from last request + public boolean updateParsedCollectionAliases(ZkStateReader zkStateReader, boolean contextualize) { + final Aliases aliases = zkStateReader.getAliases(); if (this.parsedCollectionsAliases != aliases) { if (this.parsedCollectionsAliases != null) { log.debug("Observing possibly updated alias: {}", getAliasName()); } - this.parsedCollectionsDesc = parseCollections(aliases ); + this.parsedCollectionsDesc = parseCollections(aliases); this.parsedCollectionsAliases = aliases; return true; } + if (contextualize) { + this.parsedCollectionsDesc = parseCollections(aliases); + } return false; } @@ -264,18 +268,27 @@ public class TimeRoutedAlias implements RoutedAlias { return routeField; } + @Override + public RoutedAliasTypes getRoutedAliasType() { + return RoutedAliasTypes.TIME; + } + + @SuppressWarnings("WeakerAccess") public String getIntervalMath() { return intervalMath; } + @SuppressWarnings("WeakerAccess") public long getMaxFutureMs() { return maxFutureMs; } + @SuppressWarnings("WeakerAccess") public String getPreemptiveCreateWindow() { return preemptiveCreateMath; } + @SuppressWarnings("WeakerAccess") public String getAutoDeleteAgeMath() { return autoDeleteAgeMath; } @@ -296,16 +309,17 @@ public class TimeRoutedAlias implements RoutedAlias { .add("timeZone", timeZone) .toString(); } + /** * Parses the elements of the collection list. Result is returned them in sorted order (most recent 1st) */ - List> parseCollections(Aliases aliases) { - final List collections = aliases.getCollectionAliasListMap().get(aliasName); + private List> parseCollections(Aliases aliases) { + final List collections = getCollectionList(aliases); if (collections == null) { throw RoutedAlias.newAliasMustExistException(getAliasName()); } // note: I considered TreeMap but didn't like the log(N) just to grab the most recent when we use it later - List> result = new ArrayList<>(collections.size()); + List> result = new ArrayList<>(collections.size()); for (String collection : collections) { Instant colStartTime = parseInstantFromCollectionName(aliasName, collection); result.add(new AbstractMap.SimpleImmutableEntry<>(colStartTime, collection)); @@ -314,8 +328,11 @@ public class TimeRoutedAlias implements RoutedAlias { return result; } - /** Computes the timestamp of the next collection given the timestamp of the one before. */ - public Instant computeNextCollTimestamp(Instant fromTimestamp) { + + /** + * Computes the timestamp of the next collection given the timestamp of the one before. + */ + private Instant computeNextCollTimestamp(Instant fromTimestamp) { final Instant nextCollTimestamp = DateMathParser.parseMath(Date.from(fromTimestamp), "NOW" + intervalMath, timeZone).toInstant(); assert nextCollTimestamp.isAfter(fromTimestamp); @@ -324,6 +341,7 @@ public class TimeRoutedAlias implements RoutedAlias { @Override public void validateRouteValue(AddUpdateCommand cmd) throws SolrException { + final Instant docTimestamp = parseRouteKey(cmd.getSolrInputDocument().getFieldValue(getRouteField())); @@ -333,62 +351,18 @@ public class TimeRoutedAlias implements RoutedAlias { "The document's time routed key of " + docTimestamp + " is too far in the future given " + ROUTER_MAX_FUTURE + "=" + getMaxFutureMs()); } - } - @Override - public String createCollectionsIfRequired(AddUpdateCommand cmd) { - SolrQueryRequest req = cmd.getReq(); - SolrCore core = req.getCore(); - CoreContainer coreContainer = core.getCoreContainer(); - CollectionsHandler collectionsHandler = coreContainer.getCollectionsHandler(); - final Instant docTimestamp = - parseRouteKey(cmd.getSolrInputDocument().getFieldValue(getRouteField())); - - // Even though it is possible that multiple requests hit this code in the 1-2 sec that - // it takes to create a collection, it's an established anti-pattern to feed data with a very large number - // of client connections. This in mind, we only guard against spamming the overseer within a batch of - // updates. We are intentionally tolerating a low level of redundant requests in favor of simpler code. Most - // super-sized installations with many update clients will likely be multi-tenant and multiple tenants - // probably don't write to the same alias. As such, we have deferred any solution to the "many clients causing - // collection creation simultaneously" problem until such time as someone actually has that problem in a - // real world use case that isn't just an anti-pattern. - Map.Entry candidateCollectionDesc = findCandidateGivenTimestamp(docTimestamp, cmd.getPrintableId()); - String candidateCollectionName = candidateCollectionDesc.getValue(); - - try { - switch (typeOfCreationRequired(docTimestamp, candidateCollectionDesc.getKey())) { - case SYNCHRONOUS: - // This next line blocks until all collections required by the current document have been created - return createAllRequiredCollections(docTimestamp, cmd, candidateCollectionDesc); - case ASYNC_PREEMPTIVE: - if (!preemptiveCreateOnceAlready) { - log.debug("Executing preemptive creation for {}", getAliasName()); - // It's important not to add code between here and the prior call to findCandidateGivenTimestamp() - // in processAdd() that invokes updateParsedCollectionAliases(). Doing so would update parsedCollectionsDesc - // and create a race condition. We are relying on the fact that get(0) is returning the head of the parsed - // collections that existed when candidateCollectionDesc was created. If this class updates it's notion of - // 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); - } - return candidateCollectionName; - case NONE: - return candidateCollectionName; // could use fall through, but fall through is fiddly for later editors. - default: - throw unknownCreateType(); - } - // do nothing if creationType == NONE - } catch (SolrException e) { - throw e; - } catch (Exception e) { - throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e); + // Although this is also checked later, we need to check it here too to handle the case in Dimensional Routed + // aliases where one can legally have zero collections for a newly encountered category and thus the loop later + // can't catch this. + Instant startTime = parseRouteKey(start); + if (docTimestamp.isBefore(startTime)) { + throw new SolrException(BAD_REQUEST, "The document couldn't be routed because " + docTimestamp + + " is before the start time for this alias " +start+")"); } } + @Override public Map getAliasMetadata() { return aliasMetadata; @@ -404,116 +378,12 @@ public class TimeRoutedAlias implements RoutedAlias { return OPTIONAL_ROUTER_PARAMS; } - /** - * Create as many collections as required. This method loops to allow for the possibility that the docTimestamp - * requires more than one collection to be created. Since multiple threads may be invoking maintain on separate - * requests to the same alias, we must pass in the name of the collection that this thread believes to be the most - * recent collection. This assumption is checked when the command is executed in the overseer. When this method - * finds that all collections required have been created it returns the (possibly new) most recent collection. - * The return value is ignored by the calling code in the async preemptive case. - * - * @param docTimestamp the timestamp from the document that determines routing - * @param cmd the update command being processed - * @param targetCollectionDesc the descriptor for the presently selected collection which should also be - * the most recent collection in all cases where this method is invoked. - * @return The latest collection, including collections created during maintenance - */ - private String createAllRequiredCollections( Instant docTimestamp, AddUpdateCommand cmd, - Map.Entry targetCollectionDesc) { - SolrQueryRequest req = cmd.getReq(); - SolrCore core = req.getCore(); - CoreContainer coreContainer = core.getCoreContainer(); - CollectionsHandler collectionsHandler = coreContainer.getCollectionsHandler(); - do { - switch(typeOfCreationRequired(docTimestamp, targetCollectionDesc.getKey())) { - case NONE: - return targetCollectionDesc.getValue(); // we don't need another collection - case ASYNC_PREEMPTIVE: - // can happen when preemptive interval is longer than one time slice - String mostRecentCollName = this.parsedCollectionsDesc.get(0).getValue(); - preemptiveAsync(() -> createNextCollection(mostRecentCollName, collectionsHandler), core); - return targetCollectionDesc.getValue(); - case SYNCHRONOUS: - createNextCollection(targetCollectionDesc.getValue(), collectionsHandler); // *should* throw if fails for some reason but... - ZkController zkController = coreContainer.getZkController(); - if (!updateParsedCollectionAliases(zkController)) { // thus we didn't make progress... - // this is not expected, even in known failure cases, but we check just in case - throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, - "We need to create a new time routed collection but for unknown reasons were unable to do so."); - } - // then retry the loop ... have to do find again in case other requests also added collections - // that were made visible when we called updateParsedCollectionAliases() - targetCollectionDesc = findCandidateGivenTimestamp(docTimestamp, cmd.getPrintableId()); - break; - default: - throw unknownCreateType(); - } - } while (true); + @Override + protected String getHeadCollectionIfOrdered(AddUpdateCommand cmd) { + return parsedCollectionsDesc.get(0).getValue(); } - private SolrException unknownCreateType() { - return new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown creation type while adding " + - "document to a Time Routed Alias! This is a bug caused when a creation type has been added but " + - "not all code has been updated to handle it."); - } - - private void createNextCollection(String mostRecentCollName, CollectionsHandler collHandler) { - // Invoke ROUTEDALIAS_CREATECOLL (in the Overseer, locked by alias name). It will create the collection - // and update the alias contingent on the most recent collection name being the same as - // what we think so here, otherwise it will return (without error). - try { - MaintainTimeRoutedAliasCmd.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. - - // Ensure our view of the aliases has updated. If we didn't do this, our zkStateReader might - // not yet know about the new alias (thus won't see the newly added collection to it), and we might think - // we failed. - collHandler.getCoreContainer().getZkController().getZkStateReader().aliasesManager.update(); - } catch (RuntimeException e) { - throw e; - } catch (Exception e) { - throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e); - } - } - - private void preemptiveAsync(Runnable r, SolrCore core) { - preemptiveCreateOnceAlready = true; - core.runAsync(r); - } - - /** - * Determine if the a new collection will be required based on the document timestamp. Passing null for - * preemptiveCreateInterval tells you if the document is beyond all existing collections with a response of - * {@link CreationType#NONE} or {@link CreationType#SYNCHRONOUS}, and passing a valid date math for - * preemptiveCreateMath additionally distinguishes the case where the document is close enough to the end of - * the TRA to trigger preemptive creation but not beyond all existing collections with a value of - * {@link CreationType#ASYNC_PREEMPTIVE}. - * - * @param docTimeStamp The timestamp from the document - * @param targetCollectionTimestamp The timestamp for the presently selected destination collection - * @return a {@code CreationType} indicating if and how to create a collection - */ - private CreationType typeOfCreationRequired(Instant docTimeStamp, Instant targetCollectionTimestamp) { - final Instant nextCollTimestamp = computeNextCollTimestamp(targetCollectionTimestamp); - - if (!docTimeStamp.isBefore(nextCollTimestamp)) { - // current document is destined for a collection that doesn't exist, must create the destination - // to proceed with this add command - return SYNCHRONOUS; - } - - if (isNotBlank(getPreemptiveCreateWindow())) { - Instant preemptNextColCreateTime = - calcPreemptNextColCreateTime(getPreemptiveCreateWindow(), nextCollTimestamp); - if (!docTimeStamp.isBefore(preemptNextColCreateTime)) { - return ASYNC_PREEMPTIVE; - } - } - - return NONE; - } private Instant calcPreemptNextColCreateTime(String preemptiveCreateMath, Instant nextCollTimestamp) { DateMathParser dateMathParser = new DateMathParser(); @@ -531,38 +401,194 @@ public class TimeRoutedAlias implements RoutedAlias { if (routeKey instanceof Instant) { docTimestamp = (Instant) routeKey; } else if (routeKey instanceof Date) { - docTimestamp = ((Date)routeKey).toInstant(); + docTimestamp = ((Date) routeKey).toInstant(); } else if (routeKey instanceof CharSequence) { - docTimestamp = Instant.parse((CharSequence)routeKey); + docTimestamp = Instant.parse((CharSequence) routeKey); } else { throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unexpected type of routeKey: " + routeKey); } return docTimestamp; } + /** - * Given the route key, finds the correct collection or returns the most recent collection if the doc - * is in the future. Future docs will potentially cause creation of a collection that does not yet exist - * or an error if they exceed the maxFutureMs setting. + * Given the route key, finds the correct collection and an indication of any collection that needs to be created. + * Future docs will potentially cause creation of a collection that does not yet exist. This method presumes that the + * doc time stamp has already been checked to not exceed maxFutureMs * * @throws SolrException if the doc is too old to be stored in the TRA */ - private Map.Entry findCandidateGivenTimestamp(Instant docTimestamp, String printableId) { - // Lookup targetCollection given route key. Iterates in reverse chronological order. - // We're O(N) here but N should be small, the loop is fast, and usually looking for 1st. - for (Map.Entry entry : parsedCollectionsDesc) { - Instant colStartTime = entry.getKey(); - if (!docTimestamp.isBefore(colStartTime)) { // i.e. docTimeStamp is >= the colStartTime - return entry; //found it - } - } + @Override + public CandidateCollection findCandidateGivenValue(AddUpdateCommand cmd) { + Object value = cmd.getSolrInputDocument().getFieldValue(getRouteField()); + ZkStateReader zkStateReader = cmd.getReq().getCore().getCoreContainer().getZkController().zkStateReader; + String printableId = cmd.getPrintableId(); + updateParsedCollectionAliases(zkStateReader, true); + + final Instant docTimestamp = parseRouteKey(value); + + // reparse explicitly such that if we are a dimension in a DRA, the list gets culled by our context + // This does not normally happen with the above updateParsedCollectionAliases, because at that point the aliases + // should be up to date and updateParsedCollectionAliases will short circuit + this.parsedCollectionsDesc = parseCollections(zkStateReader.getAliases()); + CandidateCollection next1 = calcCandidateCollection(docTimestamp); + if (next1 != null) return next1; + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Doc " + printableId + " couldn't be routed with " + getRouteField() + "=" + docTimestamp); } - enum CreationType { - NONE, - ASYNC_PREEMPTIVE, - SYNCHRONOUS + private CandidateCollection calcCandidateCollection(Instant docTimestamp) { + // Lookup targetCollection given route key. Iterates in reverse chronological order. + // We're O(N) here but N should be small, the loop is fast, and usually looking for 1st. + Instant next = null; + if (this.parsedCollectionsDesc.isEmpty()) { + String firstCol = computeInitialCollectionName(); + return new CandidateCollection(SYNCHRONOUS, firstCol); + } else { + Instant mostRecentCol = parsedCollectionsDesc.get(0).getKey(); + + // despite most logic hinging on the first element, we must loop so we can complain if the doc + // is too old and there's no valid collection. + for (int i = 0; i < parsedCollectionsDesc.size(); i++) { + Map.Entry entry = parsedCollectionsDesc.get(i); + Instant colStartTime = entry.getKey(); + if (i == 0) { + next = computeNextCollTimestamp(colStartTime); + } + if (!docTimestamp.isBefore(colStartTime)) { // (inclusive lower bound) + CandidateCollection candidate; + if (i == 0) { + if (docTimestamp.isBefore(next)) { // (exclusive upper bound) + candidate = new CandidateCollection(NONE, entry.getValue()); //found it + // simply goes to head collection no action required + } else { + // Although we create collections one at a time, this calculation of the ultimate destination is + // useful for contextualizing TRA's used as dimensions in DRA's + String creationCol = calcNextCollection(colStartTime); + Instant colDestTime = colStartTime; + Instant possibleDestTime = colDestTime; + while (!docTimestamp.isBefore(possibleDestTime) || docTimestamp.equals(possibleDestTime)) { + colDestTime = possibleDestTime; + possibleDestTime = computeNextCollTimestamp(colDestTime); + } + String destCol = TimeRoutedAlias.formatCollectionNameFromInstant(getAliasName(),colDestTime); + candidate = new CandidateCollection(SYNCHRONOUS, destCol, creationCol); //found it + } + } else { + // older document simply goes to existing collection, nothing created. + candidate = new CandidateCollection(NONE, entry.getValue()); //found it + } + + if (candidate.getCreationType() == NONE && isNotBlank(getPreemptiveCreateWindow()) && !this.preemptiveCreateOnceAlready) { + // are we getting close enough to the (as yet uncreated) next collection to warrant preemptive creation? + Instant time2Create = calcPreemptNextColCreateTime(getPreemptiveCreateWindow(), computeNextCollTimestamp(mostRecentCol)); + if (!docTimestamp.isBefore(time2Create)) { + String destinationCollection = candidate.getDestinationCollection(); // dest doesn't change + String creationCollection = calcNextCollection(mostRecentCol); + return new CandidateCollection(ASYNC_PREEMPTIVE, // add next collection + destinationCollection, + creationCollection); + } + } + return candidate; + } + } + } + return null; } + /** + * Deletes some of the oldest collection(s) based on {@link TimeRoutedAlias#getAutoDeleteAgeMath()}. If + * getAutoDelteAgemath is not present then this method does nothing. Per documentation is relative to a + * collection being created. Therefore if nothing is being created, nothing is deleted. + * @param actions The previously calculated add action(s). This collection should not be modified within + * this method. + */ + private List calcDeletes(List actions) { + final String autoDeleteAgeMathStr = this.getAutoDeleteAgeMath(); + if (autoDeleteAgeMathStr == null || actions .size() == 0) { + return Collections.emptyList(); + } + if (actions.size() > 1) { + throw new IllegalStateException("We are not supposed to be creating more than one collection at a time"); + } + + String deletionReferenceCollection = actions.get(0).targetCollection; + Instant deletionReferenceInstant = parseInstantFromCollectionName(getAliasName(), deletionReferenceCollection); + final Instant delBefore; + try { + delBefore = new DateMathParser(Date.from(computeNextCollTimestamp(deletionReferenceInstant)), this.getTimeZone()).parseMath(autoDeleteAgeMathStr).toInstant(); + } catch (ParseException e) { + throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e); // note: should not happen by this point + } + + List collectionsToDelete = new ArrayList<>(); + + //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. + 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 parsedCollection : parsedCollectionsDesc) { + numToKeep++; + final Instant colInstant = parsedCollection.getKey(); + if (colInstant.isBefore(delBefore) || colInstant.equals(delBefore)) { + if (log.isDebugEnabled()) { // don't perform formatting unless debugging + assert dtf != null; + 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 + assert dtf != null; + log.debug("{} is not before {} and will be retained", dtf.format(colInstant), dtf.format(delBefore)); + } + } + } + + log.debug("Collections will be deleted... parsed collections={}", parsedCollectionsDesc); + final List targetList = parsedCollectionsDesc.stream().map(Map.Entry::getValue).collect(Collectors.toList()); + log.debug("Iterating backwards on collection list to find deletions: {}", targetList); + for (int i = parsedCollectionsDesc.size() - 1; i >= numToKeep; i--) { + String toDelete = targetList.get(i); + log.debug("Adding to TRA delete list:{}", toDelete); + + collectionsToDelete.add(new Action(this, ActionType.ENSURE_REMOVED, toDelete)); + } + return collectionsToDelete; + } + + private List calcAdd(String targetCol) { + List collectionList = getCollectionList(parsedCollectionsAliases); + if (!collectionList.contains(targetCol) && !collectionList.isEmpty()) { + // Then we need to add the next one... (which may or may not be the same as our target + String mostRecentCol = collectionList.get(0); + String pfx = getRoutedAliasType().getSeparatorPrefix(); + int sepLen = mostRecentCol.contains(pfx) ? pfx.length() : 1; // __TRA__ or _ + String mostRecentTime = mostRecentCol.substring(getAliasName().length() + sepLen); + Instant parsed = DATE_TIME_FORMATTER.parse(mostRecentTime, Instant::from); + String nextCol = calcNextCollection(parsed); + return Collections.singletonList(new Action(this, ActionType.ENSURE_EXISTS, nextCol)); + } else { + return Collections.emptyList(); + } + } + + private String calcNextCollection(Instant mostRecentCollTimestamp) { + final Instant nextCollTimestamp = computeNextCollTimestamp(mostRecentCollTimestamp); + return TimeRoutedAlias.formatCollectionNameFromInstant(aliasName, nextCollTimestamp); + } + + @Override + protected List calculateActions(String targetCol) { + List actions = new ArrayList<>(); + actions.addAll(calcAdd(targetCol)); + actions.addAll(calcDeletes(actions)); + return actions; + } + + } diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java index 6471d661751..54cc5dd4a30 100644 --- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java +++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java @@ -59,6 +59,7 @@ import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder.Credential import org.apache.solr.client.solrj.util.SolrIdentifierValidator; import org.apache.solr.cloud.CloudDescriptor; import org.apache.solr.cloud.Overseer; +import org.apache.solr.cloud.OverseerTaskQueue; import org.apache.solr.cloud.ZkController; import org.apache.solr.cloud.autoscaling.AutoScalingHandler; import org.apache.solr.common.AlreadyClosedException; @@ -897,6 +898,12 @@ public class CoreContainer { } public void shutdown() { + + ZkController zkController = getZkController(); + if (zkController != null) { + OverseerTaskQueue overseerCollectionQueue = zkController.getOverseerCollectionQueue(); + overseerCollectionQueue.allowOverseerPendingTasksToComplete(); + } log.info("Shutting down CoreContainer instance=" + System.identityHashCode(this)); ExecutorUtil.shutdownAndAwaitTermination(coreContainerAsyncTaskExecutor); diff --git a/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java b/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java index 90a2dd23aa8..85033f3b184 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/BaseHandlerApiSupport.java @@ -178,6 +178,35 @@ public abstract class BaseHandlerApiSupport implements ApiSupport { return cmd.meta().getParamNamesIterator(co); } + @Override + public Map toMap(Map suppliedMap) { + for(Iterator it=getParameterNamesIterator(); it.hasNext(); ) { + final String param = it.next(); + String key = cmd.meta().getParamSubstitute(param); + Object o = key.indexOf('.') > 0 ? + Utils.getObjectByPath(map, true, splitSmart(key, '.')) : + map.get(key); + if (o == null) o = pathValues.get(key); + if (o == null && useRequestParams) o = origParams.getParams(key); + // make strings out of as many things as we can now to minimize differences from + // the standard impls that pass through a NamedList/SimpleOrderedMap... + Class oClass = o.getClass(); + if (oClass.isPrimitive() || + Number.class.isAssignableFrom(oClass) || + Character.class.isAssignableFrom(oClass) || + Boolean.class.isAssignableFrom(oClass)) { + suppliedMap.put(param,String.valueOf(o)); + } else if (List.class.isAssignableFrom(oClass) && ((List)o).get(0) instanceof String ) { + List l = (List) o; + suppliedMap.put( param, l.toArray(new String[0])); + } else { + // Lists pass through but will require special handling downstream + // if they contain non-string elements. + suppliedMap.put(param, o); + } + } + return suppliedMap; + } }); } 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 d2929768b61..5d759f9a675 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 @@ -50,8 +50,8 @@ import org.apache.solr.client.solrj.util.SolrIdentifierValidator; import org.apache.solr.cloud.OverseerSolrResponse; import org.apache.solr.cloud.OverseerTaskQueue; import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent; -import org.apache.solr.cloud.ZkController.NotInClusterStateException; import org.apache.solr.cloud.ZkController; +import org.apache.solr.cloud.ZkController.NotInClusterStateException; import org.apache.solr.cloud.ZkShardTerms; import org.apache.solr.cloud.api.collections.ReindexCollectionCmd; import org.apache.solr.cloud.api.collections.RoutedAlias; @@ -620,13 +620,27 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission String collections = req.getParams().get("collections"); RoutedAlias routedAlias = null; Exception ex = null; + HashMap possiblyModifiedParams = new HashMap<>(); try { // note that RA specific validation occurs here. - routedAlias = RoutedAlias.fromProps(alias, req.getParams().toMap(new HashMap<>())); + routedAlias = RoutedAlias.fromProps(alias, req.getParams().toMap(possiblyModifiedParams)); } catch (SolrException e) { // we'll throw this later if we are in fact creating a routed alias. ex = e; } + @SuppressWarnings("unchecked") + ModifiableSolrParams finalParams = new ModifiableSolrParams(); + for (Map.Entry entry : possiblyModifiedParams.entrySet()) { + if (entry.getValue().getClass().isArray() ) { + // v2 api hits this case + for (Object o : (Object[]) entry.getValue()) { + finalParams.add(entry.getKey(),o.toString()); + } + } else { + finalParams.add(entry.getKey(),entry.getValue().toString()); + } + } + if (collections != null) { if (routedAlias != null) { throw new SolrException(BAD_REQUEST, "Collections cannot be specified when creating a routed alias."); @@ -634,7 +648,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission ////////////////////////////////////// // Regular alias creation indicated // ////////////////////////////////////// - return copy(req.getParams().required(), null, NAME, "collections"); + return copy(finalParams.required(), null, NAME, "collections"); } } @@ -648,14 +662,15 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission } // Now filter out just the parameters we care about from the request - Map result = copy(req.getParams(), null, routedAlias.getRequiredParams()); - copy(req.getParams(), result, routedAlias.getOptionalParams()); + assert routedAlias != null; + Map result = copy(finalParams, null, routedAlias.getRequiredParams()); + copy(finalParams, result, routedAlias.getOptionalParams()); ModifiableSolrParams createCollParams = new ModifiableSolrParams(); // without prefix // add to result params that start with "create-collection.". // Additionally, save these without the prefix to createCollParams - for (Map.Entry entry : req.getParams()) { + for (Map.Entry entry : finalParams) { final String p = entry.getKey(); if (p.startsWith(CREATE_COLLECTION_PREFIX)) { // This is what SolrParams#getAll(Map, Collection)} does diff --git a/solr/core/src/java/org/apache/solr/update/processor/RoutedAliasUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/RoutedAliasUpdateProcessor.java index 384fcd81b71..d95f946578b 100644 --- a/solr/core/src/java/org/apache/solr/update/processor/RoutedAliasUpdateProcessor.java +++ b/solr/core/src/java/org/apache/solr/update/processor/RoutedAliasUpdateProcessor.java @@ -176,7 +176,7 @@ public class RoutedAliasUpdateProcessor extends UpdateRequestProcessor { // to avoid potential for race conditions, this next method should not get called again unless // we have created a collection synchronously - routedAlias.updateParsedCollectionAliases(this.zkController); + routedAlias.updateParsedCollectionAliases(this.zkController.zkStateReader, false); String targetCollection = routedAlias.createCollectionsIfRequired(cmd); @@ -269,8 +269,4 @@ public class RoutedAliasUpdateProcessor extends UpdateRequestProcessor { collection, slice.getName(), DistributedUpdateProcessor.MAX_RETRIES_ON_FORWARD_DEAULT); } - - - - } diff --git a/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java index 60d2e3b2dfc..116604df81f 100644 --- a/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java @@ -96,10 +96,10 @@ public class AliasIntegrationTest extends SolrCloudTestCase { public void testProperties() throws Exception { CollectionAdminRequest.createCollection("collection1meta", "conf", 2, 1).process(cluster.getSolrClient()); CollectionAdminRequest.createCollection("collection2meta", "conf", 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(); @@ -353,10 +353,10 @@ public class AliasIntegrationTest extends SolrCloudTestCase { private ZkStateReader createColectionsAndAlias(String aliasName) throws SolrServerException, IOException, KeeperException, InterruptedException { CollectionAdminRequest.createCollection("collection1meta", "conf", 2, 1).process(cluster.getSolrClient()); CollectionAdminRequest.createCollection("collection2meta", "conf", 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(); @@ -405,10 +405,10 @@ public class AliasIntegrationTest extends SolrCloudTestCase { public void testDeleteAliasWithExistingCollectionName() throws Exception { CollectionAdminRequest.createCollection("collection_old", "conf", 2, 1).process(cluster.getSolrClient()); CollectionAdminRequest.createCollection("collection_new", "conf", 1, 1).process(cluster.getSolrClient()); - + cluster.waitForActiveCollection("collection_old", 2, 2); cluster.waitForActiveCollection("collection_new", 1, 1); - + waitForState("Expected collection_old to be created with 2 shards and 1 replica", "collection_old", clusterShape(2, 2)); waitForState("Expected collection_new to be created with 1 shard and 1 replica", "collection_new", clusterShape(1, 1)); @@ -488,10 +488,10 @@ public class AliasIntegrationTest extends SolrCloudTestCase { public void testDeleteOneOfTwoCollectionsAliased() throws Exception { CollectionAdminRequest.createCollection("collection_one", "conf", 2, 1).process(cluster.getSolrClient()); CollectionAdminRequest.createCollection("collection_two", "conf", 1, 1).process(cluster.getSolrClient()); - + cluster.waitForActiveCollection("collection_one", 2, 2); cluster.waitForActiveCollection("collection_two", 1, 1); - + waitForState("Expected collection_one to be created with 2 shards and 1 replica", "collection_one", clusterShape(2, 2)); waitForState("Expected collection_two to be created with 1 shard and 1 replica", "collection_two", clusterShape(1, 1)); @@ -566,7 +566,7 @@ public class AliasIntegrationTest extends SolrCloudTestCase { cluster.getSolrClient().query("collection_one", new SolrQuery("*:*")); fail("should have failed"); } catch (SolrServerException | SolrException se) { - + } // Clean up @@ -591,10 +591,10 @@ public class AliasIntegrationTest extends SolrCloudTestCase { public void test() throws Exception { CollectionAdminRequest.createCollection("collection1", "conf", 2, 1).process(cluster.getSolrClient()); CollectionAdminRequest.createCollection("collection2", "conf", 1, 1).process(cluster.getSolrClient()); - + cluster.waitForActiveCollection("collection1", 2, 2); cluster.waitForActiveCollection("collection2", 1, 1); - + waitForState("Expected collection1 to be created with 2 shards and 1 replica", "collection1", clusterShape(2, 2)); waitForState("Expected collection2 to be created with 1 shard and 1 replica", "collection2", clusterShape(1, 1)); @@ -640,7 +640,7 @@ public class AliasIntegrationTest extends SolrCloudTestCase { CollectionAdminRequest.createAlias("testalias2", "collection2,collection1").process(cluster.getSolrClient()); lastVersion = waitForAliasesUpdate(lastVersion, zkStateReader); - + searchSeveralWays("testalias2", new SolrQuery("*:*"), 5); /////////////// @@ -754,12 +754,12 @@ public class AliasIntegrationTest extends SolrCloudTestCase { @Test public void testErrorChecks() throws Exception { CollectionAdminRequest.createCollection("testErrorChecks-collection", "conf", 2, 1).process(cluster.getSolrClient()); - + cluster.waitForActiveCollection("testErrorChecks-collection", 2, 2); waitForState("Expected testErrorChecks-collection to be created with 2 shards and 1 replica", "testErrorChecks-collection", clusterShape(2, 2)); - + ignoreException("."); - + // Invalid Alias name SolrException e = expectThrows(SolrException.class, () -> CollectionAdminRequest.createAlias("test:alias", "testErrorChecks-collection").process(cluster.getSolrClient())); 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 47c6b852975..9833e908913 100644 --- a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java @@ -50,7 +50,7 @@ import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; -import static org.apache.solr.cloud.api.collections.RoutedAlias.SupportedRouterTypes.TIME; +import static org.apache.solr.client.solrj.RoutedAliasTypes.TIME; /** * Direct http tests of the CreateRoutedAlias functionality. @@ -121,7 +121,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase { " \"tlogReplicas\":1,\n" + " \"pullReplicas\":1,\n" + " \"maxShardsPerNode\":4,\n" + // note: we also expect the 'policy' to work fine - " \"nodeSet\": ['" + createNode + "'],\n" + + " \"nodeSet\": '" + createNode + "',\n" + " \"properties\" : {\n" + " \"foobar\":\"bazbam\",\n" + " \"foobar2\":\"bazbam2\"\n" + @@ -136,6 +136,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase { // small chance could fail due to "NOW"; see above assertCollectionExists(initialCollectionName); + Thread.sleep(1000); // Test created collection: final DocCollection coll = solrClient.getClusterStateProvider().getState(initialCollectionName).get(); //System.err.println(coll); diff --git a/solr/core/src/test/org/apache/solr/update/processor/CategoryRoutedAliasUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/CategoryRoutedAliasUpdateProcessorTest.java index 3abf0483816..0031da81041 100644 --- a/solr/core/src/test/org/apache/solr/update/processor/CategoryRoutedAliasUpdateProcessorTest.java +++ b/solr/core/src/test/org/apache/solr/update/processor/CategoryRoutedAliasUpdateProcessorTest.java @@ -192,6 +192,11 @@ public class CategoryRoutedAliasUpdateProcessorTest extends RoutedAliasUpdatePro addDocsAndCommit(true, newDoc(SHIPS[0])); String uninitialized = getAlias() + "__CRA__" + CategoryRoutedAlias.UNINITIALIZED; + + // important to test that we don't try to delete the temp collection on the first document. If we did so + // we would be at risk of out of order execution of the deletion/creation which would leave a window + // of time where there were no collections in the alias. That would likely break all manner of other + // parts of solr. assertInvariants(colVogon, uninitialized); addDocsAndCommit(true, @@ -200,6 +205,7 @@ public class CategoryRoutedAliasUpdateProcessorTest extends RoutedAliasUpdatePro newDoc(SHIPS[3]), newDoc(SHIPS[4])); + // NOW the temp collection should be gone! assertInvariants(colVogon, colHoG, colStunt, colArk, colBistro); // make sure we fail if we have no value to route on. diff --git a/solr/core/src/test/org/apache/solr/update/processor/DimensionalRoutedAliasUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/DimensionalRoutedAliasUpdateProcessorTest.java new file mode 100644 index 00000000000..15e6ab75b3a --- /dev/null +++ b/solr/core/src/test/org/apache/solr/update/processor/DimensionalRoutedAliasUpdateProcessorTest.java @@ -0,0 +1,726 @@ +/* + * 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.update.processor; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.time.Instant; +import java.util.Collections; +import java.util.Date; +import java.util.List; +import java.util.stream.Collectors; + +import org.apache.lucene.util.IOUtils; +import org.apache.solr.client.solrj.RoutedAliasTypes; +import org.apache.solr.client.solrj.SolrServerException; +import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.CollectionAdminRequest.CreateCategoryRoutedAlias; +import org.apache.solr.client.solrj.request.CollectionAdminRequest.CreateTimeRoutedAlias; +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.CategoryRoutedAlias; +import org.apache.solr.cloud.api.collections.TimeRoutedAlias; +import org.apache.solr.common.SolrDocument; +import org.apache.solr.common.SolrDocumentList; +import org.apache.solr.common.SolrException; +import org.apache.solr.common.SolrInputDocument; +import org.apache.solr.common.params.SolrParams; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.solr.client.solrj.request.CollectionAdminRequest.createCategoryRoutedAlias; +import static org.apache.solr.client.solrj.request.CollectionAdminRequest.createTimeRoutedAlias; + +public class DimensionalRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcessorTest { + + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private static final String CRA = RoutedAliasTypes.CATEGORY.getSeparatorPrefix(); + private static final String TRA = RoutedAliasTypes.TIME.getSeparatorPrefix(); + + private static CloudSolrClient solrClient; + private int lastDocId = 0; + private int numDocsDeletedOrFailed = 0; + + private static final String timeField = "timestamp_dt"; + private static final String catField = "cat_s"; + + + @Before + public void doBefore() throws Exception { + configureCluster(4).configure(); + solrClient = getCloudSolrClient(cluster); + //log this to help debug potential causes of problems + log.info("SolrClient: {}", solrClient); + log.info("ClusterStateProvider {}", solrClient.getClusterStateProvider()); + } + + @After + public void doAfter() throws Exception { + solrClient.close(); + shutdownCluster(); + } + + @AfterClass + public static void finish() throws Exception { + IOUtils.close(solrClient); + } + @Test + public void testTimeCat() throws Exception { + String configName = getSaferTestName(); + createConfigSet(configName); + + CreateTimeRoutedAlias TRA_Dim = createTimeRoutedAlias(getAlias(), "2019-07-01T00:00:00Z", "+1DAY", + getTimeField(), null); + CreateCategoryRoutedAlias CRA_Dim = createCategoryRoutedAlias(null, getCatField(), 20, null); + + CollectionAdminRequest.DimensionalRoutedAlias dra = CollectionAdminRequest.createDimensionalRoutedAlias(getAlias(), + CollectionAdminRequest.createCollection("_unused_", configName, 2, 2) + .setMaxShardsPerNode(2), TRA_Dim, CRA_Dim); + + SolrParams params = dra.getParams(); + assertEquals("Dimensional[TIME,CATEGORY]", params.get(CollectionAdminRequest.RoutedAliasAdminRequest.ROUTER_TYPE_NAME)); + System.out.println(params); + assertEquals("20", params.get("router.1.maxCardinality")); + assertEquals("2019-07-01T00:00:00Z", params.get("router.0.start")); + + dra.process(solrClient); + + String firstCol = timeCatDraColFor("2019-07-01", CategoryRoutedAlias.UNINITIALIZED); + cluster.waitForActiveCollection(firstCol, 2, 4); + + // cat field... har har.. get it? ... category/cat... ...oh never mind. + addDocsAndCommit(true, newDoc("tabby", "2019-07-02T00:00:00Z")); + + assertCatTimeInvariants( + ap( + firstCol, + // lower dimensions are fleshed out because we need to maintain the order of the TRA dim and + // not fail if we get an older document later + timeCatDraColFor("2019-07-01", "tabby"), + timeCatDraColFor("2019-07-02", "tabby") + ), + ap( + "tabby" + ) + ); + + addDocsAndCommit(true, newDoc("calico", "2019-07-02T00:00:00Z")); + + // initial col not removed because the 07-01 CRA has not yet gained a new category (sub-dimensions are independent) + assertCatTimeInvariants( + ap( + timeCatDraColFor("2019-07-01", "calico"), + timeCatDraColFor("2019-07-02", "calico"), + timeCatDraColFor("2019-07-01", "tabby"), + timeCatDraColFor("2019-07-02", "tabby") + ), + ap( + "tabby", + "calico" + ) + ); + + testFailedDocument("shorthair", "2017-10-23T00:00:00Z", "couldn't be routed" ); + testFailedDocument("shorthair", "2020-10-23T00:00:00Z", "too far in the future" ); + testFailedDocument(null, "2019-07-02T00:00:00Z", "Route value is null"); + testFailedDocument("foo__CRA__bar", "2019-07-02T00:00:00Z", "7 character sequence __CRA__"); + testFailedDocument("fóóCRAóóbar", "2019-07-02T00:00:00Z", "7 character sequence __CRA__"); + + // hopefully nothing changed + assertCatTimeInvariants( + ap( + timeCatDraColFor("2019-07-01", "calico"), + timeCatDraColFor("2019-07-02", "calico"), + timeCatDraColFor("2019-07-01", "tabby"), + timeCatDraColFor("2019-07-02", "tabby") + ), + ap( + "tabby", + "calico" + ) + ); + + // 4 docs no new collections + addDocsAndCommit(true, + newDoc("calico", "2019-07-02T00:00:00Z"), + newDoc("tabby", "2019-07-01T00:00:00Z"), + newDoc("tabby", "2019-07-01T23:00:00Z"), + newDoc("calico", "2019-07-02T23:00:00Z") + ); + + // hopefully nothing changed + assertCatTimeInvariants( + ap( + timeCatDraColFor("2019-07-01", "calico"), + timeCatDraColFor("2019-07-02", "calico"), + timeCatDraColFor("2019-07-01", "tabby"), + timeCatDraColFor("2019-07-02", "tabby") + ), + ap( + "tabby", + "calico" + ) + ); + + // 4 docs 2 new collections, in random order and maybe not using the alias + addDocsAndCommit(false, + newDoc("calico", "2019-07-04T00:00:00Z"), + newDoc("tabby", "2019-07-01T00:00:00Z"), + newDoc("tabby", "2019-07-01T23:00:00Z"), + newDoc("calico", "2019-07-03T23:00:00Z") + ); + + assertCatTimeInvariants( + ap( + timeCatDraColFor("2019-07-01", "calico"), + timeCatDraColFor("2019-07-02", "calico"), + timeCatDraColFor("2019-07-03", "calico"), + timeCatDraColFor("2019-07-04", "calico"), + timeCatDraColFor("2019-07-01", "tabby"), + timeCatDraColFor("2019-07-02", "tabby") + ), + ap( + "tabby", + "calico" + ) + ); + + // now test with async pre-create. + CollectionAdminRequest.setAliasProperty(getAlias()) + .addProperty("router.0.preemptiveCreateMath", "30MINUTE").process(solrClient); + + addDocsAndCommit(true, + newDoc("shorthair", "2019-07-02T23:40:00Z"), // create 2 sync 1 async + newDoc("calico", "2019-07-03T23:00:00Z") // does not create + ); + + waitColAndAlias(getAlias(), "", TRA + "2019-07-03" + CRA + "shorthair", 2); + + assertCatTimeInvariants( + ap( + timeCatDraColFor("2019-07-01", "calico"), + timeCatDraColFor("2019-07-02", "calico"), + timeCatDraColFor("2019-07-03", "calico"), + timeCatDraColFor("2019-07-04", "calico"), + timeCatDraColFor("2019-07-01", "shorthair"), + timeCatDraColFor("2019-07-02", "shorthair"), + timeCatDraColFor("2019-07-03", "shorthair"), + timeCatDraColFor("2019-07-01", "tabby"), + timeCatDraColFor("2019-07-02", "tabby") + ), + ap( + "shorthair", + "tabby", + "calico" + ) + ); + + addDocsAndCommit(false, + newDoc("shorthair", "2019-07-02T23:40:00Z"), // should be no change + newDoc("calico", "2019-07-03T23:00:00Z") + ); + + /* + Here we need to be testing that something that should not be created (extra preemptive async collections) + didn't get created (a bug that actually got killed during development, and caused later asserts to + fail due to wrong number of collections). There's no way to set a watch for something that doesn't and + should never exist... Thus, the only choice is to sleep and make sure nothing appeared while we were asleep. + */ + Thread.sleep(5000); + + assertCatTimeInvariants( + ap( + timeCatDraColFor("2019-07-01", "calico"), + timeCatDraColFor("2019-07-02", "calico"), + timeCatDraColFor("2019-07-03", "calico"), + timeCatDraColFor("2019-07-04", "calico"), + timeCatDraColFor("2019-07-01", "shorthair"), + timeCatDraColFor("2019-07-02", "shorthair"), + timeCatDraColFor("2019-07-03", "shorthair"), + timeCatDraColFor("2019-07-01", "tabby"), + timeCatDraColFor("2019-07-02", "tabby") + ), + ap( + "shorthair", + "tabby", + "calico" + ) + ); + + // now test with auto-delete. + CollectionAdminRequest.setAliasProperty(getAlias()) + .addProperty("router.0.autoDeleteAge", "/DAY-5DAY").process(solrClient); + + // this one should not yet cause deletion + addDocsAndCommit(false, + newDoc("shorthair", "2019-07-02T23:00:00Z"), // no effect expected + newDoc("calico", "2019-07-05T23:00:00Z") // create 1 + ); + + assertCatTimeInvariants( + ap( + timeCatDraColFor("2019-07-01", "calico"), + timeCatDraColFor("2019-07-02", "calico"), + timeCatDraColFor("2019-07-03", "calico"), + timeCatDraColFor("2019-07-04", "calico"), + timeCatDraColFor("2019-07-05", "calico"), + timeCatDraColFor("2019-07-01", "shorthair"), + timeCatDraColFor("2019-07-02", "shorthair"), + timeCatDraColFor("2019-07-03", "shorthair"), + timeCatDraColFor("2019-07-01", "tabby"), + timeCatDraColFor("2019-07-02", "tabby") + ), + ap( + "shorthair", + "tabby", + "calico" + ) + ); + + // have to only send to alias here since one of the collections will be deleted. + addDocsAndCommit(true, + newDoc("shorthair", "2019-07-02T23:00:00Z"), // no effect expected + newDoc("calico", "2019-07-06T00:00:00Z") // create July 6, delete July 1 + ); + waitCoreCount(getAlias() + TRA + "2019-07-01" + CRA + "calico", 0); + + assertCatTimeInvariants( + ap( + timeCatDraColFor("2019-07-02", "calico"), + timeCatDraColFor("2019-07-03", "calico"), + timeCatDraColFor("2019-07-04", "calico"), + timeCatDraColFor("2019-07-05", "calico"), + timeCatDraColFor("2019-07-06", "calico"), + // note that other categories are unaffected + timeCatDraColFor("2019-07-01", "shorthair"), + timeCatDraColFor("2019-07-02", "shorthair"), + timeCatDraColFor("2019-07-03", "shorthair"), + timeCatDraColFor("2019-07-01", "tabby"), + timeCatDraColFor("2019-07-02", "tabby") + ), + ap( + "shorthair", + "tabby", + "calico" + ) + ); + + // verify that all the documents ended up in the right collections. + QueryResponse resp = solrClient.query(getAlias(), params( + "q", "*:*", + "rows", "100", + "fl","*,[shard]", + "sort", "id asc" + )); + SolrDocumentList results = resp.getResults(); + assertEquals(18, results.getNumFound()); + for (SolrDocument result : results) { + String shard = String.valueOf(result.getFieldValue("[shard]")); + String cat = String.valueOf(result.getFieldValue("cat_s")); + Date date = (Date) result.getFieldValue("timestamp_dt"); + String day = date.toInstant().toString().split("T")[0]; + assertTrue(shard.contains(cat)); + assertTrue(shard.contains(day)); + } + } + + + @Test + public void testCatTime() throws Exception { + String configName = getSaferTestName(); + createConfigSet(configName); + + CreateTimeRoutedAlias TRA_Dim = createTimeRoutedAlias(getAlias(), "2019-07-01T00:00:00Z", "+1DAY", + getTimeField(), null); + CreateCategoryRoutedAlias CRA_Dim = createCategoryRoutedAlias(null, getCatField(), 20, null); + + CollectionAdminRequest.DimensionalRoutedAlias dra = CollectionAdminRequest.createDimensionalRoutedAlias(getAlias(), + CollectionAdminRequest.createCollection("_unused_", configName, 2, 2) + .setMaxShardsPerNode(2), CRA_Dim, TRA_Dim); + + SolrParams params = dra.getParams(); + assertEquals("Dimensional[CATEGORY,TIME]", params.get(CollectionAdminRequest.RoutedAliasAdminRequest.ROUTER_TYPE_NAME)); + System.out.println(params); + assertEquals("20", params.get("router.0.maxCardinality")); + assertEquals("2019-07-01T00:00:00Z", params.get("router.1.start")); + + dra.process(solrClient); + + String firstCol = catTimeDraColFor(CategoryRoutedAlias.UNINITIALIZED, "2019-07-01"); + cluster.waitForActiveCollection(firstCol, 2, 4); + + // cat field... har har.. get it? ... category/cat... ...oh never mind. + addDocsAndCommit(true, newDoc("tabby", "2019-07-02T00:00:00Z")); + + assertCatTimeInvariants( + ap( + firstCol, + catTimeDraColFor("tabby", "2019-07-01"), + catTimeDraColFor("tabby", "2019-07-02") + ), + ap( + "tabby" + ) + ); + + addDocsAndCommit(true, newDoc("calico", "2019-07-02T00:00:00Z")); + + // initial col should be removed + assertCatTimeInvariants( + ap( + catTimeDraColFor("calico", "2019-07-01"), + catTimeDraColFor("calico", "2019-07-02"), + catTimeDraColFor("tabby", "2019-07-01"), + catTimeDraColFor("tabby", "2019-07-02") + ), + ap( + "tabby", + "calico" + ) + ); + + testFailedDocument("shorthair", "2017-10-23T00:00:00Z", "couldn't be routed" ); + testFailedDocument("shorthair", "2020-10-23T00:00:00Z", "too far in the future" ); + testFailedDocument(null, "2019-07-02T00:00:00Z", "Route value is null"); + testFailedDocument("foo__CRA__bar", "2019-07-02T00:00:00Z", "7 character sequence __CRA__"); + testFailedDocument("fóóCRAóóbar", "2019-07-02T00:00:00Z", "7 character sequence __CRA__"); + + // hopefully nothing changed + assertCatTimeInvariants( + ap( + catTimeDraColFor("calico", "2019-07-01"), + catTimeDraColFor("calico", "2019-07-02"), + catTimeDraColFor("tabby", "2019-07-01"), + catTimeDraColFor("tabby", "2019-07-02") + ), + ap( + "tabby", + "calico" + ) + ); + + // 4 docs no new collections + addDocsAndCommit(true, + newDoc("calico", "2019-07-02T00:00:00Z"), + newDoc("tabby", "2019-07-01T00:00:00Z"), + newDoc("tabby", "2019-07-01T23:00:00Z"), + newDoc("calico", "2019-07-02T23:00:00Z") + ); + + // hopefully nothing changed + assertCatTimeInvariants( + ap( + catTimeDraColFor("calico", "2019-07-01"), + catTimeDraColFor("calico", "2019-07-02"), + catTimeDraColFor("tabby", "2019-07-01"), + catTimeDraColFor("tabby", "2019-07-02") + ), + ap( + "tabby", + "calico" + ) + ); + + // 4 docs 2 new collections, in random order and maybe not using the alias + addDocsAndCommit(false, + newDoc("calico", "2019-07-04T00:00:00Z"), + newDoc("tabby", "2019-07-01T00:00:00Z"), + newDoc("tabby", "2019-07-01T23:00:00Z"), + newDoc("calico", "2019-07-03T23:00:00Z") + ); + + assertCatTimeInvariants( + ap( + catTimeDraColFor("calico", "2019-07-01"), + catTimeDraColFor("calico", "2019-07-02"), + catTimeDraColFor("calico", "2019-07-03"), + catTimeDraColFor("calico", "2019-07-04"), + catTimeDraColFor("tabby", "2019-07-01"), + catTimeDraColFor("tabby", "2019-07-02") + // tabby collections not filled in. No guarantee that time periods remain in sync + // across categories. + ), + ap( + "tabby", + "calico" + ) + ); + + // now test with async pre-create. + CollectionAdminRequest.setAliasProperty(getAlias()) + .addProperty("router.1.preemptiveCreateMath", "30MINUTE").process(solrClient); + + addDocsAndCommit(false, + newDoc("shorthair", "2019-07-02T23:40:00Z"), // create 2 sync 1 async + newDoc("calico", "2019-07-03T23:00:00Z") // does not create + ); + + waitColAndAlias(getAlias(), "", CRA + "shorthair" + TRA + "2019-07-03", 2); + + assertCatTimeInvariants( + ap( + catTimeDraColFor("calico", "2019-07-01"), + catTimeDraColFor("calico", "2019-07-02"), + catTimeDraColFor("calico", "2019-07-03"), + catTimeDraColFor("calico", "2019-07-04"), + catTimeDraColFor("shorthair", "2019-07-01"), + catTimeDraColFor("shorthair", "2019-07-02"), + catTimeDraColFor("shorthair", "2019-07-03"), + catTimeDraColFor("tabby", "2019-07-01"), + catTimeDraColFor("tabby", "2019-07-02") + ), + ap( + "shorthair", + "tabby", + "calico" + ) + ); + + addDocsAndCommit(false, + newDoc("shorthair", "2019-07-02T23:40:00Z"), // should be no change + newDoc("calico", "2019-07-03T23:00:00Z") + ); + + /* + Here we need to be testing that something that should not be created (extra preemptive async collections) + didn't get created (a bug that actually got killed during development, and caused later asserts to + fail due to wrong number of collections). There's no way to set a watch for something that doesn't and + should never exist... Thus, the only choice is to sleep and make sure nothing appeared while we were asleep. + */ + Thread.sleep(5000); + + assertCatTimeInvariants( + ap( + catTimeDraColFor("calico", "2019-07-01"), + catTimeDraColFor("calico", "2019-07-02"), + catTimeDraColFor("calico", "2019-07-03"), + catTimeDraColFor("calico", "2019-07-04"), + catTimeDraColFor("shorthair", "2019-07-01"), + catTimeDraColFor("shorthair", "2019-07-02"), + catTimeDraColFor("shorthair", "2019-07-03"), + catTimeDraColFor("tabby", "2019-07-01"), + catTimeDraColFor("tabby", "2019-07-02") + ), + ap( + "shorthair", + "tabby", + "calico" + ) + ); + + // now test with auto-delete. + CollectionAdminRequest.setAliasProperty(getAlias()) + .addProperty("router.1.autoDeleteAge", "/DAY-5DAY").process(solrClient); + + // this one should not yet cause deletion + addDocsAndCommit(false, + newDoc("shorthair", "2019-07-02T23:00:00Z"), // no effect expected + newDoc("calico", "2019-07-05T23:00:00Z") // create 1 + ); + + assertCatTimeInvariants( + ap( + catTimeDraColFor("calico", "2019-07-01"), + catTimeDraColFor("calico", "2019-07-02"), + catTimeDraColFor("calico", "2019-07-03"), + catTimeDraColFor("calico", "2019-07-04"), + catTimeDraColFor("calico", "2019-07-05"), + catTimeDraColFor("shorthair", "2019-07-01"), + catTimeDraColFor("shorthair", "2019-07-02"), + catTimeDraColFor("shorthair", "2019-07-03"), + catTimeDraColFor("tabby", "2019-07-01"), + catTimeDraColFor("tabby", "2019-07-02") + ), + ap( + "shorthair", + "tabby", + "calico" + ) + ); + + addDocsAndCommit(false, + newDoc("shorthair", "2019-07-02T23:00:00Z"), // no effect expected + newDoc("calico", "2019-07-06T00:00:00Z") // create July 6, delete July 1 + ); + waitCoreCount(getAlias() + CRA + "calico" + TRA + "2019-07-01", 0); + + assertCatTimeInvariants( + ap( + catTimeDraColFor("calico", "2019-07-02"), + catTimeDraColFor("calico", "2019-07-03"), + catTimeDraColFor("calico", "2019-07-04"), + catTimeDraColFor("calico", "2019-07-05"), + catTimeDraColFor("calico", "2019-07-06"), + // note that other categories are unaffected + catTimeDraColFor("shorthair", "2019-07-01"), + catTimeDraColFor("shorthair", "2019-07-02"), + catTimeDraColFor("shorthair", "2019-07-03"), + catTimeDraColFor("tabby", "2019-07-01"), + catTimeDraColFor("tabby", "2019-07-02") + ), + ap( + "shorthair", + "tabby", + "calico" + ) + ); + + // verify that all the documents ended up in the right collections. + QueryResponse resp = solrClient.query(getAlias(), params( + "q", "*:*", + "rows", "100", + "fl","*,[shard]", + "sort", "id asc" + )); + SolrDocumentList results = resp.getResults(); + assertEquals(18, results.getNumFound()); + for (SolrDocument result : results) { + String shard = String.valueOf(result.getFieldValue("[shard]")); + String cat = String.valueOf(result.getFieldValue("cat_s")); + Date date = (Date) result.getFieldValue("timestamp_dt"); + String day = date.toInstant().toString().split("T")[0]; + assertTrue(shard.contains(cat)); + assertTrue(shard.contains(day)); + } + + } + + public String catTimeDraColFor(String category, String timestamp) { + return getAlias() + CRA + category + TRA + timestamp; + } + + public String timeCatDraColFor(String timestamp, String category) { + return getAlias() + TRA + timestamp + CRA + category; + } + + /** + * Test for invariant conditions when dealing with a DRA that is category X time. + * + * @param expectedCols the collections we expect to see + * @param categories the categories added thus far + */ + private void assertCatTimeInvariants(String[] expectedCols, String[] categories) throws Exception { + final int expectNumFound = lastDocId - numDocsDeletedOrFailed; //lastDocId is effectively # generated docs + int totalNumFound = 0; + + final List cols = new CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(getSaferTestName()); + assert !cols.isEmpty(); + + for (String category : categories) { + List cats = cols.stream().filter(c -> c.contains(category)).collect(Collectors.toList()); + Object[] expectedColOrder = cats.stream().sorted(Collections.reverseOrder()).toArray(); + Object[] actuals = cats.toArray(); + assertArrayEquals("expected reverse sorted", + expectedColOrder, + actuals); + + Instant colEndInstant = null; // exclusive end + + for (String col : cats) { // ASSUMPTION: reverse sorted order + Instant colStartInstant; + try { + colStartInstant = TimeRoutedAlias.parseInstantFromCollectionName(getAlias(), col); + } catch (Exception e) { + String colTmp = col; + // special case for tests... all of which have no more than one TRA dimension + // This won't work if we decide to write a test with 2 time dimensions. + // (but that's an odd case so we'll wait) + int traIndex = colTmp.indexOf(TRA)+ TRA.length(); + while (colTmp.lastIndexOf("__") > traIndex) { + colTmp = colTmp.substring(0,colTmp.lastIndexOf("__")); + } + colStartInstant = TimeRoutedAlias.parseInstantFromCollectionName(getAlias(), colTmp); + } + final QueryResponse colStatsResp = solrClient.query(col, params( + "q", "*:*", + "fq", catField + ":" + category, + "rows", "0", + "stats", "true", + "stats.field", getTimeField())); + long numFound = colStatsResp.getResults().getNumFound(); + if (numFound > 0) { + totalNumFound += numFound; + final FieldStatsInfo timestampStats = colStatsResp.getFieldStatsInfo().get(getTimeField()); + assertTrue(colStartInstant.toEpochMilli() <= ((Date) timestampStats.getMin()).getTime()); + if (colEndInstant != null) { + assertTrue(colEndInstant.toEpochMilli() > ((Date) timestampStats.getMax()).getTime()); + } + } + + colEndInstant = colStartInstant; // next older segment will max out at our current start time + } + + } + + assertEquals(expectNumFound, totalNumFound); + + assertEquals("COLS FOUND:" + cols, expectedCols.length, cols.size()); + } + + private void testFailedDocument(String category, String timestamp, String errorMsg) throws SolrServerException, IOException { + try { + final UpdateResponse resp = solrClient.add(getAlias(), newDoc(category, timestamp)); + // if we have a TolerantUpdateProcessor then we see it there) + final Object errors = resp.getResponseHeader().get("errors"); // Tolerant URP + assertTrue(errors != null && errors.toString().contains(errorMsg)); + } catch (SolrException e) { + assertTrue(e.getMessage().contains(errorMsg)); + } + numDocsDeletedOrFailed++; + } + + // convenience for constructing arrays. + private String[] ap(String... p) { + return p; + } + + + private SolrInputDocument newDoc(String category, String timestamp) { + Instant instant = Instant.parse(timestamp); + return sdoc("id", Integer.toString(++lastDocId), + getTimeField(), instant.toString(), + getCatField(), category, + getIntField(), "0"); // always 0 + } + + private String getTimeField() { + return timeField; + } + + private String getCatField() { + return catField; + } + + @Override + public String getAlias() { + return getSaferTestName(); + } + + @Override + public CloudSolrClient getSolrClient() { + return solrClient; + } + +} 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 6b34ea34d79..f751066d6b0 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 @@ -79,8 +79,16 @@ public abstract class RoutedAliasUpdateProcessorTest extends SolrCloudTestCase { Thread.sleep(500); } } + try { + DocCollection confirmCollection = cluster.getSolrClient().getClusterStateProvider().getClusterState().getCollectionOrNull(collection); + assertNotNull("Unable to find collection we were waiting for after done waiting",confirmCollection); + } catch (IOException e) { + fail("exception getting collection we were waiting for and have supposedly created already"); + } } + + private boolean haveCollection(String alias, String collection) { // separated into separate lines to make it easier to track down an NPE that occurred once // 3000 runs if it shows up again... @@ -192,6 +200,37 @@ public abstract class RoutedAliasUpdateProcessorTest extends SolrCloudTestCase { } } + protected void waitCoreCount(String collection, int count) { + long start = System.nanoTime(); + int coreFooCount; + List jsrs = cluster.getJettySolrRunners(); + do { + coreFooCount = 0; + // 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 coreDescriptors = jsr.getCoreContainer().getCoreDescriptors(); + for (CoreDescriptor coreDescriptor : coreDescriptors) { + String collectionName = coreDescriptor.getCollectionName(); + if (collection.equals(collectionName)) { + coreFooCount ++; + } + } + } + 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); + } catch (InterruptedException e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + } while(coreFooCount != count); + } + public abstract String getAlias() ; public abstract CloudSolrClient getSolrClient() ; @@ -229,19 +268,26 @@ public abstract class RoutedAliasUpdateProcessorTest extends SolrCloudTestCase { if (random().nextBoolean()) { // Send in separate threads. Choose random collection & solrClient + ExecutorService exec = null; try (CloudSolrClient solrClient = getCloudSolrClient(cluster)) { - ExecutorService exec = ExecutorUtil.newMDCAwareFixedThreadPool(1 + random().nextInt(2), - new DefaultSolrThreadFactory(getSaferTestName())); - List> futures = new ArrayList<>(solrInputDocuments.length); - for (SolrInputDocument solrInputDocument : solrInputDocuments) { - String col = collections.get(random().nextInt(collections.size())); - futures.add(exec.submit(() -> solrClient.add(col, solrInputDocument, commitWithin))); + try { + exec = ExecutorUtil.newMDCAwareFixedThreadPool(1 + random().nextInt(2), + new DefaultSolrThreadFactory(getSaferTestName())); + List> futures = new ArrayList<>(solrInputDocuments.length); + for (SolrInputDocument solrInputDocument : solrInputDocuments) { + String col = collections.get(random().nextInt(collections.size())); + futures.add(exec.submit(() -> solrClient.add(col, solrInputDocument, commitWithin))); + } + for (Future future : futures) { + assertUpdateResponse(future.get()); + } + // at this point there shouldn't be any tasks running + assertEquals(0, exec.shutdownNow().size()); + } finally { + if (exec != null) { + exec.shutdownNow(); + } } - for (Future future : futures) { - assertUpdateResponse(future.get()); - } - // at this point there shouldn't be any tasks running - assertEquals(0, exec.shutdownNow().size()); } } else { // send in a batch. 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 52945223034..e0dc8e0e27a 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,7 +31,6 @@ 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; @@ -50,7 +49,6 @@ 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.CoreDescriptor; import org.apache.solr.update.UpdateCommand; import org.apache.solr.util.LogLevel; import org.junit.After; @@ -59,9 +57,8 @@ import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.solr.client.solrj.RoutedAliasTypes.TIME; 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; @@ -96,8 +93,6 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess shutdownCluster(); } - - @Slow @Test @LogLevel("org.apache.solr.update.processor.TimeRoutedAlias=DEBUG;org.apache.solr.cloud=DEBUG") @@ -154,6 +149,8 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess newDoc(Instant.parse("2017-10-24T01:00:00Z")), newDoc(Instant.parse("2017-10-24T02:00:00Z")) ); +// System.out.println(cluster.getRandomJetty(random()).getBaseUrl()); +// Thread.sleep(1000000); assertInvariants(col24th, col23rd); // assert that the IncrementURP has updated all '0' to '1' @@ -350,6 +347,11 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess waitColAndAlias(alias, TRA, "2017-10-26", numShards); waitColAndAlias(alias2, TRA, "2017-10-26", numShards); + // these next checks will be checking that a collection DID NOT get created asynchronously, there's + // no way to wait for something that should never exist to not exist... so all we can do is sleep + // a good while before checking + Thread.sleep(5000); + // after this we can ignore alias2 checkPreemptiveCase1(alias); checkPreemptiveCase1(alias2); @@ -363,21 +365,21 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess // // Start and stop some cores that have TRA's... 2x2 used to ensure every jetty gets at least one - CollectionAdminRequest.createTimeRoutedAlias("foo", "2017-10-23T00:00:00Z", "+1DAY", getTimeField(), + CollectionAdminRequest.createTimeRoutedAlias(getSaferTestName() + "foo", "2017-10-23T00:00:00Z", "+1DAY", getTimeField(), CollectionAdminRequest.createCollection("_unused_", configName, 2, 2) .setMaxShardsPerNode(numReplicas)).setPreemptiveCreateWindow("3HOUR") .process(solrClient); - 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", + waitColAndAlias(getSaferTestName() + "foo", TRA, "2017-10-23",2); + waitCoreCount(getSaferTestName() + "foo" + TRA + "2017-10-23", 4); // prove this works, for confidence in deletion checking below. + assertUpdateResponse(solrClient.add(getSaferTestName() + "foo", sdoc("id","1","timestamp_dt", "2017-10-23T00:00:00Z") // no extra collections should be created )); - assertUpdateResponse(solrClient.commit("foo")); + assertUpdateResponse(solrClient.commit(getSaferTestName() + "foo")); - List foo = solrClient.getClusterStateProvider().resolveAlias("foo"); + List foo = solrClient.getClusterStateProvider().resolveAlias(getSaferTestName() + "foo"); - CollectionAdminRequest.deleteAlias("foo").process(solrClient); + CollectionAdminRequest.deleteAlias(getSaferTestName() + "foo").process(solrClient); for (String colName : foo) { CollectionAdminRequest.deleteCollection(colName).process(solrClient); @@ -387,12 +389,16 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess // if the design for terminating our executor is correct create/delete above will not cause failures below // continue testing... - // now test with pre-create window longer than time slice, and forcing multiple creations. + cols = new CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias); + assertEquals(4,cols.size()); // only one created in async case + + // now test with pre-create window longer than time slice, only one creation per request CollectionAdminRequest.setAliasProperty(alias) .addProperty(TimeRoutedAlias.ROUTER_PREEMPTIVE_CREATE_MATH, "3DAY").process(solrClient); - assertUpdateResponse(add(alias, Collections.singletonList( - sdoc("id", "7", "timestamp_dt", "2017-10-25T23:01:00Z")), // should cause preemptive creation of 10-27 now + assertUpdateResponse(add(alias, Arrays.asList( + sdoc("id", "7", "timestamp_dt", "2017-10-25T23:01:00Z"), // should cause preemptive creation of 10-27 now + sdoc("id", "71", "timestamp_dt", "2017-10-25T23:02:00Z")), // should not cause preemptive creation of 10-28 now params)); assertUpdateResponse(solrClient.commit(alias)); waitColAndAlias(alias, TRA, "2017-10-27", numShards); @@ -401,22 +407,22 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess assertEquals(5,cols.size()); // only one created in async case assertNumDocs("2017-10-23", 1, alias); assertNumDocs("2017-10-24", 1, alias); - assertNumDocs("2017-10-25", 5, alias); + assertNumDocs("2017-10-25", 6, alias); assertNumDocs("2017-10-26", 0, alias); assertNumDocs("2017-10-27", 0, alias); + assertUpdateResponse(add(alias, Collections.singletonList( 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, 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 + assertEquals(6,cols.size()); assertNumDocs("2017-10-23", 1, alias); assertNumDocs("2017-10-24", 1, alias); - assertNumDocs("2017-10-25", 6, alias); + assertNumDocs("2017-10-25", 7, alias); assertNumDocs("2017-10-26", 0, alias); assertNumDocs("2017-10-27", 0, alias); assertNumDocs("2017-10-28", 0, alias); @@ -425,7 +431,7 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess resp = solrClient.query(alias, params( "q", "*:*", "rows", "10")); - assertEquals(8, resp.getResults().getNumFound()); + assertEquals(9, resp.getResults().getNumFound()); assertUpdateResponse(add(alias, Arrays.asList( sdoc("id", "9", "timestamp_dt", "2017-10-27T23:01:00Z"), // should cause preemptive creation @@ -448,7 +454,7 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess assertEquals(7,cols.size()); assertNumDocs("2017-10-23", 1, alias); assertNumDocs("2017-10-24", 1, alias); - assertNumDocs("2017-10-25", 6, alias); + assertNumDocs("2017-10-25", 7, alias); assertNumDocs("2017-10-26", 0, alias); assertNumDocs("2017-10-27", 1, alias); assertNumDocs("2017-10-28", 3, alias); // should get through even though preemptive creation ignored it. @@ -457,7 +463,7 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess resp = solrClient.query(alias, params( "q", "*:*", "rows", "0")); - assertEquals(12, resp.getResults().getNumFound()); + assertEquals(13, resp.getResults().getNumFound()); // Sych creation with an interval longer than the time slice for the alias.. assertUpdateResponse(add(alias, Collections.singletonList( @@ -465,86 +471,60 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess params)); assertUpdateResponse(solrClient.commit(alias)); waitColAndAlias(alias, TRA, "2017-10-30", numShards); - waitColAndAlias(alias, TRA, "2017-10-31", numShards); // spooky! async case arising in middle of sync creation!! + + // removed support for this case because it created a LOT of complexity for the benefit of attempting to + // (maybe) not pause again after already hitting a synchronous creation (but only if asynch gets it done first, + // otherwise we have a race... not enough benefit to justify the support/complexity. + // + // Now we just let the next doc take care of it... + // + // 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()); + assertEquals(8,cols.size()); assertNumDocs("2017-10-23", 1, alias); assertNumDocs("2017-10-24", 1, alias); - assertNumDocs("2017-10-25", 6, alias); + assertNumDocs("2017-10-25", 7, alias); assertNumDocs("2017-10-26", 0, alias); assertNumDocs("2017-10-27", 1, alias); assertNumDocs("2017-10-28", 3, alias); // should get through even though preemptive creation ignored it. assertNumDocs("2017-10-29", 0, alias); assertNumDocs("2017-10-30", 1, alias); - assertNumDocs("2017-10-31", 0, alias); resp = solrClient.query(alias, params( "q", "*:*", "rows", "0")); - assertEquals(13, resp.getResults().getNumFound()); + assertEquals(14, resp.getResults().getNumFound()); assertUpdateResponse(add(alias, Collections.singletonList( - sdoc("id", "14", "timestamp_dt", "2017-10-31T23:01:00Z")), // should cause preemptive creation 11-01 + sdoc("id", "14", "timestamp_dt", "2017-10-30T23:01:00Z")), // should cause preemptive creation 10-31 + params)); + waitColAndAlias(alias, TRA, "2017-10-31", numShards); + + assertUpdateResponse(add(alias, Collections.singletonList( + sdoc("id", "15", "timestamp_dt", "2017-10-30T23:01:00Z")), // should cause preemptive creation 11-01 params)); 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 + sdoc("id", "16", "timestamp_dt", "2017-10-30T23:01:00Z")), // should cause preemptive creation 11-02 params)); 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, 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 + sdoc("id", "17", "timestamp_dt", "2017-10-30T23:01:00Z")), // should NOT cause preemptive creation 11-03 params)); cols = new CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias); - assertTrue("Preemptive creation beyond ROUTER_PREEMPTIVE_CREATE_MATH setting of 3DAY!",!cols.contains("myalias" + TRA + "2017-11-04")); + assertFalse(cols.contains("myalias" + TRA + "2017-11-03")); assertUpdateResponse(add(alias, Collections.singletonList( - sdoc("id", "18", "timestamp_dt", "2017-11-01T23:01:00Z")), // should cause preemptive creation 11-04 + sdoc("id", "18", "timestamp_dt", "2017-10-31T23:01:00Z")), // should cause preemptive creation 11-03 params)); - waitColAndAlias(alias, TRA, "2017-11-04",numShards); + waitColAndAlias(alias, TRA, "2017-11-03",numShards); } - private void waitCoreCount(String collection, int count) { - long start = System.nanoTime(); - int coreFooCount; - List jsrs = cluster.getJettySolrRunners(); - do { - coreFooCount = 0; - // 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 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) > 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); - } catch (InterruptedException e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - } while(coreFooCount != count); - } - private void concurrentUpdates(ModifiableSolrParams params, String alias) throws SolrServerException, IOException { // In this method we intentionally rely on timing of a race condition but the gap in collection creation time vs // requesting the list of aliases and adding a single doc should be very large (1-2 seconds vs a few ms so we @@ -571,7 +551,8 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess // Here we quickly add another doc in a separate request, before the collection creation has completed. // This has the potential to incorrectly cause preemptive collection creation to run twice and create a - // second collection. RoutedAliasUpdateProcessor is meant to guard against this race condition. + // second collection. MaintainRoutedAliasCmd is meant to guard against this race condition by acquiring + // a lock on the collection name. assertUpdateResponse(add(alias, Collections.singletonList( sdoc("id", "6", "timestamp_dt", "2017-10-25T23:01:00Z")), // might cause duplicate preemptive creation params)); @@ -584,6 +565,8 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess assertTrue("Preemptive creation happened twice and created a collection " + "further in the future than the configured time slice!",!cols.contains("myalias" + TRA + "2017-10-27")); + validateCollectionCountAndAvailability(alias, 4, "Only 4 cols expected (premptive create happened" + + "twice among threads"); assertEquals(4, cols.size()); assertNumDocs("2017-10-23", 1, alias); assertNumDocs("2017-10-24", 1, alias); @@ -611,6 +594,24 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess assertNumDocs("2017-10-23", 0, alias); assertNumDocs("2017-10-24", 0, alias); assertNumDocs("2017-10-25", 1, alias); + + validateCollectionCountAndAvailability(alias, 3, "was expecting 3 live collections"); + } + + private void validateCollectionCountAndAvailability(String alias, int expected, String message) throws SolrServerException, IOException { + List cols; + cols = new CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias); + assertEquals(message,expected,cols.size()); // only one created in async case + + // make sure they all exist + for (String col : cols) { + try { + solrClient.query(col, params("q", "*:*","rows", "10")); + } catch (SolrException e) { + e.printStackTrace(); + fail("Unable to query " + col); + } + } } private void assertNumDocs(final String datePart, int expected, String alias) throws SolrServerException, IOException { @@ -788,9 +789,8 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess sdoc("id","6","timestamp_dt", "2017-10-26T12:00:01Z") // preemptive )); waitColAndAlias(alias, TRA,"2017-10-26",1); - checkCollectionCountIs(4) + checkCollectionCountIs(3) .containsAll(Arrays.asList( - "myalias_2017-10-23", "myalias_2017-10-24", "myalias" + TRA + "2017-10-25", "myalias" + TRA + "2017-10-26")); @@ -800,37 +800,24 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess )); waitColAndAlias(alias, TRA,"2017-10-27",1); waitCoreCount("myalias_2017-10-23",0); - checkCollectionCountIs(4) + checkCollectionCountIs(3) .containsAll(Arrays.asList( - "myalias_2017-10-24", "myalias" + TRA + "2017-10-25", "myalias" + TRA + "2017-10-26", "myalias" + TRA + "2017-10-27")); + // verify that auto-delete works on new collections. 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) + checkCollectionCountIs(3) .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); @@ -838,7 +825,7 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess "q", "*:*", "rows", "10" )); - assertEquals(4,resp.getResults().getNumFound()); + assertEquals(3,resp.getResults().getNumFound()); } @@ -848,10 +835,10 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess 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(); +// System.out.println(); +// System.out.println(clusterStateProvider.getClass()); +// System.out.println(collections); +// System.out.println(); assertEquals(num, collections.size()); // starting point return collections; } diff --git a/solr/solr-ref-guide/src/aliases.adoc b/solr/solr-ref-guide/src/aliases.adoc index 34d9b883abf..b4f5e1e6301 100644 --- a/solr/solr-ref-guide/src/aliases.adoc +++ b/solr/solr-ref-guide/src/aliases.adoc @@ -76,12 +76,12 @@ When adding data, you should usually direct documents to the alias (e.g., refere The Solr server and `CloudSolrClient` will direct an update request to the first collection that an alias points to. Once the server receives the data it will perform the necessary routing. -WARNING: It is possible to update the collections - directly, but there is no safeguard against putting data in the incorrect collection if the alias is circumvented - in this manner. +WARNING: It's extremely important with all routed aliases that the route values NOT change. Re-indexing a document +with a different route value for the same ID produces two distinct documents with the same ID accessible via the alias. +All query time behavior of the routed alias is *_undefined_* and not easily predictable once duplicate ID's exist. -CAUTION: It is a bad idea to use "data driven" mode (aka <>) with routed aliases, as duplicate schema mutations might happen -concurrently leading to errors. +CAUTION: It is a bad idea to use "data driven" mode (aka <>) with +routed aliases, as duplicate schema mutations might happen concurrently leading to errors. === Time Routed Aliases @@ -247,6 +247,120 @@ the following procedure is recommended: the name of the collection to be created has been calculated. It may not be avoided and is necessary to support future features. +=== Dimensional Routed Aliases + +For cases where the desired segregation of of data relates to two fields and combination into a single +field during indexing is impractical, or the TRA behavior is desired across multiple categories, +Dimensional Routed aliases may be used. This feature has been designed to handle an arbitrary number +and combination of category and time dimensions in any order, but users are cautioned to carefully +consider the total number of collections that will result from such configurations. Collection counts +in the high hundreds or low 1000's begin to pose significant challenges with zookeeper. + +NOTE: DRA's are a new feature and presently only 2 dimensions are supported. More dimensions will +be supported in the future (see https://issues.apache.org/jira/browse/SOLR-13628 for progress) + +==== How It Works + +First you create a dimensional routed alias with the desired router settings for each dimension. See the +<> command documentation for details on how to specify the +per-dimension configuration. Typical collection names will be of the form (example is for category x time example, +with 30 minute intervals): + + myalias__CRA__someCategory__TRA__2019-07-01_00_30 + +Note that the initial collection will be a throw away place holder for any DRA containing a category based dimension. +Name generation for each sub-part of a collection name is identical to the corresponding potion of the component +dimension type. (e.g. a category value generating __CRA__ or __TRA__ would still produce an error) + +WARNING: The prior warning about reindexing documents with different route value applies to every dimensio of +a DRA. DRA's are inappropriate for documents where categories or timestamps used in routing will change (this of +course applies to other route values in future RA types too). + +As with all Routed Aliases, DRA's impose some costs if your data is not well behaved. In addition to the +normal caveats of each component dimension there is a need for care in sending new categories after the DRA has been +running for a while. Ordered Dimensions (time) behave slightly differently from Unordered (category) dimensions. +Ordered dimensions rely on the iteration order of the collections in the alias and therefore cannot tolerate the +generation of collection names out of order. The this means that of this is that when an ordered dimension such as time +is a component of a DRA and the DRA experiences receipt of a document with a novel category with a time value +corresponding to a time slice other than the starting time-slice for the time dimension, several collections will +need to be created before the document can be indexed. This "new category effect" is identical to the behavior +you would get with a TRA if you picked a start-date too far in the past. + +For example given a Dimensional[time,category] DRA with start time of 2019-07-01T00:00:00Z the pattern of collections +created for 4 documents might look like this: + +*No documents* + +*Aliased collections:* + + // temp avoids empty alias error conditions + myalias__TRA__2019-07-01__CRA__NEW_CATEGORY_ROUTED_ALIAS_WAITING_FOR_DATA_TEMP + +*Doc 1* + + * time: 2019-07-01T00:00:00Z + * category: someCategory + +*Aliased collections:* + + // temp retained to avoid empty alias during race with collection creation + myalias__TRA__2019-07-01__CRA__NEW_CATEGORY_ROUTED_ALIAS_WAITING_FOR_DATA_TEMP + myalias__TRA__2019-07-01__CRA__someCategory + +*Doc 2* + + * time: 2019-07-02T00:04:00Z + * category: otherCategory + +*Aliased collections:* + + // temp can now be deleted without risk of having an empty alias. + myalias__TRA__2019-07-01__CRA__someCategory + myalias__TRA__2019-07-01__CRA__otherCategory // 2 collections created in one update + myalias__TRA__2019-07-02__CRA__otherCategory + +*Doc 3* + + * time: 2019-07-03T00:12:00Z + * category: thirdCategory + +*Aliased collections:* + + myalias__TRA__2019-07-01__CRA__someCategory + myalias__TRA__2019-07-01__CRA__otherCategory + myalias__TRA__2019-07-02__CRA__otherCategory + myalias__TRA__2019-07-01__CRA__thirdCategory // 3 collections created in one update! + myalias__TRA__2019-07-02__CRA__thirdCategory + myalias__TRA__2019-07-03__CRA__thirdCategory + +*Doc 4* + + * time: 2019-07-03T00:12:00Z + * category: someCategory + +*Aliased collections:* + + myalias__TRA__2019-07-01__CRA__someCategory + myalias__TRA__2019-07-01__CRA__otherCategory + myalias__TRA__2019-07-02__CRA__otherCategory + myalias__TRA__2019-07-01__CRA__thirdCategory + myalias__TRA__2019-07-02__CRA__thirdCategory + myalias__TRA__2019-07-03__CRA__thirdCategory + myalias__TRA__2019-07-02__CRA__someCategory // 2 collections created in one update + myalias__TRA__2019-07-03__CRA__someCategory + +Therefore the sweet spot for DRA's is for a data set with a well standardized set of dimensions that are not changing +and where the full set of permutations occur regularly. If a new category is introduced at a later date and +indexing latency is an important SLA feature, there are a couple strategies to mitigate this effect: + +* If the number of extra time slices to be created is not very large, then sending a single document out of band from + regular indexing, and waiting for collection creation to complete before allowing the new category to be sent via the + SLA constrained process. + +* If the above procedure is likely to create an extreme number of collections, and the earliest possible document in + the new category is known, the start time for the time dimension may be adjusted using the + <> command + === Improvement Possibilities Routed aliases are a relatively new feature of SolrCloud that can be expected to be improved. @@ -261,6 +375,8 @@ Some _potential_ areas for improvement that _are not implemented yet_ are: * *CRAs*: Supply an initial list of values for cases where these are known before hand to reduce pauses during indexing. +* *DRAs*: Support for more than 2 dimensions. + * `CloudSolrClient` could route documents to the correct collection based on the route value instead always picking the latest/first. diff --git a/solr/solr-ref-guide/src/collection-aliasing.adoc b/solr/solr-ref-guide/src/collection-aliasing.adoc index ab99db6673f..8ec6d13a7ff 100644 --- a/solr/solr-ref-guide/src/collection-aliasing.adoc +++ b/solr/solr-ref-guide/src/collection-aliasing.adoc @@ -97,7 +97,13 @@ prohibited. If routing parameters are present this parameter is prohibited. Most routed alias parameters become _alias properties_ that can subsequently be inspected and <>. `router.name`:: -The type of routing to use. Presently only `time` and `category` are valid. This parameter is required. +The type of routing to use. Presently only `time` and `category` and `Dimensional[]` are valid. +In the case of a multi dimensional routed alias (A. K. A. "DRA", see <> +documentation), it is required to express all the dimensions in the same order that they will appear in the dimension +array. The format for a DRA router.name is Dimensional[dim1,dim2] where dim1 and dim2 are valid router.name +values for each sub-dimension. Note that DRA's are very new, and only 2D DRA's are presently supported. Higher +numbers of dimensions will be supported soon. See examples below for further clarification on how to configure +individual dimensions. This parameter is required. `router.field`:: The field to inspect to determine which underlying collection an incoming document should be routed to. @@ -189,6 +195,14 @@ indexed. Any valid Java regular expression pattern may be specified. This expres each request so batching of updates is strongly recommended. Overly complex patterns will produce cpu or garbage collecting overhead during indexing as determined by the JVM's implementation of regular expressions. +==== Dimensional Routed Alias Parameters + + +`router.#.`:: +This prefix denotes which position in the dimension array is being referred to for purposes of dimension configuration. +For example in a Dimensional[time,category] router.0.start would be used to set the start time for the time dimension. + + === CREATEALIAS Response The output will simply be a responseHeader with details of the time it took to process the request. @@ -196,20 +210,49 @@ To confirm the creation of the alias, you can look in the Solr Admin UI, under t `aliases.json` file. The initial collection for routed aliases should also be visible in various parts of the admin UI. === Examples using CREATEALIAS +Create an alias named "testalias" and link it to the collections named "foo" and "bar". + +[.dynamic-tabs] +-- + +[example.tab-pane#v2createAlias] +==== +[.tab-label]*V2 API* +*Input* + +[source,json] +---- +{ + "create-alias":{ + "name":"testalias", + "collections":["foo","bar"] + } +} +---- +*Output* + +[source,json] +---- +{ + "responseHeader": { + "status": 0, + "QTime": 125 + } +} +---- +==== + +[example.tab-pane#v1createAlias] +==== +[.tab-label]*V1 API* *Input* -Create an alias named "testalias" and link it to the collections named "anotherCollection" and "testCollection". - -// tag::createalias-simple-example[] - [source,text] ---- -http://localhost:8983/solr/admin/collections?action=CREATEALIAS&name=testalias&collections=anotherCollection,testCollection&wt=xml +http://localhost:8983/solr/admin/collections?action=CREATEALIAS&name=testalias&collections=foo,bar&wt=xml ---- -//end::createalias-simple-example[] - *Output* [source,xml] @@ -221,41 +264,27 @@ http://localhost:8983/solr/admin/collections?action=CREATEALIAS&name=testalias&c ---- +==== +-- -*Input* - -Create an alias named "myTimeData" for data beginning on `2018-01-15` in the UTC time zone and partitioning daily -based on the `evt_dt` field in the incoming documents. Data more than one hour beyond the latest (most recent) -partition is to be rejected and collections are created using a configset named "myConfig". - -[source,text] ----- -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* - -[source,xml] ----- - - - 0 - 1234 - - ----- - - -*Input* - -A somewhat contrived example demonstrating the <> usage and additional collection creation options. +A somewhat contrived example demonstrating creating a TRA with many additional collection creation options. Notice that the collection creation parameters follow the v2 API naming convention, not the v1 naming conventions. +[.dynamic-tabs] +-- + +[example.tab-pane#v2createTRA] +==== +[.tab-label]*V2 API* + +*Input* + +POST /api/c + [source,json] ---- -POST /api/c { - "create-routed-alias" : { + "create-alias" : { "name": "somethingTemporalThisWayComes", "router" : { "name": "time", @@ -285,7 +314,7 @@ POST /api/c *Output* -[source,xml] +[source,json] ---- { "responseHeader": { @@ -294,6 +323,132 @@ POST /api/c } } ---- +==== + +[example.tab-pane#v1createTRA] +==== +[.tab-label]*V1 API* + +*Input* + +[source,text] +---- +http://localhost:8983/solr/admin/collections?action=CREATEALIAS + &name=somethingTemporalThisWayComes + &router.name=time + &router.start=NOW/MINUTE + &router.field=evt_dt + &router.interval=%2B2HOUR + &router.maxFutureMs=14400000 + &create-collection.collection.configName=_default + &create-collection.router.name=implicit + &create-collection.router.field=foo_s + &create-collection.numShards=3 + &create-collection.shards=foo,bar,baz + &create-collection.tlogReplicas=1 + &create-collection.pullReplicas=1 + &create-collection.maxShardsPerNode=2 + &create-collection.property.foobar=bazbam +---- + +*Output* + +[source,xml] +---- + + + 0 + 1234 + + +---- +==== +-- + +Another example, this time of a Dimensional Routed Alias demonstrating how to specify parameters for the +individual dimensions + +[.dynamic-tabs] +-- + +[example.tab-pane#v2createDRA] +==== +[.tab-label]*V2 API* + +*Input* + +POST /api/c +[source,json] +---- +{ + "create-alias":{ + "name":"dra_test1", + "router": { + "name": "Dimensional[time,category]", + "routerList" : [ { + "field":"myDate_tdt", + "start":"2019-01-01T00:00:00Z", + "interval":"+1MONTH", + "maxFutureMs":600000 + },{ + "field":"myCategory_s", + "maxCardinality":20 + }] + }, + "create-collection": { + "config":"_default", + "numShards":2 + } + } +} +---- +*Output* + +[source,json] +---- +{ + "responseHeader": { + "status": 0, + "QTime": 1234 + } +} +---- +==== + +[example.tab-pane#v1createDRA] +==== +[.tab-label]*V1 API* + +*Input* + +[source,text] +---- +http://localhost:8983/solr/admin/collections?action=CREATEALIAS + &name=dra_test1 + &router.name=Dimensional[time,category] + &router.0.start=2019-01-01T00:00:00Z + &router.0.field=myDate_tdt + &router.0.interval=%2B1MONTH + &router.0.maxFutureMs=600000 + &create-collection.collection.configName=_default + &create-collection.numShards=2 + &router.1.maxCardinality=20 + &router.1.field=myCategory_s +---- + +*Output* + +[source,xml] +---- + + + 0 + 1234 + + +---- +==== +-- [[listaliases]] == LISTALIASES: List of all aliases in the cluster diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/RoutedAliasTypes.java b/solr/solrj/src/java/org/apache/solr/client/solrj/RoutedAliasTypes.java new file mode 100644 index 00000000000..bad1bcf2bdd --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/RoutedAliasTypes.java @@ -0,0 +1,47 @@ +/* + * 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.client.solrj; + + +/** + * Types of Routed Alias supported. + * + * 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. + */ +public enum RoutedAliasTypes { + TIME { + @Override + public String getSeparatorPrefix() { + return "__TRA__"; + } + }, + CATEGORY { + @Override + public String getSeparatorPrefix() { + return "__CRA__"; + } + }, + DIMENSIONAL { + public String getSeparatorPrefix() { + throw new UnsupportedOperationException("dimensions within dimensions are not allowed"); + } + }; + public abstract String getSeparatorPrefix(); + +} 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 7f3cbd41bde..0175dd684da 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 @@ -17,16 +17,19 @@ package org.apache.solr.client.solrj.request; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.Iterator; import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.Set; import java.util.TimeZone; import java.util.UUID; import java.util.concurrent.TimeUnit; +import org.apache.solr.client.solrj.RoutedAliasTypes; import org.apache.solr.client.solrj.SolrClient; import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.SolrResponse; @@ -62,12 +65,13 @@ import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS; import static org.apache.solr.common.cloud.ZkStateReader.READ_ONLY; import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR; import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS; +import static org.apache.solr.common.params.CollectionAdminParams.ALIAS; import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF; import static org.apache.solr.common.params.CollectionAdminParams.COLOCATED_WITH; import static org.apache.solr.common.params.CollectionAdminParams.COUNT_PROP; import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SET_PARAM; import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM; -import static org.apache.solr.common.params.CollectionAdminParams.ALIAS; +import static org.apache.solr.common.params.CollectionAdminParams.ROUTER_PREFIX; import static org.apache.solr.common.params.CollectionAdminParams.WITH_COLLECTION; /** @@ -1732,13 +1736,11 @@ public abstract class CollectionAdminRequest return new CreateTimeRoutedAlias(aliasName, routerField, start, interval, createCollTemplate); } - public static class CreateTimeRoutedAlias extends AsyncCollectionAdminRequest { + public static class CreateTimeRoutedAlias extends AsyncCollectionAdminRequest implements RoutedAliasAdminRequest { // TODO: This and other commands in this file seem to need to share some sort of constants class with core // to allow this stuff not to be duplicated. (this is pasted from CreateAliasCmd.java), however I think // a comprehensive cleanup of this for all the requests in this class should be done as a separate ticket. - public static final String ROUTER_TYPE_NAME = "router.name"; - public static final String ROUTER_FIELD = "router.field"; public static final String ROUTER_START = "router.start"; public static final String ROUTER_INTERVAL = "router.interval"; public static final String ROUTER_MAX_FUTURE = "router.maxFutureMs"; @@ -1810,19 +1812,29 @@ public abstract class CollectionAdminRequest } // merge the above with collectionParams. Above takes precedence. - ModifiableSolrParams createCollParams = new ModifiableSolrParams(); // output target - final SolrParams collParams = createCollTemplate.getParams(); - final Iterator pIter = collParams.getParameterNamesIterator(); - while (pIter.hasNext()) { - String key = pIter.next(); - if (key.equals(CollectionParams.ACTION) || key.equals("name")) { - continue; - } - createCollParams.set("create-collection." + key, collParams.getParams(key)); - } + ModifiableSolrParams createCollParams = mergeCollParams(createCollTemplate); return SolrParams.wrapDefaults(params, createCollParams); } + @Override + public RoutedAliasTypes getType() { + return RoutedAliasTypes.TIME; + } + + @Override + public String getRouterField() { + return routerField; + } + + @Override + public java.util.List getParamNames() { + return java.util.List.of(ROUTER_TYPE_NAME, ROUTER_FIELD, ROUTER_START, ROUTER_INTERVAL,ROUTER_MAX_FUTURE, ROUTER_PREEMPTIVE_CREATE_WINDOW, ROUTER_AUTO_DELETE_AGE, CommonParams.TZ); + } + + @Override + public java.util.List getRequiredParamNames() { + return java.util.List.of(ROUTER_TYPE_NAME, ROUTER_FIELD,ROUTER_START, ROUTER_INTERVAL); + } } /** * Returns a SolrRequest to create a category routed alias. @@ -1840,10 +1852,8 @@ public abstract class CollectionAdminRequest return new CreateCategoryRoutedAlias(aliasName, routerField, maxCardinality, createCollTemplate); } - public static class CreateCategoryRoutedAlias extends AsyncCollectionAdminRequest { + public static class CreateCategoryRoutedAlias extends AsyncCollectionAdminRequest implements RoutedAliasAdminRequest { - public static final String ROUTER_TYPE_NAME = "router.name"; - public static final String ROUTER_FIELD = "router.field"; public static final String ROUTER_MAX_CARDINALITY = "router.maxCardinality"; public static final String ROUTER_MUST_MATCH = "router.mustMatch"; @@ -1871,7 +1881,7 @@ public abstract class CollectionAdminRequest public SolrParams getParams() { ModifiableSolrParams params = (ModifiableSolrParams) super.getParams(); params.add(CommonParams.NAME, aliasName); - params.add(ROUTER_TYPE_NAME, "category"); + params.add(ROUTER_TYPE_NAME, RoutedAliasTypes.CATEGORY.name()); params.add(ROUTER_FIELD, routerField); params.add(ROUTER_MAX_CARDINALITY, maxCardinality.toString()); @@ -1880,7 +1890,45 @@ public abstract class CollectionAdminRequest } // merge the above with collectionParams. Above takes precedence. + ModifiableSolrParams createCollParams = mergeCollParams(createCollTemplate); + return SolrParams.wrapDefaults(params, createCollParams); + } + + @Override + public RoutedAliasTypes getType() { + return RoutedAliasTypes.CATEGORY; + } + + @Override + public String getRouterField() { + return routerField; + } + + @Override + public java.util.List getParamNames() { + return java.util.List.of(ROUTER_TYPE_NAME, ROUTER_FIELD,ROUTER_MAX_CARDINALITY, ROUTER_MUST_MATCH); + } + + @Override + public java.util.List getRequiredParamNames() { + return java.util.List.of(ROUTER_TYPE_NAME, ROUTER_FIELD,ROUTER_MAX_CARDINALITY); + } + } + + public interface RoutedAliasAdminRequest { + String ROUTER_TYPE_NAME = "router.name"; + String ROUTER_FIELD = "router.field"; + + RoutedAliasTypes getType(); + String getRouterField(); + java.util.List getParamNames(); + java.util.List getRequiredParamNames(); + SolrParams getParams(); + default ModifiableSolrParams mergeCollParams(Create createCollTemplate) { ModifiableSolrParams createCollParams = new ModifiableSolrParams(); // output target + if (createCollTemplate == null) { + return createCollParams; + } final SolrParams collParams = createCollTemplate.getParams(); final Iterator pIter = collParams.getParameterNamesIterator(); while (pIter.hasNext()) { @@ -1890,9 +1938,99 @@ public abstract class CollectionAdminRequest } createCollParams.set("create-collection." + key, collParams.getParams(key)); } + return createCollParams; + } + } + + /** + * Create a Dimensional Routed alias from two or more routed alias types. + * + * @param aliasName The name of the alias + * @param createCollTemplate a create command that will be used for all collections created + * @param dims Routed Alias requests. Note that the aliasName and collection templates inside dimensions + * will be ignored and may be safely set to null + * @return An object representing a basic DimensionalRoutedAlias creation request. + */ + public static DimensionalRoutedAlias createDimensionalRoutedAlias(String aliasName, Create createCollTemplate, RoutedAliasAdminRequest... dims) { + return new DimensionalRoutedAlias(aliasName, createCollTemplate, dims); + } + + public static class DimensionalRoutedAlias extends AsyncCollectionAdminRequest implements RoutedAliasAdminRequest { + + private String aliasName; + private final Create createCollTemplate; + private final RoutedAliasAdminRequest[] dims; + + public DimensionalRoutedAlias(String aliasName, Create createCollTemplate, RoutedAliasAdminRequest... dims) { + super(CollectionAction.CREATEALIAS); + this.aliasName = aliasName; + this.createCollTemplate = createCollTemplate; + this.dims = dims; + } + + public static void addDimensionIndexIfRequired(Set params, int i, String param) { + params.add(withDimensionIndexIfRequired(param, i)); + } + + private static String withDimensionIndexIfRequired(String param, int index) { + if (param.startsWith(ROUTER_PREFIX)) { + return ROUTER_PREFIX + index + "." + param.split("\\.")[1]; + } else { + return param; + } + } + + + @Override + public SolrParams getParams() { + ModifiableSolrParams params = (ModifiableSolrParams) super.getParams(); + java.util.List types = new ArrayList<>(); + java.util.List fields = new ArrayList<>(); + for (int i = 0; i < dims.length; i++) { + RoutedAliasAdminRequest dim = dims[i]; + types.add(dim.getType().name()); + fields.add(dim.getRouterField()); + for (String param : dim.getParamNames()) { + String value = dim.getParams().get(param); + if (value != null) { + params.add(withDimensionIndexIfRequired(param, i), value); + } else { + if (dim.getRequiredParamNames().contains(param)) { + throw new IllegalArgumentException("Dimension of type " + dim.getType() + " requires a value for " + param); + } + } + } + } + params.add(CommonParams.NAME, aliasName); + params.add(ROUTER_TYPE_NAME, "Dimensional[" + String.join(",", types) + "]"); + params.add(ROUTER_FIELD, String.join(",", fields)); + + // merge the above with collectionParams. Above takes precedence. + ModifiableSolrParams createCollParams = mergeCollParams(createCollTemplate); return SolrParams.wrapDefaults(params, createCollParams); } + + + @Override + public RoutedAliasTypes getType() { + throw new UnsupportedOperationException("Dimensions of dimensions are not allowed, the multiverse might collapse!"); + } + + @Override + public String getRouterField() { + throw new UnsupportedOperationException("Dimensions of dimensions are not allowed, the multiverse might collapse!"); + } + + @Override + public java.util.List getParamNames() { + throw new UnsupportedOperationException("Dimensions of dimensions are not allowed, the multiverse might collapse!"); + } + + @Override + public java.util.List getRequiredParamNames() { + throw new UnsupportedOperationException("Dimensions of dimensions are not allowed, the multiverse might collapse!"); + } } /** diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java index 23871a7dfb5..1752377daec 100644 --- a/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java +++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java @@ -17,7 +17,6 @@ package org.apache.solr.common.cloud; import java.lang.invoke.MethodHandles; - import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -36,7 +35,7 @@ public class ConnectionManager implements Watcher { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private final String name; - + private volatile boolean connected = false; private final ZkClientConnectionStrategy connectionStrategy; @@ -73,7 +72,7 @@ public class ConnectionManager implements Watcher { || ( stateType == StateType.TRACKING_TIME && (System.nanoTime() - lastDisconnectTime > TimeUnit.NANOSECONDS.convert(timeToExpire, TimeUnit.MILLISECONDS))); } } - + public static abstract class IsClosed { public abstract boolean isClosed(); } @@ -91,7 +90,7 @@ public class ConnectionManager implements Watcher { this.beforeReconnect = beforeReconnect; this.isClosedCheck = isClosed; } - + private synchronized void connected() { connected = true; likelyExpiredState = LikelyExpiredState.NOT_EXPIRED; @@ -114,14 +113,14 @@ public class ConnectionManager implements Watcher { } else { log.debug("Watcher {} name: {} got event {} path: {} type: {}", this, name, event, event.getPath(), event.getType()); } - + if (isClosed()) { log.debug("Client->ZooKeeper status change trigger but we are already closed"); return; } - + KeeperState state = event.getState(); - + if (state == KeeperState.SyncConnected) { log.info("zkClient has connected"); connected(); @@ -133,9 +132,9 @@ public class ConnectionManager implements Watcher { // we don't call disconnected here, because we know we are expired connected = false; likelyExpiredState = LikelyExpiredState.EXPIRED; - + log.warn("Our previous ZooKeeper session was expired. Attempting to reconnect to recover relationship with ZooKeeper..."); - + if (beforeReconnect != null) { try { beforeReconnect.command(); @@ -143,7 +142,7 @@ public class ConnectionManager implements Watcher { log.warn("Exception running beforeReconnect command", e); } } - + do { // This loop will break if a valid connection is made. If a connection is not made then it will repeat and // try again to create a new connection. @@ -164,11 +163,11 @@ public class ConnectionManager implements Watcher { // we must have been asked to stop throw new RuntimeException(e); } - + if (onReconnect != null) { onReconnect.command(); } - + } catch (Exception e1) { // if there was a problem creating the new SolrZooKeeper // or if we cannot run our reconnect command, close the keeper @@ -178,15 +177,15 @@ public class ConnectionManager implements Watcher { } } }); - + break; - + } catch (Exception e) { SolrException.log(log, "", e); log.info("Could not connect due to error, sleeping for 1s and trying again"); waitSleep(1000); } - + } while (!isClosed()); log.info("zkClient Connected:" + connected); } else if (state == KeeperState.Disconnected) { @@ -201,32 +200,32 @@ public class ConnectionManager implements Watcher { public synchronized boolean isConnectedAndNotClosed() { return !isClosed() && connected; } - + public synchronized boolean isConnected() { return connected; } - + // we use a volatile rather than sync // to avoid possible deadlock on shutdown public void close() { this.isClosed = true; this.likelyExpiredState = LikelyExpiredState.EXPIRED; } - + private boolean isClosed() { return isClosed || isClosedCheck.isClosed(); } - + public boolean isLikelyExpired() { return isClosed() || likelyExpiredState.isLikelyExpired((long) (client.getZkClientTimeout() * 0.90)); } - + public synchronized void waitSleep(long waitFor) { try { wait(waitFor); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - } + } } public synchronized void waitForConnected(long waitForConnection) diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java index bd4f1ab9a6d..d77d15e54de 100644 --- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java +++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java @@ -33,20 +33,20 @@ import org.slf4j.LoggerFactory; */ public abstract class ZkClientConnectionStrategy { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - + private volatile ZkCredentialsProvider zkCredentialsToAddAutomatically; private volatile boolean zkCredentialsToAddAutomaticallyUsed; - + private List disconnectedListeners = new ArrayList<>(); private List connectedListeners = new ArrayList<>(); - + public abstract void connect(String zkServerAddress, int zkClientTimeout, Watcher watcher, ZkUpdate updater) throws IOException, InterruptedException, TimeoutException; public abstract void reconnect(String serverAddress, int zkClientTimeout, Watcher watcher, ZkUpdate updater) throws IOException, InterruptedException, TimeoutException; - + public ZkClientConnectionStrategy() { zkCredentialsToAddAutomaticallyUsed = false; } - + public synchronized void disconnected() { for (DisconnectedListener listener : disconnectedListeners) { try { @@ -56,7 +56,7 @@ public abstract class ZkClientConnectionStrategy { } } } - + public synchronized void connected() { for (ConnectedListener listener : connectedListeners) { try { @@ -66,20 +66,24 @@ public abstract class ZkClientConnectionStrategy { } } } - + public interface DisconnectedListener { void disconnected(); } - + public interface ConnectedListener { void connected(); } - - + + public synchronized void addDisconnectedListener(DisconnectedListener listener) { disconnectedListeners.add(listener); } - + + public synchronized void removeDisconnectedListener(DisconnectedListener listener) { + disconnectedListeners.remove(listener); + } + public synchronized void addConnectedListener(ConnectedListener listener) { connectedListeners.add(listener); } @@ -87,13 +91,13 @@ public abstract class ZkClientConnectionStrategy { public interface ZkUpdate { void update(SolrZooKeeper zooKeeper) throws InterruptedException, TimeoutException, IOException; } - + public void setZkCredentialsToAddAutomatically(ZkCredentialsProvider zkCredentialsToAddAutomatically) { - if (zkCredentialsToAddAutomaticallyUsed || (zkCredentialsToAddAutomatically == null)) + if (zkCredentialsToAddAutomaticallyUsed || (zkCredentialsToAddAutomatically == null)) throw new RuntimeException("Cannot change zkCredentialsToAddAutomatically after it has been (connect or reconnect was called) used or to null"); this.zkCredentialsToAddAutomatically = zkCredentialsToAddAutomatically; } - + public boolean hasZkCredentialsToAddAutomatically() { return zkCredentialsToAddAutomatically != null; } @@ -103,7 +107,7 @@ public abstract class ZkClientConnectionStrategy { protected SolrZooKeeper createSolrZooKeeper(final String serverAddress, final int zkClientTimeout, final Watcher watcher) throws IOException { SolrZooKeeper result = new SolrZooKeeper(serverAddress, zkClientTimeout, watcher); - + zkCredentialsToAddAutomaticallyUsed = true; for (ZkCredentials zkCredentials : zkCredentialsToAddAutomatically.getCredentials()) { result.addAuthInfo(zkCredentials.getScheme(), zkCredentials.getAuth()); @@ -111,5 +115,5 @@ public abstract class ZkClientConnectionStrategy { return result; } - + } 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 88b2aaa0339..89f1600722b 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 @@ -79,8 +79,7 @@ public interface CollectionParams { DELETEALIAS(true, LockLevel.COLLECTION), ALIASPROP(true, LockLevel.COLLECTION), LISTALIASES(false, LockLevel.NONE), - MAINTAINTIMEROUTEDALIAS(true, LockLevel.COLLECTION), // internal use only - MAINTAINCATEGORYROUTEDALIAS(true, LockLevel.COLLECTION), // internal use only + MAINTAINROUTEDALIAS(true, LockLevel.COLLECTION), // internal use only DELETEROUTEDALIASCOLLECTIONS(true, LockLevel.COLLECTION), SPLITSHARD(true, LockLevel.SHARD), DELETESHARD(true, LockLevel.SHARD), diff --git a/solr/solrj/src/resources/apispec/collections.Commands.json b/solr/solrj/src/resources/apispec/collections.Commands.json index 827f1a79d2f..5fc7f76561d 100644 --- a/solr/solrj/src/resources/apispec/collections.Commands.json +++ b/solr/solrj/src/resources/apispec/collections.Commands.json @@ -183,6 +183,14 @@ "mustMatch": { "type": "string", "description": "A regular expression that the value of the field specified by `router.field` must match before a corresponding collection will be created." + }, + "routerList": { + "type": "array", + "description": "A list of router property sets to be used with router type Dimensional[foo,bar] where foo and bar are valid router type names (i.e. time or category). The order must correspond to the type specification in [] in the Dimensional type, so Dimensional[category,time] would require the first set of router properties to be valid for a category routed alias, and the second set to be valid for a time routed alias. In these sets of properties, router.name will be ignored in favor of the type specified in the top level Dimensional[] router.name", + "items": { + "type": "object", + "additionalProperties": true + } } } },