You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by gu...@apache.org on 2019/10/11 00:42:28 UTC

[lucene-solr] branch branch_8x updated: SOLR-13760 - restore viability of date math in TRA start property by fixing the start date for time routed aliases upon the receipt of the first document to avoid problems with date math calculations required by DRA's

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new d5b26be  SOLR-13760 - restore viability of date math in TRA start property by fixing the start date for time routed aliases upon the receipt of the first document to avoid problems with date math calculations required by DRA's
d5b26be is described below

commit d5b26be691f526d140cf5fb937dbca8813ceafd2
Author: Gus Heck <gu...@apache.org>
AuthorDate: Thu Oct 10 18:17:22 2019 -0400

    SOLR-13760 - restore viability of date math in TRA start property by
    fixing the start date for time routed aliases
    upon the receipt of the first document to avoid problems
    with date math calculations required by DRA's
    
    (cherry picked from commit be813bd0aefcf480f854a05c7880494da5e8c8bf)
---
 .../cloud/api/collections/TimeRoutedAlias.java     | 29 ++++++++-
 .../DimensionalRoutedAliasUpdateProcessorTest.java |  3 +-
 .../TimeRoutedAliasUpdateProcessorTest.java        | 70 ++++++++++++++++++++++
 3 files changed, 100 insertions(+), 2 deletions(-)

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 6d3b53c..22024f3 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
@@ -24,6 +24,7 @@ import java.time.ZoneId;
 import java.time.ZoneOffset;
 import java.time.format.DateTimeFormatter;
 import java.time.format.DateTimeFormatterBuilder;
+import java.time.format.DateTimeParseException;
 import java.time.temporal.ChronoField;
 import java.time.temporal.ChronoUnit;
 import java.util.AbstractMap;
@@ -32,6 +33,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.Date;
 import java.util.HashSet;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -49,6 +51,7 @@ 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.SolrCore;
 import org.apache.solr.update.AddUpdateCommand;
 import org.apache.solr.update.processor.RoutedAliasUpdateProcessor;
 import org.apache.solr.util.DateMathParser;
@@ -357,7 +360,31 @@ public class TimeRoutedAlias extends RoutedAlias {
     // 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);
+
+    // SOLR-13760 - we need to fix the date math to a specific instant when the first document arrives.
+    // If we don't do this DRA's with a time dimension have variable start times across the other dimensions
+    // and logic gets much to complicated, and depends too much on queries to zookeeper. This keeps life simpler.
+    // I have to admit I'm not terribly fond of the mutation during a validate method however.
+    Instant startTime;
+    try {
+      startTime = Instant.parse(start);
+    } catch (DateTimeParseException e) {
+      startTime = DateMathParser.parseMath(new Date(), start).toInstant();
+      SolrCore core = cmd.getReq().getCore();
+      ZkStateReader zkStateReader = core.getCoreContainer().getZkController().zkStateReader;
+      Aliases aliases = zkStateReader.getAliases();
+      Map<String, String> props = new HashMap<>(aliases.getCollectionAliasProperties(aliasName));
+      start = DateTimeFormatter.ISO_INSTANT.format(startTime);
+      props.put(ROUTER_START, start);
+
+      // This could race, but it only occurs when the alias is first used and the values produced
+      // should all be identical and who wins won't matter (baring cases of Date Math involving seconds,
+      // which is pretty far fetched). Putting this in a separate thread to ensure that any failed
+      // races don't cause documents to get rejected.
+      core.runAsync(() -> zkStateReader.aliasesManager.applyModificationAndExportToZk(
+          (a) -> aliases.cloneWithCollectionAliasProperties(aliasName, props)));
+
+    }
     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+")");
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
index 738235b..3d583ac 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/DimensionalRoutedAliasUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/DimensionalRoutedAliasUpdateProcessorTest.java
@@ -686,7 +686,8 @@ public class DimensionalRoutedAliasUpdateProcessorTest extends RoutedAliasUpdate
       final Object errors = resp.getResponseHeader().get("errors"); // Tolerant URP
       assertTrue(errors != null && errors.toString().contains(errorMsg));
     } catch (SolrException e) {
-      assertTrue(e.getMessage().contains(errorMsg));
+      String message = e.getMessage();
+      assertTrue("expected message to contain" + errorMsg + " but message was " + message , message.contains(errorMsg));
     }
     numDocsDeletedOrFailed++;
   }
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 e0dc8e0..1cb5542 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
@@ -20,6 +20,8 @@ package org.apache.solr.update.processor;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.time.Instant;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeParseException;
 import java.time.temporal.ChronoUnit;
 import java.util.Arrays;
 import java.util.Collections;
