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

[lucene-solr] branch branch_8_11 updated: SOLR-16194 Backport from main, excluding new method that throws per discussion. (#2658)

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

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


The following commit(s) were added to refs/heads/branch_8_11 by this push:
     new 396cc488499 SOLR-16194 Backport from main, excluding new method that throws per discussion. (#2658)
396cc488499 is described below

commit 396cc488499e9068518849de0699356182cd48da
Author: Gus Heck <46...@users.noreply.github.com>
AuthorDate: Wed May 18 09:00:31 2022 -0400

    SOLR-16194 Backport from main, excluding new method that throws per discussion. (#2658)
---
 .../solr/cloud/api/collections/CreateAliasCmd.java |  23 ++++-
 .../solr/handler/admin/CollectionsHandler.java     |  14 ++-
 .../java/org/apache/solr/servlet/HttpSolrCall.java |   6 +-
 .../apache/solr/cloud/CreateRoutedAliasTest.java   | 113 ++++++++++++++++++---
 solr/solr-ref-guide/src/collection-aliasing.adoc   |  41 +++++---
 5 files changed, 158 insertions(+), 39 deletions(-)

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 9a9b0bbaf0a..b41eebe7fd3 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
@@ -17,6 +17,7 @@
  */
 package org.apache.solr.cloud.api.collections;
 
+import java.lang.invoke.MethodHandles;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
@@ -28,6 +29,7 @@ import java.util.stream.Collectors;
 
 import com.google.common.collect.Sets;
 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;
@@ -35,11 +37,14 @@ import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
+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;
 
 public class CreateAliasCmd extends AliasCmd {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
 
   private static boolean anyRoutingParams(ZkNodeProps message) {
@@ -132,11 +137,21 @@ public class CreateAliasCmd extends AliasCmd {
     }
 
     // Create the first collection.
-    String initialColl = routedAlias.computeInitialCollectionName();
-      ensureAliasCollection(aliasName, zkStateReader, state, routedAlias.getAliasMetadata(), initialColl);
+    Aliases aliases = zkStateReader.aliasesManager.getAliases();
+
+    List<String> collectionList = aliases.resolveAliases(aliasName);
+    String collectionListStr = String.join(",", collectionList);
+    if (!aliases.isRoutedAlias(aliasName)) {
+      // Create the first collection.
+      collectionListStr = routedAlias.computeInitialCollectionName();
+      ensureAliasCollection(
+          aliasName, zkStateReader, state, routedAlias.getAliasMetadata(), collectionListStr);
+    }
+    // Create/update the alias
+    String finalCollectionListStr = collectionListStr;
       // Create/update the alias
-      zkStateReader.aliasesManager.applyModificationAndExportToZk(aliases -> aliases
-          .cloneWithCollectionAlias(aliasName, initialColl)
+      zkStateReader.aliasesManager.applyModificationAndExportToZk(a -> a
+          .cloneWithCollectionAlias(aliasName, finalCollectionListStr)
           .cloneWithCollectionAliasProperties(aliasName, routedAlias.getAliasMetadata()));
   }
 
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 982b3ed2891..b840bcfb77d 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
@@ -670,6 +670,17 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
           //////////////////////////////////////
           return copy(finalParams.required(), null, NAME, "collections");
         }
+      } else {
+        if (routedAlias != null) {
+          CoreContainer coreContainer1 = h.getCoreContainer();
+          Aliases aliases = coreContainer1.getZkController().getZkStateReader().getAliases();
+          String aliasName = routedAlias.getAliasName();
+          if (aliases.hasAlias(aliasName) && !aliases.isRoutedAlias(aliasName)) {
+            throw new SolrException(
+                BAD_REQUEST,
+                "Cannot add routing parameters to existing non-routed Alias: " + aliasName);
+          }
+        }
       }
 
       /////////////////////////////////////////////////
@@ -824,8 +835,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       final String newShardName = SolrIdentifierValidator.validateShardName(req.getParams().get(SHARD_ID_PROP));
       boolean followAliases = req.getParams().getBool(FOLLOW_ALIASES, false);
       String extCollectionName = req.getParams().get(COLLECTION_PROP);
-      String collectionName = followAliases ? h.coreContainer.getZkController().getZkStateReader()
-          .getAliases().resolveSimpleAlias(extCollectionName) : extCollectionName;
+      String collectionName = followAliases ? h.coreContainer.getZkController().getZkStateReader().getAliases().resolveSimpleAlias(extCollectionName) : extCollectionName;
       if (!ImplicitDocRouter.NAME.equals(((Map) clusterState.getCollection(collectionName).get(DOC_ROUTER)).get(NAME)))
         throw new SolrException(ErrorCode.BAD_REQUEST, "shards can be added only to 'implicit' collections");
       copy(req.getParams(), map,
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index 0ca48edef15..bcc66c8d069 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -214,10 +214,6 @@ public class HttpSolrCall {
     return queryParams;
   }
 
-  protected Aliases getAliases() {
-    return cores.isZooKeeperAware() ? cores.getZkController().getZkStateReader().getAliases() : Aliases.EMPTY;
-  }
-
   /** The collection(s) referenced in this request. Populated in {@link #init()}. Not null. */
   public List<String> getCollectionsList() {
     return collectionsList != null ? collectionsList : Collections.emptyList();
@@ -382,7 +378,7 @@ public class HttpSolrCall {
     }
     List<String> result = null;
     LinkedHashSet<String> uniqueList = null;
-    Aliases aliases = getAliases();
+    Aliases aliases = cores.getZkController().getZkStateReader().getAliases();
     List<String> inputCollections = StrUtils.splitSmart(collectionStr, ",", true);
     if (inputCollections.size() > 1) {
       uniqueList = new LinkedHashSet<>();
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 9833e908913..b35982c361b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
@@ -18,9 +18,11 @@
 package org.apache.solr.cloud;
 
 import java.io.IOException;
+import java.time.Duration;
 import java.time.Instant;
 import java.time.temporal.ChronoUnit;
 import java.util.Date;
+import java.util.List;
 import java.util.Map;
 import java.util.TimeZone;
 
@@ -37,6 +39,7 @@ import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.cloud.api.collections.TimeRoutedAlias;
 import org.apache.solr.common.cloud.Aliases;
 import org.apache.solr.common.cloud.CompositeIdRouter;
@@ -175,20 +178,8 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
   @Test
   public void testV1() throws Exception {
     final String aliasName = getSaferTestName();
-    final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
     Instant start = Instant.now().truncatedTo(ChronoUnit.HOURS); // mostly make sure no millis
-    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=CREATEALIAS" +
-        "&wt=xml" +
-        "&name=" + aliasName +
-        "&router.field=evt_dt" +
-        "&router.name=time" +
-        "&router.start=" + start +
-        "&router.interval=%2B30MINUTE" +
-        "&create-collection.collection.configName=_default" +
-        "&create-collection.router.field=foo_s" +
-        "&create-collection.numShards=1" +
-        "&create-collection.replicationFactor=2");
-    assertSuccess(get);
+    createTRAv1(aliasName, start);
 
     String initialCollectionName = TimeRoutedAlias.formatCollectionNameFromInstant(aliasName, start);
     assertCollectionExists(initialCollectionName);
@@ -213,6 +204,102 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
     assertEquals("_default",meta.get("create-collection.collection.configName"));
     assertEquals(null,meta.get("start"));
   }
+  @Test
+  public void testUpdateRoudetedAliasDoesNotChangeCollectionList() throws Exception {
+
+    final String aliasName = getSaferTestName();
+    Instant start = Instant.now().truncatedTo(ChronoUnit.HOURS); // mostly make sure no millis
+    createTRAv1(aliasName, start);
+
+    String initialCollectionName =
+        TimeRoutedAlias.formatCollectionNameFromInstant(aliasName, start);
+    assertCollectionExists(initialCollectionName);
+
+    // Note that this is convenient for the test because it imples a different collection name, but
+    // doing this is an advanced operation, typically preceded by manual collection creations and
+    // manual tweaking of the collection list. This is here merely to test that we don't blow away the
+    // existing (possibly tweaked) list. DO NOT use this as an example of normal operations.
+    Instant earlierStart = start.minus(Duration.ofMinutes(3));
+    createTRAv1(aliasName, earlierStart);
+    assertCollectionExists(initialCollectionName);
+
+    // Test Alias metadata
+    Aliases aliases = cluster.getSolrClient().getZkStateReader().getAliases();
+    Map<String, String> collectionAliasMap = aliases.getCollectionAliasMap();
+    String alias = collectionAliasMap.get(aliasName);
+    assertNotNull(alias);
+    Map<String, String> meta = aliases.getCollectionAliasProperties(aliasName);
+    assertNotNull(meta);
+    assertEquals("evt_dt", meta.get("router.field"));
+    assertEquals("_default", meta.get("create-collection.collection.configName"));
+
+    // This should be equal to the new start value
+    assertEquals(earlierStart.toString(), meta.get("router.start"));
+    List<String> collectionList = aliases.resolveAliases(aliasName);
+    assertEquals(1, collectionList.size());
+    assertTrue(collectionList.contains(initialCollectionName));
+  }
+
+  public void testCantAddRoutingToNonRouted() throws Exception {
+    String aliasName = getSaferTestName() + "Alias";
+    createCollection();
+    final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+    HttpGet get =
+        new HttpGet(
+            baseUrl
+                + "/admin/collections?action=CREATEALIAS"
+                + "&wt=xml"
+                + "&name="
+                + aliasName
+                + "&collections="
+                + getSaferTestName());
+    assertSuccess(get);
+
+    HttpGet get2 =
+        new HttpGet(
+            baseUrl
+                + "/admin/collections?action=CREATEALIAS"
+                + "&wt=json"
+                + "&name="
+                + aliasName
+                + "&router.field=evt_dt"
+                + "&router.name=time"
+                + "&router.start=2018-01-15T00:00:00Z"
+                + "&router.interval=%2B30MINUTE"
+                + "&create-collection.collection.configName=_default"
+                + "&create-collection.numShards=1");
+    assertFailure(get2, "Cannot add routing parameters to existing non-routed Alias");
+  }
+
+  private void createCollection() throws SolrServerException, IOException {
+    final CollectionAdminResponse response =
+        CollectionAdminRequest.createCollection(getSaferTestName(), "_default", 1, 1)
+            .process(solrClient);
+    if (response.getStatus() != 0) {
+      fail("failed to create collection " + getSaferTestName());
+    }
+  }
+
+  private void createTRAv1(String aliasName, Instant start) throws IOException {
+    final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+    HttpGet get =
+        new HttpGet(
+            baseUrl
+                + "/admin/collections?action=CREATEALIAS"
+                + "&wt=xml"
+                + "&name="
+                + aliasName
+                + "&router.field=evt_dt"
+                + "&router.name=time"
+                + "&router.start="
+                + start
+                + "&router.interval=%2B30MINUTE"
+                + "&create-collection.collection.configName=_default"
+                + "&create-collection.router.field=foo_s"
+                + "&create-collection.numShards=1"
+                + "&create-collection.replicationFactor=2");
+    assertSuccess(get);
+  }
 
   // TZ should not affect the first collection name if absolute date given for start
   @Test
diff --git a/solr/solr-ref-guide/src/collection-aliasing.adoc b/solr/solr-ref-guide/src/collection-aliasing.adoc
index 0fa2e4e2d68..979d3ee1487 100644
--- a/solr/solr-ref-guide/src/collection-aliasing.adoc
+++ b/solr/solr-ref-guide/src/collection-aliasing.adoc
@@ -93,26 +93,35 @@ prohibited. If routing parameters are present this parameter is prohibited.
 
 ==== Routed Alias Parameters
 
-Most routed alias parameters become _alias properties_ that can subsequently be inspected and <<aliasprop,modified>>.
+Most routed alias parameters become _alias properties_ that can subsequently be inspected and modified either by issuing a new CREATEALIAS for the same name or via <<aliasprop,ALIASPROP>>.
+CREATEALIAS will validate against many (but not all) bad values, whereas ALIASPROP blindly accepts any key or value you give it.
+Some "valid" modifications allowed by CREATEALIAS may still be unwise, see notes below. "Expert only" modifications are technically possible, but require good understanding of how the code works and may require several precursor operations.
 
 `router.name`::
 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 <<aliases.adoc#dimensional-routed-aliases,Aliases>>
 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.
+values for each sub-dimension.
+Note that DRA's are experimental, and only 2D DRA's are presently supported.
+Higher numbers of dimensions may be supported in the future.
+Careful design of dimensional routing is required to avoid an explosion in the number of collections in the cluster.
+Solr Cloud may have difficulty managing more than a thousand collections.
+See examples below for further clarification on how to configure
+individual dimensions.
++
+This parameter is required. Modification: Do not change after creation
 
 `router.field`::
 The field to inspect to determine which underlying collection an incoming document should be routed to.
-This field is required on all incoming documents.
++
+This field is required on all incoming documents. Modification: Do not change after creation
 
 `create-collection.*`::
 The `*` wildcard can be replaced with any parameter from the <<collection-management.adoc#create,CREATE>> command except `name`. All other fields
 are identical in requirements and naming except that we insist that the configset be explicitly specified.
 The configset must be created beforehand, either uploaded or copied and modified.
-It's probably a bad idea to use "data driven" mode as schema mutations might happen concurrently leading to errors.
+It's probably a bad idea to use "data driven" mode as schema mutations might happen concurrently leading to errors. Modification: Yes, only new collections affected, use with care
 
 ==== Time Routed Alias Parameters
 
@@ -124,7 +133,9 @@ The first collection created for the alias will be internally named after this v
 If a document is submitted with an earlier value for router.field then the earliest collection the alias points to then
 it will yield an error since it can't be routed. This date/time MUST NOT have a milliseconds component other than 0.
 Particularly, this means `NOW` will fail 999 times out of 1000, though `NOW/SECOND`, `NOW/MINUTE`, etc. will work
-just fine. This parameter is required.
+just fine.
++
+This parameter is required.  Modification: Expert only
 
 `TZ`::
 The timezone to be used when evaluating any date math in router.start or router.interval. This is equivalent to the
@@ -134,20 +145,20 @@ as an alias property.
 If GMT-4 is supplied for this value then a document dated 2018-01-14T21:00:00:01.2345Z would be stored in the
 myAlias_2018-01-15_01 collection (assuming an interval of +1HOUR).
 +
-The default timezone is UTC.
+The default timezone is UTC.  Modification: Expert only
 
 `router.interval`::
 A date math expression that will be appended to a timestamp to determine the next collection in the series.
 Any date math expression that can be evaluated if appended to a timestamp of the form 2018-01-15T16:17:18 will
 work here.
 +
-This parameter is required.
+This parameter is required. Modification : Ok
 
 `router.maxFutureMs`::
 The maximum milliseconds into the future that a document is allowed to have in `router.field` for it to be accepted
 without error.  If there was no limit, than an erroneous value could trigger many collections to be created.
 +
-The default is 10 minutes.
+The default is 10 minutes. Modification : Ok
 
 `router.preemptiveCreateMath`::
 A date math expression that results in early creation of new collections.
@@ -168,7 +179,7 @@ but will not trigger additional async preemptive creation. Only one type of coll
 per document.
 Example: `90MINUTES`.
 +
-This property is blank by default indicating just-in-time, synchronous creation of new collections.
+This property is blank by default indicating just-in-time, synchronous creation of new collections. Modification : Ok
 
 `router.autoDeleteAge`::
 A date math expression that results in the oldest collections getting deleted automatically.
@@ -179,27 +190,27 @@ Collections to be deleted must have a time range that is entirely before the com
 Collections are considered for deletion immediately prior to new collections getting created.
 Example: `/DAY-90DAYS`.
 +
-The default is not to delete.
+The default is not to delete. Modification : Ok, Possible data loss, use with care!
 
 ==== Category Routed Alias Parameters
 
 `router.maxCardinality`::
 The maximum number of categories allowed for this alias.
-This setting safeguards against the inadvertent creation of an infinite number of collections in the event of bad data.
+This setting safeguards against the inadvertent creation of an infinite number of collections in the event of bad data. Modification : Ok
 
 `router.mustMatch`::
 A regular expression that the value of the field specified by `router.field` must match before a corresponding
 collection will be created. Note that changing this setting after data has been added will not alter the data already
 indexed. Any valid Java regular expression pattern may be specified. This expression is pre-compiled at the start of
 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.
+or garbage collecting overhead during indexing as determined by the JVM's implementation of regular expressions. Modification : Ok
 
 ==== 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.
+For example in a Dimensional[time,category] router.0.start would be used to set the start time for the time dimension. Modification: As per above
 
 
 === CREATEALIAS Response