You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jb...@apache.org on 2017/04/12 19:14:56 UTC

[01/12] lucene-solr:master: SOLR-10303: Initial support for common date/time Stream Evaluators

Repository: lucene-solr
Updated Branches:
  refs/heads/master b767d61b9 -> 0bcd88b18


SOLR-10303: Initial support for common date/time Stream Evaluators


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

Branch: refs/heads/master
Commit: 24ab117a41fcf9d48a85600c8fb721c75be0f982
Parents: b767d61
Author: Gethin James <ge...@alfresco.com>
Authored: Fri Mar 17 14:07:50 2017 +0100
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Apr 12 14:55:02 2017 -0400

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |   8 +-
 .../client/solrj/io/eval/DateEvaluator.java     | 133 +++++++++++++
 .../solrj/io/stream/eval/DateEvaluatorTest.java | 189 +++++++++++++++++++
 3 files changed, 329 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/24ab117a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index 6f373f6..61afea4 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -43,6 +43,7 @@ import org.apache.solr.client.solrj.io.eval.CeilingEvaluator;
 import org.apache.solr.client.solrj.io.eval.CoalesceEvaluator;
 import org.apache.solr.client.solrj.io.eval.CosineEvaluator;
 import org.apache.solr.client.solrj.io.eval.CubedRootEvaluator;
+import org.apache.solr.client.solrj.io.eval.DateEvaluator;
 import org.apache.solr.client.solrj.io.eval.DivideEvaluator;
 import org.apache.solr.client.solrj.io.eval.EqualsEvaluator;
 import org.apache.solr.client.solrj.io.eval.ExclusiveOrEvaluator;
@@ -248,12 +249,17 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("cbrt", CubedRootEvaluator.class)
       .withFunctionName("coalesce", CoalesceEvaluator.class)
       .withFunctionName("uuid", UuidEvaluator.class)
-      
+
       // Conditional Stream Evaluators
       .withFunctionName("if", IfThenElseEvaluator.class)
       .withFunctionName("analyze", AnalyzeEvaluator.class)
       ;
 
+      // Date evaluators
+      for (DateEvaluator.FUNCTION function:DateEvaluator.FUNCTION.values()) {
+        streamFactory.withFunctionName(function.toString(), DateEvaluator.class);
+      }
+
      // This pulls all the overrides and additions from the config
      List<PluginInfo> pluginInfos = core.getSolrConfig().getPluginInfos(Expressible.class.getName());
      for (PluginInfo pluginInfo : pluginInfos) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/24ab117a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DateEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DateEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DateEvaluator.java
