You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by covolution <gi...@git.apache.org> on 2017/03/20 16:36:38 UTC

[GitHub] lucene-solr pull request #171: SOLR-10303

GitHub user covolution opened a pull request:

    https://github.com/apache/lucene-solr/pull/171

    SOLR-10303

    Adding date/time Stream Evaluators for year, month, day, dayOfYear, dayOfQuarter, hour, minute, quarter, week, second, epoch.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/covolution/lucene-solr SOLR-10303

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/lucene-solr/pull/171.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #171
    
----

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #171: SOLR-10303

Posted by dennisgove <gi...@git.apache.org>.
Github user dennisgove commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/171#discussion_r107802207
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/StreamHandler.java ---
    @@ -199,10 +200,16 @@ public void inform(SolrCore core) {
           .withFunctionName("mult", MultiplyEvaluator.class)
           .withFunctionName("sub", SubtractEvaluator.class)
           .withFunctionName("log", NaturalLogEvaluator.class)
    +
           // Conditional Stream Evaluators
           .withFunctionName("if", IfThenElseEvaluator.class)
           ;
     
    +      // Date evaluators
    --- End diff --
    
    I'm not a huge fan of using the same class to handle multiple functions. There are places where we use the class to find the function name and if > 1 functions are mapped to a class then these lookups no longer work.
    
    See [this](https://github.com/dennisgove/lucene-solr/blob/master/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java#L397) for where it wouldn't work.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #171: SOLR-10303

Posted by covolution <gi...@git.apache.org>.
Github user covolution commented on the issue:

    https://github.com/apache/lucene-solr/pull/171
  
    Yes, I had some minor commits (including the delete) I hadn't pushed.  It up to date now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #171: SOLR-10303

Posted by covolution <gi...@git.apache.org>.
Github user covolution commented on the issue:

    https://github.com/apache/lucene-solr/pull/171
  
    Changes merged into master


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #171: SOLR-10303

Posted by covolution <gi...@git.apache.org>.
Github user covolution commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/171#discussion_r110107409
  
    --- Diff: solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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 {
    --- End diff --
    
    I will start working on the new design.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #171: SOLR-10303

Posted by dennisgove <gi...@git.apache.org>.
Github user dennisgove commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/171#discussion_r110244491
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/StreamHandler.java ---
    @@ -191,7 +237,20 @@ public void inform(SolrCore core) {
           .withFunctionName("lteq", LessThanEqualToEvaluator.class)
           .withFunctionName("not", NotEvaluator.class)
           .withFunctionName("or", OrEvaluator.class)
    -      
    +
    +      // Date Time Evaluators
    +      .withFunctionName(TemporalEvaluatorYear.FUNCTION_NAME, TemporalEvaluatorYear.class)
    --- End diff --
    
    I'm having trouble making up my mind about this use of the FUNCTION_NAME static property. For a while I've been considering moving the function name into the implementing class (like you've done here) but I haven't been able to convince myself that it is inherently better. I'd like to discuss it though, so I'll list my reasons both for and against.
    
    1. Function names were originally assigned in a single place, the StreamHandler, to allow for easy overrides via the config file (eg. using business specific logic for an innerJoin you could override the assignment of `innerJoin` to your own class via the config file). Having the assignments in a single place made this easier.
    
    2. Having assignments in a single place makes it easy to see what the full list of available function names is. Among other things, this has helped prevent me from accidentally using an already used function name for a different class.
    
    3. Function names are now being assigned in at least 3 places ([StreamHandler](https://github.com/dennisgove/lucene-solr/blob/master/solr/core/src/java/org/apache/solr/handler/StreamHandler.java), [GraphHandler](https://github.com/dennisgove/lucene-solr/blob/master/solr/core/src/java/org/apache/solr/handler/GraphHandler.java), and [SolrTable](https://github.com/dennisgove/lucene-solr/blob/master/solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java)) so all the benefits of a single assignment is somewhat out the window.
    
    4. By assigning the function names in the implementing class, we no longer need to rely on the StreamFactory for knowing what the assigned function name for a class is. This means we can use the function name in more logging and error messages where a StreamFactory instance may not be readily available. 
    
    5. It will still be possible for people to override the default assigned function names and classes, but the logic that currently does that will have to be carefully changed to ensure full backward compatibility.
    
    I'm in favor of making this change, but it's larger than something that should probably be done in this PR. 
    
    @joel-bernstein, @covolution - your thoughts?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #171: SOLR-10303

Posted by dennisgove <gi...@git.apache.org>.
Github user dennisgove commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/171#discussion_r107805373
  
    --- Diff: solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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 {
    --- End diff --
    
    If you were to break this class into multiple (one for each evaluator) then this function should def exist in a parent class so keep the logic in a singular place.
    
    The way I could see the hierarchy working is something like this
    ```
    ComplexEvaluator
      TemporalEvaluator
         YearEvaluator
         MonthOfYearEvaluator
         DayOfMonthEvalator
         DayNameEvaluator
         ....
    ```
    
    Basically, if some evaluator extends NumberEvaluator then that means this evaluator works over numbers, not that it returns a number. In a sense, when the sub-evaluators are executed they need to return something that is a Number. Following that logic, these evaluators work over temporal objects, so they extend TemporalEvaluator.
    
    They can return whatever type they need to (String, Number, etc...) but they must have a temporal value to work from.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #171: SOLR-10303

Posted by dennisgove <gi...@git.apache.org>.
Github user dennisgove commented on the issue:

    https://github.com/apache/lucene-solr/pull/171
  
    Sounds good to me. I like the work that was done here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #171: SOLR-10303

Posted by dennisgove <gi...@git.apache.org>.
Github user dennisgove commented on the issue:

    https://github.com/apache/lucene-solr/pull/171
  
    I think this all looks good.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #171: SOLR-10303

Posted by covolution <gi...@git.apache.org>.
Github user covolution commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/171#discussion_r110346756
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/StreamHandler.java ---
    @@ -191,7 +237,20 @@ public void inform(SolrCore core) {
           .withFunctionName("lteq", LessThanEqualToEvaluator.class)
           .withFunctionName("not", NotEvaluator.class)
           .withFunctionName("or", OrEvaluator.class)
    -      
    +
    +      // Date Time Evaluators
    +      .withFunctionName(TemporalEvaluatorYear.FUNCTION_NAME, TemporalEvaluatorYear.class)
    --- End diff --
    
    I agree with what you are saying. Its a tricky question.
    
    The function definitions in StreamHandler are all static, there are no class instances.  This makes sense if a factory is creating them.  If we went for an instance method on an implementation class then that wouldn't really work.  (It would allow us to have multiple implementations in the same class though)!
    
    So instances vs static is a bit of a problem. I like that you can also use aliases at the moment so wouldn't want to lose that. Are function classes being created on-demand, per request?
    
    If we keep with the static implementation, perhaps we can use an interface called FunctionNames to put all the string constants?  It would be one central definition list for StreamHandler, GraphHandler and SolrTable.  Function implementation classes could still reference the FunctionNames in their implementation.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #171: SOLR-10303

Posted by dennisgove <gi...@git.apache.org>.
Github user dennisgove commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/171#discussion_r110127292
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/StreamHandler.java ---
    @@ -199,10 +200,16 @@ public void inform(SolrCore core) {
           .withFunctionName("mult", MultiplyEvaluator.class)
           .withFunctionName("sub", SubtractEvaluator.class)
           .withFunctionName("log", NaturalLogEvaluator.class)
    +
           // Conditional Stream Evaluators
           .withFunctionName("if", IfThenElseEvaluator.class)
           ;
     
    +      // Date evaluators
    --- End diff --
    
    Yeah, I agree - maybe we should take a look at that assumption of 1 class == 1 function and re-evaluate if there's a better way.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #171: SOLR-10303

Posted by dennisgove <gi...@git.apache.org>.
Github user dennisgove commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/171#discussion_r107804517
  
    --- Diff: solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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();
    --- End diff --
    
    Even in a case like this, why not pass the instant down to the `private evaluate(Instant)` function. That way all the real decisions are made there.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #171: SOLR-10303

Posted by covolution <gi...@git.apache.org>.
Github user covolution commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/171#discussion_r110107074
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/StreamHandler.java ---
    @@ -199,10 +200,16 @@ public void inform(SolrCore core) {
           .withFunctionName("mult", MultiplyEvaluator.class)
           .withFunctionName("sub", SubtractEvaluator.class)
           .withFunctionName("log", NaturalLogEvaluator.class)
    +
           // Conditional Stream Evaluators
           .withFunctionName("if", IfThenElseEvaluator.class)
           ;
     
    +      // Date evaluators
    --- End diff --
    
    I figured using one class (< 170 lines) to implement 11 date functions was preferable to lots of little classes.  Now I know there's an assumption that 1 class = 1 function, I can re-factor to add all the extra classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #171: SOLR-10303

Posted by covolution <gi...@git.apache.org>.
Github user covolution commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/171#discussion_r110107246
  
    --- Diff: solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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();
    --- End diff --
    
    This is an optimization to avoid creating a Date object unnecessarily.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #171: SOLR-10303

Posted by covolution <gi...@git.apache.org>.
Github user covolution closed the pull request at:

    https://github.com/apache/lucene-solr/pull/171


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #171: SOLR-10303

Posted by joel-bernstein <gi...@git.apache.org>.
Github user joel-bernstein commented on the issue:

    https://github.com/apache/lucene-solr/pull/171
  
    Ok great. Just running the test suite now...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #171: SOLR-10303

Posted by covolution <gi...@git.apache.org>.
Github user covolution commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/171#discussion_r110107181
  
    --- Diff: solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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) {
    --- End diff --
    
    Instant does not work with human units of time(eg. year, month, or day), a timezone is required. So I am converting it to a LocalDateTime using ZoneOffset.UTC.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #171: SOLR-10303

Posted by dennisgove <gi...@git.apache.org>.
Github user dennisgove commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/171#discussion_r111227848
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/StreamHandler.java ---
    @@ -191,7 +237,20 @@ public void inform(SolrCore core) {
           .withFunctionName("lteq", LessThanEqualToEvaluator.class)
           .withFunctionName("not", NotEvaluator.class)
           .withFunctionName("or", OrEvaluator.class)
    -      
    +
    +      // Date Time Evaluators
    +      .withFunctionName(TemporalEvaluatorYear.FUNCTION_NAME, TemporalEvaluatorYear.class)
    --- End diff --
    
    I'd like to approach this in a separate ticket. Would it be alright if we change the use of TemporalEvaluatorYear.FUNCTION_NAME to "year" in the call to `withFunctionName` for now?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #171: SOLR-10303

Posted by dennisgove <gi...@git.apache.org>.
Github user dennisgove commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/171#discussion_r107804393
  
    --- Diff: solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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) {
    --- End diff --
    
    I'd rather see this continue to act on an Instant instead of a Date. If the value is a TemporalAccessor (above if statement) you can convert that into an Instant.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #171: SOLR-10303

Posted by dennisgove <gi...@git.apache.org>.
Github user dennisgove commented on the issue:

    https://github.com/apache/lucene-solr/pull/171
  
    I noticed that solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DatePartEvaluator.java is still part of the PR. Is that intentional?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #171: SOLR-10303

Posted by joel-bernstein <gi...@git.apache.org>.
Github user joel-bernstein commented on the issue:

    https://github.com/apache/lucene-solr/pull/171
  
    Hi @dennisgove,
    
    I've been working with this ticket today. I'm commit it to master so you and I can make changes more easily to it. I also introduced the concept of a TupleContext in this ticket which may need some re-working. When we feel it's ready we can backport.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org