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/02/10 05:37:33 UTC

[3/3] lucene-solr:branch_6x: SOLR-9916: Adds Stream Evaluators to support evaluating values from tuples

SOLR-9916: Adds Stream Evaluators to support evaluating values from tuples


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

Branch: refs/heads/branch_6x
Commit: 7372df9957b75c08283af6db47234df1787f1490
Parents: f919485
Author: Dennis Gove <dp...@gmail.com>
Authored: Fri Feb 10 00:37:10 2017 -0500
Committer: Dennis Gove <dp...@gmail.com>
Committed: Fri Feb 10 00:37:10 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   6 +-
 .../org/apache/solr/handler/StreamHandler.java  |  94 +++++-
 .../solrj/io/eval/AbsoluteValueEvaluator.java   |  60 ++++
 .../solr/client/solrj/io/eval/AddEvaluator.java |  61 ++++
 .../solr/client/solrj/io/eval/AndEvaluator.java |  90 +++++
 .../client/solrj/io/eval/BooleanEvaluator.java  |  86 +++++
 .../client/solrj/io/eval/ComplexEvaluator.java  |  99 ++++++
 .../solrj/io/eval/ConditionalEvaluator.java     |  59 ++++
 .../client/solrj/io/eval/DivideEvaluator.java   |  78 +++++
 .../client/solrj/io/eval/EqualsEvaluator.java   | 112 +++++++
 .../solrj/io/eval/ExclusiveOrEvaluator.java     |  67 ++++
 .../client/solrj/io/eval/FieldEvaluator.java    |  62 ++++
 .../io/eval/GreaterThanEqualToEvaluator.java    |  99 ++++++
 .../solrj/io/eval/GreaterThanEvaluator.java     |  99 ++++++
 .../solrj/io/eval/IfThenElseEvaluator.java      |  62 ++++
 .../solrj/io/eval/LessThanEqualToEvaluator.java |  99 ++++++
 .../client/solrj/io/eval/LessThanEvaluator.java |  99 ++++++
 .../client/solrj/io/eval/MultiplyEvaluator.java |  62 ++++
 .../solr/client/solrj/io/eval/NotEvaluator.java |  62 ++++
 .../client/solrj/io/eval/NumberEvaluator.java   |  79 +++++
 .../solr/client/solrj/io/eval/OrEvaluator.java  |  90 +++++
 .../client/solrj/io/eval/RawValueEvaluator.java |  90 +++++
 .../client/solrj/io/eval/SimpleEvaluator.java   |  29 ++
 .../client/solrj/io/eval/StreamEvaluator.java   |  30 ++
 .../client/solrj/io/eval/SubtractEvaluator.java |  61 ++++
 .../solr/client/solrj/io/ops/AndOperation.java  | 101 ------
 .../client/solrj/io/ops/BooleanOperation.java   |  26 --
 .../client/solrj/io/ops/EqualsOperation.java    |  70 ----
 .../io/ops/GreaterThanEqualToOperation.java     |  70 ----
 .../solrj/io/ops/GreaterThanOperation.java      |  70 ----
 .../solr/client/solrj/io/ops/LeafOperation.java |  67 ----
 .../solrj/io/ops/LessThanEqualToOperation.java  |  70 ----
 .../client/solrj/io/ops/LessThanOperation.java  |  70 ----
 .../solr/client/solrj/io/ops/NotOperation.java  |  87 -----
 .../solr/client/solrj/io/ops/OrOperation.java   |  71 ----
 .../client/solrj/io/stream/HavingStream.java    |  46 ++-
 .../client/solrj/io/stream/SelectStream.java    |  71 +++-
 .../solrj/io/stream/expr/Explanation.java       |   1 +
 .../solrj/io/stream/expr/StreamFactory.java     |  34 ++
 .../io/stream/SelectWithEvaluatorsTest.java     | 259 ++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   |  55 +--
 .../stream/eval/AbsoluteValueEvaluatorTest.java |  96 ++++++
 .../solrj/io/stream/eval/AddEvaluatorTest.java  | 336 +++++++++++++++++++
 .../solrj/io/stream/eval/AndEvaluatorTest.java  | 123 +++++++
 .../io/stream/eval/CompoundEvaluatorTest.java   |  85 +++++
 .../io/stream/eval/DivideEvaluatorTest.java     | 164 +++++++++
 .../io/stream/eval/EqualsEvaluatorTest.java     | 263 +++++++++++++++
 .../stream/eval/ExclusiveOrEvaluatorTest.java   | 123 +++++++
 .../eval/GreaterThanEqualToEvaluatorTest.java   | 249 ++++++++++++++
 .../stream/eval/GreaterThanEvaluatorTest.java   | 249 ++++++++++++++
 .../eval/LessThanEqualToEvaluatorTest.java      | 256 ++++++++++++++
 .../io/stream/eval/LessThanEvaluatorTest.java   | 249 ++++++++++++++
 .../io/stream/eval/MultiplyEvaluatorTest.java   | 179 ++++++++++
 .../solrj/io/stream/eval/NotEvaluatorTest.java  |  80 +++++
 .../solrj/io/stream/eval/OrEvaluatorTest.java   | 123 +++++++
 .../io/stream/eval/RawValueEvaluatorTest.java   |  69 ++++
 .../io/stream/eval/SubtractEvaluatorTest.java   | 188 +++++++++++
 57 files changed, 5055 insertions(+), 780 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0f08b51..f730f7f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -49,7 +49,11 @@ New Features
 * SOLR-9926: Allow passing arbitrary java system properties to zkcli. (Hrishikesh Gadre via Mark Miller)
 
 * SOLR-9885: Allow pre-startup Solr log management in Solr bin scripts to be disabled. (Mano Kovacs via Mark Miller)