new file mode 100644
index 0000000..f59cade
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DateEvaluator.java
@@ -0,0 +1,133 @@
+/*
+ * 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.io.eval;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeParseException;
+import java.time.temporal.IsoFields;
+import java.util.Arrays;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides numeric Date/Time stream evaluators
+ */
+public class DateEvaluator extends NumberEvaluator {
+
+  public enum FUNCTION {year, month, day, dayofyear, dayofquarter, hour, minute, quarter, week, second, epoch};
+
+  private FUNCTION function;
+  private String fieldName;
+
+  public DateEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+
+    String functionName = expression.getFunctionName();
+
+    try {
+      this.function = FUNCTION.valueOf(functionName);
+    } catch (IllegalArgumentException e) {
+      throw new IOException(String.format(Locale.ROOT,"Invalid date expression %s - expecting one of %s",functionName, Arrays.toString(FUNCTION.values())));
+    }
+
+    fieldName = factory.getValueOperand(expression, 0);
+
+    //Taken from Field evaluator
+    if(fieldName != null && fieldName.startsWith("'") && fieldName.endsWith("'") && fieldName.length() > 1){
+      fieldName = fieldName.substring(1, fieldName.length() - 1);
+    }
+
+    if(1 != subEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  //TODO: Support non-string, eg. java.util.date or instant
+
+  @Override
+  public Number evaluate(Tuple tuple) throws IOException {
+
+    try {
+      String dateStr = (String) tuple.get(fieldName);
+      if (dateStr != null && !dateStr.isEmpty()) {
+        Instant instant = Instant.parse(dateStr);
+        if (function.equals(FUNCTION.epoch)) return instant.toEpochMilli();
+
+        LocalDateTime date = LocalDateTime.ofInstant(instant, ZoneOffset.UTC);
+        return evaluate(date);
+      }
+    } catch (ClassCastException | DateTimeParseException e) {
+      throw new IOException(String.format(Locale.ROOT,"Invalid field %s - The field must be a string formatted in the ISO_INSTANT date format.",fieldName));
+    }
+
+    return null;
+  }
+
+  /**
+   * Evaluate the date based on the specified function
+   * @param date
+   * @return the evaluated value
+   */
+  private Number evaluate(LocalDateTime date) {
+    switch (function) {
+      case year:
+        return date.getYear();
+      case month:
+        return date.getMonthValue();
+      case day:
+        return date.getDayOfMonth();
+      case dayofyear:
+        return date.getDayOfYear();
+      case hour:
+        return date.getHour();
+      case minute:
+        return date.getMinute();
+      case second:
+        return date.getSecond();
+      case dayofquarter:
+        return date.get(IsoFields.DAY_OF_QUARTER);
+      case quarter:
+        return date.get(IsoFields.QUARTER_OF_YEAR);
+      case week:
+        return date.get(IsoFields.WEEK_OF_WEEK_BASED_YEAR);
+    }
+    return null;
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    return new StreamExpression(function.toString()).withParameter(fieldName);
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(nodeId.toString())
+        .withExpressionType(Explanation.ExpressionType.EVALUATOR)
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/24ab117a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DateEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DateEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DateEvaluatorTest.java
new file mode 100644
index 0000000..a76bd38
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DateEvaluatorTest.java
@@ -0,0 +1,189 @@
+/*
+ * 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.io.stream.eval;
+
+import java.io.IOException;
+import java.util.Date;
+import java.util.Map;
+
+import org.apache.commons.collections.map.HashedMap;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.eval.DateEvaluator;
+import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParser;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.junit.Test;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertNotNull;
+import static junit.framework.Assert.assertNull;
+import static junit.framework.Assert.assertTrue;
+
+/**
+ * Tests numeric Date/Time stream evaluators
+ */
+public class DateEvaluatorTest {
+
+
+  StreamFactory factory;
+  Map<String, Object> values;
+
+  public DateEvaluatorTest() {
+    super();
+
+    factory = new StreamFactory();
+
+    factory.withFunctionName("nope", DateEvaluator.class);
+    for (DateEvaluator.FUNCTION function : DateEvaluator.FUNCTION.values()) {
+      factory.withFunctionName(function.toString(), DateEvaluator.class);
+    }
+    values = new HashedMap();
+  }
+
+  @Test
+  public void testInvalidExpression() throws Exception {
+
+    StreamEvaluator evaluator;
+
+    try {
+      evaluator = factory.constructEvaluator("nope(a)");
+      evaluator.evaluate(new Tuple(null));
+      assertTrue(false);
+    } catch (Exception e) {
+      assertTrue(e.getCause().getCause().getMessage().contains("Invalid date expression nope"));
+      assertTrue(e.getCause().getCause().getMessage().contains("expecting one of [year, month, day"));
+    }
+
+    try {
+      evaluator = factory.constructEvaluator("week()");
+      assertTrue(false);
+    } catch (Exception e) {
+      assertTrue(e.getCause().getCause().getMessage().contains("Invalid expression week()"));
+    }
+
+    try {
+      evaluator = factory.constructEvaluator("week(a, b)");
+      assertTrue(false);
+    } catch (Exception e) {
+      assertTrue(e.getCause().getCause().getMessage().contains("expecting one value but found 2"));
+    }
+
+    try {
+      evaluator = factory.constructEvaluator("Week()");
+      assertTrue(false);
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains("Invalid evaluator expression Week() - function 'Week' is unknown"));
+    }
+  }
+
+
+  @Test
+  public void testInvalidValues() throws Exception {
+    StreamEvaluator evaluator = factory.constructEvaluator("year(a)");
+
+    try {
+      values.clear();
+      values.put("a", 12);
+      Object result = evaluator.evaluate(new Tuple(values));
+      assertTrue(false);
+    } catch (Exception e) {
+      assertEquals("Invalid field a - The field must be a string formatted in the ISO_INSTANT date format.", e.getMessage());
+    }
+
+    try {
+      values.clear();
+      values.put("a", "1995-12-31");
+      Object result = evaluator.evaluate(new Tuple(values));
+      assertTrue(false);
+    } catch (Exception e) {
+      assertEquals("Invalid field a - The field must be a string formatted in the ISO_INSTANT date format.", e.getMessage());
+    }
+
+    values.clear();
+    values.put("a", null);
+    assertNull(evaluator.evaluate(new Tuple(values)));
+  }
+
+  @Test
+  public void testAllFunctions() throws Exception {
+
+    //year, month, day, dayofyear, hour, minute, quarter, week, second, epoch
+    testFunction("year(a)", "1995-12-31T23:59:59Z", 1995);
+    testFunction("month(a)","1995-12-31T23:59:59Z", 12);
+    testFunction("day(a)",  "1995-12-31T23:59:59Z", 31);
+    testFunction("dayofyear(a)",  "1995-12-31T23:59:59Z", 365);
+    testFunction("dayofquarter(a)",  "1995-12-31T23:59:59Z", 92);
+    testFunction("hour(a)",   "1995-12-31T23:59:59Z", 23);
+    testFunction("minute(a)", "1995-12-31T23:59:59Z", 59);
+    testFunction("quarter(a)","1995-12-31T23:59:59Z", 4);
+    testFunction("week(a)",   "1995-12-31T23:59:59Z", 52);
+    testFunction("second(a)", "1995-12-31T23:59:58Z", 58);
+    testFunction("epoch(a)",  "1995-12-31T23:59:59Z", 820454399000l);
+
+    testFunction("year(a)", "2017-03-17T10:30:45Z", 2017);
+    testFunction("year('a')", "2017-03-17T10:30:45Z", 2017);
+    testFunction("month(a)","2017-03-17T10:30:45Z", 3);
+    testFunction("day(a)",  "2017-03-17T10:30:45Z", 17);
+    testFunction("day('a')",  "2017-03-17T10:30:45Z", 17);
+    testFunction("dayofyear(a)",  "2017-03-17T10:30:45Z", 76);
+    testFunction("dayofquarter(a)",  "2017-03-17T10:30:45Z", 76);
+    testFunction("hour(a)",   "2017-03-17T10:30:45Z", 10);
+    testFunction("minute(a)", "2017-03-17T10:30:45Z", 30);
+    testFunction("quarter(a)","2017-03-17T10:30:45Z", 1);
+    testFunction("week(a)",   "2017-03-17T10:30:45Z", 11);
+    testFunction("second(a)", "2017-03-17T10:30:45Z", 45);
+    testFunction("epoch(a)",  "2017-03-17T10:30:45Z", 1489746645000l);
+
+    testFunction("epoch(a)",  new Date(1489746645500l).toInstant().toString(), 1489746645500l);
+    testFunction("epoch(a)",  new Date(820454399990l).toInstant().toString(), 820454399990l);
+
+    //Additionally test all functions to make sure they return a non-null number
+    for (DateEvaluator.FUNCTION function : DateEvaluator.FUNCTION.values()) {
+      StreamEvaluator evaluator = factory.constructEvaluator(function+"(a)");
+      values.clear();
+      values.put("a", "2017-03-17T10:30:45Z");
+      Object result = evaluator.evaluate(new Tuple(values));
+      assertNotNull(function+" should return a result",result);
+      assertTrue(function+" should return a number", result instanceof Number);
+    }
+  }
+
+  public void testFunction(String expression, String value, Number expected) throws Exception {
+    StreamEvaluator evaluator = factory.constructEvaluator(expression);
+    values.clear();
+    values.put("a", value);
+    Object result = evaluator.evaluate(new Tuple(values));
+    assertTrue(result instanceof Number);
+    assertEquals(expected, result);
+  }
+
+  @Test
+  public void testExplain() throws IOException {
+    StreamExpression express = StreamExpressionParser.parse("month('myfield')");
+    DateEvaluator dateEvaluator = new DateEvaluator(express,factory);
+    Explanation explain = dateEvaluator.toExplanation(factory);
+    assertEquals("month(myfield)", explain.getExpression());
+
+    express = StreamExpressionParser.parse("day(aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbb)");
+    dateEvaluator = new DateEvaluator(express,factory);
+    explain = dateEvaluator.toExplanation(factory);
+    assertEquals("day(aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbb)", explain.getExpression());
+  }
+}


[12/12] lucene-solr:master: SOLR-10303: Fix precommit

Posted by jb...@apache.org.
SOLR-10303: Fix precommit


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

Branch: refs/heads/master
Commit: 0bcd88b181b3f9d927ae5507b700321eb6e68af6
Parents: 5e40364
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Apr 12 14:46:31 2017 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Apr 12 15:04:52 2017 -0400

----------------------------------------------------------------------
 .../solr/client/solrj/io/eval/TemporalEvaluatorEpoch.java   | 1 -
 .../client/solrj/io/stream/eval/TemporalEvaluatorsTest.java | 9 +++++----
 2 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0bcd88b1/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorEpoch.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorEpoch.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorEpoch.java
index bea4f5d..a8554b3 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorEpoch.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorEpoch.java
@@ -20,7 +20,6 @@ package org.apache.solr.client.solrj.io.eval;
 import java.io.IOException;
 import java.time.LocalDateTime;
 import java.time.ZoneOffset;
-import java.time.temporal.ChronoField;
 import java.time.temporal.TemporalAccessor;
 import java.util.Locale;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0bcd88b1/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
index d3b839b..8205cea 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
@@ -26,6 +26,7 @@ import java.time.ZoneOffset;
 import java.util.Calendar;
 import java.util.Date;
 import java.util.GregorianCalendar;
+import java.util.Locale;
 import java.util.Map;
 import java.util.TimeZone;
 
@@ -197,8 +198,8 @@ public class TemporalEvaluatorsTest {
 
   @Test
   public void testFunctionsOnDate() throws Exception {
-    Calendar calendar = new GregorianCalendar(2017,12,5, 23, 59);
-    calendar.setTimeZone(TimeZone.getTimeZone("UTC"));
+    Calendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"), Locale.ROOT);
+    calendar.set(2017, 12, 5, 23, 59);
     Date aDate = calendar.getTime();
     testFunction("year(a)", aDate, calendar.get(Calendar.YEAR));
     testFunction("month(a)", aDate, calendar.get(Calendar.MONTH)+1);
@@ -210,8 +211,8 @@ public class TemporalEvaluatorsTest {
 
   @Test
   public void testFunctionsOnInstant() throws Exception {
-    Calendar calendar = new GregorianCalendar(2017,12,5, 23, 59);
-    calendar.setTimeZone(TimeZone.getTimeZone("UTC"));
+    Calendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"), Locale.ROOT);
+    calendar.set(2017, 12, 5, 23, 59);
     Date aDate = calendar.getTime();
     Instant instant = aDate.toInstant();
     testFunction("year(a)", instant, calendar.get(Calendar.YEAR));


[10/12] lucene-solr:master: SOLR-10303: Removing the unused class DatePartEvaluator from the test

Posted by jb...@apache.org.
SOLR-10303:  Removing the unused class DatePartEvaluator from the test


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

Branch: refs/heads/master
Commit: b78a270c9d3f0040325bbe4eb044384173bff963
Parents: 8642ed9
Author: Gethin James <ge...@alfresco.com>
Authored: Wed Apr 12 18:00:35 2017 +0200
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Apr 12 15:04:52 2017 -0400

----------------------------------------------------------------------
 .../io/stream/eval/TemporalEvaluatorsTest.java  | 26 +++-----------------
 1 file changed, 3 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b78a270c/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
index 6f8c788..d44d8c2 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
@@ -38,7 +38,6 @@ import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorEpoch;
 import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorHour;
 import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorMinute;
 import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorMonth;
-import org.apache.solr.client.solrj.io.eval.DatePartEvaluator;
 import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
 import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorQuarter;
 import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorSecond;
@@ -69,7 +68,6 @@ public class TemporalEvaluatorsTest {
 
     factory = new StreamFactory();
 
-    factory.withFunctionName("nope", DatePartEvaluator.class);
     factory.withFunctionName(TemporalEvaluatorYear.FUNCTION_NAME,  TemporalEvaluatorYear.class);
     factory.withFunctionName(TemporalEvaluatorMonth.FUNCTION_NAME, TemporalEvaluatorMonth.class);
     factory.withFunctionName(TemporalEvaluatorDay.FUNCTION_NAME,   TemporalEvaluatorDay.class);
@@ -91,15 +89,6 @@ public class TemporalEvaluatorsTest {
     StreamEvaluator evaluator;
 
     try {
-      evaluator = factory.constructEvaluator("nope(a)");
-      evaluator.evaluate(new Tuple(null));
-      assertTrue(false);
-    } catch (IOException e) {
-      assertTrue(e.getCause().getCause().getMessage().contains("Invalid date expression nope"));
-      assertTrue(e.getCause().getCause().getMessage().contains("expecting one of [year, month, day"));
-    }
-
-    try {
       evaluator = factory.constructEvaluator("week()");
       assertTrue(false);
     } catch (IOException e) {
@@ -191,15 +180,6 @@ public class TemporalEvaluatorsTest {
     testFunction("epoch(a)",  new Date(1489746645500l).toInstant().toString(), 1489746645500l);
     testFunction("epoch(a)",  new Date(820454399990l).toInstant().toString(), 820454399990l);
 
-    //Additionally test all functions to make sure they return a non-null number
-    for (DatePartEvaluator.FUNCTION function : DatePartEvaluator.FUNCTION.values()) {
-      StreamEvaluator evaluator = factory.constructEvaluator(function+"(a)");
-      values.clear();
-      values.put("a", "2017-03-17T10:30:45Z");
-      Object result = evaluator.evaluate(new Tuple(values));
-      assertNotNull(function+" should return a result",result);
-      assertTrue(function+" should return a number", result instanceof Number);
-    }
   }
 
   @Test
@@ -297,13 +277,13 @@ public class TemporalEvaluatorsTest {
   @Test
   public void testExplain() throws IOException {
     StreamExpression express = StreamExpressionParser.parse("month('myfield')");
-    DatePartEvaluator datePartEvaluator = new DatePartEvaluator(express,factory);
+    TemporalEvaluatorMonth datePartEvaluator = new TemporalEvaluatorMonth(express,factory);
     Explanation explain = datePartEvaluator.toExplanation(factory);
     assertEquals("month(myfield)", explain.getExpression());
 
     express = StreamExpressionParser.parse("day(aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbb)");
-    datePartEvaluator = new DatePartEvaluator(express,factory);
-    explain = datePartEvaluator.toExplanation(factory);
+    TemporalEvaluatorDay dayPartEvaluator = new TemporalEvaluatorDay(express,factory);
+    explain = dayPartEvaluator.toExplanation(factory);
     assertEquals("day(aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbb)", explain.getExpression());
   }
 }


[04/12] lucene-solr:master: SOLR-10303: Switching from the fieldName param to subEvaluators

Posted by jb...@apache.org.
SOLR-10303:  Switching from the fieldName param to subEvaluators


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

Branch: refs/heads/master
Commit: c3d205cdccf1175d9960723d0fe4e0d4391dff11
Parents: b13945b
Author: Gethin James <ge...@alfresco.com>
Authored: Fri Mar 17 18:11:00 2017 +0100
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Apr 12 15:00:06 2017 -0400

----------------------------------------------------------------------
 .../client/solrj/io/eval/DatePartEvaluator.java | 84 ++++++++++----------
 .../io/stream/eval/DatePartEvaluatorTest.java   |  4 +-
 2 files changed, 45 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c3d205cd/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
index 1423364..e18de3e 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
@@ -38,10 +38,9 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
  */
 public class DatePartEvaluator extends NumberEvaluator {
 
-  public enum FUNCTION {year, month, day, dayofyear, dayofquarter, hour, minute, quarter, week, second, epoch};
+  public enum FUNCTION {year, month, day, dayofyear, dayofquarter, hour, minute, quarter, week, second, epoch}
 
   private FUNCTION function;
-  private String fieldName;
 
   public DatePartEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
     super(expression, factory);
@@ -51,66 +50,63 @@ public class DatePartEvaluator extends NumberEvaluator {
     try {
       this.function = FUNCTION.valueOf(functionName);
     } catch (IllegalArgumentException e) {
-      throw new IOException(String.format(Locale.ROOT,"Invalid date expression %s - expecting one of %s",functionName, Arrays.toString(FUNCTION.values())));
+      throw new IOException(String.format(Locale.ROOT, "Invalid date expression %s - expecting one of %s", functionName, Arrays.toString(FUNCTION.values())));
     }
 
-    fieldName = factory.getValueOperand(expression, 0);
-
-    //Taken from Field evaluator
-    if(fieldName != null && fieldName.startsWith("'") && fieldName.endsWith("'") && fieldName.length() > 1){
-      fieldName = fieldName.substring(1, fieldName.length() - 1);
-    }
-
-    if(1 != subEvaluators.size()){
-      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size()));
+    if (1 != subEvaluators.size()) {
+      throw new IOException(String.format(Locale.ROOT, "Invalid expression %s - expecting one value but found %d", expression, subEvaluators.size()));
     }
   }
 
   @Override
   public Number evaluate(Tuple tuple) throws IOException {
 
-    try {
-      Object fieldValue = tuple.get(fieldName);
-      Instant instant = null;
-      LocalDateTime date = null;
-
-      if (fieldValue == null) return null;
-
-      if (fieldValue instanceof String) {
-        instant = getInstant((String)fieldValue);
-      } else if (fieldValue instanceof Instant) {
-        instant = (Instant) fieldValue;
-      } else if (fieldValue instanceof Date) {
-        instant = ((Date) fieldValue).toInstant();
-      } else if (fieldValue instanceof LocalDateTime) {
-        date = ((LocalDateTime) fieldValue);
-      }
+    Instant instant = null;
+    LocalDateTime date = null;
 
-      if (instant != null) {
-        if (function.equals(FUNCTION.epoch)) return instant.toEpochMilli();
-        date = LocalDateTime.ofInstant(instant, ZoneOffset.UTC);
-      }
+    //First evaluate the parameter
+    StreamEvaluator streamEvaluator = subEvaluators.get(0);
+    Object tupleValue = streamEvaluator.evaluate(tuple);
 
-      if (date != null) {
-        return evaluate(date);
-      }
+    if (tupleValue == null) return null;
+
+    if (tupleValue instanceof String) {
+      instant = getInstant((String) tupleValue);
+    } else if (tupleValue instanceof Instant) {
+      instant = (Instant) tupleValue;
+    } else if (tupleValue instanceof Date) {
+      instant = ((Date) tupleValue).toInstant();
+    } else if (tupleValue instanceof LocalDateTime) {
+      date = ((LocalDateTime) tupleValue);
+    }
+
+    if (instant != null) {
+      if (function.equals(FUNCTION.epoch)) return instant.toEpochMilli();
+      date = LocalDateTime.ofInstant(instant, ZoneOffset.UTC);
+    }
 
-    } catch (DateTimeParseException e) {
-      throw new IOException(String.format(Locale.ROOT,"Invalid field %s - The field must be a string formatted in the ISO_INSTANT date format.",fieldName));
+    if (date != null) {
+      return evaluate(date);
     }
 
-    throw new IOException(String.format(Locale.ROOT,"Invalid field %s - The field must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.",fieldName));
+    throw new IOException(String.format(Locale.ROOT, "Invalid parameter %s - The parameter must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", String.valueOf(tupleValue)));
   }
 
-  private Instant getInstant(String dateStr) {
+  private Instant getInstant(String dateStr) throws IOException {
+
     if (dateStr != null && !dateStr.isEmpty()) {
-      return Instant.parse(dateStr);
+      try {
+        return Instant.parse(dateStr);
+      } catch (DateTimeParseException e) {
+        throw new IOException(String.format(Locale.ROOT, "Invalid parameter %s - The String must be formatted in the ISO_INSTANT date format.", dateStr));
+      }
     }
     return null;
   }
 
   /**
    * Evaluate the date based on the specified function
+   *
    * @param date
    * @return the evaluated value
    */
@@ -142,7 +138,13 @@ public class DatePartEvaluator extends NumberEvaluator {
 
   @Override
   public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
-    return new StreamExpression(function.toString()).withParameter(fieldName);
+    StreamExpression expression = new StreamExpression(function.toString());
+
+    for (StreamEvaluator evaluator : subEvaluators) {
+      expression.addParameter(evaluator.toExpression(factory));
+    }
+
+    return expression;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c3d205cd/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
index 307575c..6240bf8 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
@@ -109,7 +109,7 @@ public class DatePartEvaluatorTest {
       Object result = evaluator.evaluate(new Tuple(values));
       assertTrue(false);
     } catch (Exception e) {
-      assertEquals("Invalid field a - The field must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", e.getMessage());
+      assertEquals("Invalid parameter 12 - The parameter must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", e.getMessage());
     }
 
     try {
@@ -118,7 +118,7 @@ public class DatePartEvaluatorTest {
       Object result = evaluator.evaluate(new Tuple(values));
       assertTrue(false);
     } catch (Exception e) {
-      assertEquals("Invalid field a - The field must be a string formatted in the ISO_INSTANT date format.", e.getMessage());
+      assertEquals("Invalid parameter 1995-12-31 - The String must be formatted in the ISO_INSTANT date format.", e.getMessage());
     }
 
     values.clear();


[03/12] lucene-solr:master: SOLR-10303: Supporting more datatypes via a TemporalAccessor

Posted by jb...@apache.org.
SOLR-10303:  Supporting more datatypes via a TemporalAccessor


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

Branch: refs/heads/master
Commit: c6fbb27376d7ac53e149da2e420bb81bdb2513be
Parents: cf14b4b
Author: Gethin James <ge...@alfresco.com>
Authored: Mon Mar 20 17:02:41 2017 +0100
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Apr 12 15:00:06 2017 -0400

----------------------------------------------------------------------
 .../client/solrj/io/eval/DatePartEvaluator.java | 65 +++++++++++---------
 .../io/stream/eval/DatePartEvaluatorTest.java   | 44 ++++++++++---
 2 files changed, 74 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c6fbb273/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
index dc0e986..c3c5a61 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
@@ -22,7 +22,10 @@ import java.time.Instant;
 import java.time.LocalDateTime;
 import java.time.ZoneOffset;
 import java.time.format.DateTimeParseException;
+import java.time.temporal.ChronoField;
 import java.time.temporal.IsoFields;
+import java.time.temporal.TemporalAccessor;
+import java.time.temporal.UnsupportedTemporalTypeException;
 import java.util.Arrays;
 import java.util.Date;
 import java.util.Locale;
@@ -62,7 +65,7 @@ public class DatePartEvaluator extends NumberEvaluator {
   public Number evaluate(Tuple tuple) throws IOException {
 
     Instant instant = null;
-    LocalDateTime date = null;
+    TemporalAccessor date = null;
 
     //First evaluate the parameter
     StreamEvaluator streamEvaluator = subEvaluators.get(0);
@@ -76,8 +79,8 @@ public class DatePartEvaluator extends NumberEvaluator {
       instant = (Instant) tupleValue;
     } else if (tupleValue instanceof Date) {
       instant = ((Date) tupleValue).toInstant();
-    } else if (tupleValue instanceof LocalDateTime) {
-      date = ((LocalDateTime) tupleValue);
+    } else if (tupleValue instanceof TemporalAccessor) {
+      date = ((TemporalAccessor) tupleValue);
     }
 
     if (instant != null) {
@@ -110,32 +113,38 @@ public class DatePartEvaluator extends NumberEvaluator {
    * @param date
    * @return the evaluated value
    */
-  private Number evaluate(LocalDateTime date) throws IOException {
-    switch (function) {
-      case year:
-        return date.getYear();
-      case month:
-        return date.getMonthValue();
-      case day:
-        return date.getDayOfMonth();
-      case dayofyear:
-        return date.getDayOfYear();
-      case hour:
-        return date.getHour();
-      case minute:
-        return date.getMinute();
-      case second:
-        return date.getSecond();
-      case dayofquarter:
-        return date.get(IsoFields.DAY_OF_QUARTER);
-      case quarter:
-        return date.get(IsoFields.QUARTER_OF_YEAR);
-      case week:
-        return date.get(IsoFields.WEEK_OF_WEEK_BASED_YEAR);
-      case epoch:
-        return date.atZone(ZoneOffset.UTC).toInstant().toEpochMilli();
+  private Number evaluate(TemporalAccessor date) throws IOException {
+    try {
+      switch (function) {
+        case year:
+          return date.get(ChronoField.YEAR);
+        case month:
+          return date.get(ChronoField.MONTH_OF_YEAR);
+        case day:
+          return date.get(ChronoField.DAY_OF_MONTH);
+        case dayofyear:
+          return date.get(ChronoField.DAY_OF_YEAR);
+        case hour:
+          return date.get(ChronoField.HOUR_OF_DAY);
+        case minute:
+          return date.get(ChronoField.MINUTE_OF_HOUR);
+        case second:
+          return date.get(ChronoField.SECOND_OF_MINUTE);
+        case dayofquarter:
+          return date.get(IsoFields.DAY_OF_QUARTER);
+        case quarter:
+          return date.get(IsoFields.QUARTER_OF_YEAR);
+        case week:
+          return date.get(IsoFields.WEEK_OF_WEEK_BASED_YEAR);
+        case epoch:
+          if (date instanceof LocalDateTime) {
+            return ((LocalDateTime)date).atZone(ZoneOffset.UTC).toInstant().toEpochMilli();
+          }
+      }
+    } catch (UnsupportedTemporalTypeException utte) {
+      throw new IOException(String.format(Locale.ROOT, "It is not possible to call '%s' function on %s", function, date.getClass().getName()));
     }
-    throw new IOException(String.format(Locale.ROOT, "Unsupported function %s called on LocalDateTime %s", function, date.toString()));
+    throw new IOException(String.format(Locale.ROOT, "Unsupported function '%s' called on %s", function, date.toString()));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c6fbb273/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
index 703b015..2e5c4b0 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
@@ -20,6 +20,8 @@ package org.apache.solr.client.solrj.io.stream.eval;
 import java.io.IOException;
 import java.time.Instant;
 import java.time.LocalDateTime;
+import java.time.MonthDay;
+import java.time.YearMonth;
 import java.time.ZoneOffset;
 import java.util.Calendar;
 import java.util.Date;
@@ -72,7 +74,7 @@ public class DatePartEvaluatorTest {
       evaluator = factory.constructEvaluator("nope(a)");
       evaluator.evaluate(new Tuple(null));
       assertTrue(false);
-    } catch (Exception e) {
+    } catch (IOException e) {
       assertTrue(e.getCause().getCause().getMessage().contains("Invalid date expression nope"));
       assertTrue(e.getCause().getCause().getMessage().contains("expecting one of [year, month, day"));
     }
@@ -80,21 +82,21 @@ public class DatePartEvaluatorTest {
     try {
       evaluator = factory.constructEvaluator("week()");
       assertTrue(false);
-    } catch (Exception e) {
+    } catch (IOException e) {
       assertTrue(e.getCause().getCause().getMessage().contains("Invalid expression week()"));
     }
 
     try {
       evaluator = factory.constructEvaluator("week(a, b)");
       assertTrue(false);
-    } catch (Exception e) {
+    } catch (IOException e) {
       assertTrue(e.getCause().getCause().getMessage().contains("expecting one value but found 2"));
     }
 
     try {
       evaluator = factory.constructEvaluator("Week()");
       assertTrue(false);
-    } catch (Exception e) {
+    } catch (IOException e) {
       assertTrue(e.getMessage().contains("Invalid evaluator expression Week() - function 'Week' is unknown"));
     }
   }
@@ -109,7 +111,7 @@ public class DatePartEvaluatorTest {
       values.put("a", 12);
       Object result = evaluator.evaluate(new Tuple(values));
       assertTrue(false);
-    } catch (Exception e) {
+    } catch (IOException e) {
       assertEquals("Invalid parameter 12 - The parameter must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", e.getMessage());
     }
 
@@ -118,7 +120,7 @@ public class DatePartEvaluatorTest {
       values.put("a", "1995-12-31");
       Object result = evaluator.evaluate(new Tuple(values));
       assertTrue(false);
-    } catch (Exception e) {
+    } catch (IOException e) {
       assertEquals("Invalid parameter 1995-12-31 - The String must be formatted in the ISO_INSTANT date format.", e.getMessage());
     }
 
@@ -127,7 +129,7 @@ public class DatePartEvaluatorTest {
       values.put("a", "");
       Object result = evaluator.evaluate(new Tuple(values));
       assertTrue(false);
-    } catch (Exception e) {
+    } catch (IOException e) {
       assertEquals("Invalid parameter  - The parameter must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", e.getMessage());
     }
 
@@ -220,6 +222,34 @@ public class DatePartEvaluatorTest {
     testFunction("epoch(a)", localDateTime, aDate.getTime());
   }
 
+  @Test
+  public void testLimitedFunctions() throws Exception {
+
+    MonthDay monthDay = MonthDay.of(12,5);
+    testFunction("month(a)", monthDay, 12);
+    testFunction("day(a)", monthDay, 5);
+
+    try {
+      testFunction("year(a)", monthDay, 2017);
+      assertTrue(false);
+    } catch (IOException e) {
+      assertEquals("It is not possible to call 'year' function on java.time.MonthDay", e.getMessage());
+    }
+
+    YearMonth yearMonth = YearMonth.of(2018, 4);
+    testFunction("month(a)", yearMonth, 4);
+    testFunction("year(a)", yearMonth, 2018);
+
+    try {
+      testFunction("day(a)", yearMonth, 5);
+      assertTrue(false);
+    } catch (IOException e) {
+      assertEquals("It is not possible to call 'day' function on java.time.YearMonth", e.getMessage());
+    }
+
+  }
+
+
   public void testFunction(String expression, Object value, Number expected) throws Exception {
     StreamEvaluator evaluator = factory.constructEvaluator(expression);
     values.clear();


[06/12] lucene-solr:master: SOLR-10303: Supporting epoch for LocalDateTime

Posted by jb...@apache.org.
SOLR-10303:  Supporting epoch for LocalDateTime


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

Branch: refs/heads/master
Commit: cf14b4be034975417bbdc1185b5aef392c00ae91
Parents: c3d205c
Author: Gethin James <ge...@alfresco.com>
Authored: Sat Mar 18 10:42:19 2017 +0100
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Apr 12 15:00:06 2017 -0400

----------------------------------------------------------------------
 .../solr/client/solrj/io/eval/DatePartEvaluator.java   |  8 +++++---
 .../solrj/io/stream/eval/DatePartEvaluatorTest.java    | 13 +++++++++++++
 2 files changed, 18 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cf14b4be/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
index e18de3e..dc0e986 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
@@ -40,7 +40,7 @@ public class DatePartEvaluator extends NumberEvaluator {
 
   public enum FUNCTION {year, month, day, dayofyear, dayofquarter, hour, minute, quarter, week, second, epoch}
 
-  private FUNCTION function;
+  private final FUNCTION function;
 
   public DatePartEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
     super(expression, factory);
@@ -110,7 +110,7 @@ public class DatePartEvaluator extends NumberEvaluator {
    * @param date
    * @return the evaluated value
    */
-  private Number evaluate(LocalDateTime date) {
+  private Number evaluate(LocalDateTime date) throws IOException {
     switch (function) {
       case year:
         return date.getYear();
@@ -132,8 +132,10 @@ public class DatePartEvaluator extends NumberEvaluator {
         return date.get(IsoFields.QUARTER_OF_YEAR);
       case week:
         return date.get(IsoFields.WEEK_OF_WEEK_BASED_YEAR);
+      case epoch:
+        return date.atZone(ZoneOffset.UTC).toInstant().toEpochMilli();
     }
-    return null;
+    throw new IOException(String.format(Locale.ROOT, "Unsupported function %s called on LocalDateTime %s", function, date.toString()));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cf14b4be/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
index 6240bf8..703b015 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
@@ -20,6 +20,7 @@ package org.apache.solr.client.solrj.io.stream.eval;
 import java.io.IOException;
 import java.time.Instant;
 import java.time.LocalDateTime;
+import java.time.ZoneOffset;
 import java.util.Calendar;
 import java.util.Date;
 import java.util.GregorianCalendar;
@@ -121,6 +122,15 @@ public class DatePartEvaluatorTest {
       assertEquals("Invalid parameter 1995-12-31 - The String must be formatted in the ISO_INSTANT date format.", e.getMessage());
     }
 
+    try {
+      values.clear();
+      values.put("a", "");
+      Object result = evaluator.evaluate(new Tuple(values));
+      assertTrue(false);
+    } catch (Exception e) {
+      assertEquals("Invalid parameter  - The parameter must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", e.getMessage());
+    }
+
     values.clear();
     values.put("a", null);
     assertNull(evaluator.evaluate(new Tuple(values)));
@@ -199,12 +209,15 @@ public class DatePartEvaluatorTest {
 
   @Test
   public void testFunctionsLocalDateTime() throws Exception {
+
     LocalDateTime localDateTime = LocalDateTime.of(2017,12,5, 23, 59);
+    Date aDate = Date.from(localDateTime.atZone(ZoneOffset.UTC).toInstant());
     testFunction("year(a)", localDateTime, 2017);
     testFunction("month(a)", localDateTime, 12);
     testFunction("day(a)", localDateTime, 5);
     testFunction("hour(a)", localDateTime, 23);
     testFunction("minute(a)", localDateTime, 59);
+    testFunction("epoch(a)", localDateTime, aDate.getTime());
   }
 
   public void testFunction(String expression, Object value, Number expected) throws Exception {


[05/12] lucene-solr:master: SOLR-10303: Renamed to DatePartEvaluator and adding support for Instant, Date, LocalDateTime

Posted by jb...@apache.org.
SOLR-10303: Renamed to DatePartEvaluator and adding support for Instant, Date, LocalDateTime


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

Branch: refs/heads/master
Commit: b13945b1ef82235bb28257c566fe9a95ba801da6
Parents: 24ab117
Author: Gethin James <ge...@alfresco.com>
Authored: Fri Mar 17 15:30:12 2017 +0100
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Apr 12 15:00:06 2017 -0400

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |   6 +-
 .../client/solrj/io/eval/DateEvaluator.java     | 133 -----------
 .../client/solrj/io/eval/DatePartEvaluator.java | 156 +++++++++++++
 .../solrj/io/stream/eval/DateEvaluatorTest.java | 189 ---------------
 .../io/stream/eval/DatePartEvaluatorTest.java   | 231 +++++++++++++++++++
 5 files changed, 390 insertions(+), 325 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b13945b1/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index 61afea4..1f087b6 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -43,7 +43,7 @@ import org.apache.solr.client.solrj.io.eval.CeilingEvaluator;
 import org.apache.solr.client.solrj.io.eval.CoalesceEvaluator;
 import org.apache.solr.client.solrj.io.eval.CosineEvaluator;
 import org.apache.solr.client.solrj.io.eval.CubedRootEvaluator;
-import org.apache.solr.client.solrj.io.eval.DateEvaluator;
+import org.apache.solr.client.solrj.io.eval.DatePartEvaluator;
 import org.apache.solr.client.solrj.io.eval.DivideEvaluator;
 import org.apache.solr.client.solrj.io.eval.EqualsEvaluator;
 import org.apache.solr.client.solrj.io.eval.ExclusiveOrEvaluator;
@@ -256,8 +256,8 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       ;
 
       // Date evaluators
-      for (DateEvaluator.FUNCTION function:DateEvaluator.FUNCTION.values()) {
-        streamFactory.withFunctionName(function.toString(), DateEvaluator.class);
+      for (DatePartEvaluator.FUNCTION function: DatePartEvaluator.FUNCTION.values()) {
+        streamFactory.withFunctionName(function.toString(), DatePartEvaluator.class);
       }
 
      // This pulls all the overrides and additions from the config

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b13945b1/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DateEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DateEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DateEvaluator.java
deleted file mode 100644
index f59cade..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DateEvaluator.java
+++ /dev/null
@@ -1,133 +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.client.solrj.io.eval;
-
-import java.io.IOException;
-import java.time.Instant;
-import java.time.LocalDateTime;
-import java.time.ZoneOffset;
-import java.time.format.DateTimeParseException;
-import java.time.temporal.IsoFields;
-import java.util.Arrays;
-import java.util.Locale;
-
-import org.apache.solr.client.solrj.io.Tuple;
-import org.apache.solr.client.solrj.io.stream.expr.Explanation;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
-import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
-
-/**
- * Provides numeric Date/Time stream evaluators
- */
-public class DateEvaluator extends NumberEvaluator {
-
-  public enum FUNCTION {year, month, day, dayofyear, dayofquarter, hour, minute, quarter, week, second, epoch};
-
-  private FUNCTION function;
-  private String fieldName;
-
-  public DateEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
-    super(expression, factory);
-
-    String functionName = expression.getFunctionName();
-
-    try {
-      this.function = FUNCTION.valueOf(functionName);
-    } catch (IllegalArgumentException e) {
-      throw new IOException(String.format(Locale.ROOT,"Invalid date expression %s - expecting one of %s",functionName, Arrays.toString(FUNCTION.values())));
-    }
-
-    fieldName = factory.getValueOperand(expression, 0);
-
-    //Taken from Field evaluator
-    if(fieldName != null && fieldName.startsWith("'") && fieldName.endsWith("'") && fieldName.length() > 1){
-      fieldName = fieldName.substring(1, fieldName.length() - 1);
-    }
-
-    if(1 != subEvaluators.size()){
-      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size()));
-    }
-  }
-
-  //TODO: Support non-string, eg. java.util.date or instant
-
-  @Override
-  public Number evaluate(Tuple tuple) throws IOException {
-
-    try {
-      String dateStr = (String) tuple.get(fieldName);
-      if (dateStr != null && !dateStr.isEmpty()) {
-        Instant instant = Instant.parse(dateStr);
-        if (function.equals(FUNCTION.epoch)) return instant.toEpochMilli();
-
-        LocalDateTime date = LocalDateTime.ofInstant(instant, ZoneOffset.UTC);
-        return evaluate(date);
-      }
-    } catch (ClassCastException | DateTimeParseException e) {
-      throw new IOException(String.format(Locale.ROOT,"Invalid field %s - The field must be a string formatted in the ISO_INSTANT date format.",fieldName));
-    }
-
-    return null;
-  }
-
-  /**
-   * Evaluate the date based on the specified function
-   * @param date
-   * @return the evaluated value
-   */
-  private Number evaluate(LocalDateTime date) {
-    switch (function) {
-      case year:
-        return date.getYear();
-      case month:
-        return date.getMonthValue();
-      case day:
-        return date.getDayOfMonth();
-      case dayofyear:
-        return date.getDayOfYear();
-      case hour:
-        return date.getHour();
-      case minute:
-        return date.getMinute();
-      case second:
-        return date.getSecond();
-      case dayofquarter:
-        return date.get(IsoFields.DAY_OF_QUARTER);
-      case quarter:
-        return date.get(IsoFields.QUARTER_OF_YEAR);
-      case week:
-        return date.get(IsoFields.WEEK_OF_WEEK_BASED_YEAR);
-    }
-    return null;
-  }
-
-  @Override
-  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
-    return new StreamExpression(function.toString()).withParameter(fieldName);
-  }
-
-  @Override
-  public Explanation toExplanation(StreamFactory factory) throws IOException {
-    return new Explanation(nodeId.toString())
-        .withExpressionType(Explanation.ExpressionType.EVALUATOR)
-        .withImplementingClass(getClass().getName())
-        .withExpression(toExpression(factory).toString());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b13945b1/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
new file mode 100644
index 0000000..1423364
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
@@ -0,0 +1,156 @@
+/*
+ * 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.io.eval;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeParseException;
+import java.time.temporal.IsoFields;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides numeric Date/Time stream evaluators
+ */
+public class DatePartEvaluator extends NumberEvaluator {
+
+  public enum FUNCTION {year, month, day, dayofyear, dayofquarter, hour, minute, quarter, week, second, epoch};
+
+  private FUNCTION function;
+  private String fieldName;
+
+  public DatePartEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+
+    String functionName = expression.getFunctionName();
+
+    try {
+      this.function = FUNCTION.valueOf(functionName);
+    } catch (IllegalArgumentException e) {
+      throw new IOException(String.format(Locale.ROOT,"Invalid date expression %s - expecting one of %s",functionName, Arrays.toString(FUNCTION.values())));
+    }
+
+    fieldName = factory.getValueOperand(expression, 0);
+
+    //Taken from Field evaluator
+    if(fieldName != null && fieldName.startsWith("'") && fieldName.endsWith("'") && fieldName.length() > 1){
+      fieldName = fieldName.substring(1, fieldName.length() - 1);
+    }
+
+    if(1 != subEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Number evaluate(Tuple tuple) throws IOException {
+
+    try {
+      Object fieldValue = tuple.get(fieldName);
+      Instant instant = null;
+      LocalDateTime date = null;
+
+      if (fieldValue == null) return null;
+
+      if (fieldValue instanceof String) {
+        instant = getInstant((String)fieldValue);
+      } else if (fieldValue instanceof Instant) {
+        instant = (Instant) fieldValue;
+      } else if (fieldValue instanceof Date) {
+        instant = ((Date) fieldValue).toInstant();
+      } else if (fieldValue instanceof LocalDateTime) {
+        date = ((LocalDateTime) fieldValue);
+      }
+
+      if (instant != null) {
+        if (function.equals(FUNCTION.epoch)) return instant.toEpochMilli();
+        date = LocalDateTime.ofInstant(instant, ZoneOffset.UTC);
+      }
+
+      if (date != null) {
+        return evaluate(date);
+      }
+
+    } catch (DateTimeParseException e) {
+      throw new IOException(String.format(Locale.ROOT,"Invalid field %s - The field must be a string formatted in the ISO_INSTANT date format.",fieldName));
+    }
+
+    throw new IOException(String.format(Locale.ROOT,"Invalid field %s - The field must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.",fieldName));
+  }
+
+  private Instant getInstant(String dateStr) {
+    if (dateStr != null && !dateStr.isEmpty()) {
+      return Instant.parse(dateStr);
+    }
+    return null;
+  }
+
+  /**
+   * Evaluate the date based on the specified function
+   * @param date
+   * @return the evaluated value
+   */
+  private Number evaluate(LocalDateTime date) {
+    switch (function) {
+      case year:
+        return date.getYear();
+      case month:
+        return date.getMonthValue();
+      case day:
+        return date.getDayOfMonth();
+      case dayofyear:
+        return date.getDayOfYear();
+      case hour:
+        return date.getHour();
+      case minute:
+        return date.getMinute();
+      case second:
+        return date.getSecond();
+      case dayofquarter:
+        return date.get(IsoFields.DAY_OF_QUARTER);
+      case quarter:
+        return date.get(IsoFields.QUARTER_OF_YEAR);
+      case week:
+        return date.get(IsoFields.WEEK_OF_WEEK_BASED_YEAR);
+    }
+    return null;
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    return new StreamExpression(function.toString()).withParameter(fieldName);
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(nodeId.toString())
+        .withExpressionType(Explanation.ExpressionType.EVALUATOR)
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b13945b1/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DateEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DateEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DateEvaluatorTest.java
deleted file mode 100644
index a76bd38..0000000
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DateEvaluatorTest.java
+++ /dev/null
@@ -1,189 +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.client.solrj.io.stream.eval;
-
-import java.io.IOException;
-import java.util.Date;
-import java.util.Map;
-
-import org.apache.commons.collections.map.HashedMap;
-import org.apache.solr.client.solrj.io.Tuple;
-import org.apache.solr.client.solrj.io.eval.DateEvaluator;
-import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
-import org.apache.solr.client.solrj.io.stream.expr.Explanation;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParser;
-import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
-import org.junit.Test;
-
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertNotNull;
-import static junit.framework.Assert.assertNull;
-import static junit.framework.Assert.assertTrue;
-
-/**
- * Tests numeric Date/Time stream evaluators
- */
-public class DateEvaluatorTest {
-
-
-  StreamFactory factory;
-  Map<String, Object> values;
-
-  public DateEvaluatorTest() {
-    super();
-
-    factory = new StreamFactory();
-
-    factory.withFunctionName("nope", DateEvaluator.class);
-    for (DateEvaluator.FUNCTION function : DateEvaluator.FUNCTION.values()) {
-      factory.withFunctionName(function.toString(), DateEvaluator.class);
-    }
-    values = new HashedMap();
-  }
-
-  @Test
-  public void testInvalidExpression() throws Exception {
-
-    StreamEvaluator evaluator;
-
-    try {
-      evaluator = factory.constructEvaluator("nope(a)");
-      evaluator.evaluate(new Tuple(null));
-      assertTrue(false);
-    } catch (Exception e) {
-      assertTrue(e.getCause().getCause().getMessage().contains("Invalid date expression nope"));
-      assertTrue(e.getCause().getCause().getMessage().contains("expecting one of [year, month, day"));
-    }
-
-    try {
-      evaluator = factory.constructEvaluator("week()");
-      assertTrue(false);
-    } catch (Exception e) {
-      assertTrue(e.getCause().getCause().getMessage().contains("Invalid expression week()"));
-    }
-
-    try {
-      evaluator = factory.constructEvaluator("week(a, b)");
-      assertTrue(false);
-    } catch (Exception e) {
-      assertTrue(e.getCause().getCause().getMessage().contains("expecting one value but found 2"));
-    }
-
-    try {
-      evaluator = factory.constructEvaluator("Week()");
-      assertTrue(false);
-    } catch (Exception e) {
-      assertTrue(e.getMessage().contains("Invalid evaluator expression Week() - function 'Week' is unknown"));
-    }
-  }
-
-
-  @Test
-  public void testInvalidValues() throws Exception {
-    StreamEvaluator evaluator = factory.constructEvaluator("year(a)");
-
-    try {
-      values.clear();
-      values.put("a", 12);
-      Object result = evaluator.evaluate(new Tuple(values));
-      assertTrue(false);
-    } catch (Exception e) {
-      assertEquals("Invalid field a - The field must be a string formatted in the ISO_INSTANT date format.", e.getMessage());
-    }
-
-    try {
-      values.clear();
-      values.put("a", "1995-12-31");
-      Object result = evaluator.evaluate(new Tuple(values));
-      assertTrue(false);
-    } catch (Exception e) {
-      assertEquals("Invalid field a - The field must be a string formatted in the ISO_INSTANT date format.", e.getMessage());
-    }
-
-    values.clear();
-    values.put("a", null);
-    assertNull(evaluator.evaluate(new Tuple(values)));
-  }
-
-  @Test
-  public void testAllFunctions() throws Exception {
-
-    //year, month, day, dayofyear, hour, minute, quarter, week, second, epoch
-    testFunction("year(a)", "1995-12-31T23:59:59Z", 1995);
-    testFunction("month(a)","1995-12-31T23:59:59Z", 12);
-    testFunction("day(a)",  "1995-12-31T23:59:59Z", 31);
-    testFunction("dayofyear(a)",  "1995-12-31T23:59:59Z", 365);
-    testFunction("dayofquarter(a)",  "1995-12-31T23:59:59Z", 92);
-    testFunction("hour(a)",   "1995-12-31T23:59:59Z", 23);
-    testFunction("minute(a)", "1995-12-31T23:59:59Z", 59);
-    testFunction("quarter(a)","1995-12-31T23:59:59Z", 4);
-    testFunction("week(a)",   "1995-12-31T23:59:59Z", 52);
-    testFunction("second(a)", "1995-12-31T23:59:58Z", 58);
-    testFunction("epoch(a)",  "1995-12-31T23:59:59Z", 820454399000l);
-
-    testFunction("year(a)", "2017-03-17T10:30:45Z", 2017);
-    testFunction("year('a')", "2017-03-17T10:30:45Z", 2017);
-    testFunction("month(a)","2017-03-17T10:30:45Z", 3);
-    testFunction("day(a)",  "2017-03-17T10:30:45Z", 17);
-    testFunction("day('a')",  "2017-03-17T10:30:45Z", 17);
-    testFunction("dayofyear(a)",  "2017-03-17T10:30:45Z", 76);
-    testFunction("dayofquarter(a)",  "2017-03-17T10:30:45Z", 76);
-    testFunction("hour(a)",   "2017-03-17T10:30:45Z", 10);
-    testFunction("minute(a)", "2017-03-17T10:30:45Z", 30);
-    testFunction("quarter(a)","2017-03-17T10:30:45Z", 1);
-    testFunction("week(a)",   "2017-03-17T10:30:45Z", 11);
-    testFunction("second(a)", "2017-03-17T10:30:45Z", 45);
-    testFunction("epoch(a)",  "2017-03-17T10:30:45Z", 1489746645000l);
-
-    testFunction("epoch(a)",  new Date(1489746645500l).toInstant().toString(), 1489746645500l);
-    testFunction("epoch(a)",  new Date(820454399990l).toInstant().toString(), 820454399990l);
-
-    //Additionally test all functions to make sure they return a non-null number
-    for (DateEvaluator.FUNCTION function : DateEvaluator.FUNCTION.values()) {
-      StreamEvaluator evaluator = factory.constructEvaluator(function+"(a)");
-      values.clear();
-      values.put("a", "2017-03-17T10:30:45Z");
-      Object result = evaluator.evaluate(new Tuple(values));
-      assertNotNull(function+" should return a result",result);
-      assertTrue(function+" should return a number", result instanceof Number);
-    }
-  }
-
-  public void testFunction(String expression, String value, Number expected) throws Exception {
-    StreamEvaluator evaluator = factory.constructEvaluator(expression);
-    values.clear();
-    values.put("a", value);
-    Object result = evaluator.evaluate(new Tuple(values));
-    assertTrue(result instanceof Number);
-    assertEquals(expected, result);
-  }
-
-  @Test
-  public void testExplain() throws IOException {
-    StreamExpression express = StreamExpressionParser.parse("month('myfield')");
-    DateEvaluator dateEvaluator = new DateEvaluator(express,factory);
-    Explanation explain = dateEvaluator.toExplanation(factory);
-    assertEquals("month(myfield)", explain.getExpression());
-
-    express = StreamExpressionParser.parse("day(aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbb)");
-    dateEvaluator = new DateEvaluator(express,factory);
-    explain = dateEvaluator.toExplanation(factory);
-    assertEquals("day(aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbb)", explain.getExpression());
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b13945b1/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
new file mode 100644
index 0000000..307575c
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
@@ -0,0 +1,231 @@
+/*
+ * 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.io.stream.eval;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.Map;
+import java.util.TimeZone;
+
+import org.apache.commons.collections.map.HashedMap;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.eval.DatePartEvaluator;
+import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParser;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.junit.Test;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertNotNull;
+import static junit.framework.Assert.assertNull;
+import static junit.framework.Assert.assertTrue;
+
+/**
+ * Tests numeric Date/Time stream evaluators
+ */
+public class DatePartEvaluatorTest {
+
+
+  StreamFactory factory;
+  Map<String, Object> values;
+
+  public DatePartEvaluatorTest() {
+    super();
+
+    factory = new StreamFactory();
+
+    factory.withFunctionName("nope", DatePartEvaluator.class);
+    for (DatePartEvaluator.FUNCTION function : DatePartEvaluator.FUNCTION.values()) {
+      factory.withFunctionName(function.toString(), DatePartEvaluator.class);
+    }
+    values = new HashedMap();
+  }
+
+  @Test
+  public void testInvalidExpression() throws Exception {
+
+    StreamEvaluator evaluator;
+
+    try {
+      evaluator = factory.constructEvaluator("nope(a)");
+      evaluator.evaluate(new Tuple(null));
+      assertTrue(false);
+    } catch (Exception e) {
+      assertTrue(e.getCause().getCause().getMessage().contains("Invalid date expression nope"));
+      assertTrue(e.getCause().getCause().getMessage().contains("expecting one of [year, month, day"));
+    }
+
+    try {
+      evaluator = factory.constructEvaluator("week()");
+      assertTrue(false);
+    } catch (Exception e) {
+      assertTrue(e.getCause().getCause().getMessage().contains("Invalid expression week()"));
+    }
+
+    try {
+      evaluator = factory.constructEvaluator("week(a, b)");
+      assertTrue(false);
+    } catch (Exception e) {
+      assertTrue(e.getCause().getCause().getMessage().contains("expecting one value but found 2"));
+    }
+
+    try {
+      evaluator = factory.constructEvaluator("Week()");
+      assertTrue(false);
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains("Invalid evaluator expression Week() - function 'Week' is unknown"));
+    }
+  }
+
+
+  @Test
+  public void testInvalidValues() throws Exception {
+    StreamEvaluator evaluator = factory.constructEvaluator("year(a)");
+
+    try {
+      values.clear();
+      values.put("a", 12);
+      Object result = evaluator.evaluate(new Tuple(values));
+      assertTrue(false);
+    } catch (Exception e) {
+      assertEquals("Invalid field a - The field must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", e.getMessage());
+    }
+
+    try {
+      values.clear();
+      values.put("a", "1995-12-31");
+      Object result = evaluator.evaluate(new Tuple(values));
+      assertTrue(false);
+    } catch (Exception e) {
+      assertEquals("Invalid field a - The field must be a string formatted in the ISO_INSTANT date format.", e.getMessage());
+    }
+
+    values.clear();
+    values.put("a", null);
+    assertNull(evaluator.evaluate(new Tuple(values)));
+  }
+
+  @Test
+  public void testAllFunctions() throws Exception {
+
+    //year, month, day, dayofyear, hour, minute, quarter, week, second, epoch
+    testFunction("year(a)", "1995-12-31T23:59:59Z", 1995);
+    testFunction("month(a)","1995-12-31T23:59:59Z", 12);
+    testFunction("day(a)",  "1995-12-31T23:59:59Z", 31);
+    testFunction("dayofyear(a)",  "1995-12-31T23:59:59Z", 365);
+    testFunction("dayofquarter(a)",  "1995-12-31T23:59:59Z", 92);
+    testFunction("hour(a)",   "1995-12-31T23:59:59Z", 23);
+    testFunction("minute(a)", "1995-12-31T23:59:59Z", 59);
+    testFunction("quarter(a)","1995-12-31T23:59:59Z", 4);
+    testFunction("week(a)",   "1995-12-31T23:59:59Z", 52);
+    testFunction("second(a)", "1995-12-31T23:59:58Z", 58);
+    testFunction("epoch(a)",  "1995-12-31T23:59:59Z", 820454399000l);
+
+    testFunction("year(a)", "2017-03-17T10:30:45Z", 2017);
+    testFunction("year('a')", "2017-03-17T10:30:45Z", 2017);
+    testFunction("month(a)","2017-03-17T10:30:45Z", 3);
+    testFunction("day(a)",  "2017-03-17T10:30:45Z", 17);
+    testFunction("day('a')",  "2017-03-17T10:30:45Z", 17);
+    testFunction("dayofyear(a)",  "2017-03-17T10:30:45Z", 76);
+    testFunction("dayofquarter(a)",  "2017-03-17T10:30:45Z", 76);
+    testFunction("hour(a)",   "2017-03-17T10:30:45Z", 10);
+    testFunction("minute(a)", "2017-03-17T10:30:45Z", 30);
+    testFunction("quarter(a)","2017-03-17T10:30:45Z", 1);
+    testFunction("week(a)",   "2017-03-17T10:30:45Z", 11);
+    testFunction("second(a)", "2017-03-17T10:30:45Z", 45);
+    testFunction("epoch(a)",  "2017-03-17T10:30:45Z", 1489746645000l);
+
+    testFunction("epoch(a)",  new Date(1489746645500l).toInstant().toString(), 1489746645500l);
+    testFunction("epoch(a)",  new Date(820454399990l).toInstant().toString(), 820454399990l);
+
+    //Additionally test all functions to make sure they return a non-null number
+    for (DatePartEvaluator.FUNCTION function : DatePartEvaluator.FUNCTION.values()) {
+      StreamEvaluator evaluator = factory.constructEvaluator(function+"(a)");
+      values.clear();
+      values.put("a", "2017-03-17T10:30:45Z");
+      Object result = evaluator.evaluate(new Tuple(values));
+      assertNotNull(function+" should return a result",result);
+      assertTrue(function+" should return a number", result instanceof Number);
+    }
+  }
+
+  @Test
+  public void testFunctionsOnDate() throws Exception {
+    Calendar calendar = new GregorianCalendar(2017,12,5, 23, 59);
+    calendar.setTimeZone(TimeZone.getTimeZone("UTC"));
+    Date aDate = calendar.getTime();
+    testFunction("year(a)", aDate, calendar.get(Calendar.YEAR));
+    testFunction("month(a)", aDate, calendar.get(Calendar.MONTH)+1);
+    testFunction("day(a)", aDate, calendar.get(Calendar.DAY_OF_MONTH));
+    testFunction("hour(a)", aDate, calendar.get(Calendar.HOUR_OF_DAY));
+    testFunction("minute(a)", aDate, calendar.get(Calendar.MINUTE));
+    testFunction("epoch(a)", aDate, aDate.getTime());
+  }
+
+  @Test
+  public void testFunctionsOnInstant() throws Exception {
+    Calendar calendar = new GregorianCalendar(2017,12,5, 23, 59);
+    calendar.setTimeZone(TimeZone.getTimeZone("UTC"));
+    Date aDate = calendar.getTime();
+    Instant instant = aDate.toInstant();
+    testFunction("year(a)", instant, calendar.get(Calendar.YEAR));
+    testFunction("month(a)", instant, calendar.get(Calendar.MONTH)+1);
+    testFunction("day(a)", instant, calendar.get(Calendar.DAY_OF_MONTH));
+    testFunction("hour(a)", instant, calendar.get(Calendar.HOUR_OF_DAY));
+    testFunction("minute(a)", instant, calendar.get(Calendar.MINUTE));
+    testFunction("epoch(a)", instant, aDate.getTime());
+  }
+
+  @Test
+  public void testFunctionsLocalDateTime() throws Exception {
+    LocalDateTime localDateTime = LocalDateTime.of(2017,12,5, 23, 59);
+    testFunction("year(a)", localDateTime, 2017);
+    testFunction("month(a)", localDateTime, 12);
+    testFunction("day(a)", localDateTime, 5);
+    testFunction("hour(a)", localDateTime, 23);
+    testFunction("minute(a)", localDateTime, 59);
+  }
+
+  public void testFunction(String expression, Object value, Number expected) throws Exception {
+    StreamEvaluator evaluator = factory.constructEvaluator(expression);
+    values.clear();
+    values.put("a", value);
+    Object result = evaluator.evaluate(new Tuple(values));
+    assertTrue(result instanceof Number);
+    assertEquals(expected, result);
+  }
+
+  @Test
+  public void testExplain() throws IOException {
+    StreamExpression express = StreamExpressionParser.parse("month('myfield')");
+    DatePartEvaluator datePartEvaluator = new DatePartEvaluator(express,factory);
+    Explanation explain = datePartEvaluator.toExplanation(factory);
+    assertEquals("month(myfield)", explain.getExpression());
+
+    express = StreamExpressionParser.parse("day(aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbb)");
+    datePartEvaluator = new DatePartEvaluator(express,factory);
+    explain = datePartEvaluator.toExplanation(factory);
+    assertEquals("day(aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbb)", explain.getExpression());
+  }
+}


[08/12] lucene-solr:master: SOLR-10303: Error message formatting for TemporalEvaluator

Posted by jb...@apache.org.
SOLR-10303:  Error message formatting for TemporalEvaluator


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

Branch: refs/heads/master
Commit: 8642ed9f88e2c8667ff80835b2b3b632786884ab
Parents: 1c333c7
Author: Gethin James <ge...@alfresco.com>
Authored: Thu Apr 6 17:19:31 2017 +0200
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Apr 12 15:04:51 2017 -0400

----------------------------------------------------------------------
 .../org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java  | 2 +-
 .../solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8642ed9f/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java
index 884653f..1694236 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java
@@ -83,7 +83,7 @@ public abstract class TemporalEvaluator extends ComplexEvaluator {
       }
     }
 
-    throw new IOException(String.format(Locale.ROOT, "Invalid parameter %s - The parameter must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", String.valueOf(tupleValue)));
+    throw new IOException(String.format(Locale.ROOT, "Invalid parameter %s - The parameter must be a string formatted ISO_INSTANT or of type Long,Instant,Date,LocalDateTime or TemporalAccessor.", String.valueOf(tupleValue)));
   }
 
   public abstract Object evaluateDate(TemporalAccessor aDate) throws IOException;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8642ed9f/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
index a6c590b..6f8c788 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
@@ -132,7 +132,7 @@ public class TemporalEvaluatorsTest {
       Object result = evaluator.evaluate(new Tuple(values));
       assertTrue(false);
     } catch (IOException e) {
-      assertEquals("Invalid parameter 12 - The parameter must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", e.getMessage());
+      assertEquals("Invalid parameter 12 - The parameter must be a string formatted ISO_INSTANT or of type Long,Instant,Date,LocalDateTime or TemporalAccessor.", e.getMessage());
     }
 
     try {
@@ -150,7 +150,7 @@ public class TemporalEvaluatorsTest {
       Object result = evaluator.evaluate(new Tuple(values));
       assertTrue(false);
     } catch (IOException e) {
-      assertEquals("Invalid parameter  - The parameter must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", e.getMessage());
+      assertEquals("Invalid parameter  - The parameter must be a string formatted ISO_INSTANT or of type Long,Instant,Date,LocalDateTime or TemporalAccessor.", e.getMessage());
     }
 
     values.clear();


[11/12] lucene-solr:master: SOLR-10303: Add the tuple context to avoid creating multiple LocalDateTime instances for the same Tuple

Posted by jb...@apache.org.
SOLR-10303: Add the tuple context to avoid creating multiple LocalDateTime instances for the same Tuple


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

Branch: refs/heads/master
Commit: 5e403647de109685cbfc0c81943ae9e79638348f
Parents: b78a270
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Apr 12 13:18:19 2017 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Apr 12 15:04:52 2017 -0400

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  | 42 ++++++++++----------
 .../client/solrj/io/eval/TemporalEvaluator.java | 34 +++++++++++-----
 .../client/solrj/io/stream/HavingStream.java    |  3 ++
 .../client/solrj/io/stream/SelectStream.java    | 10 +++++
 .../client/solrj/io/stream/StreamContext.java   |  5 +++
 .../io/stream/eval/TemporalEvaluatorsTest.java  | 17 +++++++-
 6 files changed, 79 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e403647/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index 8ec5536..b955463 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -185,14 +185,14 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName(SORT, SortStream.class)
       .withFunctionName("train", TextLogitStream.class)
       .withFunctionName("features", FeaturesSelectionStream.class)
-      .withFunctionName("daemon", DaemonStream.class)
+         .withFunctionName("daemon", DaemonStream.class)
       .withFunctionName("shortestPath", ShortestPathStream.class)
       .withFunctionName("gatherNodes", GatherNodesStream.class)
       .withFunctionName("nodes", GatherNodesStream.class)
       .withFunctionName("select", SelectStream.class)
       .withFunctionName("shortestPath", ShortestPathStream.class)
       .withFunctionName("gatherNodes", GatherNodesStream.class)
-      .withFunctionName("nodes", GatherNodesStream.class)
+         .withFunctionName("nodes", GatherNodesStream.class)
       .withFunctionName("scoreNodes", ScoreNodesStream.class)
       .withFunctionName("model", ModelStream.class)
       .withFunctionName("classify", ClassifyStream.class)
@@ -202,29 +202,29 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("priority", PriorityStream.class)
       .withFunctionName("significantTerms", SignificantTermsStream.class)
       .withFunctionName("cartesianProduct", CartesianProductStream.class)
-      .withFunctionName("shuffle", ShuffleStream.class)
-      
-      // metrics
-      .withFunctionName("min", MinMetric.class)
+         .withFunctionName("shuffle", ShuffleStream.class)
+
+             // metrics
+         .withFunctionName("min", MinMetric.class)
       .withFunctionName("max", MaxMetric.class)
       .withFunctionName("avg", MeanMetric.class)
       .withFunctionName("sum", SumMetric.class)
       .withFunctionName("count", CountMetric.class)
       
       // tuple manipulation operations
-      .withFunctionName("replace", ReplaceOperation.class)
+         .withFunctionName("replace", ReplaceOperation.class)
       .withFunctionName("concat", ConcatOperation.class)
       
       // stream reduction operations
-      .withFunctionName("group", GroupOperation.class)
+         .withFunctionName("group", GroupOperation.class)
       .withFunctionName("distinct", DistinctOperation.class)
       .withFunctionName("having", HavingStream.class)
       
       // Stream Evaluators
-      .withFunctionName("val", RawValueEvaluator.class)
+         .withFunctionName("val", RawValueEvaluator.class)
       
       // Boolean Stream Evaluators
-      .withFunctionName("and", AndEvaluator.class)
+         .withFunctionName("and", AndEvaluator.class)
       .withFunctionName("eor", ExclusiveOrEvaluator.class)
       .withFunctionName("eq", EqualsEvaluator.class)
       .withFunctionName("gt", GreaterThanEvaluator.class)
@@ -232,23 +232,23 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("lt", LessThanEvaluator.class)
       .withFunctionName("lteq", LessThanEqualToEvaluator.class)
       .withFunctionName("not", NotEvaluator.class)
-      .withFunctionName("or", OrEvaluator.class)
+         .withFunctionName("or", OrEvaluator.class)
 
       // Date Time Evaluators
-      .withFunctionName(TemporalEvaluatorYear.FUNCTION_NAME, TemporalEvaluatorYear.class)
+         .withFunctionName(TemporalEvaluatorYear.FUNCTION_NAME, TemporalEvaluatorYear.class)
       .withFunctionName(TemporalEvaluatorMonth.FUNCTION_NAME, TemporalEvaluatorMonth.class)
       .withFunctionName(TemporalEvaluatorDay.FUNCTION_NAME, TemporalEvaluatorDay.class)
       .withFunctionName(TemporalEvaluatorDayOfYear.FUNCTION_NAME, TemporalEvaluatorDayOfYear.class)
-      .withFunctionName(TemporalEvaluatorHour.FUNCTION_NAME, TemporalEvaluatorHour.class)
+         .withFunctionName(TemporalEvaluatorHour.FUNCTION_NAME, TemporalEvaluatorHour.class)
       .withFunctionName(TemporalEvaluatorMinute.FUNCTION_NAME, TemporalEvaluatorMinute.class)
-      .withFunctionName(TemporalEvaluatorSecond.FUNCTION_NAME, TemporalEvaluatorSecond.class)
+         .withFunctionName(TemporalEvaluatorSecond.FUNCTION_NAME, TemporalEvaluatorSecond.class)
       .withFunctionName(TemporalEvaluatorEpoch.FUNCTION_NAME, TemporalEvaluatorEpoch.class)
       .withFunctionName(TemporalEvaluatorWeek.FUNCTION_NAME, TemporalEvaluatorWeek.class)
-      .withFunctionName(TemporalEvaluatorQuarter.FUNCTION_NAME, TemporalEvaluatorQuarter.class)
-      .withFunctionName(TemporalEvaluatorDayOfQuarter.FUNCTION_NAME, TemporalEvaluatorDayOfQuarter.class)
+         .withFunctionName(TemporalEvaluatorQuarter.FUNCTION_NAME, TemporalEvaluatorQuarter.class)
+         .withFunctionName(TemporalEvaluatorDayOfQuarter.FUNCTION_NAME, TemporalEvaluatorDayOfQuarter.class)
 
       // Number Stream Evaluators
-      .withFunctionName("abs", AbsoluteValueEvaluator.class)
+         .withFunctionName("abs", AbsoluteValueEvaluator.class)
       .withFunctionName("add", AddEvaluator.class)
       .withFunctionName("div", DivideEvaluator.class)
       .withFunctionName("mult", MultiplyEvaluator.class)
@@ -256,7 +256,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("log", NaturalLogEvaluator.class)
       .withFunctionName("pow", PowerEvaluator.class)
       .withFunctionName("mod", ModuloEvaluator.class)
-      .withFunctionName("ceil", CeilingEvaluator.class)
+         .withFunctionName("ceil", CeilingEvaluator.class)
       .withFunctionName("floor", FloorEvaluator.class)
       .withFunctionName("sin", SineEvaluator.class)
       .withFunctionName("asin", ArcSineEvaluator.class)
@@ -267,7 +267,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("tan", TangentEvaluator.class)
       .withFunctionName("atan", ArcTangentEvaluator.class)
       .withFunctionName("tanh", HyperbolicTangentEvaluator.class)
-      .withFunctionName("round", RoundEvaluator.class)
+         .withFunctionName("round", RoundEvaluator.class)
       .withFunctionName("sqrt", SquareRootEvaluator.class)
       .withFunctionName("cbrt", CubedRootEvaluator.class)
       .withFunctionName("coalesce", CoalesceEvaluator.class)
@@ -275,8 +275,8 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
 
 
       // Conditional Stream Evaluators
-      .withFunctionName("if", IfThenElseEvaluator.class)
-      .withFunctionName("analyze", AnalyzeEvaluator.class)
+         .withFunctionName("if", IfThenElseEvaluator.class)
+         .withFunctionName("analyze", AnalyzeEvaluator.class)
       ;
 
      // This pulls all the overrides and additions from the config

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e403647/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java
index 1694236..4af1f354 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java
@@ -26,6 +26,7 @@ import java.time.temporal.TemporalAccessor;
 import java.time.temporal.UnsupportedTemporalTypeException;
 import java.util.Date;
 import java.util.Locale;
+import java.util.Map;
 
 import org.apache.solr.client.solrj.io.Tuple;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation;
@@ -38,6 +39,8 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
  */
 public abstract class TemporalEvaluator extends ComplexEvaluator {
 
+  private String field;
+
   public TemporalEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
     super(expression, factory);
 
@@ -58,21 +61,32 @@ public abstract class TemporalEvaluator extends ComplexEvaluator {
 
     if (tupleValue == null) return null;
 
-    if (tupleValue instanceof String) {
-      instant = getInstant((String) tupleValue);
-    } else if (tupleValue instanceof Long) {
-      instant = Instant.ofEpochMilli((Long)tupleValue);
-    } else if (tupleValue instanceof Instant) {
-      instant = (Instant) tupleValue;
-    } else if (tupleValue instanceof Date) {
-      instant = ((Date) tupleValue).toInstant();
-    } else if (tupleValue instanceof TemporalAccessor) {
-      date = ((TemporalAccessor) tupleValue);
+    if(field == null) {
+      field = streamEvaluator.toExpression(constructingFactory).toString();
+    }
+
+    Map tupleContext = streamContext.getTupleContext();
+    date = (LocalDateTime)tupleContext.get(field); // Check to see if the date has already been created for this field
+
+    if(date == null) {
+      if (tupleValue instanceof String) {
+        instant = getInstant((String) tupleValue);
+      } else if (tupleValue instanceof Long) {
+        instant = Instant.ofEpochMilli((Long) tupleValue);
+      } else if (tupleValue instanceof Instant) {
+        instant = (Instant) tupleValue;
+      } else if (tupleValue instanceof Date) {
+        instant = ((Date) tupleValue).toInstant();
+      } else if (tupleValue instanceof TemporalAccessor) {
+        date = ((TemporalAccessor) tupleValue);
+        tupleContext.put(field, date); // Cache the date in the TupleContext
+      }
     }
 
     if (instant != null) {
       if (TemporalEvaluatorEpoch.FUNCTION_NAME.equals(getFunction())) return instant.toEpochMilli();
       date = LocalDateTime.ofInstant(instant, ZoneOffset.UTC);
+      tupleContext.put(field, date); // Cache the date in the TupleContext
     }
 
     if (date != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e403647/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
index 35e8952..2f74bc5 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
@@ -43,6 +43,7 @@ public class HavingStream extends TupleStream implements Expressible {
 
   private TupleStream stream;
   private BooleanEvaluator evaluator;
+  private StreamContext streamContext;
 
   private transient Tuple currentGroupHead;
 
@@ -128,6 +129,7 @@ public class HavingStream extends TupleStream implements Expressible {
   }
 
   public void setStreamContext(StreamContext context) {
+    this.streamContext = context;
     this.stream.setStreamContext(context);
   }
 
@@ -152,6 +154,7 @@ public class HavingStream extends TupleStream implements Expressible {
         return tuple;
       }
 
+      streamContext.getTupleContext().clear();
       if(evaluator.evaluate(tuple)){
         return tuple;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e403647/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SelectStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SelectStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SelectStream.java
index c0cbc17..95926e3 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SelectStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SelectStream.java
@@ -49,6 +49,7 @@ public class SelectStream extends TupleStream implements Expressible {
   private static final long serialVersionUID = 1;
 
   private TupleStream stream;
+  private StreamContext streamContext;
   private Map<String,String> selectedFields;
   private Map<StreamEvaluator,String> selectedEvaluators;
   private List<StreamOperation> operations;
@@ -213,6 +214,7 @@ public class SelectStream extends TupleStream implements Expressible {
   }
 
   public void setStreamContext(StreamContext context) {
+    this.streamContext = context;
     this.stream.setStreamContext(context);
     Set<StreamEvaluator> evaluators = selectedEvaluators.keySet();
 
@@ -245,6 +247,14 @@ public class SelectStream extends TupleStream implements Expressible {
     // create a copy with the limited set of fields
     Tuple workingToReturn = new Tuple(new HashMap<>());
     Tuple workingForEvaluators = new Tuple(new HashMap<>());
+
+    //Clear the TupleContext before running the evaluators.
+    //The TupleContext allows evaluators to cache values within the scope of a single tuple.
+    //For example a LocalDateTime could be parsed by one evaluator and used by other evaluators within the scope of the tuple.
+    //This avoids the need to create multiple LocalDateTime instances for the same tuple to satisfy a select expression.
+
+    streamContext.getTupleContext().clear();
+
     for(Object fieldName : original.fields.keySet()){
       workingForEvaluators.put(fieldName, original.get(fieldName));
       if(selectedFields.containsKey(fieldName)){

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e403647/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StreamContext.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StreamContext.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StreamContext.java
index d1460ea..60a9274 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StreamContext.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StreamContext.java
@@ -36,6 +36,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 public class StreamContext implements Serializable{
 
   private Map entries = new HashMap();
+  private Map tupleContext = new HashMap();
   public int workerID;
   public int numWorkers;
   private SolrClientCache clientCache;
@@ -78,6 +79,10 @@ public class StreamContext implements Serializable{
     this.streamFactory = streamFactory;
   }
 
+  public Map getTupleContext() {
+    return tupleContext;
+  }
+
   public StreamFactory getStreamFactory() {
     return this.streamFactory;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e403647/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
index d44d8c2..d3b839b 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
@@ -43,6 +43,7 @@ import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorQuarter;
 import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorSecond;
 import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorWeek;
 import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorYear;
+import org.apache.solr.client.solrj.io.stream.StreamContext;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParser;
@@ -50,7 +51,6 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 import org.junit.Test;
 
 import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertNotNull;
 import static junit.framework.Assert.assertNull;
 import static junit.framework.Assert.assertTrue;
 
@@ -90,6 +90,8 @@ public class TemporalEvaluatorsTest {
 
     try {
       evaluator = factory.constructEvaluator("week()");
+      StreamContext streamContext = new StreamContext();
+      evaluator.setStreamContext(streamContext);
       assertTrue(false);
     } catch (IOException e) {
       assertTrue(e.getCause().getCause().getMessage().contains("Invalid expression week()"));
@@ -97,6 +99,8 @@ public class TemporalEvaluatorsTest {
 
     try {
       evaluator = factory.constructEvaluator("week(a, b)");
+      StreamContext streamContext = new StreamContext();
+      evaluator.setStreamContext(streamContext);
       assertTrue(false);
     } catch (IOException e) {
       assertTrue(e.getCause().getCause().getMessage().contains("expecting one value but found 2"));
@@ -104,6 +108,8 @@ public class TemporalEvaluatorsTest {
 
     try {
       evaluator = factory.constructEvaluator("Week()");
+      StreamContext streamContext = new StreamContext();
+      evaluator.setStreamContext(streamContext);
       assertTrue(false);
     } catch (IOException e) {
       assertTrue(e.getMessage().contains("Invalid evaluator expression Week() - function 'Week' is unknown"));
@@ -115,9 +121,12 @@ public class TemporalEvaluatorsTest {
   public void testInvalidValues() throws Exception {
     StreamEvaluator evaluator = factory.constructEvaluator("year(a)");
 
+
     try {
       values.clear();
       values.put("a", 12);
+      StreamContext streamContext = new StreamContext();
+      evaluator.setStreamContext(streamContext);
       Object result = evaluator.evaluate(new Tuple(values));
       assertTrue(false);
     } catch (IOException e) {
@@ -127,6 +136,8 @@ public class TemporalEvaluatorsTest {
     try {
       values.clear();
       values.put("a", "1995-12-31");
+      StreamContext streamContext = new StreamContext();
+      evaluator.setStreamContext(streamContext);
       Object result = evaluator.evaluate(new Tuple(values));
       assertTrue(false);
     } catch (IOException e) {
@@ -136,6 +147,8 @@ public class TemporalEvaluatorsTest {
     try {
       values.clear();
       values.put("a", "");
+      StreamContext streamContext = new StreamContext();
+      evaluator.setStreamContext(streamContext);
       Object result = evaluator.evaluate(new Tuple(values));
       assertTrue(false);
     } catch (IOException e) {
@@ -267,6 +280,8 @@ public class TemporalEvaluatorsTest {
 
   public void testFunction(String expression, Object value, Number expected) throws Exception {
     StreamEvaluator evaluator = factory.constructEvaluator(expression);
+    StreamContext streamContext = new StreamContext();
+    evaluator.setStreamContext(streamContext);
     values.clear();
     values.put("a", value);
     Object result = evaluator.evaluate(new Tuple(values));


[09/12] lucene-solr:master: SOLR-10303: Removing the unused class, replaced by TemporalEvaluator

Posted by jb...@apache.org.
SOLR-10303:  Removing the unused class, replaced by TemporalEvaluator


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

Branch: refs/heads/master
Commit: 1c333c79d0cd3d94e85222030788bf5597732f2c
Parents: d70fc96
Author: Gethin James <ge...@alfresco.com>
Authored: Thu Apr 6 17:19:02 2017 +0200
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Apr 12 15:04:51 2017 -0400

----------------------------------------------------------------------
 .../client/solrj/io/eval/DatePartEvaluator.java | 169 -------------------
 1 file changed, 169 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1c333c79/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
deleted file mode 100644
index 592d4af..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
+++ /dev/null
@@ -1,169 +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.client.solrj.io.eval;
-
-import java.io.IOException;
-import java.time.Instant;
-import java.time.LocalDateTime;
-import java.time.ZoneOffset;
-import java.time.format.DateTimeParseException;
-import java.time.temporal.ChronoField;
-import java.time.temporal.IsoFields;
-import java.time.temporal.TemporalAccessor;
-import java.time.temporal.UnsupportedTemporalTypeException;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.Locale;
-
-import org.apache.solr.client.solrj.io.Tuple;
-import org.apache.solr.client.solrj.io.stream.expr.Explanation;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
-import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
-
-/**
- * Provides numeric Date/Time stream evaluators
- */
-public class DatePartEvaluator extends NumberEvaluator {
-
-  public enum FUNCTION {year, month, day, dayOfYear, dayOfQuarter, hour, minute, quarter, week, second, epoch}
-
-  private final FUNCTION function;
-
-  public DatePartEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
-    super(expression, factory);
-
-    String functionName = expression.getFunctionName();
-
-    try {
-      this.function = FUNCTION.valueOf(functionName);
-    } catch (IllegalArgumentException e) {
-      throw new IOException(String.format(Locale.ROOT, "Invalid date expression %s - expecting one of %s", functionName, Arrays.toString(FUNCTION.values())));
-    }
-
-    if (1 != subEvaluators.size()) {
-      throw new IOException(String.format(Locale.ROOT, "Invalid expression %s - expecting one value but found %d", expression, subEvaluators.size()));
-    }
-  }
-
-  @Override
-  public Number evaluate(Tuple tuple) throws IOException {
-
-    Instant instant = null;
-    TemporalAccessor date = null;
-
-    //First evaluate the parameter
-    StreamEvaluator streamEvaluator = subEvaluators.get(0);
-    Object tupleValue = streamEvaluator.evaluate(tuple);
-
-    if (tupleValue == null) return null;
-
-    if (tupleValue instanceof String) {
-      instant = getInstant((String) tupleValue);
-    } else if (tupleValue instanceof Instant) {
-      instant = (Instant) tupleValue;
-    } else if (tupleValue instanceof Date) {
-      instant = ((Date) tupleValue).toInstant();
-    } else if (tupleValue instanceof TemporalAccessor) {
-      date = ((TemporalAccessor) tupleValue);
-    }
-
-    if (instant != null) {
-      if (function.equals(FUNCTION.epoch)) return instant.toEpochMilli();
-      date = LocalDateTime.ofInstant(instant, ZoneOffset.UTC);
-    }
-
-    if (date != null) {
-      return evaluate(date);
-    }
-
-    throw new IOException(String.format(Locale.ROOT, "Invalid parameter %s - The parameter must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", String.valueOf(tupleValue)));
-  }
-
-  private Instant getInstant(String dateStr) throws IOException {
-
-    if (dateStr != null && !dateStr.isEmpty()) {
-      try {
-        return Instant.parse(dateStr);
-      } catch (DateTimeParseException e) {
-        throw new IOException(String.format(Locale.ROOT, "Invalid parameter %s - The String must be formatted in the ISO_INSTANT date format.", dateStr));
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Evaluate the date based on the specified function
-   *
-   * @param date
-   * @return the evaluated value
-   */
-  private Number evaluate(TemporalAccessor date) throws IOException {
-    try {
-      switch (function) {
-        case year:
-          return date.get(ChronoField.YEAR);
-        case month:
-          return date.get(ChronoField.MONTH_OF_YEAR);
-        case day:
-          return date.get(ChronoField.DAY_OF_MONTH);
-        case dayOfYear:
-          return date.get(ChronoField.DAY_OF_YEAR);
-        case hour:
-          return date.get(ChronoField.HOUR_OF_DAY);
-        case minute:
-          return date.get(ChronoField.MINUTE_OF_HOUR);
-        case second:
-          return date.get(ChronoField.SECOND_OF_MINUTE);
-        case dayOfQuarter:
-          return date.get(IsoFields.DAY_OF_QUARTER);
-        case quarter:
-          return date.get(IsoFields.QUARTER_OF_YEAR);
-        case week:
-          return date.get(IsoFields.WEEK_OF_WEEK_BASED_YEAR);
-        case epoch:
-          if (date instanceof LocalDateTime) {
-            return ((LocalDateTime)date).atZone(ZoneOffset.UTC).toInstant().toEpochMilli();
-          }
-      }
-    } catch (UnsupportedTemporalTypeException utte) {
-      throw new IOException(String.format(Locale.ROOT, "It is not possible to call '%s' function on %s", function, date.getClass().getName()));
-    }
-    throw new IOException(String.format(Locale.ROOT, "Unsupported function '%s' called on %s", function, date.toString()));
-  }
-
-  @Override
-  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
-    StreamExpression expression = new StreamExpression(function.toString());
-
-    for (StreamEvaluator evaluator : subEvaluators) {
-      expression.addParameter(evaluator.toExpression(factory));
-    }
-
-    return expression;
-  }
-
-  @Override
-  public Explanation toExplanation(StreamFactory factory) throws IOException {
-    return new Explanation(nodeId.toString())
-        .withExpressionType(Explanation.ExpressionType.EVALUATOR)
-        .withImplementingClass(getClass().getName())
-        .withExpression(toExpression(factory).toString());
-  }
-
-}


[07/12] lucene-solr:master: SOLR-10303: Refactored to multiple TemporalEvaluator classes based on feedback

Posted by jb...@apache.org.
SOLR-10303:  Refactored to multiple TemporalEvaluator classes based on feedback


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

Branch: refs/heads/master
Commit: d70fc967d44fd0a7e3706707479853dfd43ea908
Parents: b314bc6
Author: Gethin James <ge...@alfresco.com>
Authored: Thu Apr 6 11:58:26 2017 +0200
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Apr 12 15:04:51 2017 -0400

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |  33 +-
 .../client/solrj/io/eval/TemporalEvaluator.java | 122 ++++++++
 .../solrj/io/eval/TemporalEvaluatorDay.java     |  48 +++
 .../io/eval/TemporalEvaluatorDayOfQuarter.java  |  48 +++
 .../io/eval/TemporalEvaluatorDayOfYear.java     |  48 +++
 .../solrj/io/eval/TemporalEvaluatorEpoch.java   |  54 ++++
 .../solrj/io/eval/TemporalEvaluatorHour.java    |  48 +++
 .../solrj/io/eval/TemporalEvaluatorMinute.java  |  48 +++
 .../solrj/io/eval/TemporalEvaluatorMonth.java   |  48 +++
 .../solrj/io/eval/TemporalEvaluatorQuarter.java |  48 +++
 .../solrj/io/eval/TemporalEvaluatorSecond.java  |  48 +++
 .../solrj/io/eval/TemporalEvaluatorWeek.java    |  48 +++
 .../solrj/io/eval/TemporalEvaluatorYear.java    |  48 +++
 .../io/stream/eval/DatePartEvaluatorTest.java   | 274 ----------------
 .../io/stream/eval/TemporalEvaluatorsTest.java  | 309 +++++++++++++++++++
 15 files changed, 991 insertions(+), 281 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index 1f087b6..8ec5536 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -43,7 +43,6 @@ import org.apache.solr.client.solrj.io.eval.CeilingEvaluator;
 import org.apache.solr.client.solrj.io.eval.CoalesceEvaluator;
 import org.apache.solr.client.solrj.io.eval.CosineEvaluator;
 import org.apache.solr.client.solrj.io.eval.CubedRootEvaluator;
-import org.apache.solr.client.solrj.io.eval.DatePartEvaluator;
 import org.apache.solr.client.solrj.io.eval.DivideEvaluator;
 import org.apache.solr.client.solrj.io.eval.EqualsEvaluator;
 import org.apache.solr.client.solrj.io.eval.ExclusiveOrEvaluator;
@@ -86,6 +85,17 @@ import org.apache.solr.client.solrj.io.stream.metrics.MaxMetric;
 import org.apache.solr.client.solrj.io.stream.metrics.MeanMetric;
 import org.apache.solr.client.solrj.io.stream.metrics.MinMetric;
 import org.apache.solr.client.solrj.io.stream.metrics.SumMetric;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorDay;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorDayOfQuarter;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorDayOfYear;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorEpoch;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorHour;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorMinute;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorMonth;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorQuarter;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorSecond;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorWeek;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorYear;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -223,7 +233,20 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("lteq", LessThanEqualToEvaluator.class)
       .withFunctionName("not", NotEvaluator.class)
       .withFunctionName("or", OrEvaluator.class)
-      
+
+      // Date Time Evaluators
+      .withFunctionName(TemporalEvaluatorYear.FUNCTION_NAME, TemporalEvaluatorYear.class)
+      .withFunctionName(TemporalEvaluatorMonth.FUNCTION_NAME, TemporalEvaluatorMonth.class)
+      .withFunctionName(TemporalEvaluatorDay.FUNCTION_NAME, TemporalEvaluatorDay.class)
+      .withFunctionName(TemporalEvaluatorDayOfYear.FUNCTION_NAME, TemporalEvaluatorDayOfYear.class)
+      .withFunctionName(TemporalEvaluatorHour.FUNCTION_NAME, TemporalEvaluatorHour.class)
+      .withFunctionName(TemporalEvaluatorMinute.FUNCTION_NAME, TemporalEvaluatorMinute.class)
+      .withFunctionName(TemporalEvaluatorSecond.FUNCTION_NAME, TemporalEvaluatorSecond.class)
+      .withFunctionName(TemporalEvaluatorEpoch.FUNCTION_NAME, TemporalEvaluatorEpoch.class)
+      .withFunctionName(TemporalEvaluatorWeek.FUNCTION_NAME, TemporalEvaluatorWeek.class)
+      .withFunctionName(TemporalEvaluatorQuarter.FUNCTION_NAME, TemporalEvaluatorQuarter.class)
+      .withFunctionName(TemporalEvaluatorDayOfQuarter.FUNCTION_NAME, TemporalEvaluatorDayOfQuarter.class)
+
       // Number Stream Evaluators
       .withFunctionName("abs", AbsoluteValueEvaluator.class)
       .withFunctionName("add", AddEvaluator.class)
@@ -250,16 +273,12 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("coalesce", CoalesceEvaluator.class)
       .withFunctionName("uuid", UuidEvaluator.class)
 
+
       // Conditional Stream Evaluators
       .withFunctionName("if", IfThenElseEvaluator.class)
       .withFunctionName("analyze", AnalyzeEvaluator.class)
       ;
 
-      // Date evaluators
-      for (DatePartEvaluator.FUNCTION function: DatePartEvaluator.FUNCTION.values()) {
-        streamFactory.withFunctionName(function.toString(), DatePartEvaluator.class);
-      }
-
      // This pulls all the overrides and additions from the config
      List<PluginInfo> pluginInfos = core.getSolrConfig().getPluginInfos(Expressible.class.getName());
      for (PluginInfo pluginInfo : pluginInfos) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java
new file mode 100644
index 0000000..884653f
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluator.java
@@ -0,0 +1,122 @@
+/*
+ * 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.io.eval;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeParseException;
+import java.time.temporal.TemporalAccessor;
+import java.time.temporal.UnsupportedTemporalTypeException;
+import java.util.Date;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * A generic date evaluator for use with a TemporalAccessor
+ */
+public abstract class TemporalEvaluator extends ComplexEvaluator {
+
+  public TemporalEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+
+    if (1 != subEvaluators.size()) {
+      throw new IOException(String.format(Locale.ROOT, "Invalid expression %s - expecting one value but found %d", expression, subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Object evaluate(Tuple tuple) throws IOException {
+
+    Instant instant = null;
+    TemporalAccessor date = null;
+
+    //First evaluate the parameter
+    StreamEvaluator streamEvaluator = subEvaluators.get(0);
+    Object tupleValue = streamEvaluator.evaluate(tuple);
+
+    if (tupleValue == null) return null;
+
+    if (tupleValue instanceof String) {
+      instant = getInstant((String) tupleValue);
+    } else if (tupleValue instanceof Long) {
+      instant = Instant.ofEpochMilli((Long)tupleValue);
+    } else if (tupleValue instanceof Instant) {
+      instant = (Instant) tupleValue;
+    } else if (tupleValue instanceof Date) {
+      instant = ((Date) tupleValue).toInstant();
+    } else if (tupleValue instanceof TemporalAccessor) {
+      date = ((TemporalAccessor) tupleValue);
+    }
+
+    if (instant != null) {
+      if (TemporalEvaluatorEpoch.FUNCTION_NAME.equals(getFunction())) return instant.toEpochMilli();
+      date = LocalDateTime.ofInstant(instant, ZoneOffset.UTC);
+    }
+
+    if (date != null) {
+      try {
+        return evaluateDate(date);
+      } catch (UnsupportedTemporalTypeException utte) {
+        throw new IOException(String.format(Locale.ROOT, "It is not possible to call '%s' function on %s", getFunction(), date.getClass().getName()));
+      }
+    }
+
+    throw new IOException(String.format(Locale.ROOT, "Invalid parameter %s - The parameter must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", String.valueOf(tupleValue)));
+  }
+
+  public abstract Object evaluateDate(TemporalAccessor aDate) throws IOException;
+  public abstract String getFunction();
+
+  protected Instant getInstant(String dateStr) throws IOException {
+
+    if (dateStr != null && !dateStr.isEmpty()) {
+      try {
+        return Instant.parse(dateStr);
+      } catch (DateTimeParseException e) {
+        throw new IOException(String.format(Locale.ROOT, "Invalid parameter %s - The String must be formatted in the ISO_INSTANT date format.", dateStr));
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(getFunction());
+
+    for (StreamEvaluator evaluator : subEvaluators) {
+      expression.addParameter(evaluator.toExpression(factory));
+    }
+
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(nodeId.toString())
+        .withExpressionType(Explanation.ExpressionType.EVALUATOR)
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDay.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDay.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDay.java
new file mode 100644
index 0000000..88b5043
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDay.java
@@ -0,0 +1,48 @@
+/*
+ * 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.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a day stream evaluator
+ */
+public class TemporalEvaluatorDay extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "day";
+
+  public TemporalEvaluatorDay(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(ChronoField.DAY_OF_MONTH);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfQuarter.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfQuarter.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfQuarter.java
new file mode 100644
index 0000000..f034377
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfQuarter.java
@@ -0,0 +1,48 @@
+/*
+ * 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.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.IsoFields;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a dayOfQuarter stream evaluator
+ */
+public class TemporalEvaluatorDayOfQuarter extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "dayOfQuarter";
+
+  public TemporalEvaluatorDayOfQuarter(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(IsoFields.DAY_OF_QUARTER);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfYear.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfYear.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfYear.java
new file mode 100644
index 0000000..339f938
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorDayOfYear.java
@@ -0,0 +1,48 @@
+/*
+ * 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.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a dayOfYear stream evaluator
+ */
+public class TemporalEvaluatorDayOfYear extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "dayOfYear";
+
+  public TemporalEvaluatorDayOfYear(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(ChronoField.DAY_OF_YEAR);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorEpoch.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorEpoch.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorEpoch.java
new file mode 100644
index 0000000..bea4f5d
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorEpoch.java
@@ -0,0 +1,54 @@
+/*
+ * 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.io.eval;
+
+import java.io.IOException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a epoch stream evaluator
+ */
+public class TemporalEvaluatorEpoch extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "epoch";
+
+  public TemporalEvaluatorEpoch(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) throws IOException {
+    if (aDate instanceof LocalDateTime) {
+      return ((LocalDateTime)aDate).atZone(ZoneOffset.UTC).toInstant().toEpochMilli();
+    }
+    throw new IOException(String.format(Locale.ROOT, "Unsupported function '%s' called on %s", FUNCTION_NAME, aDate.toString()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorHour.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorHour.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorHour.java
new file mode 100644
index 0000000..5480870
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorHour.java
@@ -0,0 +1,48 @@
+/*
+ * 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.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a hour stream evaluator
+ */
+public class TemporalEvaluatorHour extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "hour";
+
+  public TemporalEvaluatorHour(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(ChronoField.HOUR_OF_DAY);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMinute.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMinute.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMinute.java
new file mode 100644
index 0000000..9438e9d
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMinute.java
@@ -0,0 +1,48 @@
+/*
+ * 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.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a minute stream evaluator
+ */
+public class TemporalEvaluatorMinute extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "minute";
+
+  public TemporalEvaluatorMinute(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(ChronoField.MINUTE_OF_HOUR);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMonth.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMonth.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMonth.java
new file mode 100644
index 0000000..b9a70fe
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorMonth.java
@@ -0,0 +1,48 @@
+/*
+ * 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.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a month stream evaluator
+ */
+public class TemporalEvaluatorMonth extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "month";
+
+  public TemporalEvaluatorMonth(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(ChronoField.MONTH_OF_YEAR);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorQuarter.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorQuarter.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorQuarter.java
new file mode 100644
index 0000000..0144311
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorQuarter.java
@@ -0,0 +1,48 @@
+/*
+ * 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.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.IsoFields;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a quarter stream evaluator
+ */
+public class TemporalEvaluatorQuarter extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "quarter";
+
+  public TemporalEvaluatorQuarter(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(IsoFields.QUARTER_OF_YEAR);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorSecond.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorSecond.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorSecond.java
new file mode 100644
index 0000000..f5b71fc
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorSecond.java
@@ -0,0 +1,48 @@
+/*
+ * 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.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a second stream evaluator
+ */
+public class TemporalEvaluatorSecond extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "second";
+
+  public TemporalEvaluatorSecond(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(ChronoField.SECOND_OF_MINUTE);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorWeek.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorWeek.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorWeek.java
new file mode 100644
index 0000000..1a2974a
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorWeek.java
@@ -0,0 +1,48 @@
+/*
+ * 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.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.IsoFields;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a week stream evaluator
+ */
+public class TemporalEvaluatorWeek extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "week";
+
+  public TemporalEvaluatorWeek(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(IsoFields.WEEK_OF_WEEK_BASED_YEAR);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorYear.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorYear.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorYear.java
new file mode 100644
index 0000000..0b8d69c
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TemporalEvaluatorYear.java
@@ -0,0 +1,48 @@
+/*
+ * 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.io.eval;
+
+import java.io.IOException;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * Provides a year stream evaluator
+ */
+public class TemporalEvaluatorYear extends TemporalEvaluator {
+
+  public static final String FUNCTION_NAME = "year";
+
+  public TemporalEvaluatorYear(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public String getFunction() {
+    return FUNCTION_NAME;
+  }
+
+  @Override
+  public Object evaluateDate(TemporalAccessor aDate) {
+    return aDate.get(ChronoField.YEAR);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
deleted file mode 100644
index 8b3317e..0000000
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
+++ /dev/null
@@ -1,274 +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.client.solrj.io.stream.eval;
-
-import java.io.IOException;
-import java.time.Instant;
-import java.time.LocalDateTime;
-import java.time.MonthDay;
-import java.time.YearMonth;
-import java.time.ZoneOffset;
-import java.util.Calendar;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.Map;
-import java.util.TimeZone;
-
-import org.apache.commons.collections.map.HashedMap;
-import org.apache.solr.client.solrj.io.Tuple;
-import org.apache.solr.client.solrj.io.eval.DatePartEvaluator;
-import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
-import org.apache.solr.client.solrj.io.stream.expr.Explanation;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParser;
-import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
-import org.junit.Test;
-
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertNotNull;
-import static junit.framework.Assert.assertNull;
-import static junit.framework.Assert.assertTrue;
-
-/**
- * Tests numeric Date/Time stream evaluators
- */
-public class DatePartEvaluatorTest {
-
-
-  StreamFactory factory;
-  Map<String, Object> values;
-
-  public DatePartEvaluatorTest() {
-    super();
-
-    factory = new StreamFactory();
-
-    factory.withFunctionName("nope", DatePartEvaluator.class);
-    for (DatePartEvaluator.FUNCTION function : DatePartEvaluator.FUNCTION.values()) {
-      factory.withFunctionName(function.toString(), DatePartEvaluator.class);
-    }
-    values = new HashedMap();
-  }
-
-  @Test
-  public void testInvalidExpression() throws Exception {
-
-    StreamEvaluator evaluator;
-
-    try {
-      evaluator = factory.constructEvaluator("nope(a)");
-      evaluator.evaluate(new Tuple(null));
-      assertTrue(false);
-    } catch (IOException e) {
-      assertTrue(e.getCause().getCause().getMessage().contains("Invalid date expression nope"));
-      assertTrue(e.getCause().getCause().getMessage().contains("expecting one of [year, month, day"));
-    }
-
-    try {
-      evaluator = factory.constructEvaluator("week()");
-      assertTrue(false);
-    } catch (IOException e) {
-      assertTrue(e.getCause().getCause().getMessage().contains("Invalid expression week()"));
-    }
-
-    try {
-      evaluator = factory.constructEvaluator("week(a, b)");
-      assertTrue(false);
-    } catch (IOException e) {
-      assertTrue(e.getCause().getCause().getMessage().contains("expecting one value but found 2"));
-    }
-
-    try {
-      evaluator = factory.constructEvaluator("Week()");
-      assertTrue(false);
-    } catch (IOException e) {
-      assertTrue(e.getMessage().contains("Invalid evaluator expression Week() - function 'Week' is unknown"));
-    }
-  }
-
-
-  @Test
-  public void testInvalidValues() throws Exception {
-    StreamEvaluator evaluator = factory.constructEvaluator("year(a)");
-
-    try {
-      values.clear();
-      values.put("a", 12);
-      Object result = evaluator.evaluate(new Tuple(values));
-      assertTrue(false);
-    } catch (IOException e) {
-      assertEquals("Invalid parameter 12 - The parameter must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", e.getMessage());
-    }
-
-    try {
-      values.clear();
-      values.put("a", "1995-12-31");
-      Object result = evaluator.evaluate(new Tuple(values));
-      assertTrue(false);
-    } catch (IOException e) {
-      assertEquals("Invalid parameter 1995-12-31 - The String must be formatted in the ISO_INSTANT date format.", e.getMessage());
-    }
-
-    try {
-      values.clear();
-      values.put("a", "");
-      Object result = evaluator.evaluate(new Tuple(values));
-      assertTrue(false);
-    } catch (IOException e) {
-      assertEquals("Invalid parameter  - The parameter must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", e.getMessage());
-    }
-
-    values.clear();
-    values.put("a", null);
-    assertNull(evaluator.evaluate(new Tuple(values)));
-  }
-
-  @Test
-  public void testAllFunctions() throws Exception {
-
-    //year, month, day, dayofyear, hour, minute, quarter, week, second, epoch
-    testFunction("year(a)", "1995-12-31T23:59:59Z", 1995);
-    testFunction("month(a)","1995-12-31T23:59:59Z", 12);
-    testFunction("day(a)",  "1995-12-31T23:59:59Z", 31);
-    testFunction("dayOfYear(a)",  "1995-12-31T23:59:59Z", 365);
-    testFunction("dayOfQuarter(a)",  "1995-12-31T23:59:59Z", 92);
-    testFunction("hour(a)",   "1995-12-31T23:59:59Z", 23);
-    testFunction("minute(a)", "1995-12-31T23:59:59Z", 59);
-    testFunction("quarter(a)","1995-12-31T23:59:59Z", 4);
-    testFunction("week(a)",   "1995-12-31T23:59:59Z", 52);
-    testFunction("second(a)", "1995-12-31T23:59:58Z", 58);
-    testFunction("epoch(a)",  "1995-12-31T23:59:59Z", 820454399000l);
-
-    testFunction("year(a)", "2017-03-17T10:30:45Z", 2017);
-    testFunction("year('a')", "2017-03-17T10:30:45Z", 2017);
-    testFunction("month(a)","2017-03-17T10:30:45Z", 3);
-    testFunction("day(a)",  "2017-03-17T10:30:45Z", 17);
-    testFunction("day('a')",  "2017-03-17T10:30:45Z", 17);
-    testFunction("dayOfYear(a)",  "2017-03-17T10:30:45Z", 76);
-    testFunction("dayOfQuarter(a)",  "2017-03-17T10:30:45Z", 76);
-    testFunction("hour(a)",   "2017-03-17T10:30:45Z", 10);
-    testFunction("minute(a)", "2017-03-17T10:30:45Z", 30);
-    testFunction("quarter(a)","2017-03-17T10:30:45Z", 1);
-    testFunction("week(a)",   "2017-03-17T10:30:45Z", 11);
-    testFunction("second(a)", "2017-03-17T10:30:45Z", 45);
-    testFunction("epoch(a)",  "2017-03-17T10:30:45Z", 1489746645000l);
-
-    testFunction("epoch(a)",  new Date(1489746645500l).toInstant().toString(), 1489746645500l);
-    testFunction("epoch(a)",  new Date(820454399990l).toInstant().toString(), 820454399990l);
-
-    //Additionally test all functions to make sure they return a non-null number
-    for (DatePartEvaluator.FUNCTION function : DatePartEvaluator.FUNCTION.values()) {
-      StreamEvaluator evaluator = factory.constructEvaluator(function+"(a)");
-      values.clear();
-      values.put("a", "2017-03-17T10:30:45Z");
-      Object result = evaluator.evaluate(new Tuple(values));
-      assertNotNull(function+" should return a result",result);
-      assertTrue(function+" should return a number", result instanceof Number);
-    }
-  }
-
-  @Test
-  public void testFunctionsOnDate() throws Exception {
-    Calendar calendar = new GregorianCalendar(2017,12,5, 23, 59);
-    calendar.setTimeZone(TimeZone.getTimeZone("UTC"));
-    Date aDate = calendar.getTime();
-    testFunction("year(a)", aDate, calendar.get(Calendar.YEAR));
-    testFunction("month(a)", aDate, calendar.get(Calendar.MONTH)+1);
-    testFunction("day(a)", aDate, calendar.get(Calendar.DAY_OF_MONTH));
-    testFunction("hour(a)", aDate, calendar.get(Calendar.HOUR_OF_DAY));
-    testFunction("minute(a)", aDate, calendar.get(Calendar.MINUTE));
-    testFunction("epoch(a)", aDate, aDate.getTime());
-  }
-
-  @Test
-  public void testFunctionsOnInstant() throws Exception {
-    Calendar calendar = new GregorianCalendar(2017,12,5, 23, 59);
-    calendar.setTimeZone(TimeZone.getTimeZone("UTC"));
-    Date aDate = calendar.getTime();
-    Instant instant = aDate.toInstant();
-    testFunction("year(a)", instant, calendar.get(Calendar.YEAR));
-    testFunction("month(a)", instant, calendar.get(Calendar.MONTH)+1);
-    testFunction("day(a)", instant, calendar.get(Calendar.DAY_OF_MONTH));
-    testFunction("hour(a)", instant, calendar.get(Calendar.HOUR_OF_DAY));
-    testFunction("minute(a)", instant, calendar.get(Calendar.MINUTE));
-    testFunction("epoch(a)", instant, aDate.getTime());
-  }
-
-  @Test
-  public void testFunctionsLocalDateTime() throws Exception {
-
-    LocalDateTime localDateTime = LocalDateTime.of(2017,12,5, 23, 59);
-    Date aDate = Date.from(localDateTime.atZone(ZoneOffset.UTC).toInstant());
-    testFunction("year(a)", localDateTime, 2017);
-    testFunction("month(a)", localDateTime, 12);
-    testFunction("day(a)", localDateTime, 5);
-    testFunction("hour(a)", localDateTime, 23);
-    testFunction("minute(a)", localDateTime, 59);
-    testFunction("epoch(a)", localDateTime, aDate.getTime());
-  }
-
-  @Test
-  public void testLimitedFunctions() throws Exception {
-
-    MonthDay monthDay = MonthDay.of(12,5);
-    testFunction("month(a)", monthDay, 12);
-    testFunction("day(a)", monthDay, 5);
-
-    try {
-      testFunction("year(a)", monthDay, 2017);
-      assertTrue(false);
-    } catch (IOException e) {
-      assertEquals("It is not possible to call 'year' function on java.time.MonthDay", e.getMessage());
-    }
-
-    YearMonth yearMonth = YearMonth.of(2018, 4);
-    testFunction("month(a)", yearMonth, 4);
-    testFunction("year(a)", yearMonth, 2018);
-
-    try {
-      testFunction("day(a)", yearMonth, 5);
-      assertTrue(false);
-    } catch (IOException e) {
-      assertEquals("It is not possible to call 'day' function on java.time.YearMonth", e.getMessage());
-    }
-
-  }
-
-
-  public void testFunction(String expression, Object value, Number expected) throws Exception {
-    StreamEvaluator evaluator = factory.constructEvaluator(expression);
-    values.clear();
-    values.put("a", value);
-    Object result = evaluator.evaluate(new Tuple(values));
-    assertTrue(result instanceof Number);
-    assertEquals(expected, result);
-  }
-
-  @Test
-  public void testExplain() throws IOException {
-    StreamExpression express = StreamExpressionParser.parse("month('myfield')");
-    DatePartEvaluator datePartEvaluator = new DatePartEvaluator(express,factory);
-    Explanation explain = datePartEvaluator.toExplanation(factory);
-    assertEquals("month(myfield)", explain.getExpression());
-
-    express = StreamExpressionParser.parse("day(aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbb)");
-    datePartEvaluator = new DatePartEvaluator(express,factory);
-    explain = datePartEvaluator.toExplanation(factory);
-    assertEquals("day(aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbb)", explain.getExpression());
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d70fc967/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
new file mode 100644
index 0000000..a6c590b
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TemporalEvaluatorsTest.java
@@ -0,0 +1,309 @@
+/*
+ * 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.io.stream.eval;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.MonthDay;
+import java.time.YearMonth;
+import java.time.ZoneOffset;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.Map;
+import java.util.TimeZone;
+
+import org.apache.commons.collections.map.HashedMap;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorDay;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorDayOfQuarter;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorDayOfYear;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorEpoch;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorHour;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorMinute;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorMonth;
+import org.apache.solr.client.solrj.io.eval.DatePartEvaluator;
+import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorQuarter;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorSecond;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorWeek;
+import org.apache.solr.client.solrj.io.eval.TemporalEvaluatorYear;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParser;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.junit.Test;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertNotNull;
+import static junit.framework.Assert.assertNull;
+import static junit.framework.Assert.assertTrue;
+
+/**
+ * Tests numeric Date/Time stream evaluators
+ */
+public class TemporalEvaluatorsTest {
+
+
+  StreamFactory factory;
+  Map<String, Object> values;
+
+  public TemporalEvaluatorsTest() {
+    super();
+
+    factory = new StreamFactory();
+
+    factory.withFunctionName("nope", DatePartEvaluator.class);
+    factory.withFunctionName(TemporalEvaluatorYear.FUNCTION_NAME,  TemporalEvaluatorYear.class);
+    factory.withFunctionName(TemporalEvaluatorMonth.FUNCTION_NAME, TemporalEvaluatorMonth.class);
+    factory.withFunctionName(TemporalEvaluatorDay.FUNCTION_NAME,   TemporalEvaluatorDay.class);
+    factory.withFunctionName(TemporalEvaluatorDayOfYear.FUNCTION_NAME,   TemporalEvaluatorDayOfYear.class);
+    factory.withFunctionName(TemporalEvaluatorHour.FUNCTION_NAME,   TemporalEvaluatorHour.class);
+    factory.withFunctionName(TemporalEvaluatorMinute.FUNCTION_NAME,   TemporalEvaluatorMinute.class);
+    factory.withFunctionName(TemporalEvaluatorSecond.FUNCTION_NAME,   TemporalEvaluatorSecond.class);
+    factory.withFunctionName(TemporalEvaluatorEpoch.FUNCTION_NAME,   TemporalEvaluatorEpoch.class);
+    factory.withFunctionName(TemporalEvaluatorWeek.FUNCTION_NAME,   TemporalEvaluatorWeek.class);
+    factory.withFunctionName(TemporalEvaluatorQuarter.FUNCTION_NAME,   TemporalEvaluatorQuarter.class);
+    factory.withFunctionName(TemporalEvaluatorDayOfQuarter.FUNCTION_NAME,   TemporalEvaluatorDayOfQuarter.class);
+
+    values = new HashedMap();
+  }
+
+  @Test
+  public void testInvalidExpression() throws Exception {
+
+    StreamEvaluator evaluator;
+
+    try {
+      evaluator = factory.constructEvaluator("nope(a)");
+      evaluator.evaluate(new Tuple(null));
+      assertTrue(false);
+    } catch (IOException e) {
+      assertTrue(e.getCause().getCause().getMessage().contains("Invalid date expression nope"));
+      assertTrue(e.getCause().getCause().getMessage().contains("expecting one of [year, month, day"));
+    }
+
+    try {
+      evaluator = factory.constructEvaluator("week()");
+      assertTrue(false);
+    } catch (IOException e) {
+      assertTrue(e.getCause().getCause().getMessage().contains("Invalid expression week()"));
+    }
+
+    try {
+      evaluator = factory.constructEvaluator("week(a, b)");
+      assertTrue(false);
+    } catch (IOException e) {
+      assertTrue(e.getCause().getCause().getMessage().contains("expecting one value but found 2"));
+    }
+
+    try {
+      evaluator = factory.constructEvaluator("Week()");
+      assertTrue(false);
+    } catch (IOException e) {
+      assertTrue(e.getMessage().contains("Invalid evaluator expression Week() - function 'Week' is unknown"));
+    }
+  }
+
+
+  @Test
+  public void testInvalidValues() throws Exception {
+    StreamEvaluator evaluator = factory.constructEvaluator("year(a)");
+
+    try {
+      values.clear();
+      values.put("a", 12);
+      Object result = evaluator.evaluate(new Tuple(values));
+      assertTrue(false);
+    } catch (IOException e) {
+      assertEquals("Invalid parameter 12 - The parameter must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", e.getMessage());
+    }
+
+    try {
+      values.clear();
+      values.put("a", "1995-12-31");
+      Object result = evaluator.evaluate(new Tuple(values));
+      assertTrue(false);
+    } catch (IOException e) {
+      assertEquals("Invalid parameter 1995-12-31 - The String must be formatted in the ISO_INSTANT date format.", e.getMessage());
+    }
+
+    try {
+      values.clear();
+      values.put("a", "");
+      Object result = evaluator.evaluate(new Tuple(values));
+      assertTrue(false);
+    } catch (IOException e) {
+      assertEquals("Invalid parameter  - The parameter must be a string formatted ISO_INSTANT or of type Instant,Date or LocalDateTime.", e.getMessage());
+    }
+
+    values.clear();
+    values.put("a", null);
+    assertNull(evaluator.evaluate(new Tuple(values)));
+  }
+
+  @Test
+  public void testAllFunctions() throws Exception {
+
+    //year, month, day, dayofyear, hour, minute, quarter, week, second, epoch
+    testFunction("year(a)", "1995-12-31T23:59:59Z", 1995);
+    testFunction("month(a)","1995-12-31T23:59:59Z", 12);
+    testFunction("day(a)",  "1995-12-31T23:59:59Z", 31);
+    testFunction("dayOfYear(a)",  "1995-12-31T23:59:59Z", 365);
+    testFunction("dayOfQuarter(a)",  "1995-12-31T23:59:59Z", 92);
+    testFunction("hour(a)",   "1995-12-31T23:59:59Z", 23);
+    testFunction("minute(a)", "1995-12-31T23:59:59Z", 59);
+    testFunction("quarter(a)","1995-12-31T23:59:59Z", 4);
+    testFunction("week(a)",   "1995-12-31T23:59:59Z", 52);
+    testFunction("second(a)", "1995-12-31T23:59:58Z", 58);
+    testFunction("epoch(a)",  "1995-12-31T23:59:59Z", 820454399000l);
+
+    testFunction("year(a)", "2017-03-17T10:30:45Z", 2017);
+    testFunction("year('a')", "2017-03-17T10:30:45Z", 2017);
+    testFunction("month(a)","2017-03-17T10:30:45Z", 3);
+    testFunction("day(a)",  "2017-03-17T10:30:45Z", 17);
+    testFunction("day('a')",  "2017-03-17T10:30:45Z", 17);
+    testFunction("dayOfYear(a)",  "2017-03-17T10:30:45Z", 76);
+    testFunction("dayOfQuarter(a)",  "2017-03-17T10:30:45Z", 76);
+    testFunction("hour(a)",   "2017-03-17T10:30:45Z", 10);
+    testFunction("minute(a)", "2017-03-17T10:30:45Z", 30);
+    testFunction("quarter(a)","2017-03-17T10:30:45Z", 1);
+    testFunction("week(a)",   "2017-03-17T10:30:45Z", 11);
+    testFunction("second(a)", "2017-03-17T10:30:45Z", 45);
+    testFunction("epoch(a)",  "2017-03-17T10:30:45Z", 1489746645000l);
+
+    testFunction("epoch(a)",  new Date(1489746645500l).toInstant().toString(), 1489746645500l);
+    testFunction("epoch(a)",  new Date(820454399990l).toInstant().toString(), 820454399990l);
+
+    //Additionally test all functions to make sure they return a non-null number
+    for (DatePartEvaluator.FUNCTION function : DatePartEvaluator.FUNCTION.values()) {
+      StreamEvaluator evaluator = factory.constructEvaluator(function+"(a)");
+      values.clear();
+      values.put("a", "2017-03-17T10:30:45Z");
+      Object result = evaluator.evaluate(new Tuple(values));
+      assertNotNull(function+" should return a result",result);
+      assertTrue(function+" should return a number", result instanceof Number);
+    }
+  }
+
+  @Test
+  public void testFunctionsOnDate() throws Exception {
+    Calendar calendar = new GregorianCalendar(2017,12,5, 23, 59);
+    calendar.setTimeZone(TimeZone.getTimeZone("UTC"));
+    Date aDate = calendar.getTime();
+    testFunction("year(a)", aDate, calendar.get(Calendar.YEAR));
+    testFunction("month(a)", aDate, calendar.get(Calendar.MONTH)+1);
+    testFunction("day(a)", aDate, calendar.get(Calendar.DAY_OF_MONTH));
+    testFunction("hour(a)", aDate, calendar.get(Calendar.HOUR_OF_DAY));
+    testFunction("minute(a)", aDate, calendar.get(Calendar.MINUTE));
+    testFunction("epoch(a)", aDate, aDate.getTime());
+  }
+
+  @Test
+  public void testFunctionsOnInstant() throws Exception {
+    Calendar calendar = new GregorianCalendar(2017,12,5, 23, 59);
+    calendar.setTimeZone(TimeZone.getTimeZone("UTC"));
+    Date aDate = calendar.getTime();
+    Instant instant = aDate.toInstant();
+    testFunction("year(a)", instant, calendar.get(Calendar.YEAR));
+    testFunction("month(a)", instant, calendar.get(Calendar.MONTH)+1);
+    testFunction("day(a)", instant, calendar.get(Calendar.DAY_OF_MONTH));
+    testFunction("hour(a)", instant, calendar.get(Calendar.HOUR_OF_DAY));
+    testFunction("minute(a)", instant, calendar.get(Calendar.MINUTE));
+    testFunction("epoch(a)", instant, aDate.getTime());
+  }
+
+  @Test
+  public void testFunctionsLocalDateTime() throws Exception {
+
+    LocalDateTime localDateTime = LocalDateTime.of(2017,12,5, 23, 59);
+    Date aDate = Date.from(localDateTime.atZone(ZoneOffset.UTC).toInstant());
+    testFunction("year(a)", localDateTime, 2017);
+    testFunction("month(a)", localDateTime, 12);
+    testFunction("day(a)", localDateTime, 5);
+    testFunction("hour(a)", localDateTime, 23);
+    testFunction("minute(a)", localDateTime, 59);
+    testFunction("epoch(a)", localDateTime, aDate.getTime());
+  }
+
+  @Test
+  public void testFunctionsOnLong() throws Exception {
+
+    Long longDate = 1512518340000l;
+
+    testFunction("year(a)", longDate, 2017);
+    testFunction("month(a)", longDate, 12);
+    testFunction("day(a)", longDate, 5);
+    testFunction("hour(a)", longDate, 23);
+    testFunction("minute(a)", longDate, 59);
+    testFunction("second(a)", longDate, 0);
+    testFunction("epoch(a)", longDate, longDate);
+
+  }
+
+  @Test
+  public void testLimitedFunctions() throws Exception {
+
+    MonthDay monthDay = MonthDay.of(12,5);
+    testFunction("month(a)", monthDay, 12);
+    testFunction("day(a)", monthDay, 5);
+
+    try {
+      testFunction("year(a)", monthDay, 2017);
+      assertTrue(false);
+    } catch (IOException e) {
+      assertEquals("It is not possible to call 'year' function on java.time.MonthDay", e.getMessage());
+    }
+
+    YearMonth yearMonth = YearMonth.of(2018, 4);
+    testFunction("month(a)", yearMonth, 4);
+    testFunction("year(a)", yearMonth, 2018);
+
+    try {
+      testFunction("day(a)", yearMonth, 5);
+      assertTrue(false);
+    } catch (IOException e) {
+      assertEquals("It is not possible to call 'day' function on java.time.YearMonth", e.getMessage());
+    }
+
+  }
+
+
+  public void testFunction(String expression, Object value, Number expected) throws Exception {
+    StreamEvaluator evaluator = factory.constructEvaluator(expression);
+    values.clear();
+    values.put("a", value);
+    Object result = evaluator.evaluate(new Tuple(values));
+    assertTrue(result instanceof Number);
+    assertEquals(expected, result);
+  }
+
+  @Test
+  public void testExplain() throws IOException {
+    StreamExpression express = StreamExpressionParser.parse("month('myfield')");
+    DatePartEvaluator datePartEvaluator = new DatePartEvaluator(express,factory);
+    Explanation explain = datePartEvaluator.toExplanation(factory);
+    assertEquals("month(myfield)", explain.getExpression());
+
+    express = StreamExpressionParser.parse("day(aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbb)");
+    datePartEvaluator = new DatePartEvaluator(express,factory);
+    explain = datePartEvaluator.toExplanation(factory);
+    assertEquals("day(aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbb)", explain.getExpression());
+  }
+}


[02/12] lucene-solr:master: SOLR-10303: Switched to pascal casing

Posted by jb...@apache.org.
SOLR-10303:  Switched to pascal casing


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

Branch: refs/heads/master
Commit: b314bc67764131bb677f25d98212577182af0b1e
Parents: c6fbb27
Author: Gethin James <ge...@alfresco.com>
Authored: Mon Mar 20 17:08:15 2017 +0100
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Apr 12 15:00:06 2017 -0400

----------------------------------------------------------------------
 .../apache/solr/client/solrj/io/eval/DatePartEvaluator.java  | 6 +++---
 .../client/solrj/io/stream/eval/DatePartEvaluatorTest.java   | 8 ++++----
 2 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b314bc67/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
index c3c5a61..592d4af 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java
@@ -41,7 +41,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
  */
 public class DatePartEvaluator extends NumberEvaluator {
 
-  public enum FUNCTION {year, month, day, dayofyear, dayofquarter, hour, minute, quarter, week, second, epoch}
+  public enum FUNCTION {year, month, day, dayOfYear, dayOfQuarter, hour, minute, quarter, week, second, epoch}
 
   private final FUNCTION function;
 
@@ -122,7 +122,7 @@ public class DatePartEvaluator extends NumberEvaluator {
           return date.get(ChronoField.MONTH_OF_YEAR);
         case day:
           return date.get(ChronoField.DAY_OF_MONTH);
-        case dayofyear:
+        case dayOfYear:
           return date.get(ChronoField.DAY_OF_YEAR);
         case hour:
           return date.get(ChronoField.HOUR_OF_DAY);
@@ -130,7 +130,7 @@ public class DatePartEvaluator extends NumberEvaluator {
           return date.get(ChronoField.MINUTE_OF_HOUR);
         case second:
           return date.get(ChronoField.SECOND_OF_MINUTE);
-        case dayofquarter:
+        case dayOfQuarter:
           return date.get(IsoFields.DAY_OF_QUARTER);
         case quarter:
           return date.get(IsoFields.QUARTER_OF_YEAR);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b314bc67/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
index 2e5c4b0..8b3317e 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DatePartEvaluatorTest.java
@@ -145,8 +145,8 @@ public class DatePartEvaluatorTest {
     testFunction("year(a)", "1995-12-31T23:59:59Z", 1995);
     testFunction("month(a)","1995-12-31T23:59:59Z", 12);
     testFunction("day(a)",  "1995-12-31T23:59:59Z", 31);
-    testFunction("dayofyear(a)",  "1995-12-31T23:59:59Z", 365);
-    testFunction("dayofquarter(a)",  "1995-12-31T23:59:59Z", 92);
+    testFunction("dayOfYear(a)",  "1995-12-31T23:59:59Z", 365);
+    testFunction("dayOfQuarter(a)",  "1995-12-31T23:59:59Z", 92);
     testFunction("hour(a)",   "1995-12-31T23:59:59Z", 23);
     testFunction("minute(a)", "1995-12-31T23:59:59Z", 59);
     testFunction("quarter(a)","1995-12-31T23:59:59Z", 4);
@@ -159,8 +159,8 @@ public class DatePartEvaluatorTest {
     testFunction("month(a)","2017-03-17T10:30:45Z", 3);
     testFunction("day(a)",  "2017-03-17T10:30:45Z", 17);
     testFunction("day('a')",  "2017-03-17T10:30:45Z", 17);
-    testFunction("dayofyear(a)",  "2017-03-17T10:30:45Z", 76);
-    testFunction("dayofquarter(a)",  "2017-03-17T10:30:45Z", 76);
+    testFunction("dayOfYear(a)",  "2017-03-17T10:30:45Z", 76);
+    testFunction("dayOfQuarter(a)",  "2017-03-17T10:30:45Z", 76);
     testFunction("hour(a)",   "2017-03-17T10:30:45Z", 10);
     testFunction("minute(a)", "2017-03-17T10:30:45Z", 30);
     testFunction("quarter(a)","2017-03-17T10:30:45Z", 1);