You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dp...@apache.org on 2017/04/02 03:22:39 UTC

[2/3] lucene-solr:master: SOLR-10356: Adds basic math streaming evaluators

SOLR-10356: Adds basic math streaming 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/674ce4e8
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/674ce4e8
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/674ce4e8

Branch: refs/heads/master
Commit: 674ce4e89393efe3147629e76f053c9901c182dc
Parents: b02626d
Author: Dennis Gove <dp...@gmail.com>
Authored: Thu Mar 23 20:08:11 2017 -0400
Committer: Dennis Gove <dp...@gmail.com>
Committed: Sat Apr 1 21:58:04 2017 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../org/apache/solr/handler/StreamHandler.java  |  79 ++++++++-
 .../solrj/io/eval/ArcCosineEvaluator.java       |  60 +++++++
 .../client/solrj/io/eval/ArcSineEvaluator.java  |  60 +++++++
 .../solrj/io/eval/ArcTangentEvaluator.java      |  60 +++++++
 .../client/solrj/io/eval/BooleanEvaluator.java  |   7 -
 .../client/solrj/io/eval/CeilingEvaluator.java  |  61 +++++++
 .../client/solrj/io/eval/CoalesceEvaluator.java |  52 ++++++
 .../client/solrj/io/eval/ComplexEvaluator.java  |   6 +
 .../solrj/io/eval/ConditionalEvaluator.java     |   6 -
 .../client/solrj/io/eval/CosineEvaluator.java   |  60 +++++++
 .../solrj/io/eval/CubedRootEvaluator.java       |  60 +++++++
 .../client/solrj/io/eval/FloorEvaluator.java    |  61 +++++++
 .../io/eval/HyperbolicCosineEvaluator.java      |  60 +++++++
 .../solrj/io/eval/HyperbolicSineEvaluator.java  |  60 +++++++
 .../io/eval/HyperbolicTangentEvaluator.java     |  60 +++++++
 .../client/solrj/io/eval/ModuloEvaluator.java   |  78 +++++++++
 .../client/solrj/io/eval/NumberEvaluator.java   |   6 -
 .../client/solrj/io/eval/PowerEvaluator.java    |  61 +++++++
 .../client/solrj/io/eval/RoundEvaluator.java    |  60 +++++++
 .../client/solrj/io/eval/SineEvaluator.java     |  60 +++++++
 .../solrj/io/eval/SquareRootEvaluator.java      |  60 +++++++
 .../client/solrj/io/eval/TangentEvaluator.java  |  60 +++++++
 .../io/stream/eval/ArcCosineEvaluatorTest.java  |  91 ++++++++++
 .../io/stream/eval/ArcSineEvaluatorTest.java    |  91 ++++++++++
 .../io/stream/eval/ArcTangentEvaluatorTest.java |  91 ++++++++++
 .../io/stream/eval/CeilingEvaluatorTest.java    |  96 +++++++++++
 .../io/stream/eval/CoalesceEvaluatorTest.java   | 112 +++++++++++++
 .../io/stream/eval/CosineEvaluatorTest.java     |  91 ++++++++++
 .../io/stream/eval/CubedRootEvaluatorTest.java  |  91 ++++++++++
 .../io/stream/eval/FloorEvaluatorTest.java      |  96 +++++++++++
 .../eval/HyperbolicCosineEvaluatorTest.java     |  91 ++++++++++
 .../eval/HyperbolicSineEvaluatorTest.java       |  91 ++++++++++
 .../eval/HyperbolicTangentEvaluatorTest.java    |  91 ++++++++++
 .../io/stream/eval/ModuloEvaluatorTest.java     | 164 +++++++++++++++++++
 .../io/stream/eval/PowerEvaluatorTest.java      | 119 ++++++++++++++
 .../io/stream/eval/RoundEvaluatorTest.java      |  95 +++++++++++
 .../solrj/io/stream/eval/SineEvaluatorTest.java |  91 ++++++++++
 .../io/stream/eval/SquareRootEvaluatorTest.java |  91 ++++++++++
 .../io/stream/eval/TangentEvaluatorTest.java    |  91 ++++++++++
 40 files changed, 2799 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 99edab4..1c3aaf7 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -61,6 +61,8 @@ New Features
 * SOLR-10292: Adds CartesianProductStream which turns a single tuple with a multi-valued field into N 
   tuples, one for each value in the multi-valued field. (Dennis Gove)
 
+* SOLR-10356: Adds basic math Streaming Evaluators (Dennis Gove)
+
 Bug Fixes
 ----------------------
 * SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/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 3ede732..b508754 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -16,6 +16,9 @@
  */
 package org.apache.solr.handler;
 