-  
+ 
+* SOLR-9916: Adds Stream Evaluators to support evaluating values from tuples. Supports boolean,
+  numeric, and conditional evaluators. BooleanOperations have been removed in preference of
+  BooleanEvaluators. (Dennis Gove)
+ 
 Optimizations
 ----------------------
 * SOLR-9941: Clear the deletes lists at UpdateLog before replaying from log. This prevents redundantly pre-applying

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/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 0b41ef4..825cbac 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -29,21 +29,63 @@ import org.apache.solr.client.solrj.io.ModelCache;
 import org.apache.solr.client.solrj.io.SolrClientCache;
 import org.apache.solr.client.solrj.io.Tuple;
 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.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.GreaterThanEqualToEvaluator;
+import org.apache.solr.client.solrj.io.eval.GreaterThanEvaluator;
+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.MultiplyEvaluator;
+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.RawValueEvaluator;
+import org.apache.solr.client.solrj.io.eval.SubtractEvaluator;
 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.AndOperation;
 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.EqualsOperation;
-import org.apache.solr.client.solrj.io.ops.GreaterThanEqualToOperation;
-import org.apache.solr.client.solrj.io.ops.GreaterThanOperation;
 import org.apache.solr.client.solrj.io.ops.GroupOperation;
-import org.apache.solr.client.solrj.io.ops.LessThanEqualToOperation;
-import org.apache.solr.client.solrj.io.ops.LessThanOperation;
-import org.apache.solr.client.solrj.io.ops.NotOperation;
-import org.apache.solr.client.solrj.io.ops.OrOperation;
 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.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.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;
@@ -152,6 +194,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("executor", ExecutorStream.class)
       .withFunctionName("null", NullStream.class)
       .withFunctionName("priority", PriorityStream.class)
+      
       // metrics
       .withFunctionName("min", MinMetric.class)
       .withFunctionName("max", MaxMetric.class)
@@ -167,14 +210,31 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("group", GroupOperation.class)
       .withFunctionName("distinct", DistinctOperation.class)
       .withFunctionName("having", HavingStream.class)
-      .withFunctionName("and", AndOperation.class)
-      .withFunctionName("or", OrOperation.class)
-      .withFunctionName("not", NotOperation.class)
-      .withFunctionName("gt", GreaterThanOperation.class)
-      .withFunctionName("lt", LessThanOperation.class)
-      .withFunctionName("eq", EqualsOperation.class)
-      .withFunctionName("lteq", LessThanEqualToOperation.class)
-      .withFunctionName("gteq", GreaterThanEqualToOperation.class);
+      
+      // Stream Evaluators
+      .withFunctionName("val", RawValueEvaluator.class)
+      
+      // Boolean Stream Evaluators
+      .withFunctionName("and", AndEvaluator.class)
+      .withFunctionName("eor", ExclusiveOrEvaluator.class)
+      .withFunctionName("eq", EqualsEvaluator.class)
+      .withFunctionName("gt", GreaterThanEvaluator.class)
+      .withFunctionName("gteq", GreaterThanEqualToEvaluator.class)
+      .withFunctionName("lt", LessThanEvaluator.class)
+      .withFunctionName("lteq", LessThanEqualToEvaluator.class)
+      .withFunctionName("not", NotEvaluator.class)
+      .withFunctionName("or", OrEvaluator.class)
+      
+      // Number Stream Evaluators
+      .withFunctionName("abs", AbsoluteValueEvaluator.class)
+      .withFunctionName("add", AddEvaluator.class)
+      .withFunctionName("div", DivideEvaluator.class)
+      .withFunctionName("mult", MultiplyEvaluator.class)
+      .withFunctionName("sub", SubtractEvaluator.class)
+      
+      // Conditional Stream Evaluators
+      .withFunctionName("if", IfThenElseEvaluator.class)
+      ;
 
      // This pulls all the overrides and additions from the config
      List<PluginInfo> pluginInfos = core.getSolrConfig().getPluginInfos(Expressible.class.getName());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AbsoluteValueEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AbsoluteValueEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AbsoluteValueEvaluator.java
