You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2018/03/15 10:37:30 UTC

lucene-solr:branch_7x: SOLR-11670: Use TimeSource's value of NOW consistently when parsing date math. Add a unit test for TimeSource's epochTime.

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x 17cfd87a2 -> e1b0f796a


SOLR-11670: Use TimeSource's value of NOW consistently when parsing date math.
Add a unit test for TimeSource's epochTime.


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

Branch: refs/heads/branch_7x
Commit: e1b0f796af9363b5496cc4ca8f17755f3c10e59b
Parents: 17cfd87
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Mar 13 13:46:11 2018 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Thu Mar 15 11:37:17 2018 +0100

----------------------------------------------------------------------
 .../cloud/autoscaling/ScheduledTrigger.java     | 14 +++---
 .../org/apache/solr/common/util/TimeSource.java |  9 ++++
 .../apache/solr/common/util/TestTimeSource.java | 51 ++++++++++++++++++++
 3 files changed, 68 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e1b0f796/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
index 2bfa7f5..367f07a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
@@ -76,9 +76,11 @@ public class ScheduledTrigger extends TriggerBase {
     preferredOp = (String) properties.get(PREFERRED_OP);
 
     // attempt parsing to validate date math strings
-    Instant startTime = parseStartTime(startTimeStr, timeZoneStr);
-    DateMathParser.parseMath(null, startTime + everyStr, timeZone);
-    DateMathParser.parseMath(null, startTime + graceDurationStr, timeZone);
+    // explicitly set NOW because it may be different for simulated time
+    Date now = new Date(TimeUnit.NANOSECONDS.toMillis(cloudManager.getTimeSource().getEpochTime()));
+    Instant startTime = parseStartTime(now, startTimeStr, timeZoneStr);
+    DateMathParser.parseMath(now, startTime + everyStr, timeZone);
+    DateMathParser.parseMath(now, startTime + graceDurationStr, timeZone);
 
     // We set lastRunAt to be the startTime (which could be a date math expression such as 'NOW')
     // Ordinarily, NOW will always be evaluated in this constructor so it may seem that
@@ -88,13 +90,13 @@ public class ScheduledTrigger extends TriggerBase {
     this.lastRunAt = startTime;
   }
 
-  private Instant parseStartTime(String startTimeStr, String timeZoneStr) {
+  private Instant parseStartTime(Date now, String startTimeStr, String timeZoneStr) {
     if (startTimeStr == null) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Parameter 'startTime' cannot be null");
     }
     try {
       // try parsing startTime as an ISO-8601 date time string
-      return DateMathParser.parseMath(null, startTimeStr).toInstant();
+      return DateMathParser.parseMath(now, startTimeStr).toInstant();
     } catch (SolrException e) {
       if (e.code() != SolrException.ErrorCode.BAD_REQUEST.code)  throw e;
     }
@@ -174,7 +176,7 @@ public class ScheduledTrigger extends TriggerBase {
       }
       // Even though we are skipping the event, we need to notify any listeners of the IGNORED stage
       // so we create a dummy event with the ignored=true flag and ScheduledTriggers will do the rest
-      if (processor != null && processor.process(new ScheduledEvent(getEventType(), getName(), nextRunTime.toEpochMilli(),
+      if (processor != null && processor.process(new ScheduledEvent(getEventType(), getName(), timeSource.getTime(),
           preferredOp, now.toEpochMilli(), true))) {
         lastRunAt = nextRunTime;
         return;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e1b0f796/solr/solrj/src/java/org/apache/solr/common/util/TimeSource.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/TimeSource.java b/solr/solrj/src/java/org/apache/solr/common/util/TimeSource.java
index 35ef069..9690537 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/TimeSource.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/TimeSource.java
@@ -136,6 +136,11 @@ public abstract class TimeSource {
       long nano = Math.round((double)TimeUnit.NANOSECONDS.convert(value, fromUnit) / multiplier);
       return toUnit.convert(nano, TimeUnit.NANOSECONDS);
     }
+
+    @Override
+    public String toString() {
+      return super.toString() + ":" + multiplier;
+    }
   }
 
   /** This instance uses {@link CurrentTimeSource} for generating timestamps. */
@@ -196,4 +201,8 @@ public abstract class TimeSource {
   public abstract void sleep(long ms) throws InterruptedException;
 
   public abstract long convertDelay(TimeUnit fromUnit, long value, TimeUnit toUnit);
+
+  public String toString() {
+    return getClass().getSimpleName();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e1b0f796/solr/solrj/src/test/org/apache/solr/common/util/TestTimeSource.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/common/util/TestTimeSource.java b/solr/solrj/src/test/org/apache/solr/common/util/TestTimeSource.java
new file mode 100644
index 0000000..b37f026
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/common/util/TestTimeSource.java
@@ -0,0 +1,51 @@
+/*
+ * 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.common.util;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.junit.Test;
+
+/**
+ *
+ */
+public class TestTimeSource extends SolrTestCaseJ4 {
+
+  @Test
+  public void testEpochTime() throws Exception {
+    doTestEpochTime(TimeSource.CURRENT_TIME);
+    doTestEpochTime(TimeSource.NANO_TIME);
+    doTestEpochTime(TimeSource.get("simTime:50"));
+  }
+
+  private void doTestEpochTime(TimeSource ts) throws Exception {
+    long prevTime = ts.getTime();
+    long prevEpochTime = ts.getEpochTime();
+    long delta = 500000000; // 500 ms
+    for (int i = 0; i < 10; i++) {
+      ts.sleep(500);
+      long curTime = ts.getTime();
+      long curEpochTime = ts.getEpochTime();
+      long diff = prevTime + delta - curTime;
+      assertTrue(ts + " time diff=" + diff, diff < 100000);
+      diff = prevEpochTime + delta - curEpochTime;
+      assertTrue(ts + " epochTime diff=" + diff, diff < 100000);
+      prevTime = curTime;
+      prevEpochTime = curEpochTime;
+    }
+  }
+}