+import static org.apache.solr.common.params.CommonParams.ID;
+import static org.apache.solr.common.params.CommonParams.SORT;
+
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
@@ -33,27 +36,80 @@ import org.apache.solr.client.solrj.io.comp.StreamComparator;
 import org.apache.solr.client.solrj.io.eval.AbsoluteValueEvaluator;
 import org.apache.solr.client.solrj.io.eval.AddEvaluator;
 import org.apache.solr.client.solrj.io.eval.AndEvaluator;
+import org.apache.solr.client.solrj.io.eval.ArcCosineEvaluator;
+import org.apache.solr.client.solrj.io.eval.ArcSineEvaluator;
+import org.apache.solr.client.solrj.io.eval.ArcTangentEvaluator;
+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.DivideEvaluator;
 import org.apache.solr.client.solrj.io.eval.EqualsEvaluator;
 import org.apache.solr.client.solrj.io.eval.ExclusiveOrEvaluator;
+import org.apache.solr.client.solrj.io.eval.FloorEvaluator;
 import org.apache.solr.client.solrj.io.eval.GreaterThanEqualToEvaluator;
 import org.apache.solr.client.solrj.io.eval.GreaterThanEvaluator;
+import org.apache.solr.client.solrj.io.eval.HyperbolicCosineEvaluator;
+import org.apache.solr.client.solrj.io.eval.HyperbolicSineEvaluator;
+import org.apache.solr.client.solrj.io.eval.HyperbolicTangentEvaluator;
 import org.apache.solr.client.solrj.io.eval.IfThenElseEvaluator;
 import org.apache.solr.client.solrj.io.eval.LessThanEqualToEvaluator;
 import org.apache.solr.client.solrj.io.eval.LessThanEvaluator;
+import org.apache.solr.client.solrj.io.eval.ModuloEvaluator;
 import org.apache.solr.client.solrj.io.eval.MultiplyEvaluator;
 import org.apache.solr.client.solrj.io.eval.NaturalLogEvaluator;
 import org.apache.solr.client.solrj.io.eval.NotEvaluator;
 import org.apache.solr.client.solrj.io.eval.OrEvaluator;
+import org.apache.solr.client.solrj.io.eval.PowerEvaluator;
 import org.apache.solr.client.solrj.io.eval.RawValueEvaluator;
+import org.apache.solr.client.solrj.io.eval.RoundEvaluator;
+import org.apache.solr.client.solrj.io.eval.SineEvaluator;
+import org.apache.solr.client.solrj.io.eval.SquareRootEvaluator;
 import org.apache.solr.client.solrj.io.eval.SubtractEvaluator;
+import org.apache.solr.client.solrj.io.eval.TangentEvaluator;
 import org.apache.solr.client.solrj.io.graph.GatherNodesStream;
 import org.apache.solr.client.solrj.io.graph.ShortestPathStream;
 import org.apache.solr.client.solrj.io.ops.ConcatOperation;
 import org.apache.solr.client.solrj.io.ops.DistinctOperation;
 import org.apache.solr.client.solrj.io.ops.GroupOperation;
 import org.apache.solr.client.solrj.io.ops.ReplaceOperation;
-import org.apache.solr.client.solrj.io.stream.*;
+import org.apache.solr.client.solrj.io.stream.CartesianProductStream;
+import org.apache.solr.client.solrj.io.stream.CloudSolrStream;
+import org.apache.solr.client.solrj.io.stream.CommitStream;
+import org.apache.solr.client.solrj.io.stream.ComplementStream;
+import org.apache.solr.client.solrj.io.stream.DaemonStream;
+import org.apache.solr.client.solrj.io.stream.ExceptionStream;
+import org.apache.solr.client.solrj.io.stream.ExecutorStream;
+import org.apache.solr.client.solrj.io.stream.FacetStream;
+import org.apache.solr.client.solrj.io.stream.FeaturesSelectionStream;
+import org.apache.solr.client.solrj.io.stream.FetchStream;
+import org.apache.solr.client.solrj.io.stream.HashJoinStream;
+import org.apache.solr.client.solrj.io.stream.HavingStream;
+import org.apache.solr.client.solrj.io.stream.InnerJoinStream;
+import org.apache.solr.client.solrj.io.stream.IntersectStream;
+import org.apache.solr.client.solrj.io.stream.JDBCStream;
+import org.apache.solr.client.solrj.io.stream.LeftOuterJoinStream;
+import org.apache.solr.client.solrj.io.stream.MergeStream;
+import org.apache.solr.client.solrj.io.stream.ModelStream;
+import org.apache.solr.client.solrj.io.stream.NullStream;
+import org.apache.solr.client.solrj.io.stream.OuterHashJoinStream;
+import org.apache.solr.client.solrj.io.stream.ParallelStream;
+import org.apache.solr.client.solrj.io.stream.PriorityStream;
+import org.apache.solr.client.solrj.io.stream.RandomStream;
+import org.apache.solr.client.solrj.io.stream.RankStream;
+import org.apache.solr.client.solrj.io.stream.ReducerStream;
+import org.apache.solr.client.solrj.io.stream.RollupStream;
+import org.apache.solr.client.solrj.io.stream.ScoreNodesStream;
+import org.apache.solr.client.solrj.io.stream.SelectStream;
+import org.apache.solr.client.solrj.io.stream.SignificantTermsStream;
+import org.apache.solr.client.solrj.io.stream.SortStream;
+import org.apache.solr.client.solrj.io.stream.StatsStream;
+import org.apache.solr.client.solrj.io.stream.StreamContext;
+import org.apache.solr.client.solrj.io.stream.TextLogitStream;
+import org.apache.solr.client.solrj.io.stream.TopicStream;
+import org.apache.solr.client.solrj.io.stream.TupleStream;
+import org.apache.solr.client.solrj.io.stream.UniqueStream;
+import org.apache.solr.client.solrj.io.stream.UpdateStream;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
 import org.apache.solr.client.solrj.io.stream.expr.Expressible;