@@ -50,7 +52,12 @@ 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.update.UpdateCommand;
+import org.apache.solr.util.DateMathParser;
 import org.apache.solr.util.LogLevel;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -59,6 +66,7 @@ import org.slf4j.LoggerFactory;
 
 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.TimeRoutedAlias.ROUTER_START;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTIONS_ZKNODE;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROPS_ZKNODE;
 
@@ -706,6 +714,68 @@ public class TimeRoutedAliasUpdateProcessorTest extends RoutedAliasUpdateProcess
       TimeRoutedAlias.parseInstantFromCollectionName(alias, alias + TRA + "2017-10-02"));
   }
 
+  @Test
+  public void testDateMathInStart() throws Exception {
+    ClusterStateProvider clusterStateProvider = solrClient.getClusterStateProvider();
+    Class<? extends ClusterStateProvider> aClass = clusterStateProvider.getClass();
+    System.out.println("CSPROVIDER:" + aClass);
+
+    // This test prevents recurrence of SOLR-13760
+
+    String configName = getSaferTestName();
+    createConfigSet(configName);
+    CountDownLatch aliasUpdate = new CountDownLatch(1);
+    monitorAlias(aliasUpdate);
+
+    // each collection has 4 shards with 3 replicas for 12 possible destinations
+    // 4 of which are leaders, and 8 of which should fail this test.
+    final int numShards = 1 + random().nextInt(4);
+    final int numReplicas = 1 + random().nextInt(3);
+    CollectionAdminRequest.createTimeRoutedAlias(alias, "2019-09-14T03:00:00Z/DAY", "+1DAY", getTimeField(),
+        CollectionAdminRequest.createCollection("_unused_", configName, numShards, numReplicas)
+            .setMaxShardsPerNode(numReplicas))
+        .process(solrClient);
+
+    aliasUpdate.await();
+    if (BaseHttpClusterStateProvider.class.isAssignableFrom(aClass)) {
+      ((BaseHttpClusterStateProvider)clusterStateProvider).resolveAlias(getAlias(), true);
+    }
+    aliasUpdate = new CountDownLatch(1);
+    monitorAlias(aliasUpdate);
+
+    ModifiableSolrParams params = params();
+    String nowDay = DateTimeFormatter.ISO_INSTANT.format(DateMathParser.parseMath(new Date(), "2019-09-14T01:00:00Z").toInstant());
+    assertUpdateResponse(add(alias, Arrays.asList(
+        sdoc("id", "1", "timestamp_dt", nowDay)), // should not cause preemptive creation of 10-28 now
+        params));
+
+    // this process should have lead to the modification of the start time for the alias, converting it into
+    // a parsable date, removing the DateMath
+
+    // what we test next happens in a separate thread, so we have to give it some time to happen
+    aliasUpdate.await();
+    if (BaseHttpClusterStateProvider.class.isAssignableFrom(aClass)) {
+      ((BaseHttpClusterStateProvider)clusterStateProvider).resolveAlias(getAlias(), true);
+    }
+
+    String hopeFullyModified = clusterStateProvider.getAliasProperties(getAlias()).get(ROUTER_START);
+    try {
+      Instant.parse(hopeFullyModified);
+    } catch (DateTimeParseException e) {
+      fail(ROUTER_START + " should not have any date math by this point and parse as an instant. Using "+ aClass +" Found:" + hopeFullyModified);
+    }
+  }
+
+  private void monitorAlias(CountDownLatch aliasUpdate) throws KeeperException, InterruptedException {
+    Stat stat = new Stat();
+    zkClient().getData("/aliases.json", new Watcher() {
+      @Override
+      public void process(WatchedEvent watchedEvent) {
+        aliasUpdate.countDown();
+      }
+    }, stat, true);
+  }
+
   /**
    * Need to ensure that the existing TRA's gracefully handle, old, new and mixtures thereof. TRA's with
    * an autoDeleteAge setting will gracefully convert to the new format over time.