You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/02/15 19:00:06 UTC

[02/52] [abbrv] lucene-solr:jira/solr-9858: SOLR-9916: Adds Stream Evaluators to support evaluating values from tuples

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/StreamEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/StreamEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/StreamEvaluator.java
new file mode 100644
index 0000000..6bc4d50
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/StreamEvaluator.java
@@ -0,0 +1,30 @@
+/*
+ * 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.io.Serializable;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+
+public interface StreamEvaluator extends Expressible, Serializable {
+  Object evaluate(final Tuple tuple) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SubtractEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SubtractEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SubtractEvaluator.java
new file mode 100644
index 0000000..3bf62b7
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SubtractEvaluator.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 SubtractEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+  
+  public SubtractEvaluator(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.subtract(results.get(idx));
+      }
+    }
+    
+    return normalizeType(result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/AndOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/AndOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/AndOperation.java
deleted file mode 100644
index f095f63..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/AndOperation.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.io.ops;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.UUID;
-
-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.Expressible;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
-import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
-
-public class AndOperation implements BooleanOperation {
-
-  private static final long serialVersionUID = 1;
-  private UUID operationNodeId = UUID.randomUUID();
-
-  protected BooleanOperation leftOperand;
-  protected BooleanOperation rightOperand;
-
-  public void operate(Tuple tuple) {
-    leftOperand.operate(tuple);
-    rightOperand.operate(tuple);
-  }
-
-  public AndOperation(BooleanOperation leftOperand, BooleanOperation rightOperand) {
-    this.leftOperand = leftOperand;
-    this.rightOperand = rightOperand;
-  }
-
-  public AndOperation(StreamExpression expression, StreamFactory factory) throws IOException {
-      List<StreamExpression> operationExpressions = factory.getExpressionOperandsRepresentingTypes(expression, BooleanOperation.class);
-      if(operationExpressions != null && operationExpressions.size() == 2) {
-        StreamExpression left = operationExpressions.get(0);
-        StreamOperation leftOp = factory.constructOperation(left);
-        if(leftOp instanceof BooleanOperation) {
-          leftOperand = (BooleanOperation) leftOp;
-        } else {
-          throw new IOException("The And/Or Operation requires a BooleanOperation.");
-        }
-
-        StreamExpression right = operationExpressions.get(1);
-        StreamOperation rightOp = factory.constructOperation(right);
-        if(rightOp instanceof BooleanOperation) {
-          rightOperand = (BooleanOperation) rightOp;
-        } else {
-          throw new IOException("The And/Or Operation requires a BooleanOperation.");
-        }
-      } else {
-        throw new IOException("The And/Or Operation requires a BooleanOperations.");
-      }
-  }
-
-  public boolean evaluate() {
-    return leftOperand.evaluate() && rightOperand.evaluate();
-  }
-
-  @Override
-  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
-    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
-    if(leftOperand instanceof Expressible) {
-      expression.addParameter(leftOperand.toExpression(factory));
-    } else {
-      throw new IOException("This left operand of the AndOperation contains a non-expressible operation - it cannot be converted to an expression");
-    }
-
-    if(rightOperand instanceof Expressible) {
-      expression.addParameter(rightOperand.toExpression(factory));
-    } else {
-      throw new IOException("This the right operand of the AndOperation contains a non-expressible operation - it cannot be converted to an expression");
-    }
-    return expression;
-  }
-
-  @Override
-  public Explanation toExplanation(StreamFactory factory) throws IOException {
-    return new Explanation(operationNodeId.toString())
-        .withExpressionType(ExpressionType.OPERATION)
-        .withFunctionName(factory.getFunctionName(getClass()))
-        .withImplementingClass(getClass().getName())
-        .withExpression(toExpression(factory).toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
deleted file mode 100644
index d455999..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.io.ops;
-
-/**
- *  A BooleanOperation returns true or false for each tuple that it evaluates. The HavingStream applies a BooleanOperation to
- *  determine which tuples to emit.
- */
-
-public interface BooleanOperation extends StreamOperation {
-  public abstract boolean evaluate();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java
deleted file mode 100644
index 1958551..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.io.ops;
-
-import java.io.IOException;
-import java.util.UUID;
-
-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.StreamFactory;
-
-public class EqualsOperation extends LeafOperation {
-
-  private static final long serialVersionUID = 1;
-  private UUID operationNodeId = UUID.randomUUID();
-
-  public void operate(Tuple tuple) {
-    this.tuple = tuple;
-  }
-
-  public EqualsOperation(String field, double val) {
-    super(field, val);
-  }
-
-  public EqualsOperation(StreamExpression expression, StreamFactory factory) throws IOException {
-    super(expression, factory);
-  }
-
-  public boolean evaluate() {
-    Double d = tuple.getDouble(field);
-
-    if(d == null) {
-      return false;
-    }
-
-    return d.doubleValue() == val;
-  }
-
-  public StreamExpression toExpression(StreamFactory factory) throws IOException {
-    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
-    expression.addParameter(field);
-    expression.addParameter(Double.toString(val));
-    return expression;
-  }
-
-  @Override
-  public Explanation toExplanation(StreamFactory factory) throws IOException {
-    return new Explanation(operationNodeId.toString())
-        .withExpressionType(ExpressionType.OPERATION)
-        .withFunctionName(factory.getFunctionName(getClass()))
-        .withImplementingClass(getClass().getName())
-        .withExpression(toExpression(factory).toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java
deleted file mode 100644
index 87c8364..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.io.ops;
-
-import java.io.IOException;
-import java.util.UUID;
-
-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.StreamFactory;
-
-public class GreaterThanEqualToOperation extends LeafOperation {
-
-  private static final long serialVersionUID = 1;
-  private UUID operationNodeId = UUID.randomUUID();
-
-  public void operate(Tuple tuple) {
-    this.tuple = tuple;
-  }
-
-  public GreaterThanEqualToOperation(String field, double val) {
-    super(field, val);
-  }
-
-  public GreaterThanEqualToOperation(StreamExpression expression, StreamFactory factory) throws IOException {
-    super(expression, factory);
-  }
-
-  public boolean evaluate() {
-    Double d = tuple.getDouble(field);
-
-    if(d == null) {
-      return false;
-    }
-
-    return d.doubleValue() >= val;
-  }
-
-  public StreamExpression toExpression(StreamFactory factory) throws IOException {
-    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
-    expression.addParameter(field);
-    expression.addParameter(Double.toString(val));
-    return expression;
-  }
-
-  @Override
-  public Explanation toExplanation(StreamFactory factory) throws IOException {
-    return new Explanation(operationNodeId.toString())
-        .withExpressionType(ExpressionType.OPERATION)
-        .withFunctionName(factory.getFunctionName(getClass()))
-        .withImplementingClass(getClass().getName())
-        .withExpression(toExpression(factory).toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java
deleted file mode 100644
index 664438a..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.io.ops;
-
-import java.io.IOException;
-import java.util.UUID;
-
-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.StreamFactory;
-
-public class GreaterThanOperation extends LeafOperation {
-
-  private static final long serialVersionUID = 1;
-  private UUID operationNodeId = UUID.randomUUID();
-
-  public void operate(Tuple tuple) {
-    this.tuple = tuple;
-  }
-
-  public GreaterThanOperation(String field, double val) {
-    super(field, val);
-  }
-
-  public GreaterThanOperation(StreamExpression expression, StreamFactory factory) throws IOException {
-    super(expression, factory);
-  }
-
-  public boolean evaluate() {
-    Double d = tuple.getDouble(field);
-
-    if(d == null) {
-      return false;
-    }
-
-    return d.doubleValue() > val;
-  }
-
-  public StreamExpression toExpression(StreamFactory factory) throws IOException {
-    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
-    expression.addParameter(field);
-    expression.addParameter(Double.toString(val));
-    return expression;
-  }
-
-  @Override
-  public Explanation toExplanation(StreamFactory factory) throws IOException {
-    return new Explanation(operationNodeId.toString())
-        .withExpressionType(ExpressionType.OPERATION)
-        .withFunctionName(factory.getFunctionName(getClass()))
-        .withImplementingClass(getClass().getName())
-        .withExpression(toExpression(factory).toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
deleted file mode 100644
index 691a328..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.io.ops;
-
-import java.io.IOException;
-import java.util.UUID;
-
-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.StreamFactory;
-
-public abstract class LeafOperation implements BooleanOperation {
-
-  private static final long serialVersionUID = 1;
-  private UUID operationNodeId = UUID.randomUUID();
-
-  protected String field;
-  protected Double val;
-  protected Tuple tuple;
-
-  public void operate(Tuple tuple) {
-    this.tuple = tuple;
-  }
-
-  public LeafOperation(String field, double val) {
-    this.field = field;
-    this.val = val;
-  }
-
-  public LeafOperation(StreamExpression expression, StreamFactory factory) throws IOException {
-    this.field = factory.getValueOperand(expression, 0);
-    this.val = Double.parseDouble(factory.getValueOperand(expression, 1));
-  }
-
-  @Override
-  public Explanation toExplanation(StreamFactory factory) throws IOException {
-    return new Explanation(operationNodeId.toString())
-        .withExpressionType(ExpressionType.OPERATION)
-        .withFunctionName(factory.getFunctionName(getClass()))
-        .withImplementingClass(getClass().getName())
-        .withExpression(toExpression(factory).toString());
-  }
-
-  protected String quote(String s) {
-    if(s.contains("(")) {
-      return "'"+s+"'";
-    }
-
-    return s;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java
deleted file mode 100644
index 2da3274..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.io.ops;
-
-import java.io.IOException;
-import java.util.UUID;
-
-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.StreamFactory;
-
-public class LessThanEqualToOperation extends LeafOperation {
-
-  private static final long serialVersionUID = 1;
-  private UUID operationNodeId = UUID.randomUUID();
-
-  public void operate(Tuple tuple) {
-    this.tuple = tuple;
-  }
-
-  public LessThanEqualToOperation(String field, double val) {
-    super(field, val);
-  }
-
-  public LessThanEqualToOperation(StreamExpression expression, StreamFactory factory) throws IOException {
-    super(expression, factory);
-  }
-
-  public boolean evaluate() {
-    Double d = tuple.getDouble(field);
-
-    if(d == null) {
-      return true;
-    }
-
-    return d.doubleValue() <= val;
-  }
-
-  public StreamExpression toExpression(StreamFactory factory) throws IOException {
-    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
-    expression.addParameter(field);
-    expression.addParameter(Double.toString(val));
-    return expression;
-  }
-
-  @Override
-  public Explanation toExplanation(StreamFactory factory) throws IOException {
-    return new Explanation(operationNodeId.toString())
-        .withExpressionType(ExpressionType.OPERATION)
-        .withFunctionName(factory.getFunctionName(getClass()))
-        .withImplementingClass(getClass().getName())
-        .withExpression(toExpression(factory).toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java
deleted file mode 100644
index c1cec95..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.io.ops;
-
-import java.io.IOException;
-import java.util.UUID;
-
-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.StreamFactory;
-
-public class LessThanOperation extends LeafOperation {
-
-  private static final long serialVersionUID = 1;
-  private UUID operationNodeId = UUID.randomUUID();
-
-  public void operate(Tuple tuple) {
-    this.tuple = tuple;
-  }
-
-  public LessThanOperation(String field, double val) {
-    super(field, val);
-  }
-
-  public LessThanOperation(StreamExpression expression, StreamFactory factory) throws IOException {
-    super(expression, factory);
-  }
-
-  public boolean evaluate() {
-    Double d = tuple.getDouble(field);
-
-    if(d == null) {
-      return true;
-    }
-    
-    return d.doubleValue() < val;
-  }
-
-  public StreamExpression toExpression(StreamFactory factory) throws IOException {
-    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
-    expression.addParameter(field);
-    expression.addParameter(Double.toString(val));
-    return expression;
-  }
-
-  @Override
-  public Explanation toExplanation(StreamFactory factory) throws IOException {
-    return new Explanation(operationNodeId.toString())
-        .withExpressionType(ExpressionType.OPERATION)
-        .withFunctionName(factory.getFunctionName(getClass()))
-        .withImplementingClass(getClass().getName())
-        .withExpression(toExpression(factory).toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/NotOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/NotOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/NotOperation.java
deleted file mode 100644
index 0e40b72..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/NotOperation.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.io.ops;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.UUID;
-
-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.Expressible;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
-import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
-
-
-public class NotOperation implements BooleanOperation {
-
-  private static final long serialVersionUID = 1;
-  private UUID operationNodeId = UUID.randomUUID();
-
-  protected BooleanOperation operand;
-
-  public void operate(Tuple tuple) {
-    operand.operate(tuple);
-  }
-
-  public NotOperation(BooleanOperation operand) {
-    this.operand = operand;
-  }
-
-  public NotOperation(StreamExpression expression, StreamFactory factory) throws IOException {
-    List<StreamExpression> operationExpressions = factory.getExpressionOperandsRepresentingTypes(expression, BooleanOperation.class);
-    if(operationExpressions != null && operationExpressions.size() == 1) {
-      StreamExpression op = operationExpressions.get(0);
-      StreamOperation streamOp = factory.constructOperation(op);
-      if(streamOp instanceof BooleanOperation) {
-        operand = (BooleanOperation) streamOp;
-      } else {
-        throw new IOException("The NotOperation requires a BooleanOperation.");
-      }
-
-    } else {
-      throw new IOException("The NotOperation requires a BooleanOperations.");
-    }
-  }
-
-  public boolean evaluate() {
-    return !operand.evaluate();
-  }
-
-  @Override
-  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
-    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
-    if(operand instanceof Expressible) {
-      expression.addParameter(operand.toExpression(factory));
-    } else {
-      throw new IOException("The operand of the NotOperation contains a non-expressible operation - it cannot be converted to an expression");
-    }
-    return expression;
-  }
-
-  @Override
-  public Explanation toExplanation(StreamFactory factory) throws IOException {
-    return new Explanation(operationNodeId.toString())
-        .withExpressionType(ExpressionType.OPERATION)
-        .withFunctionName(factory.getFunctionName(getClass()))
-        .withImplementingClass(getClass().getName())
-        .withExpression(toExpression(factory).toString());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/OrOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/OrOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/OrOperation.java
deleted file mode 100644
index faac5cd..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/OrOperation.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.io.ops;
-
-import java.io.IOException;
-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.Expressible;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
-import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
-
-public class OrOperation extends AndOperation {
-
-  private static final long serialVersionUID = 1;
-  private UUID operationNodeId = UUID.randomUUID();
-
-  public OrOperation(BooleanOperation leftOperand, BooleanOperation rightOperand) {
-    super(leftOperand, rightOperand);
-  }
-
-  public OrOperation(StreamExpression expression, StreamFactory factory) throws IOException {
-    super(expression, factory);
-  }
-
-  public boolean evaluate() {
-    return leftOperand.evaluate() || rightOperand.evaluate();
-  }
-
-  @Override
-  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
-    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
-    if(leftOperand instanceof Expressible) {
-      expression.addParameter(leftOperand.toExpression(factory));
-    } else {
-      throw new IOException("This left operand of the OrOperation contains a non-expressible operation - it cannot be converted to an expression");
-    }
-
-    if(rightOperand instanceof Expressible) {
-      expression.addParameter(rightOperand.toExpression(factory));
-    } else {
-      throw new IOException("This the right operand of the OrOperation contains a non-expressible operation - it cannot be converted to an expression");
-    }
-    return expression;
-  }
-
-  @Override
-  public Explanation toExplanation(StreamFactory factory) throws IOException {
-    return new Explanation(operationNodeId.toString())
-        .withExpressionType(ExpressionType.OPERATION)
-        .withFunctionName(factory.getFunctionName(getClass()))
-        .withImplementingClass(getClass().getName())
-        .withExpression(toExpression(factory).toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
index 38c1a6b..35e8952 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
@@ -23,8 +23,8 @@ import java.util.Locale;
 
 import org.apache.solr.client.solrj.io.Tuple;
 import org.apache.solr.client.solrj.io.comp.StreamComparator;
-import org.apache.solr.client.solrj.io.ops.BooleanOperation;
-import org.apache.solr.client.solrj.io.ops.StreamOperation;
+import org.apache.solr.client.solrj.io.eval.BooleanEvaluator;
+import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
 import org.apache.solr.client.solrj.io.stream.expr.Expressible;
@@ -42,19 +42,19 @@ public class HavingStream extends TupleStream implements Expressible {
   private static final long serialVersionUID = 1;
 
   private TupleStream stream;
-  private BooleanOperation op;
+  private BooleanEvaluator evaluator;
 
   private transient Tuple currentGroupHead;
 
-  public HavingStream(TupleStream stream, BooleanOperation op) throws IOException {
-    init(stream, op);
+  public HavingStream(TupleStream stream, BooleanEvaluator evaluator) throws IOException {
+    init(stream, evaluator);
   }
 
 
   public HavingStream(StreamExpression expression, StreamFactory factory) throws IOException{
     // grab all parameters out
     List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
-    List<StreamExpression> operationExpressions = factory.getExpressionOperandsRepresentingTypes(expression, BooleanOperation.class);
+    List<StreamExpression> evaluatorExpressions = factory.getExpressionOperandsRepresentingTypes(expression, BooleanEvaluator.class);
 
     // validate expression contains only what we want.
     if(expression.getParameters().size() != streamExpressions.size() + 1){
@@ -66,25 +66,23 @@ public class HavingStream extends TupleStream implements Expressible {
     }
 
 
-    BooleanOperation booleanOperation = null;
-    if(operationExpressions != null && operationExpressions.size() == 1) {
-      StreamExpression ex = operationExpressions.get(0);
-      StreamOperation operation = factory.constructOperation(ex);
-      if(operation instanceof BooleanOperation) {
-        booleanOperation = (BooleanOperation) operation;
-      } else {
-        throw new IOException("The HavingStream requires a BooleanOperation. A StreamOperation was provided.");
+    StreamEvaluator evaluator = null;
+    if(evaluatorExpressions != null && evaluatorExpressions.size() == 1) {
+      StreamExpression ex = evaluatorExpressions.get(0);
+      evaluator = factory.constructEvaluator(ex);
+      if(!(evaluator instanceof BooleanEvaluator)) {
+        throw new IOException("The HavingStream requires a BooleanEvaluator. A StreamEvaluator was provided.");
       }
     } else {
-      throw new IOException("The HavingStream requires a BooleanOperation.");
+      throw new IOException("The HavingStream requires a BooleanEvaluator.");
     }
 
-    init(factory.constructStream(streamExpressions.get(0)), booleanOperation);
+    init(factory.constructStream(streamExpressions.get(0)), (BooleanEvaluator)evaluator);
   }
 
-  private void init(TupleStream stream, BooleanOperation op) throws IOException{
+  private void init(TupleStream stream, BooleanEvaluator evaluator) throws IOException{
     this.stream = stream;
-    this.op = op;
+    this.evaluator = evaluator;
   }
 
   @Override
@@ -104,10 +102,10 @@ public class HavingStream extends TupleStream implements Expressible {
       expression.addParameter("<stream>");
     }
 
-    if(op instanceof Expressible) {
-      expression.addParameter(op.toExpression(factory));
+    if(evaluator instanceof Expressible) {
+      expression.addParameter(evaluator.toExpression(factory));
     } else {
-      throw new IOException("This ReducerStream contains a non-expressible operation - it cannot be converted to an expression");
+      throw new IOException("This HavingStream contains a non-expressible evaluator - it cannot be converted to an expression");
     }
 
     return expression;
@@ -125,7 +123,7 @@ public class HavingStream extends TupleStream implements Expressible {
         .withExpressionType(ExpressionType.STREAM_DECORATOR)
         .withExpression(toExpression(factory, false).toString())
         .withHelpers(new Explanation[]{
-            op.toExplanation(factory)
+            evaluator.toExplanation(factory)
         });
   }
 
@@ -154,9 +152,7 @@ public class HavingStream extends TupleStream implements Expressible {
         return tuple;
       }
 
-      op.operate(tuple);
-
-      if(op.evaluate()) {
+      if(evaluator.evaluate(tuple)){
         return tuple;
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SelectStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SelectStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SelectStream.java
index b0a1e05..eed8182 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SelectStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SelectStream.java
@@ -25,6 +25,7 @@ import java.util.Map;
 
 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.StreamEvaluator;
 import org.apache.solr.client.solrj.io.ops.StreamOperation;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
@@ -32,6 +33,7 @@ import org.apache.solr.client.solrj.io.stream.expr.Expressible;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
 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.StreamExpressionParser;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
 import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 
@@ -47,6 +49,7 @@ public class SelectStream extends TupleStream implements Expressible {
 
   private TupleStream stream;
   private Map<String,String> selectedFields;
+  private Map<StreamEvaluator,String> selectedEvaluators;
   private List<StreamOperation> operations;
 
   public SelectStream(TupleStream stream, List<String> selectedFields) throws IOException {
@@ -56,22 +59,25 @@ public class SelectStream extends TupleStream implements Expressible {
       this.selectedFields.put(selectedField, selectedField);
     }
     operations = new ArrayList<>();
+    selectedEvaluators = new HashMap<>();
   }
   
   public SelectStream(TupleStream stream, Map<String,String> selectedFields) throws IOException {
     this.stream = stream;
     this.selectedFields = selectedFields;
     operations = new ArrayList<>();
+    selectedEvaluators = new HashMap<>();
   }
   
   public SelectStream(StreamExpression expression,StreamFactory factory) throws IOException {
     // grab all parameters out
     List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
-    List<StreamExpressionParameter> selectFieldsExpressions = factory.getOperandsOfType(expression, StreamExpressionValue.class);
+    List<StreamExpressionParameter> selectAsFieldsExpressions = factory.getOperandsOfType(expression, StreamExpressionValue.class);
     List<StreamExpression> operationExpressions = factory.getExpressionOperandsRepresentingTypes(expression, StreamOperation.class);
+    List<StreamExpression> evaluatorExpressions = factory.getExpressionOperandsRepresentingTypes(expression, StreamEvaluator.class);
     
     // validate expression contains only what we want.
-    if(expression.getParameters().size() != streamExpressions.size() + selectFieldsExpressions.size() + operationExpressions.size()){
+    if(expression.getParameters().size() != streamExpressions.size() + selectAsFieldsExpressions.size() + operationExpressions.size()){
       throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - unknown operands found", expression));
     }
     
@@ -79,14 +85,19 @@ public class SelectStream extends TupleStream implements Expressible {
       throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting single stream but found %d (must be TupleStream types)",expression, streamExpressions.size()));
     }
 
-    if(0 == selectFieldsExpressions.size()){
+    if(0 == selectAsFieldsExpressions.size()){
       throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at least one select field but found %d",expression, streamExpressions.size()));
     }
+    
+    if(0 != evaluatorExpressions.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - evaluators must be given a name, like 'add(...) as result' but found %d evaluators without names",expression, evaluatorExpressions.size()));
+    }
 
     stream = factory.constructStream(streamExpressions.get(0));
     
-    selectedFields = new HashMap<String,String>(selectFieldsExpressions.size());
-    for(StreamExpressionParameter parameter : selectFieldsExpressions){
+    selectedFields = new HashMap<String,String>();
+    selectedEvaluators = new HashMap<StreamEvaluator, String>();
+    for(StreamExpressionParameter parameter : selectAsFieldsExpressions){
       StreamExpressionValue selectField = (StreamExpressionValue)parameter;
       String value = selectField.getValue().trim();
       
@@ -99,7 +110,28 @@ public class SelectStream extends TupleStream implements Expressible {
         if(2 != parts.length){
           throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting select field of form 'fieldA' or 'fieldA as alias' but found %s",expression, value));
         }
-        selectedFields.put(parts[0].trim(), parts[1].trim());
+        
+        String asValue = parts[0].trim();
+        String asName = parts[1].trim();
+        
+        boolean handled = false;
+        if(asValue.contains("(")){
+          // possible evaluator
+          try{
+            StreamExpression asValueExpression = StreamExpressionParser.parse(asValue);
+            if(factory.doesRepresentTypes(asValueExpression, StreamEvaluator.class)){
+              selectedEvaluators.put(factory.constructEvaluator(asValueExpression), asName);
+              handled = true;
+            }
+          }
+          catch(Throwable e){
+            // it was not handled, so treat as a non-evaluator
+          }
+        }
+        
+        if(!handled){        
+          selectedFields.put(asValue, asName);
+        }
       }
       else{
         selectedFields.put(value,value);
@@ -134,7 +166,7 @@ public class SelectStream extends TupleStream implements Expressible {
       expression.addParameter("<stream>");
     }
     
-    // selects
+    // selected fields
     for(Map.Entry<String, String> selectField : selectedFields.entrySet()) {
       if(selectField.getKey().equals(selectField.getValue())){
         expression.addParameter(selectField.getKey());
@@ -144,6 +176,11 @@ public class SelectStream extends TupleStream implements Expressible {
       }
     }
     
+    // selected evaluators
+    for(Map.Entry<StreamEvaluator, String> selectedEvaluator : selectedEvaluators.entrySet()) {
+      expression.addParameter(String.format(Locale.ROOT, "%s as %s", selectedEvaluator.getKey().toExpression(factory), selectedEvaluator.getValue()));
+    }
+    
     for(StreamOperation operation : operations){
       expression.addParameter(operation.toExpression(factory));
     }
@@ -163,6 +200,10 @@ public class SelectStream extends TupleStream implements Expressible {
       .withExpressionType(ExpressionType.STREAM_DECORATOR)
       .withExpression(toExpression(factory, false).toString());   
     
+    for(StreamEvaluator evaluator : selectedEvaluators.keySet()){
+      explanation.addHelper(evaluator.toExplanation(factory));
+    }
+    
     for(StreamOperation operation : operations){
       explanation.addHelper(operation.toExplanation(factory));
     }
@@ -196,19 +237,27 @@ public class SelectStream extends TupleStream implements Expressible {
     }
 
     // create a copy with the limited set of fields
-    Tuple working = new Tuple(new HashMap<>());
+    Tuple workingToReturn = new Tuple(new HashMap<>());
+    Tuple workingForEvaluators = new Tuple(new HashMap<>());
     for(Object fieldName : original.fields.keySet()){
+      workingForEvaluators.put(fieldName, original.get(fieldName));
       if(selectedFields.containsKey(fieldName)){
-        working.put(selectedFields.get(fieldName), original.get(fieldName));
+        workingToReturn.put(selectedFields.get(fieldName), original.get(fieldName));
       }
     }
     
     // apply all operations
     for(StreamOperation operation : operations){
-      operation.operate(working);
+      operation.operate(workingToReturn);
+      operation.operate(workingForEvaluators);
+    }
+    
+    // Apply all evaluators
+    for(Map.Entry<StreamEvaluator, String> selectedEvaluator : selectedEvaluators.entrySet()) {
+      workingToReturn.put(selectedEvaluator.getValue(), selectedEvaluator.getKey().evaluate(workingForEvaluators));
     }
     
-    return working;
+    return workingToReturn;
   }
   
   /** Return the stream sort - ie, the order in which records are returned */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/Explanation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/Explanation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/Explanation.java
index e72d6ed..acaefbf 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/Explanation.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/Explanation.java
@@ -155,6 +155,7 @@ public class Explanation implements MapSerializable {
     public static final String DATASTORE = "datastore";
     public static final String METRIC = "metric";
     public static final String OPERATION = "operation";
+    public static final String EVALUATOR = "evaluator";
     public static final String EQUALITOR = "equalitor";
     public static final String SORTER = "sorter";
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java
index bf20a1e..f57319d 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java
@@ -26,12 +26,14 @@ import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.stream.Collectors;
 
 import org.apache.solr.client.solrj.io.comp.ComparatorOrder;
 import org.apache.solr.client.solrj.io.comp.MultipleFieldComparator;
 import org.apache.solr.client.solrj.io.comp.StreamComparator;
 import org.apache.solr.client.solrj.io.eq.MultipleFieldEqualitor;
 import org.apache.solr.client.solrj.io.eq.StreamEqualitor;
+import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
 import org.apache.solr.client.solrj.io.ops.StreamOperation;
 import org.apache.solr.client.solrj.io.stream.TupleStream;
 import org.apache.solr.client.solrj.io.stream.metrics.Metric;
@@ -87,6 +89,10 @@ public class StreamFactory implements Serializable {
     return expression.getParameters().get(parameterIndex);
   }
   
+  public List<String> getValueOperands(StreamExpression expression){
+    return getOperandsOfType(expression, StreamExpressionValue.class).stream().map(item -> ((StreamExpressionValue)item).getValue()).collect(Collectors.toList());
+  }
+  
   /** Given an expression, will return the value parameter at the given index, or null if doesn't exist */
   public String getValueOperand(StreamExpression expression, int parameterIndex){
     StreamExpressionParameter parameter = getOperand(expression, parameterIndex);
@@ -176,6 +182,19 @@ public class StreamFactory implements Serializable {
     return matchingStreamExpressions;   
   }
   
+  public boolean doesRepresentTypes(StreamExpression expression, Class ... clazzes){
+    if(functionNames.containsKey(expression.getFunctionName())){
+      for(Class clazz : clazzes){
+        if(!clazz.isAssignableFrom(functionNames.get(expression.getFunctionName()))){
+          return false;
+        }
+      }
+      return true;
+    }
+    
+    return false;    
+  }
+  
   public int getIntOperand(StreamExpression expression, String paramName, Integer defaultValue) throws IOException{
     StreamExpressionNamedParameter param = getNamedOperand(expression, paramName);
     
@@ -343,6 +362,21 @@ public class StreamFactory implements Serializable {
     throw new IOException(String.format(Locale.ROOT,"Invalid operation expression %s - function '%s' is unknown (not mapped to a valid StreamOperation)", expression, expression.getFunctionName()));
   }
 
+  public StreamEvaluator constructEvaluator(String expressionClause) throws IOException {
+    return constructEvaluator(StreamExpressionParser.parse(expressionClause));
+  }
+  public StreamEvaluator constructEvaluator(StreamExpression expression) throws IOException{
+    String function = expression.getFunctionName();
+    if(functionNames.containsKey(function)){
+      Class<? extends Expressible> clazz = functionNames.get(function);
+      if(Expressible.class.isAssignableFrom(clazz) && StreamEvaluator.class.isAssignableFrom(clazz)){
+        return (StreamEvaluator)createInstance(functionNames.get(function), new Class[]{ StreamExpression.class, StreamFactory.class }, new Object[]{ expression, this});
+      }
+    }
+    
+    throw new IOException(String.format(Locale.ROOT,"Invalid evaluator expression %s - function '%s' is unknown (not mapped to a valid StreamEvaluator)", expression, expression.getFunctionName()));
+  }
+
 
   public <T> T createInstance(Class<T> clazz, Class<?>[] paramTypes, Object[] params) throws IOException{
     Constructor<T> ctor;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/SelectWithEvaluatorsTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/SelectWithEvaluatorsTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/SelectWithEvaluatorsTest.java
new file mode 100644
index 0000000..b91df8d
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/SelectWithEvaluatorsTest.java
@@ -0,0 +1,259 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.eval.AddEvaluator;
+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.stream.expr.StreamFactory;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.cloud.AbstractDistribZkTestBase;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ *  All base tests will be done with CloudSolrStream. Under the covers CloudSolrStream uses SolrStream so
+ *  SolrStream will get fully exercised through these tests.
+ *
+ **/
+
+@Slow
+@LuceneTestCase.SuppressCodecs({"Lucene3x", "Lucene40","Lucene41","Lucene42","Lucene45"})
+public class SelectWithEvaluatorsTest extends SolrCloudTestCase {
+
+  private static final String COLLECTIONORALIAS = "collection1";
+  private static final int TIMEOUT = DEFAULT_TIMEOUT;
+  private static final String id = "id";
+
+  private static boolean useAlias;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(4)
+        .addConfig("conf", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf"))
+        .addConfig("ml", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("ml").resolve("conf"))
+        .configure();
+
+    String collection;
+    useAlias = random().nextBoolean();
+    if (useAlias) {
+      collection = COLLECTIONORALIAS + "_collection";
+    } else {
+      collection = COLLECTIONORALIAS;
+    }
+    CollectionAdminRequest.createCollection(collection, "conf", 2, 1).process(cluster.getSolrClient());
+    AbstractDistribZkTestBase.waitForRecoveriesToFinish(collection, cluster.getSolrClient().getZkStateReader(),
+        false, true, TIMEOUT);
+    if (useAlias) {
+      CollectionAdminRequest.createAlias(COLLECTIONORALIAS, collection).process(cluster.getSolrClient());
+    }
+  }
+
+  @Before
+  public void cleanIndex() throws Exception {
+    new UpdateRequest()
+        .deleteByQuery("*:*")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+  }
+
+  @Test
+  public void testSelectWithEvaluatorsStream() throws Exception {
+
+    new UpdateRequest()
+        .add(id, "1", "a_s", "foo", "b_i", "1", "c_d", "3.3", "d_b", "true")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    String clause;
+    TupleStream stream;
+    List<Tuple> tuples;
+    
+    StreamFactory factory = new StreamFactory()
+      .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress())
+      .withFunctionName("search", CloudSolrStream.class)
+      .withFunctionName("select", SelectStream.class)
+      .withFunctionName("add", AddEvaluator.class)
+      .withFunctionName("if", IfThenElseEvaluator.class)
+      .withFunctionName("gt", GreaterThanEvaluator.class)
+      ;
+    
+    // Basic test
+    clause = "select("
+            +   "id,"
+            +   "add(b_i,c_d) as result,"
+            +   "search(collection1, q=*:*, fl=\"id,a_s,b_i,c_d,d_b\", sort=\"id asc\")"
+            + ")";
+    stream = factory.constructStream(clause);
+    tuples = getTuples(stream);
+    assertFields(tuples, "id", "result");
+    assertNotFields(tuples, "a_s", "b_i", "c_d", "d_b");
+    assertEquals(1, tuples.size());
+    assertDouble(tuples.get(0), "result", 4.3);
+    assertEquals(4.3, tuples.get(0).get("result"));
+
+  }
+  
+  protected List<Tuple> getTuples(TupleStream tupleStream) throws IOException {
+    tupleStream.open();
+    List<Tuple> tuples = new ArrayList<Tuple>();
+    for(Tuple t = tupleStream.read(); !t.EOF; t = tupleStream.read()) {
+      tuples.add(t);
+    }
+    tupleStream.close();
+    return tuples;
+  }
+  protected boolean assertOrder(List<Tuple> tuples, int... ids) throws Exception {
+    return assertOrderOf(tuples, "id", ids);
+  }
+  protected boolean assertOrderOf(List<Tuple> tuples, String fieldName, int... ids) throws Exception {
+    int i = 0;
+    for(int val : ids) {
+      Tuple t = tuples.get(i);
+      String tip = t.getString(fieldName);
+      if(!tip.equals(Integer.toString(val))) {
+        throw new Exception("Found value:"+tip+" expecting:"+val);
+      }
+      ++i;
+    }
+    return true;
+  }
+
+  protected boolean assertMapOrder(List<Tuple> tuples, int... ids) throws Exception {
+    int i = 0;
+    for(int val : ids) {
+      Tuple t = tuples.get(i);
+      List<Map> tip = t.getMaps("group");
+      int id = (int)tip.get(0).get("id");
+      if(id != val) {
+        throw new Exception("Found value:"+id+" expecting:"+val);
+      }
+      ++i;
+    }
+    return true;
+  }
+
+  protected boolean assertFields(List<Tuple> tuples, String ... fields) throws Exception{
+    for(Tuple tuple : tuples){
+      for(String field : fields){
+        if(!tuple.fields.containsKey(field)){
+          throw new Exception(String.format(Locale.ROOT, "Expected field '%s' not found", field));
+        }
+      }
+    }
+    return true;
+  }
+  protected boolean assertNotFields(List<Tuple> tuples, String ... fields) throws Exception{
+    for(Tuple tuple : tuples){
+      for(String field : fields){
+        if(tuple.fields.containsKey(field)){
+          throw new Exception(String.format(Locale.ROOT, "Unexpected field '%s' found", field));
+        }
+      }
+    }
+    return true;
+  }  
+
+  protected boolean assertGroupOrder(Tuple tuple, int... ids) throws Exception {
+    List<?> group = (List<?>)tuple.get("tuples");
+    int i=0;
+    for(int val : ids) {
+      Map<?,?> t = (Map<?,?>)group.get(i);
+      Long tip = (Long)t.get("id");
+      if(tip.intValue() != val) {
+        throw new Exception("Found value:"+tip.intValue()+" expecting:"+val);
+      }
+      ++i;
+    }
+    return true;
+  }
+
+  public boolean assertLong(Tuple tuple, String fieldName, long l) throws Exception {
+    long lv = (long)tuple.get(fieldName);
+    if(lv != l) {
+      throw new Exception("Longs not equal:"+l+" : "+lv);
+    }
+
+    return true;
+  }
+  
+  public boolean assertDouble(Tuple tuple, String fieldName, double expectedValue) throws Exception {
+    double value = (double)tuple.get(fieldName);
+    if(expectedValue != value) {
+      throw new Exception("Doubles not equal:"+value+" : "+expectedValue);
+    }
+
+    return true;
+  }
+  
+  public boolean assertString(Tuple tuple, String fieldName, String expected) throws Exception {
+    String actual = (String)tuple.get(fieldName);
+    
+    if( (null == expected && null != actual) ||
+        (null != expected && null == actual) ||
+        (null != expected && !expected.equals(actual))){
+      throw new Exception("Longs not equal:"+expected+" : "+actual);
+    }
+
+    return true;
+  }
+  
+  protected boolean assertMaps(List<Map> maps, int... ids) throws Exception {
+    if(maps.size() != ids.length) {
+      throw new Exception("Expected id count != actual map count:"+ids.length+":"+maps.size());
+    }
+
+    int i=0;
+    for(int val : ids) {
+      Map t = maps.get(i);
+      String tip = (String)t.get("id");
+      if(!tip.equals(Integer.toString(val))) {
+        throw new Exception("Found value:"+tip+" expecting:"+val);
+      }
+      ++i;
+    }
+    return true;
+  }
+
+  private boolean assertList(List list, Object... vals) throws Exception {
+
+    if(list.size() != vals.length) {
+      throw new Exception("Lists are not the same size:"+list.size() +" : "+vals.length);
+    }
+
+    for(int i=0; i<list.size(); i++) {
+      Object a = list.get(i);
+      Object b = vals[i];
+      if(!a.equals(b)) {
+        throw new Exception("List items not equals:"+a+" : "+b);
+      }
+    }
+
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index 5b806a8..ebc3250 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -33,16 +33,19 @@ import org.apache.solr.client.solrj.io.SolrClientCache;
 import org.apache.solr.client.solrj.io.Tuple;
 import org.apache.solr.client.solrj.io.comp.ComparatorOrder;
 import org.apache.solr.client.solrj.io.comp.FieldComparator;
-import org.apache.solr.client.solrj.io.ops.AndOperation;
+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.EqualsEvaluator;
+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.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.ops.ConcatOperation;
-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.expr.StreamExpression;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParser;
@@ -848,14 +851,14 @@ public class StreamExpressionTest extends SolrCloudTestCase {
         .withFunctionName("having", HavingStream.class)
         .withFunctionName("rollup", RollupStream.class)
         .withFunctionName("sum", SumMetric.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);
+        .withFunctionName("and", AndEvaluator.class)
+        .withFunctionName("or", OrEvaluator.class)
+        .withFunctionName("not", NotEvaluator.class)
+        .withFunctionName("gt", GreaterThanEvaluator.class)
+        .withFunctionName("lt", LessThanEvaluator.class)
+        .withFunctionName("eq", EqualsEvaluator.class)
+        .withFunctionName("lteq", LessThanEqualToEvaluator.class)
+        .withFunctionName("gteq", GreaterThanEqualToEvaluator.class);
 
     stream = factory.constructStream("having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), eq(a_i, 9))");
     StreamContext context = new StreamContext();
@@ -956,14 +959,15 @@ public class StreamExpressionTest extends SolrCloudTestCase {
         .withFunctionName("having", HavingStream.class)
         .withFunctionName("rollup", RollupStream.class)
         .withFunctionName("sum", SumMetric.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)
+        .withFunctionName("and", AndEvaluator.class)
+        .withFunctionName("or", OrEvaluator.class)
+        .withFunctionName("not", NotEvaluator.class)
+        .withFunctionName("gt", GreaterThanEvaluator.class)
+        .withFunctionName("lt", LessThanEvaluator.class)
+        .withFunctionName("eq", EqualsEvaluator.class)
+        .withFunctionName("lteq", LessThanEqualToEvaluator.class)
+        .withFunctionName("gteq", GreaterThanEqualToEvaluator.class)
+        .withFunctionName("val", RawValueEvaluator.class)
         .withFunctionName("parallel", ParallelStream.class);
 
     stream = factory.constructStream("parallel(" + COLLECTIONORALIAS + ", workers=2, sort=\"a_f asc\", having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=id), eq(a_i, 9)))");
@@ -2192,6 +2196,9 @@ public class StreamExpressionTest extends SolrCloudTestCase {
       .withFunctionName("select", SelectStream.class)
       .withFunctionName("replace", ReplaceOperation.class)
       .withFunctionName("concat", ConcatOperation.class)
+      .withFunctionName("add", AddEvaluator.class)
+      .withFunctionName("if", IfThenElseEvaluator.class)
+      .withFunctionName("gt", GreaterThanEvaluator.class)
       ;
     
     // Basic test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/62489678/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AbsoluteValueEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AbsoluteValueEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AbsoluteValueEvaluatorTest.java
new file mode 100644
index 0000000..88d3447
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AbsoluteValueEvaluatorTest.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.Map;
+
+import junit.framework.Assert;
+
+import org.apache.commons.collections.map.HashedMap;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.eval.AbsoluteValueEvaluator;
+import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.junit.Test;
+
+public class AbsoluteValueEvaluatorTest extends LuceneTestCase {
+
+  StreamFactory factory;
+  Map<String, Object> values;
+  
+  public AbsoluteValueEvaluatorTest() {
+    super();
+    
+    factory = new StreamFactory()
+      .withFunctionName("abs", AbsoluteValueEvaluator.class);
+    values = new HashedMap();
+  }
+    
+  @Test
+  public void absoluteValueOneField() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("abs(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 Double);
+    Assert.assertEquals(1.1D, result);
+    
+    values.clear();
+    values.put("a", -1.1);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(1.1D, result);
+  }
+
+  @Test(expected = IOException.class)
+  public void absNoField() throws Exception{
+    factory.constructEvaluator("abs()");
+  }
+  
+  @Test(expected = IOException.class)
+  public void absTwoFields() throws Exception{
+    factory.constructEvaluator("abs(a,b)");
+  }
+  
+  @Test
+  public void absNoValue() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("abs(a)");
+    
+    values.clear();
+    Object result = evaluator.evaluate(new Tuple(values));
+    assertNull(result);
+  }
+  @Test
+  public void absNullValue() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("abs(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/62489678/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AddEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AddEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AddEvaluatorTest.java
new file mode 100644
index 0000000..7115452
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AddEvaluatorTest.java
@@ -0,0 +1,336 @@
+/*
+ * 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.Map;
+
+import junit.framework.Assert;
+
+import org.apache.commons.collections.map.HashedMap;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.eval.AddEvaluator;
+import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.junit.Test;
+
+public class AddEvaluatorTest extends LuceneTestCase {
+
+  StreamFactory factory;
+  Map<String, Object> values;
+  
+  public AddEvaluatorTest() {
+    super();
+    
+    factory = new StreamFactory()
+      .withFunctionName("add", AddEvaluator.class);
+    values = new HashedMap();
+  }
+    
+  @Test
+  public void addTwoFieldsWithValues() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("add(a,b)");
+    Object result;
+    
+    values.clear();
+    values.put("a", 1);
+    values.put("b", 2);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Long);
+    Assert.assertEquals(3L, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", 2);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(3.1D, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", 2.1);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(3.2D, result);
+  }
+
+  @Test(expected = IOException.class)
+  public void addOneField() throws Exception{
+    factory.constructEvaluator("add(a)");
+  }
+  
+  @Test
+  public void addTwoFieldWithNulls() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("add(a,b)");
+    Object result;
+    
+    values.clear();
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertNull(result);
+  }
+  
+  @Test
+  public void addTwoFieldsWithNull() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("add(a,b)");
+    Object result;
+    
+    values.clear();
+    values.put("a", 1);
+    values.put("b", null);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertNull(result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", null);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertNull(result);
+    
+    values.clear();
+    values.put("a", null);
+    values.put("b", 1.1);    
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertNull(result);
+  }
+
+  @Test
+  public void addTwoFieldsWithMissingField() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("add(a,b)");
+    Object result;
+    
+    values.clear();
+    values.put("a", 1);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertNull(result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertNull(result);
+    
+    values.clear();
+    values.put("b", 1.1);    
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertNull(result);
+  }
+
+  @Test
+  public void addManyFieldsWithValues() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("add(a,b,c,d)");
+    Object result;
+    
+    values.clear();
+    values.put("a", 1);
+    values.put("b", 2);
+    values.put("c", 3);
+    values.put("d", 4);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Long);
+    Assert.assertEquals(10L, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", 2);
+    values.put("c", 3);
+    values.put("d", 4);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(10.1D, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", 2.1);
+    values.put("c", 3.1);
+    values.put("d", 4.1);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(10.4D, result);
+  }
+  
+  @Test
+  public void addManyFieldsWithSubAdds() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("add(a,b,add(c,d))");
+    Object result;
+    
+    values.clear();
+    values.put("a", 1);
+    values.put("b", 2);
+    values.put("c", 3);
+    values.put("d", 4);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Long);
+    Assert.assertEquals(10L, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", 2);
+    values.put("c", 3);
+    values.put("d", 4);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(10.1D, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", 2.1);
+    values.put("c", 3.1);
+    values.put("d", 4.1);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(10.4D, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", 2.1);
+    values.put("c", 3.1);
+    values.put("d", 4.123456789123456);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(10.423456789123456, result);
+    
+    values.clear();
+    values.put("a", 123456789123456789L);
+    values.put("b", 123456789123456789L);
+    values.put("c", 123456789123456789L);
+    values.put("d", 123456789123456789L);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Long);
+    Assert.assertEquals(4 * 123456789123456789L, result);
+  }
+  
+  @Test
+  public void addManyFieldsWithManySubAdds() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("add(add(a,b),add(c,d),add(c,a))");
+    Object result;
+    
+    values.clear();
+    values.put("a", 1);
+    values.put("b", 2);
+    values.put("c", 3);
+    values.put("d", 4);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Long);
+    Assert.assertEquals(14L, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", 2);
+    values.put("c", 3);
+    values.put("d", 4);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(14.2D, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", 2.1);
+    values.put("c", 3.1);
+    values.put("d", 4.1);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(14.6D, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", 2.1);
+    values.put("c", 3.1);
+    values.put("d", 4.123456789123456);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(14.623456789123456, result);
+    
+    values.clear();
+    values.put("a", 123456789123456789L);
+    values.put("b", 123456789123456789L);
+    values.put("c", 123456789123456789L);
+    values.put("d", 123456789123456789L);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Long);
+    Assert.assertEquals(6 * 123456789123456789L, result);
+    
+    values.clear();
+    values.put("a", 4.123456789123456);
+    values.put("b", 4.123456789123456);
+    values.put("c", 4.123456789123456);
+    values.put("d", 4.123456789123456);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(6 * 4.123456789123456, result);
+  }
+  
+  @Test
+  public void addManyFieldsWithManySubAddsWithNegative() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("add(add(a,b),add(c,d),add(c,a))");
+    Object result;
+    
+    values.clear();
+    values.put("a", -1);
+    values.put("b", 2);
+    values.put("c", 3);
+    values.put("d", 4);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Long);
+    Assert.assertEquals(10L, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", 2);
+    values.put("c", -3);
+    values.put("d", 4);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(2.2D, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", 2.1);
+    values.put("c", -3.1);
+    values.put("d", 4.1);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(2.2D, result);
+    
+    values.clear();
+    values.put("a", 1.1);
+    values.put("b", 2.1);
+    values.put("c", -3.1);
+    values.put("d", 5.223456789123456);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(3.323456789123456, result);
+    
+    values.clear();
+    values.put("a", 123456789123456789L);
+    values.put("b", -123456789123456789L);
+    values.put("c", 123456789123456789L);
+    values.put("d", 123456789123456789L);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Long);
+    Assert.assertEquals(4 * 123456789123456789L, result);
+    
+    values.clear();
+    values.put("a", -4.123456789123456);
+    values.put("b", -4.123456789123456);
+    values.put("c", -4.123456789123456);
+    values.put("d", -4.123456789123456);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertEquals(6 * -4.123456789123456, result);
+  }
+
+}