@@ -80,9 +136,6 @@ import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.common.params.CommonParams.ID;
-import static org.apache.solr.common.params.CommonParams.SORT;
-
 public class StreamHandler extends RequestHandlerBase implements SolrCoreAware, PermissionNameProvider {
 
   static SolrClientCache clientCache = new SolrClientCache();
@@ -207,6 +260,24 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("mult", MultiplyEvaluator.class)
       .withFunctionName("sub", SubtractEvaluator.class)
       .withFunctionName("log", NaturalLogEvaluator.class)
+      .withFunctionName("pow", PowerEvaluator.class)
+      .withFunctionName("mod", ModuloEvaluator.class)
+      .withFunctionName("ceil", CeilingEvaluator.class)
+      .withFunctionName("floor", FloorEvaluator.class)
+      .withFunctionName("sin", SineEvaluator.class)
+      .withFunctionName("asin", ArcSineEvaluator.class)
+      .withFunctionName("sinh", HyperbolicSineEvaluator.class)
+      .withFunctionName("cos", CosineEvaluator.class)
+      .withFunctionName("acos", ArcCosineEvaluator.class)
+      .withFunctionName("cosh", HyperbolicCosineEvaluator.class)
+      .withFunctionName("tan", TangentEvaluator.class)
+      .withFunctionName("atan", ArcTangentEvaluator.class)
+      .withFunctionName("tanh", HyperbolicTangentEvaluator.class)
+      .withFunctionName("round", RoundEvaluator.class)
+      .withFunctionName("sqrt", SquareRootEvaluator.class)
+      .withFunctionName("cbrt", CubedRootEvaluator.class)
+      .withFunctionName("coalesce", CoalesceEvaluator.class)
+      
       // Conditional Stream Evaluators
       .withFunctionName("if", IfThenElseEvaluator.class)
       .withFunctionName("analyze", AnalyzeEvaluator.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcCosineEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcCosineEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcCosineEvaluator.java
new file mode 100644
index 0000000..0c8e383
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcCosineEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.math.BigDecimal;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class ArcCosineEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public ArcCosineEvaluator(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 Number evaluate(Tuple tuple) throws IOException {
+    
+    List<BigDecimal> results = evaluateAll(tuple);
+    
+    // we're still doing these checks because if we ever add an array-flatten evaluator, 
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+    
+    if(null == results.get(0)){
+      return null;
+    }
+    
+    return Math.acos(results.get(0).doubleValue());
+  }  
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcSineEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcSineEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcSineEvaluator.java
new file mode 100644
index 0000000..ed95165
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcSineEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.math.BigDecimal;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class ArcSineEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public ArcSineEvaluator(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 Number evaluate(Tuple tuple) throws IOException {
+    
+    List<BigDecimal> results = evaluateAll(tuple);
+    
+    // we're still doing these checks because if we ever add an array-flatten evaluator, 
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+    
+    if(null == results.get(0)){
+      return null;
+    }
+    
+    return Math.asin(results.get(0).doubleValue());
+  }  
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcTangentEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcTangentEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcTangentEvaluator.java
new file mode 100644
index 0000000..9325b41
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcTangentEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.math.BigDecimal;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class ArcTangentEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public ArcTangentEvaluator(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 Number evaluate(Tuple tuple) throws IOException {
+    
+    List<BigDecimal> results = evaluateAll(tuple);
+    
+    // we're still doing these checks because if we ever add an array-flatten evaluator, 
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+    
+    if(null == results.get(0)){
+      return null;
+    }
+    
+    return Math.atan(results.get(0).doubleValue());
+  }  
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/BooleanEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/BooleanEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/BooleanEvaluator.java
index f02f1fa..908562f 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/BooleanEvaluator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/BooleanEvaluator.java
@@ -24,13 +24,11 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.solr.client.solrj.io.Tuple;
-import org.apache.solr.client.solrj.io.stream.StreamContext;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
 import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 
 public abstract class BooleanEvaluator extends ComplexEvaluator {
   protected static final long serialVersionUID = 1L;
-  protected StreamContext streamContext;
   
   public BooleanEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
     super(expression, factory);
@@ -48,11 +46,6 @@ public abstract class BooleanEvaluator extends ComplexEvaluator {
     return results;
   }
 
-  public void setStreamContext(StreamContext streamContext) {
-    this.streamContext = streamContext;
-  }
-
-
   public interface Checker {
     default boolean isNullAllowed(){
       return false;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CeilingEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CeilingEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CeilingEvaluator.java
new file mode 100644
index 0000000..e2ccc8f
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CeilingEvaluator.java
@@ -0,0 +1,61 @@
+/*
+ * 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.math.BigDecimal;
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class CeilingEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+
+  public CeilingEvaluator(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 Number evaluate(Tuple tuple) throws IOException {
+
+    List<BigDecimal> results = evaluateAll(tuple);
+
+    // we're still doing these checks because if we ever add an array-flatten evaluator,
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+
+    if(null == results.get(0)){
+      return null;
+    }
+
+    return normalizeType(results.get(0).setScale(0, RoundingMode.CEILING));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CoalesceEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CoalesceEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CoalesceEvaluator.java
new file mode 100644
index 0000000..8a6eda4
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CoalesceEvaluator.java
@@ -0,0 +1,52 @@
+/*
+ * 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.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class CoalesceEvaluator extends ComplexEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public CoalesceEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+    
+    if(subEvaluators.size() < 1){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at least one value but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Object evaluate(Tuple tuple) throws IOException {
+    
+    for(StreamEvaluator evaluator : subEvaluators){
+      Object result = evaluator.evaluate(tuple);
+      if(null != result){
+        return result;
+      }
+    }
+        
+    return null;    
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ComplexEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ComplexEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ComplexEvaluator.java
index 1e56d12..59a4653 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ComplexEvaluator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ComplexEvaluator.java
@@ -22,6 +22,7 @@ import java.util.List;
 import java.util.Locale;
 import java.util.UUID;
 
+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.Explanation.ExpressionType;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
@@ -31,6 +32,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 
 public abstract class ComplexEvaluator implements StreamEvaluator {
   protected static final long serialVersionUID = 1L;
+  protected StreamContext streamContext;
   
   protected UUID nodeId = UUID.randomUUID();
   
@@ -96,4 +98,8 @@ public abstract class ComplexEvaluator implements StreamEvaluator {
       .withImplementingClass(getClass().getName())
       .withExpression(toExpression(factory).toString());
   }
+  
+  public void setStreamContext(StreamContext context) {
+    this.streamContext = context;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConditionalEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConditionalEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConditionalEvaluator.java
index 6126544..025bfae 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConditionalEvaluator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConditionalEvaluator.java
@@ -24,13 +24,11 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.solr.client.solrj.io.Tuple;
-import org.apache.solr.client.solrj.io.stream.StreamContext;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
 import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 
 public abstract class ConditionalEvaluator extends ComplexEvaluator {
   protected static final long serialVersionUID = 1L;
-  protected StreamContext streamContext;
   
   public ConditionalEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
     super(expression, factory);
@@ -45,10 +43,6 @@ public abstract class ConditionalEvaluator extends ComplexEvaluator {
     return results;
   }
 
-  public void setStreamContext(StreamContext streamContext) {
-    this.streamContext = streamContext;
-  }
-  
   public interface Checker {
     default boolean isNullAllowed(){
       return false;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CosineEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CosineEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CosineEvaluator.java
new file mode 100644
index 0000000..6adbb81
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CosineEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.math.BigDecimal;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class CosineEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public CosineEvaluator(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 Number evaluate(Tuple tuple) throws IOException {
+    
+    List<BigDecimal> results = evaluateAll(tuple);
+    
+    // we're still doing these checks because if we ever add an array-flatten evaluator, 
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+    
+    if(null == results.get(0)){
+      return null;
+    }
+    
+    return Math.cos(results.get(0).doubleValue());
+  }  
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CubedRootEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CubedRootEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CubedRootEvaluator.java
new file mode 100644
index 0000000..4cd9277
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CubedRootEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.math.BigDecimal;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class CubedRootEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public CubedRootEvaluator(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 Number evaluate(Tuple tuple) throws IOException {
+    
+    List<BigDecimal> results = evaluateAll(tuple);
+    
+    // we're still doing these checks because if we ever add an array-flatten evaluator, 
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+    
+    if(null == results.get(0)){
+      return null;
+    }
+    
+    return Math.cbrt(results.get(0).doubleValue());
+  }  
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FloorEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FloorEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FloorEvaluator.java
new file mode 100644
index 0000000..0191a8e
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FloorEvaluator.java
@@ -0,0 +1,61 @@
+/*
+ * 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.math.BigDecimal;
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class FloorEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+
+  public FloorEvaluator(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 Number evaluate(Tuple tuple) throws IOException {
+
+    List<BigDecimal> results = evaluateAll(tuple);
+
+    // we're still doing these checks because if we ever add an array-flatten evaluator,
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+
+    if(null == results.get(0)){
+      return null;
+    }
+
+    return normalizeType(results.get(0).setScale(0, RoundingMode.FLOOR));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicCosineEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicCosineEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicCosineEvaluator.java
new file mode 100644
index 0000000..4e973a4
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicCosineEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.math.BigDecimal;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class HyperbolicCosineEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public HyperbolicCosineEvaluator(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 Number evaluate(Tuple tuple) throws IOException {
+    
+    List<BigDecimal> results = evaluateAll(tuple);
+    
+    // we're still doing these checks because if we ever add an array-flatten evaluator, 
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+    
+    if(null == results.get(0)){
+      return null;
+    }
+    
+    return Math.cosh(results.get(0).doubleValue());
+  }  
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicSineEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicSineEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicSineEvaluator.java
new file mode 100644
index 0000000..5bf4a38
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicSineEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.math.BigDecimal;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class HyperbolicSineEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public HyperbolicSineEvaluator(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 Number evaluate(Tuple tuple) throws IOException {
+    
+    List<BigDecimal> results = evaluateAll(tuple);
+    
+    // we're still doing these checks because if we ever add an array-flatten evaluator, 
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+    
+    if(null == results.get(0)){
+      return null;
+    }
+    
+    return Math.sinh(results.get(0).doubleValue());
+  }  
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicTangentEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicTangentEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicTangentEvaluator.java
new file mode 100644
index 0000000..89aacd1
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicTangentEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.math.BigDecimal;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class HyperbolicTangentEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public HyperbolicTangentEvaluator(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 Number evaluate(Tuple tuple) throws IOException {
+    
+    List<BigDecimal> results = evaluateAll(tuple);
+    
+    // we're still doing these checks because if we ever add an array-flatten evaluator, 
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+    
+    if(null == results.get(0)){
+      return null;
+    }
+    
+    return Math.tanh(results.get(0).doubleValue());
+  }  
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ModuloEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ModuloEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ModuloEvaluator.java
new file mode 100644
index 0000000..928754b
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ModuloEvaluator.java
@@ -0,0 +1,78 @@
+/*
+ * 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.math.BigDecimal;
+import java.math.MathContext;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class ModuloEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public ModuloEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+    
+    if(2 != subEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting two values but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Number evaluate(Tuple tuple) throws IOException {
+    
+    List<BigDecimal> results = evaluateAll(tuple);
+    
+    // we're still doing these checks because if we ever add an array-flatten evaluator, 
+    // two found in the constructor could become != 2
+    if(2 != results.size()){
+      String message = null;
+      if(1 == results.size()){
+        message = String.format(Locale.ROOT,"%s(...) only works with a 2 values (numerator,denominator) but 1 was provided", constructingFactory.getFunctionName(getClass())); 
+      }
+      else{
+        message = String.format(Locale.ROOT,"%s(...) only works with a 2 values (numerator,denominator) but %d were provided", constructingFactory.getFunctionName(getClass()), results.size());
+      }
+      throw new IOException(message);
+    }
+    
+    BigDecimal numerator = results.get(0);
+    BigDecimal denominator = results.get(1);
+    
+    if(null == numerator){
+      throw new IOException(String.format(Locale.ROOT,"Unable to %s(...) with a null numerator", constructingFactory.getFunctionName(getClass())));
+    }
+    
+    if(null == denominator){
+      throw new IOException(String.format(Locale.ROOT,"Unable to %s(...) with a null denominator", constructingFactory.getFunctionName(getClass())));
+    }
+    
+    if(0 == denominator.compareTo(BigDecimal.ZERO)){
+      throw new IOException(String.format(Locale.ROOT,"Unable to %s(...) with a 0 denominator", constructingFactory.getFunctionName(getClass())));
+    }
+    
+    return normalizeType(numerator.remainder(denominator, MathContext.DECIMAL64));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NumberEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NumberEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NumberEvaluator.java
index 283c7b1..f4491fd 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NumberEvaluator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NumberEvaluator.java
@@ -26,13 +26,11 @@ import java.util.List;
 import java.util.Locale;
 
 import org.apache.solr.client.solrj.io.Tuple;
-import org.apache.solr.client.solrj.io.stream.StreamContext;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
 import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 
 public abstract class NumberEvaluator extends ComplexEvaluator {
   protected static final long serialVersionUID = 1L;
-  protected StreamContext streamContext;
   
   public NumberEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
     super(expression, factory);
@@ -40,10 +38,6 @@ public abstract class NumberEvaluator extends ComplexEvaluator {
   
   // restrict result to a Number
   public abstract Number evaluate(Tuple tuple) throws IOException;
-
-  public void setStreamContext(StreamContext context) {
-    this.streamContext = context;
-  }
   
   public List<BigDecimal> evaluateAll(final Tuple tuple) throws IOException {
     // evaluate each and confirm they are all either null or numeric

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PowerEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PowerEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PowerEvaluator.java
new file mode 100644
index 0000000..a8245b6
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PowerEvaluator.java
@@ -0,0 +1,61 @@
+/*
+ * 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.math.BigDecimal;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class PowerEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public PowerEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+    
+    if(2 != subEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting exactly two values but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Number evaluate(Tuple tuple) throws IOException {
+    
+    List<BigDecimal> results = evaluateAll(tuple);
+    
+    if(results.stream().anyMatch(item -> null == item)){
+      return null;
+    }
+    
+    BigDecimal value = results.get(0);
+    BigDecimal exponent = results.get(1);
+    
+    double result = Math.pow(value.doubleValue(), exponent.doubleValue());
+    if(Double.isNaN(result)){
+      return result;
+    }
+    
+    return normalizeType(BigDecimal.valueOf(result));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RoundEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RoundEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RoundEvaluator.java
new file mode 100644
index 0000000..a34cdf4
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RoundEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.math.BigDecimal;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class RoundEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public RoundEvaluator(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 Number evaluate(Tuple tuple) throws IOException {
+    
+    List<BigDecimal> results = evaluateAll(tuple);
+    
+    // we're still doing these checks because if we ever add an array-flatten evaluator, 
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+    
+    if(null == results.get(0)){
+      return null;
+    }
+    
+    return Math.round(results.get(0).doubleValue());
+  }  
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SineEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SineEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SineEvaluator.java
new file mode 100644
index 0000000..1e2fbb5
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SineEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.math.BigDecimal;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class SineEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public SineEvaluator(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 Number evaluate(Tuple tuple) throws IOException {
+    
+    List<BigDecimal> results = evaluateAll(tuple);
+    
+    // we're still doing these checks because if we ever add an array-flatten evaluator, 
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+    
+    if(null == results.get(0)){
+      return null;
+    }
+    
+    return Math.sin(results.get(0).doubleValue());
+  }  
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SquareRootEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SquareRootEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SquareRootEvaluator.java
new file mode 100644
index 0000000..74b9d81
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SquareRootEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.math.BigDecimal;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class SquareRootEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public SquareRootEvaluator(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 Number evaluate(Tuple tuple) throws IOException {
+    
+    List<BigDecimal> results = evaluateAll(tuple);
+    
+    // we're still doing these checks because if we ever add an array-flatten evaluator, 
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+    
+    if(null == results.get(0)){
+      return null;
+    }
+    
+    return Math.sqrt(results.get(0).doubleValue());
+  }  
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TangentEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TangentEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TangentEvaluator.java
new file mode 100644
index 0000000..d2a0476
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TangentEvaluator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.math.BigDecimal;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class TangentEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public TangentEvaluator(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 Number evaluate(Tuple tuple) throws IOException {
+    
+    List<BigDecimal> results = evaluateAll(tuple);
+    
+    // we're still doing these checks because if we ever add an array-flatten evaluator, 
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+    
+    if(null == results.get(0)){
+      return null;
+    }
+    
+    return Math.tan(results.get(0).doubleValue());
+  }  
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcCosineEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcCosineEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcCosineEvaluatorTest.java
new file mode 100644
index 0000000..6a99a1c
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcCosineEvaluatorTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.eval.ArcCosineEvaluator;
+import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+public class ArcCosineEvaluatorTest extends LuceneTestCase {
+
+  StreamFactory factory;
+  Map<String, Object> values;
+  
+  public ArcCosineEvaluatorTest() {
+    super();
+    
+    factory = new StreamFactory()
+      .withFunctionName("acos", ArcCosineEvaluator.class);
+    values = new HashMap<String,Object>();
+  }
+  
+  private void test(Double value) throws IOException{
+    StreamEvaluator evaluator = factory.constructEvaluator("acos(a)");
+    
+    values.clear();
+    values.put("a", value);
+    Object result = evaluator.evaluate(new Tuple(values));
+    
+    if(null == value){
+      Assert.assertNull(result);
+    }
+    else{
+      Assert.assertTrue(result instanceof Double);
+      Assert.assertEquals(Math.acos(value), result);
+    }
+  }
+    
+  @Test
+  public void oneField() throws Exception{
+    test(90D);
+    test(45D);
+    test(12.4D);
+    test(-45D);
+  }
+
+  @Test(expected = IOException.class)
+  public void noField() throws Exception{
+    factory.constructEvaluator("acos()");
+  }
+  
+  @Test(expected = IOException.class)
+  public void twoFields() throws Exception{
+    factory.constructEvaluator("acos(a,b)");
+  }
+  
+  @Test
+  public void noValue() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("acos(a)");
+    
+    values.clear();
+    Object result = evaluator.evaluate(new Tuple(values));
+    assertNull(result);
+  }
+  @Test
+  public void nullValue() throws Exception{
+    test(null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcSineEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcSineEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcSineEvaluatorTest.java
new file mode 100644
index 0000000..79e934b
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcSineEvaluatorTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.eval.ArcSineEvaluator;
+import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+public class ArcSineEvaluatorTest extends LuceneTestCase {
+
+  StreamFactory factory;
+  Map<String, Object> values;
+  
+  public ArcSineEvaluatorTest() {
+    super();
+    
+    factory = new StreamFactory()
+      .withFunctionName("asin", ArcSineEvaluator.class);
+    values = new HashMap<String,Object>();
+  }
+  
+  private void test(Double value) throws IOException{
+    StreamEvaluator evaluator = factory.constructEvaluator("asin(a)");
+    
+    values.clear();
+    values.put("a", value);
+    Object result = evaluator.evaluate(new Tuple(values));
+    
+    if(null == value){
+      Assert.assertNull(result);
+    }
+    else{
+      Assert.assertTrue(result instanceof Double);
+      Assert.assertEquals(Math.asin(value), result);
+    }
+  }
+    
+  @Test
+  public void oneField() throws Exception{
+    test(90D);
+    test(45D);
+    test(12.4D);
+    test(-45D);
+  }
+
+  @Test(expected = IOException.class)
+  public void noField() throws Exception{
+    factory.constructEvaluator("asin()");
+  }
+  
+  @Test(expected = IOException.class)
+  public void twoFields() throws Exception{
+    factory.constructEvaluator("asin(a,b)");
+  }
+  
+  @Test
+  public void noValue() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("asin(a)");
+    
+    values.clear();
+    Object result = evaluator.evaluate(new Tuple(values));
+    assertNull(result);
+  }
+  @Test
+  public void nullValue() throws Exception{
+    test(null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcTangentEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcTangentEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcTangentEvaluatorTest.java
new file mode 100644
index 0000000..7af225b
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcTangentEvaluatorTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.eval.ArcTangentEvaluator;
+import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+public class ArcTangentEvaluatorTest extends LuceneTestCase {
+
+  StreamFactory factory;
+  Map<String, Object> values;
+  
+  public ArcTangentEvaluatorTest() {
+    super();
+    
+    factory = new StreamFactory()
+      .withFunctionName("atan", ArcTangentEvaluator.class);
+    values = new HashMap<String,Object>();
+  }
+  
+  private void test(Double value) throws IOException{
+    StreamEvaluator evaluator = factory.constructEvaluator("atan(a)");
+    
+    values.clear();
+    values.put("a", value);
+    Object result = evaluator.evaluate(new Tuple(values));
+    
+    if(null == value){
+      Assert.assertNull(result);
+    }
+    else{
+      Assert.assertTrue(result instanceof Double);
+      Assert.assertEquals(Math.atan(value), result);
+    }
+  }
+    
+  @Test
+  public void oneField() throws Exception{
+    test(90D);
+    test(45D);
+    test(12.4D);
+    test(-45D);
+  }
+
+  @Test(expected = IOException.class)
+  public void noField() throws Exception{
+    factory.constructEvaluator("atan()");
+  }
+  
+  @Test(expected = IOException.class)
+  public void twoFields() throws Exception{
+    factory.constructEvaluator("atan(a,b)");
+  }
+  
+  @Test
+  public void noValue() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("atan(a)");
+    
+    values.clear();
+    Object result = evaluator.evaluate(new Tuple(values));
+    assertNull(result);
+  }
+  @Test
+  public void nullValue() throws Exception{
+    test(null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CeilingEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CeilingEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CeilingEvaluatorTest.java
new file mode 100644
index 0000000..03395d2
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CeilingEvaluatorTest.java
@@ -0,0 +1,96 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.eval.CeilingEvaluator;
+import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+public class CeilingEvaluatorTest extends LuceneTestCase {
+
+  StreamFactory factory;
+  Map<String, Object> values;
+  
+  public CeilingEvaluatorTest() {
+    super();
+    
+    factory = new StreamFactory()
+      .withFunctionName("ceil", CeilingEvaluator.class);
+    values = new HashMap<String,Object>();
+  }
+    
+  @Test
+  public void ceilingOneField() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("ceil(a)");
+    Object result;
+    
+    values.clear();
+    values.put("a", 1);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Long);
+    Assert.assertEquals(1L, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Long);
+    Assert.assertEquals(2L, result);
+    
+    values.clear();
+    values.put("a", -1.1);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Long);
+    Assert.assertEquals(-1L, result);
+  }
+
+  @Test(expected = IOException.class)
+  public void ceilNoField() throws Exception{
+    factory.constructEvaluator("ceil()");
+  }
+  
+  @Test(expected = IOException.class)
+  public void ceilTwoFields() throws Exception{
+    factory.constructEvaluator("ceil(a,b)");
+  }
+  
+  @Test
+  public void ceilNoValue() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("ceil(a)");
+    
+    values.clear();
+    Object result = evaluator.evaluate(new Tuple(values));
+    assertNull(result);
+  }
+  @Test
+  public void ceilNullValue() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("ceil(a)");
+    
+    values.clear();
+    values.put("a", null);
+    Object result = evaluator.evaluate(new Tuple(values));
+    assertNull(result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CoalesceEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CoalesceEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CoalesceEvaluatorTest.java
new file mode 100644
index 0000000..79f46e7
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CoalesceEvaluatorTest.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for multitional 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.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.eval.CoalesceEvaluator;
+import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+public class CoalesceEvaluatorTest extends LuceneTestCase {
+
+  StreamFactory factory;
+  Map<String, Object> values;
+  
+  public CoalesceEvaluatorTest() {
+    super();
+    
+    factory = new StreamFactory()
+      .withFunctionName("coalesce", CoalesceEvaluator.class);
+    values = new HashMap<String,Object>();
+  }
+    
+  @Test
+  public void twoFieldsWithValues() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("coalesce(a,b)");
+    Object result;
+    
+    values.clear();
+    values.put("a", null);
+    values.put("b", 2);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertEquals(2, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", null);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertEquals(1.1D, result);
+    
+    values.clear();
+    values.put("a", "foo");
+    values.put("b", 2.1);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertEquals("foo", result);
+
+    values.clear();
+    values.put("a", true);
+    values.put("b", 2.1);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertEquals(true, result);
+    
+
+    values.clear();
+    values.put("a", null);
+    values.put("b", false);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertEquals(false, result);
+
+    values.clear();
+    values.put("a", null);
+    values.put("b", null);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertNull(result);
+  }
+  
+
+  @Test
+  public void twoFieldsWithMissingField() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("coalesce(a,b)");
+    Object result;
+    
+    values.clear();
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertNull(result);
+    
+  }
+  
+  @Test
+  public void manyFieldsWithSubcoalesces() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("coalesce(a,b,coalesce(c,d))");
+    Object result;
+    
+    values.clear();
+    values.put("a", 1);
+    values.put("b", null);
+    values.put("c", null);
+    values.put("d", 4);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertEquals(1, result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/674ce4e8/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CosineEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CosineEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CosineEvaluatorTest.java
new file mode 100644
index 0000000..6bb6913
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CosineEvaluatorTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.eval.CosineEvaluator;
+import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+public class CosineEvaluatorTest extends LuceneTestCase {
+
+  StreamFactory factory;
+  Map<String, Object> values;
+  
+  public CosineEvaluatorTest() {
+    super();
+    
+    factory = new StreamFactory()
+      .withFunctionName("cos", CosineEvaluator.class);
+    values = new HashMap<String,Object>();
+  }
+  
+  private void test(Double value) throws IOException{
+    StreamEvaluator evaluator = factory.constructEvaluator("cos(a)");
+    
+    values.clear();
+    values.put("a", value);
+    Object result = evaluator.evaluate(new Tuple(values));
+    
+    if(null == value){
+      Assert.assertNull(result);
+    }
+    else{
+      Assert.assertTrue(result instanceof Double);
+      Assert.assertEquals(Math.cos(value), result);
+    }
+  }
+    
+  @Test
+  public void oneField() throws Exception{
+    test(90D);
+    test(45D);
+    test(12.4D);
+    test(-45D);
+  }
+
+  @Test(expected = IOException.class)
+  public void noField() throws Exception{
+    factory.constructEvaluator("cos()");
+  }
+  
+  @Test(expected = IOException.class)
+  public void twoFields() throws Exception{
+    factory.constructEvaluator("cos(a,b)");
+  }
+  
+  @Test
+  public void noValue() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("cos(a)");
+    
+    values.clear();
+    Object result = evaluator.evaluate(new Tuple(values));
+    assertNull(result);
+  }
+  @Test
+  public void nullValue() throws Exception{
+    test(null);
+  }
+}