new file mode 100644
index 0000000..38b3bb5
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AbsoluteValueEvaluator.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 AbsoluteValueEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public AbsoluteValueEvaluator(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).abs());
+  }  
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AddEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AddEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AddEvaluator.java
new file mode 100644
index 0000000..317741e
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AddEvaluator.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 AddEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public AddEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+    
+    if(subEvaluators.size() < 2){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at least 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 result = null;
+    if(results.size() > 0){
+      result = results.get(0);
+      for(int idx = 1; idx < results.size(); ++idx){
+        result = result.add(results.get(idx));
+      }
+    }
+    
+    return normalizeType(result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AndEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AndEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AndEvaluator.java
new file mode 100644
index 0000000..290bd98
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AndEvaluator.java
@@ -0,0 +1,90 @@
+/*
+ * 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.List;
+import java.util.Locale;
+import java.util.stream.Collectors;
+
+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 AndEvaluator extends BooleanEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public AndEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+    
+    if(subEvaluators.size() < 2){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at least two values but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Boolean evaluate(Tuple tuple) throws IOException {
+    
+    List<Object> results = evaluateAll(tuple);
+    
+    if(results.size() < 2){
+      String message = null;
+      if(1 == results.size()){
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 1 was provided", constructingFactory.getFunctionName(getClass())); 
+      }
+      else{
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 0 were provided", constructingFactory.getFunctionName(getClass()));
+      }
+      throw new IOException(message);
+    }
+    
+    Checker checker = constructChecker(results.get(0));
+    if(results.stream().anyMatch(result -> null == result && !checker.isNullAllowed())){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) because a null value was found", constructingFactory.getFunctionName(getClass())));
+    }
+    if(results.stream().anyMatch(result -> !checker.isCorrectType(result))){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) of differing types [%s]", constructingFactory.getFunctionName(getClass()), results.stream().map(item -> item.getClass().getSimpleName()).collect(Collectors.joining(","))));
+    }
+
+    for(int idx = 1; idx < results.size(); ++idx){
+      if(!checker.test(results.get(0), results.get(idx))){
+        return false;
+      }
+    }
+    
+    return true;
+  }
+  
+  private Checker constructChecker(Object fromValue) throws IOException{
+    if(null == fromValue){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) because a null value was found", constructingFactory.getFunctionName(getClass())));
+    }
+    else if(fromValue instanceof Boolean){
+      return new BooleanChecker(){
+        @Override
+        public boolean test(Object left, Object right) {
+          return (boolean)left && (boolean)right;
+        }
+      };
+    }
+    
+    throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) for values of type '%s'", constructingFactory.getFunctionName(getClass()), fromValue.getClass().getSimpleName()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/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
new file mode 100644
index 0000000..bf21f1d
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/BooleanEvaluator.java
@@ -0,0 +1,86 @@
+/*
+ * 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.ArrayList;
+import java.util.List;
+
+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 abstract class BooleanEvaluator extends ComplexEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public BooleanEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+  }
+  
+  // restrict result to a Boolean
+  public abstract Boolean evaluate(Tuple tuple) throws IOException;
+  
+  public List<Object> evaluateAll(final Tuple tuple) throws IOException {
+    List<Object> results = new ArrayList<Object>();
+    for(StreamEvaluator subEvaluator : subEvaluators){
+      results.add(subEvaluator.evaluate(tuple));
+    }
+    
+    return results;
+  }
+  
+  public interface Checker {
+    default boolean isNullAllowed(){
+      return false;
+    }
+    boolean isCorrectType(Object value);
+    boolean test(Object left, Object right);
+  }
+  
+  public interface NullChecker extends Checker {
+    default boolean isNullAllowed(){
+      return true;
+    }
+    default boolean isCorrectType(Object value){
+      return true;
+    }
+    default boolean test(Object left, Object right){
+      return null == left && null == right;
+    }
+  }
+  
+  public interface BooleanChecker extends Checker {
+    default boolean isCorrectType(Object value){
+      return value instanceof Boolean;
+    }
+  }
+  
+  public interface NumberChecker extends Checker {
+    default boolean isCorrectType(Object value){
+      return value instanceof Number;
+    }
+  }
+  
+  public interface StringChecker extends Checker {
+    default boolean isCorrectType(Object value){
+      return value instanceof String;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/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
new file mode 100644
index 0000000..1e56d12
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ComplexEvaluator.java
@@ -0,0 +1,99 @@
+/*
+ * 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.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.UUID;
+
+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;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public abstract class ComplexEvaluator implements StreamEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  protected UUID nodeId = UUID.randomUUID();
+  
+  protected StreamFactory constructingFactory;
+  protected List<StreamEvaluator> subEvaluators = new ArrayList<StreamEvaluator>();
+  
+  public ComplexEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    constructingFactory = factory;
+    
+    // We have to do this because order of the parameters matter
+    List<StreamExpressionParameter> parameters = factory.getOperandsOfType(expression, StreamExpressionParameter.class);
+    
+    for(StreamExpressionParameter parameter : parameters){
+      if(parameter instanceof StreamExpression){
+        // possible evaluator
+        StreamExpression streamExpression = (StreamExpression)parameter;
+        if(factory.doesRepresentTypes(streamExpression, ComplexEvaluator.class)){
+          subEvaluators.add(factory.constructEvaluator(streamExpression));
+        }
+        else if(factory.doesRepresentTypes(streamExpression, SimpleEvaluator.class)){
+          subEvaluators.add(factory.constructEvaluator(streamExpression));
+        }
+        else{
+          // Will be treated as a field name
+          subEvaluators.add(new FieldEvaluator(streamExpression.toString()));
+        }
+      }
+      else if(parameter instanceof StreamExpressionValue){
+        if(0 != ((StreamExpressionValue)parameter).getValue().length()){
+          // special case - if evaluates to a number, boolean, or null then we'll treat it 
+          // as a RawValueEvaluator
+          Object value = factory.constructPrimitiveObject(((StreamExpressionValue)parameter).getValue());
+          if(null == value || value instanceof Boolean || value instanceof Number){
+            subEvaluators.add(new RawValueEvaluator(value));
+          }
+          else if(value instanceof String){
+            subEvaluators.add(new FieldEvaluator((String)value));
+          }
+        }
+      }
+    }
+    
+    if(expression.getParameters().size() != subEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - unknown operands found - expecting only StreamEvaluators or field names", expression));
+    }
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    
+    for(StreamEvaluator evaluator : subEvaluators){
+      expression.addParameter(evaluator.toExpression(factory));
+    }
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(nodeId.toString())
+      .withExpressionType(ExpressionType.EVALUATOR)
+      .withFunctionName(factory.getFunctionName(getClass()))
+      .withImplementingClass(getClass().getName())
+      .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/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
new file mode 100644
index 0000000..499e2f8
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConditionalEvaluator.java
@@ -0,0 +1,59 @@
+/*
+ * 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.ArrayList;
+import java.util.List;
+
+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 abstract class ConditionalEvaluator extends ComplexEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public ConditionalEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+  }
+  
+  public List<Object> evaluateAll(final Tuple tuple) throws IOException {
+    List<Object> results = new ArrayList<Object>();
+    for(StreamEvaluator subEvaluator : subEvaluators){
+      results.add(subEvaluator.evaluate(tuple));
+    }
+    
+    return results;
+  }
+  
+  public interface Checker {
+    default boolean isNullAllowed(){
+      return false;
+    }
+    boolean isCorrectType(Object value);
+    boolean test(Object left, Object right);
+  }
+    
+  public interface BooleanChecker extends Checker {
+    default boolean isCorrectType(Object value){
+      return value instanceof Boolean;
+    }
+  }  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DivideEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DivideEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DivideEvaluator.java
new file mode 100644
index 0000000..f21a7f3
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DivideEvaluator.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 DivideEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public DivideEvaluator(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.divide(denominator, MathContext.DECIMAL64));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/EqualsEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/EqualsEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/EqualsEvaluator.java
new file mode 100644
index 0000000..051a2de
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/EqualsEvaluator.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 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 java.util.stream.Collectors;
+
+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 EqualsEvaluator extends BooleanEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public EqualsEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+    
+    if(subEvaluators.size() < 2){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at least two values but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Boolean evaluate(Tuple tuple) throws IOException {
+    
+    List<Object> results = evaluateAll(tuple);
+    
+    if(results.size() < 2){
+      String message = null;
+      if(1 == results.size()){
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 1 was provided", constructingFactory.getFunctionName(getClass())); 
+      }
+      else{
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 0 were provided", constructingFactory.getFunctionName(getClass()));
+      }
+      throw new IOException(message);
+    }
+    
+    Checker checker = constructChecker(results.get(0));
+    if(results.stream().anyMatch(result -> null == result && !checker.isNullAllowed())){
+      return false;
+    }
+    if(results.stream().anyMatch(result -> !checker.isCorrectType(result))){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) of differing types [%s]", constructingFactory.getFunctionName(getClass()), results.stream().map(item -> item.getClass().getSimpleName()).collect(Collectors.joining(","))));
+    }
+
+    for(int idx = 1; idx < results.size(); ++idx){
+      if(!checker.test(results.get(0), results.get(idx))){
+        return false;
+      }
+    }
+    
+    return true;
+  }
+  
+  private Checker constructChecker(Object fromValue) throws IOException{
+    if(null == fromValue){
+      return new NullChecker() {
+        @Override
+        public boolean test(Object left, Object right) {
+          return null == left && null == right;
+        }
+      };
+    }
+    else if(fromValue instanceof Boolean){
+      return new BooleanChecker(){
+        @Override
+        public boolean test(Object left, Object right) {
+          return (boolean)left.equals((boolean)right);
+        }
+      };
+    }
+    else if(fromValue instanceof Number){
+      return new NumberChecker(){
+        @Override
+        public boolean test(Object left, Object right) {
+          return 0 == (new BigDecimal(left.toString())).compareTo(new BigDecimal(right.toString()));
+        }
+      };
+    }
+    else if(fromValue instanceof String){
+      return new StringChecker(){
+        @Override
+        public boolean test(Object left, Object right) {
+          return left.equals(right);
+        }
+      };
+    }
+    
+    throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) for values of type '%s'", constructingFactory.getFunctionName(getClass()), fromValue.getClass().getSimpleName()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ExclusiveOrEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ExclusiveOrEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ExclusiveOrEvaluator.java
new file mode 100644
index 0000000..e63cab0
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ExclusiveOrEvaluator.java
@@ -0,0 +1,67 @@
+/*
+ * 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.List;
+import java.util.Locale;
+import java.util.stream.Collectors;
+
+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 ExclusiveOrEvaluator extends BooleanEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public ExclusiveOrEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+    
+    if(subEvaluators.size() < 2){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at least two values but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Boolean evaluate(Tuple tuple) throws IOException {
+    
+    List<Object> results = evaluateAll(tuple);
+    
+    if(results.size() < 2){
+      String message = null;
+      if(1 == results.size()){
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 1 was provided", constructingFactory.getFunctionName(getClass())); 
+      }
+      else{
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 0 were provided", constructingFactory.getFunctionName(getClass()));
+      }
+      throw new IOException(message);
+    }
+    
+    if(results.stream().anyMatch(result -> null == result)){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) because a null value was found", constructingFactory.getFunctionName(getClass())));
+    }
+    if(results.stream().anyMatch(result -> !(result instanceof Boolean))){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) of non-boolean values [%s]", constructingFactory.getFunctionName(getClass()), results.stream().map(item -> item.getClass().getSimpleName()).collect(Collectors.joining(","))));
+    }
+
+    return 1 == results.stream().filter(result -> (boolean)result).count();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FieldEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FieldEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FieldEvaluator.java
new file mode 100644
index 0000000..0ebe729
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FieldEvaluator.java
@@ -0,0 +1,62 @@
+/*
+ * 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 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.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class FieldEvaluator extends SimpleEvaluator {
+  private static final long serialVersionUID = 1L;
+  
+  private String fieldName;
+  
+  public FieldEvaluator(String fieldName) {
+    if(fieldName.startsWith("'") && fieldName.endsWith("'") && fieldName.length() > 1){
+      fieldName = fieldName.substring(1, fieldName.length() - 1);
+    }
+    
+    this.fieldName = fieldName;
+  }
+  
+  @Override
+  public Object evaluate(Tuple tuple) {
+    return tuple.get(fieldName); // returns null if field doesn't exist in tuple
+  }
+  
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    return new StreamExpressionValue(fieldName);
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(nodeId.toString())
+      .withExpressionType(ExpressionType.EVALUATOR)
+      .withImplementingClass(getClass().getName())
+      .withExpression(toExpression(factory).toString());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/GreaterThanEqualToEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/GreaterThanEqualToEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/GreaterThanEqualToEvaluator.java
new file mode 100644
index 0000000..ad79e82
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/GreaterThanEqualToEvaluator.java
@@ -0,0 +1,99 @@
+/*
+ * 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 java.util.stream.Collectors;
+
+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 GreaterThanEqualToEvaluator extends BooleanEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public GreaterThanEqualToEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+    
+    if(subEvaluators.size() < 2){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at least two values but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Boolean evaluate(Tuple tuple) throws IOException {
+    
+    List<Object> results = evaluateAll(tuple);
+    
+    if(results.size() < 2){
+      String message = null;
+      if(1 == results.size()){
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 1 was provided", constructingFactory.getFunctionName(getClass())); 
+      }
+      else{
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 0 were provided", constructingFactory.getFunctionName(getClass()));
+      }
+      throw new IOException(message);
+    }
+    
+    Checker checker = constructChecker(results.get(0));
+    if(results.stream().anyMatch(result -> null == result)){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) because a null value was found", constructingFactory.getFunctionName(getClass())));
+    }
+    if(results.stream().anyMatch(result -> !checker.isCorrectType(result))){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) of differing types [%s]", constructingFactory.getFunctionName(getClass()), results.stream().map(item -> item.getClass().getSimpleName()).collect(Collectors.joining(","))));
+    }
+
+    for(int idx = 1; idx < results.size(); ++idx){
+      if(!checker.test(results.get(idx - 1), results.get(idx))){
+        return false;
+      }
+    }
+    
+    return true;
+  }
+  
+  private Checker constructChecker(Object fromValue) throws IOException{
+    if(null == fromValue){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) because a null value was found", constructingFactory.getFunctionName(getClass())));
+    }
+    else if(fromValue instanceof Number){
+      return new NumberChecker(){
+        @Override
+        public boolean test(Object left, Object right) {
+          return (new BigDecimal(left.toString())).compareTo(new BigDecimal(right.toString())) >= 0;
+        }
+      };
+    }
+    else if(fromValue instanceof String){
+      return new StringChecker(){
+        @Override
+        public boolean test(Object left, Object right) {
+          return ((String)left).compareToIgnoreCase((String)right) >= 0;
+        }
+      };
+    }
+    
+    throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) for values of type '%s'", constructingFactory.getFunctionName(getClass()), fromValue.getClass().getSimpleName()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/GreaterThanEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/GreaterThanEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/GreaterThanEvaluator.java
new file mode 100644
index 0000000..0b0e6e3
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/GreaterThanEvaluator.java
@@ -0,0 +1,99 @@
+/*
+ * 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 java.util.stream.Collectors;
+
+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 GreaterThanEvaluator extends BooleanEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public GreaterThanEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+    
+    if(subEvaluators.size() < 2){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at least two values but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Boolean evaluate(Tuple tuple) throws IOException {
+    
+    List<Object> results = evaluateAll(tuple);
+    
+    if(results.size() < 2){
+      String message = null;
+      if(1 == results.size()){
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 1 was provided", constructingFactory.getFunctionName(getClass())); 
+      }
+      else{
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 0 were provided", constructingFactory.getFunctionName(getClass()));
+      }
+      throw new IOException(message);
+    }
+    
+    Checker checker = constructChecker(results.get(0));
+    if(results.stream().anyMatch(result -> null == result)){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) because a null value was found", constructingFactory.getFunctionName(getClass())));
+    }
+    if(results.stream().anyMatch(result -> !checker.isCorrectType(result))){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) of differing types [%s]", constructingFactory.getFunctionName(getClass()), results.stream().map(item -> item.getClass().getSimpleName()).collect(Collectors.joining(","))));
+    }
+
+    for(int idx = 1; idx < results.size(); ++idx){
+      if(!checker.test(results.get(idx - 1), results.get(idx))){
+        return false;
+      }
+    }
+    
+    return true;
+  }
+  
+  private Checker constructChecker(Object fromValue) throws IOException{
+    if(null == fromValue){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) because a null value was found", constructingFactory.getFunctionName(getClass())));
+    }
+    else if(fromValue instanceof Number){
+      return new NumberChecker(){
+        @Override
+        public boolean test(Object left, Object right) {
+          return (new BigDecimal(left.toString())).compareTo(new BigDecimal(right.toString())) > 0;
+        }
+      };
+    }
+    else if(fromValue instanceof String){
+      return new StringChecker(){
+        @Override
+        public boolean test(Object left, Object right) {
+          return ((String)left).compareToIgnoreCase((String)right) > 0;
+        }
+      };
+    }
+    
+    throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) for values of type '%s'", constructingFactory.getFunctionName(getClass()), fromValue.getClass().getSimpleName()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/IfThenElseEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/IfThenElseEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/IfThenElseEvaluator.java
new file mode 100644
index 0000000..346b743
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/IfThenElseEvaluator.java
@@ -0,0 +1,62 @@
+/*
+ * 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.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 IfThenElseEvaluator extends ConditionalEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public IfThenElseEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+    
+    if(3 != subEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting three values but found %d",expression,subEvaluators.size()));
+    }
+    
+    if(!(subEvaluators.get(0) instanceof BooleanEvaluator)){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a boolean as the first parameter but found %s",expression,subEvaluators.get(0).getClass().getSimpleName()));
+    }
+
+  }
+
+  @Override
+  public Object evaluate(Tuple tuple) throws IOException {
+    
+    List<Object> results = evaluateAll(tuple);
+    
+    if(3 != results.size()){
+      String message = String.format(Locale.ROOT,"%s(...) only works with 3 values but %s were provided", constructingFactory.getFunctionName(getClass()), results.size());
+      throw new IOException(message);
+    }
+    
+    if(!(results.get(0) instanceof Boolean)){
+      throw new IOException(String.format(Locale.ROOT,"$s(...) only works with a boolean as the first parameter but found %s",results.get(0).getClass().getSimpleName()));
+    }
+  
+    return (boolean)results.get(0) ? results.get(1) : results.get(2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LessThanEqualToEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LessThanEqualToEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LessThanEqualToEvaluator.java
new file mode 100644
index 0000000..cb2fc7a
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LessThanEqualToEvaluator.java
@@ -0,0 +1,99 @@
+/*
+ * 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 java.util.stream.Collectors;
+
+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 LessThanEqualToEvaluator extends BooleanEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public LessThanEqualToEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+    
+    if(subEvaluators.size() < 2){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at least two values but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Boolean evaluate(Tuple tuple) throws IOException {
+    
+    List<Object> results = evaluateAll(tuple);
+    
+    if(results.size() < 2){
+      String message = null;
+      if(1 == results.size()){
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 1 was provided", constructingFactory.getFunctionName(getClass())); 
+      }
+      else{
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 0 were provided", constructingFactory.getFunctionName(getClass()));
+      }
+      throw new IOException(message);
+    }
+    
+    Checker checker = constructChecker(results.get(0));
+    if(results.stream().anyMatch(result -> null == result)){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) because a null value was found", constructingFactory.getFunctionName(getClass())));
+    }
+    if(results.stream().anyMatch(result -> !checker.isCorrectType(result))){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) of differing types [%s]", constructingFactory.getFunctionName(getClass()), results.stream().map(item -> item.getClass().getSimpleName()).collect(Collectors.joining(","))));
+    }
+
+    for(int idx = 1; idx < results.size(); ++idx){
+      if(!checker.test(results.get(idx - 1), results.get(idx))){
+        return false;
+      }
+    }
+    
+    return true;
+  }
+  
+  private Checker constructChecker(Object fromValue) throws IOException{
+    if(null == fromValue){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) because a null value was found", constructingFactory.getFunctionName(getClass())));
+    }
+    else if(fromValue instanceof Number){
+      return new NumberChecker(){
+        @Override
+        public boolean test(Object left, Object right) {
+          return (new BigDecimal(left.toString())).compareTo(new BigDecimal(right.toString())) <= 0;
+        }
+      };
+    }
+    else if(fromValue instanceof String){
+      return new StringChecker(){
+        @Override
+        public boolean test(Object left, Object right) {
+          return ((String)left).compareToIgnoreCase((String)right) <= 0;
+        }
+      };
+    }
+    
+    throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) for values of type '%s'", constructingFactory.getFunctionName(getClass()), fromValue.getClass().getSimpleName()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LessThanEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LessThanEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LessThanEvaluator.java
new file mode 100644
index 0000000..40796b8
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LessThanEvaluator.java
@@ -0,0 +1,99 @@
+/*
+ * 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 java.util.stream.Collectors;
+
+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 LessThanEvaluator extends BooleanEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public LessThanEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+    
+    if(subEvaluators.size() < 2){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at least two values but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Boolean evaluate(Tuple tuple) throws IOException {
+    
+    List<Object> results = evaluateAll(tuple);
+    
+    if(results.size() < 2){
+      String message = null;
+      if(1 == results.size()){
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 1 was provided", constructingFactory.getFunctionName(getClass())); 
+      }
+      else{
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 0 were provided", constructingFactory.getFunctionName(getClass()));
+      }
+      throw new IOException(message);
+    }
+    
+    Checker checker = constructChecker(results.get(0));
+    if(results.stream().anyMatch(result -> null == result)){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) because a null value was found", constructingFactory.getFunctionName(getClass())));
+    }
+    if(results.stream().anyMatch(result -> !checker.isCorrectType(result))){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) of differing types [%s]", constructingFactory.getFunctionName(getClass()), results.stream().map(item -> item.getClass().getSimpleName()).collect(Collectors.joining(","))));
+    }
+
+    for(int idx = 1; idx < results.size(); ++idx){
+      if(!checker.test(results.get(idx - 1), results.get(idx))){
+        return false;
+      }
+    }
+    
+    return true;
+  }
+  
+  private Checker constructChecker(Object fromValue) throws IOException{
+    if(null == fromValue){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) because a null value was found", constructingFactory.getFunctionName(getClass())));
+    }
+    else if(fromValue instanceof Number){
+      return new NumberChecker(){
+        @Override
+        public boolean test(Object left, Object right) {
+          return (new BigDecimal(left.toString())).compareTo(new BigDecimal(right.toString())) < 0;
+        }
+      };
+    }
+    else if(fromValue instanceof String){
+      return new StringChecker(){
+        @Override
+        public boolean test(Object left, Object right) {
+          return ((String)left).compareToIgnoreCase((String)right) < 0;
+        }
+      };
+    }
+    
+    throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) for values of type '%s'", constructingFactory.getFunctionName(getClass()), fromValue.getClass().getSimpleName()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/MultiplyEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/MultiplyEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/MultiplyEvaluator.java
new file mode 100644
index 0000000..44b0b26
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/MultiplyEvaluator.java
@@ -0,0 +1,62 @@
+/*
+ * 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 MultiplyEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public MultiplyEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+    
+    if(subEvaluators.size() < 2){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at least 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 result = null;
+    if(results.size() > 0){
+      result = results.get(0);
+      for(int idx = 1; idx < results.size(); ++idx){
+        result = result.multiply(results.get(idx), MathContext.DECIMAL64);
+      }
+    }
+    
+    return normalizeType(result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NotEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NotEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NotEvaluator.java
new file mode 100644
index 0000000..da2eeff
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NotEvaluator.java
@@ -0,0 +1,62 @@
+/*
+ * 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.List;
+import java.util.Locale;
+import java.util.stream.Collectors;
+
+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 NotEvaluator extends BooleanEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public NotEvaluator(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 Boolean evaluate(Tuple tuple) throws IOException {
+    
+    List<Object> results = evaluateAll(tuple);
+    
+    if(1 != results.size()){
+      String message = String.format(Locale.ROOT,"%s(...) only works with 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size());
+      throw new IOException(message);
+    }
+
+    Object result = results.get(0);
+    if(null == result){
+      throw new IOException(String.format(Locale.ROOT,"Unable to evaluate %s(...) because a null value was found", constructingFactory.getFunctionName(getClass())));
+    }
+    if(!(result instanceof Boolean)){
+      throw new IOException(String.format(Locale.ROOT,"Unable to evaluate %s(...) of a non-boolean value [%s]", constructingFactory.getFunctionName(getClass()), results.stream().map(item -> item.getClass().getSimpleName()).collect(Collectors.joining(","))));
+    }
+    
+    return !((Boolean)result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/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
new file mode 100644
index 0000000..f4491fd
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NumberEvaluator.java
@@ -0,0 +1,79 @@
+/*
+ * 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.ArrayList;
+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 abstract class NumberEvaluator extends ComplexEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public NumberEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+  }
+  
+  // restrict result to a Number
+  public abstract Number evaluate(Tuple tuple) throws IOException;
+  
+  public List<BigDecimal> evaluateAll(final Tuple tuple) throws IOException {
+    // evaluate each and confirm they are all either null or numeric
+    List<BigDecimal> results = new ArrayList<BigDecimal>();
+    for(StreamEvaluator subEvaluator : subEvaluators){
+      Object result = subEvaluator.evaluate(tuple);
+      
+      if(null == result){
+        results.add(null);
+      }
+      else if(result instanceof Number){
+        results.add(new BigDecimal(result.toString()));
+      }
+      else{
+        String message = String.format(Locale.ROOT,"Failed to evaluate to a numeric value - evaluator '%s' resulted in type '%s' and value '%s'", 
+                                        subEvaluator.toExpression(constructingFactory),
+                                        result.getClass().getName(),
+                                        result.toString());
+        throw new IOException(message);
+      }
+    }
+    
+    return results;
+  }
+  
+  public Number normalizeType(BigDecimal value){
+    if(null == value){
+      return null;
+    }
+    
+    if(value.signum() == 0 || value.scale() <= 0 || value.stripTrailingZeros().scale() <= 0){
+      return value.longValue();
+    }
+    
+    return value.doubleValue();
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/OrEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/OrEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/OrEvaluator.java
new file mode 100644
index 0000000..1cd9df8
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/OrEvaluator.java
@@ -0,0 +1,90 @@
+/*
+ * 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.List;
+import java.util.Locale;
+import java.util.stream.Collectors;
+
+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 OrEvaluator extends BooleanEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public OrEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+    
+    if(subEvaluators.size() < 2){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at least two values but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Boolean evaluate(Tuple tuple) throws IOException {
+    
+    List<Object> results = evaluateAll(tuple);
+    
+    if(results.size() < 2){
+      String message = null;
+      if(1 == results.size()){
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 1 was provided", constructingFactory.getFunctionName(getClass())); 
+      }
+      else{
+        message = String.format(Locale.ROOT,"%s(...) only works with at least 2 values but 0 were provided", constructingFactory.getFunctionName(getClass()));
+      }
+      throw new IOException(message);
+    }
+    
+    Checker checker = constructChecker(results.get(0));
+    if(results.stream().anyMatch(result -> null == result && !checker.isNullAllowed())){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) because a null value was found", constructingFactory.getFunctionName(getClass())));
+    }
+    if(results.stream().anyMatch(result -> !checker.isCorrectType(result))){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) of differing types [%s]", constructingFactory.getFunctionName(getClass()), results.stream().map(item -> item.getClass().getSimpleName()).collect(Collectors.joining(","))));
+    }
+
+    for(int idx = 1; idx < results.size(); ++idx){
+      if(!checker.test(results.get(0), results.get(idx))){
+        return false;
+      }
+    }
+    
+    return true;
+  }
+  
+  private Checker constructChecker(Object fromValue) throws IOException{
+    if(null == fromValue){
+      throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) because a null value was found", constructingFactory.getFunctionName(getClass())));
+    }
+    else if(fromValue instanceof Boolean){
+      return new BooleanChecker(){
+        @Override
+        public boolean test(Object left, Object right) {
+          return (boolean)left || (boolean)right;
+        }
+      };
+    }
+    
+    throw new IOException(String.format(Locale.ROOT,"Unable to check %s(...) for values of type '%s'", constructingFactory.getFunctionName(getClass()), fromValue.getClass().getSimpleName()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RawValueEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RawValueEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RawValueEvaluator.java
new file mode 100644
index 0000000..1751380
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RawValueEvaluator.java
@@ -0,0 +1,90 @@
+/*
+ * 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.List;
+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.Explanation.ExpressionType;
+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.StreamExpressionValue;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class RawValueEvaluator extends SimpleEvaluator {
+  private static final long serialVersionUID = 1L;
+  
+  private Object value;
+  
+  public RawValueEvaluator(Object value){
+    init(value);
+  }
+  
+  public RawValueEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    // We have to do this because order of the parameters matter
+    List<StreamExpressionParameter> parameters = factory.getOperandsOfType(expression, StreamExpressionValue.class);
+    
+    if(expression.getParameters().size() != parameters.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - unknown operands found - expecting only raw values", expression));
+    }
+    
+    if(1 != parameters.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - only 1 value can exist in a %s(...) evaluator", expression, factory.getFunctionName(getClass())));
+    }
+    
+    init(factory.constructPrimitiveObject(((StreamExpressionValue)parameters.get(0)).getValue()));
+  }
+  
+  private void init(Object value){
+    if(value instanceof Integer){
+      this.value = (Long)value;
+    }
+    else if(value instanceof Float){
+      this.value = ((Float)value).doubleValue();
+    }
+    else{
+      this.value = value;
+    }
+  }
+  
+  @Override
+  public Object evaluate(Tuple tuple) {
+    return value;
+  }
+  
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    expression.addParameter(new StreamExpressionValue(value.toString()));
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(nodeId.toString())
+      .withExpressionType(ExpressionType.EVALUATOR)
+      .withImplementingClass(getClass().getName())
+      .withExpression(toExpression(factory).toString());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7372df99/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SimpleEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SimpleEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SimpleEvaluator.java
new file mode 100644
index 0000000..79d1799
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SimpleEvaluator.java
@@ -0,0 +1,29 @@
+/*
+ * 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.util.UUID;
+
+public abstract class SimpleEvaluator implements StreamEvaluator {
+  private static final long serialVersionUID = 1L;
+  
+  protected UUID nodeId = UUID.randomUUID();
+
+}