You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cp...@apache.org on 2017/05/25 17:55:47 UTC

[38/44] lucene-solr:jira/solr-8668: SOLR-10746: Move all Stream Evaluators to the eval package

SOLR-10746: Move all Stream Evaluators to the eval package


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

Branch: refs/heads/jira/solr-8668
Commit: 7fef7e3374da8ea79616d56d583465dea7ead20b
Parents: 6d80320
Author: Joel Bernstein <jb...@apache.org>
Authored: Thu May 25 10:32:28 2017 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Thu May 25 10:32:48 2017 -0400

----------------------------------------------------------------------
 .../client/solrj/io/eval/ColumnEvaluator.java   |  76 +++++++++++
 .../solrj/io/eval/ConvolutionEvaluator.java     |  80 ++++++++++++
 .../client/solrj/io/eval/CopyOfEvaluator.java   |  82 ++++++++++++
 .../solrj/io/eval/CopyOfRangeEvaluator.java     |  83 ++++++++++++
 .../solrj/io/eval/CorrelationEvaluator.java     |  75 +++++++++++
 .../solrj/io/eval/CovarianceEvaluator.java      |  75 +++++++++++
 .../client/solrj/io/eval/DescribeEvaluator.java |  90 +++++++++++++
 .../client/solrj/io/eval/DistanceEvaluator.java |  75 +++++++++++
 .../io/eval/EmpiricalDistributionEvaluator.java | 127 ++++++++++++++++++
 .../solrj/io/eval/FindDelayEvaluator.java       |  90 +++++++++++++
 .../client/solrj/io/eval/LengthEvaluator.java   |  58 +++++++++
 .../solrj/io/eval/NormalizeEvaluator.java       |  74 +++++++++++
 .../solrj/io/eval/PercentileEvaluator.java      |  66 ++++++++++
 .../client/solrj/io/eval/PredictEvaluator.java  |  66 ++++++++++
 .../client/solrj/io/eval/RankEvaluator.java     |  73 +++++++++++
 .../solrj/io/eval/RegressionEvaluator.java      | 110 ++++++++++++++++
 .../client/solrj/io/eval/ReverseEvaluator.java  |  65 ++++++++++
 .../client/solrj/io/eval/ScaleEvaluator.java    |  76 +++++++++++
 .../client/solrj/io/stream/ColumnEvaluator.java |  77 -----------
 .../solrj/io/stream/ConvolutionEvaluator.java   |  82 ------------
 .../client/solrj/io/stream/CopyOfEvaluator.java |  84 ------------
 .../solrj/io/stream/CopyOfRangeEvaluator.java   |  85 ------------
 .../solrj/io/stream/CorrelationEvaluator.java   |  77 -----------
 .../solrj/io/stream/CovarianceEvaluator.java    |  77 -----------
 .../solrj/io/stream/DescribeEvaluator.java      |  92 -------------
 .../solrj/io/stream/DistanceEvaluator.java      |  77 -----------
 .../stream/EmpiricalDistributionEvaluator.java  | 129 -------------------
 .../solrj/io/stream/FindDelayEvaluator.java     |  92 -------------
 .../client/solrj/io/stream/LengthEvaluator.java |  60 ---------
 .../solrj/io/stream/NormalizeEvaluator.java     |  76 -----------
 .../solrj/io/stream/PercentileEvaluator.java    |  68 ----------
 .../solrj/io/stream/PredictEvaluator.java       |  68 ----------
 .../client/solrj/io/stream/RankEvaluator.java   |  75 -----------
 .../solrj/io/stream/RegressionEvaluator.java    | 112 ----------------
 .../solrj/io/stream/ReverseEvaluator.java       |  67 ----------
 .../client/solrj/io/stream/ScaleEvaluator.java  |  78 -----------
 36 files changed, 1441 insertions(+), 1476 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ColumnEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ColumnEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ColumnEvaluator.java
new file mode 100644
index 0000000..dbb17b0
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ColumnEvaluator.java
@@ -0,0 +1,76 @@
+/*
+ * 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.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 ColumnEvaluator extends SimpleEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+  private String name;
+  private String fieldName;
+ ;
+  public ColumnEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    String name = factory.getValueOperand(expression, 0);
+    String fieldName = factory.getValueOperand(expression, 1);
+    init(name, fieldName);
+  }
+
+  private void init(String name, String fieldName) {
+    this.name = name;
+    this.fieldName = fieldName;
+  }
+
+  public List<Number> evaluate(Tuple tuple) throws IOException {
+    List<Tuple> tuples = (List<Tuple>)tuple.get(name);
+    List<Number> column = new ArrayList(tuples.size());
+    for(Tuple t : tuples) {
+      Object o = t.get(fieldName);
+      if(o instanceof Number) {
+        column.add((Number)o);
+      } else {
+        throw new IOException("Found non-numeric in column:"+o.toString());
+      }
+    }
+    return column;
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConvolutionEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConvolutionEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConvolutionEvaluator.java
new file mode 100644
index 0000000..6ca178d
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConvolutionEvaluator.java
@@ -0,0 +1,80 @@
+/*
+ * 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.ArrayList;
+
+import org.apache.commons.math3.util.MathArrays;
+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 ConvolutionEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public ConvolutionEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public List<Number> evaluate(Tuple tuple) throws IOException {
+    StreamEvaluator colEval1 = subEvaluators.get(0);
+    StreamEvaluator colEval2 = subEvaluators.get(1);
+
+    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
+    List<Number> numbers2 = (List<Number>)colEval2.evaluate(tuple);
+    double[] column1 = new double[numbers1.size()];
+    double[] column2 = new double[numbers2.size()];
+
+    for(int i=0; i<numbers1.size(); i++) {
+      column1[i] = numbers1.get(i).doubleValue();
+    }
+
+    for(int i=0; i<numbers2.size(); i++) {
+      column2[i] = numbers2.get(i).doubleValue();
+    }
+
+    double[] conArray = MathArrays.convolve(column1, column2);
+    List<Number> conList = new ArrayList();
+    for(double d :conArray) {
+      conList.add(d);
+    }
+
+    return conList;
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CopyOfEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CopyOfEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CopyOfEvaluator.java
new file mode 100644
index 0000000..d379c41
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CopyOfEvaluator.java
@@ -0,0 +1,82 @@
+/*
+ * 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.Arrays;
+import java.util.List;
+
+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 CopyOfEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public CopyOfEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public List<Number> evaluate(Tuple tuple) throws IOException {
+    StreamEvaluator colEval1 = subEvaluators.get(0);
+
+    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
+    double[] vals = new double[numbers1.size()];
+
+    for(int i=0; i<vals.length; i++) {
+      vals[i] = numbers1.get(i).doubleValue();
+    }
+
+    if(subEvaluators.size() == 2) {
+      StreamEvaluator lengthEval = subEvaluators.get(1);
+      Number lengthNum = (Number)lengthEval.evaluate(tuple);
+      int length = lengthNum.intValue();
+      vals = Arrays.copyOf(vals, length);
+    } else {
+      vals = Arrays.copyOf(vals, vals.length);
+    }
+
+    List<Number> copyOf = new ArrayList(vals.length);
+
+    for(int i=0; i<vals.length; i++) {
+      copyOf.add(vals[i]);
+    }
+
+    return copyOf;
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CopyOfRangeEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CopyOfRangeEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CopyOfRangeEvaluator.java
new file mode 100644
index 0000000..f1e56dd
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CopyOfRangeEvaluator.java
@@ -0,0 +1,83 @@
+/*
+ * 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.Arrays;
+import java.util.List;
+
+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 CopyOfRangeEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public CopyOfRangeEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public List<Number> evaluate(Tuple tuple) throws IOException {
+    StreamEvaluator colEval1 = subEvaluators.get(0);
+
+    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
+    double[] vals = new double[numbers1.size()];
+
+    for(int i=0; i<vals.length; i++) {
+      vals[i] = numbers1.get(i).doubleValue();
+    }
+
+    StreamEvaluator startIndexEval = subEvaluators.get(1);
+    Number startIndexNum = (Number)startIndexEval.evaluate(tuple);
+    int startIndex = startIndexNum.intValue();
+
+    StreamEvaluator endIndexEval = subEvaluators.get(2);
+    Number endIndexNum = (Number)endIndexEval.evaluate(tuple);
+    int endIndex = endIndexNum.intValue();
+
+    vals = Arrays.copyOfRange(vals, startIndex, endIndex);
+
+    List<Number> copyOf = new ArrayList(vals.length);
+
+    for(int i=0; i<vals.length; i++) {
+      copyOf.add(vals[i]);
+    }
+
+    return copyOf;
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CorrelationEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CorrelationEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CorrelationEvaluator.java
new file mode 100644
index 0000000..fc3d8c3
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CorrelationEvaluator.java
@@ -0,0 +1,75 @@
+/*
+ * 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 org.apache.commons.math3.stat.correlation.PearsonsCorrelation;
+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 CorrelationEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public CorrelationEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public Number evaluate(Tuple tuple) throws IOException {
+    StreamEvaluator colEval1 = subEvaluators.get(0);
+    StreamEvaluator colEval2 = subEvaluators.get(1);
+
+    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
+    List<Number> numbers2 = (List<Number>)colEval2.evaluate(tuple);
+    double[] column1 = new double[numbers1.size()];
+    double[] column2 = new double[numbers2.size()];
+
+    for(int i=0; i<numbers1.size(); i++) {
+      column1[i] = numbers1.get(i).doubleValue();
+    }
+
+    for(int i=0; i<numbers2.size(); i++) {
+      column2[i] = numbers2.get(i).doubleValue();
+    }
+
+    PearsonsCorrelation pearsonsCorrelation = new PearsonsCorrelation();
+
+    return pearsonsCorrelation.correlation(column1, column2);
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CovarianceEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CovarianceEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CovarianceEvaluator.java
new file mode 100644
index 0000000..7a6de68
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CovarianceEvaluator.java
@@ -0,0 +1,75 @@
+/*
+ * 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 org.apache.commons.math3.stat.correlation.Covariance;
+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 CovarianceEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public CovarianceEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public Number evaluate(Tuple tuple) throws IOException {
+    StreamEvaluator colEval1 = subEvaluators.get(0);
+    StreamEvaluator colEval2 = subEvaluators.get(1);
+
+    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
+    List<Number> numbers2 = (List<Number>)colEval2.evaluate(tuple);
+    double[] column1 = new double[numbers1.size()];
+    double[] column2 = new double[numbers2.size()];
+
+    for(int i=0; i<numbers1.size(); i++) {
+      column1[i] = numbers1.get(i).doubleValue();
+    }
+
+    for(int i=0; i<numbers2.size(); i++) {
+      column2[i] = numbers2.get(i).doubleValue();
+    }
+
+    Covariance covariance = new Covariance();
+
+    return covariance.covariance(column1, column2);
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DescribeEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DescribeEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DescribeEvaluator.java
new file mode 100644
index 0000000..196afe5
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DescribeEvaluator.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.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics;
+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 DescribeEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public DescribeEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public Tuple evaluate(Tuple tuple) throws IOException {
+
+    if(subEvaluators.size() != 1) {
+      throw new IOException("describe expects 1 column as a parameters");
+    }
+
+    StreamEvaluator colEval = subEvaluators.get(0);
+
+    List<Number> numbers = (List<Number>)colEval.evaluate(tuple);
+    DescriptiveStatistics descriptiveStatistics = new DescriptiveStatistics();
+
+    for(Number n : numbers) {
+      descriptiveStatistics.addValue(n.doubleValue());
+    }
+
+
+    Map map = new HashMap();
+
+    map.put("max", descriptiveStatistics.getMax());
+    map.put("mean", descriptiveStatistics.getMean());
+    map.put("min", descriptiveStatistics.getMin());
+    map.put("stdev", descriptiveStatistics.getStandardDeviation());
+    map.put("sum", descriptiveStatistics.getSum());
+    map.put("N", descriptiveStatistics.getN());
+    map.put("var", descriptiveStatistics.getVariance());
+    map.put("kurtosis", descriptiveStatistics.getKurtosis());
+    map.put("skewness", descriptiveStatistics.getSkewness());
+    map.put("popVar", descriptiveStatistics.getPopulationVariance());
+    map.put("geometricMean", descriptiveStatistics.getGeometricMean());
+    map.put("sumsq", descriptiveStatistics.getSumsq());
+
+    return new Tuple(map);
+  }
+
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DistanceEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DistanceEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DistanceEvaluator.java
new file mode 100644
index 0000000..201da4b
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DistanceEvaluator.java
@@ -0,0 +1,75 @@
+/*
+ * 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 org.apache.commons.math3.ml.distance.EuclideanDistance;
+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 DistanceEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public DistanceEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public Number evaluate(Tuple tuple) throws IOException {
+    StreamEvaluator colEval1 = subEvaluators.get(0);
+    StreamEvaluator colEval2 = subEvaluators.get(1);
+
+    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
+    List<Number> numbers2 = (List<Number>)colEval2.evaluate(tuple);
+    double[] column1 = new double[numbers1.size()];
+    double[] column2 = new double[numbers2.size()];
+
+    for(int i=0; i<numbers1.size(); i++) {
+      column1[i] = numbers1.get(i).doubleValue();
+    }
+
+    for(int i=0; i<numbers2.size(); i++) {
+      column2[i] = numbers2.get(i).doubleValue();
+    }
+
+    EuclideanDistance distance = new EuclideanDistance();
+    return distance.compute(column1, column2);
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/EmpiricalDistributionEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/EmpiricalDistributionEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/EmpiricalDistributionEvaluator.java
new file mode 100644
index 0000000..6885352
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/EmpiricalDistributionEvaluator.java
@@ -0,0 +1,127 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Arrays;
+
+import org.apache.commons.math3.random.EmpiricalDistribution;
+import org.apache.commons.math3.stat.descriptive.StatisticalSummary;
+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 EmpiricalDistributionEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public EmpiricalDistributionEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public Tuple evaluate(Tuple tuple) throws IOException {
+
+    if(subEvaluators.size() != 1) {
+      throw new IOException("Empirical dist expects 1 column as a parameters");
+    }
+
+    StreamEvaluator colEval1 = subEvaluators.get(0);
+
+    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
+    double[] column1 = new double[numbers1.size()];
+
+    for(int i=0; i<numbers1.size(); i++) {
+      column1[i] = numbers1.get(i).doubleValue();
+    }
+
+    Arrays.sort(column1);
+    EmpiricalDistribution empiricalDistribution = new EmpiricalDistribution();
+    empiricalDistribution.load(column1);
+
+    Map map = new HashMap();
+    StatisticalSummary statisticalSummary = empiricalDistribution.getSampleStats();
+
+    map.put("max", statisticalSummary.getMax());
+    map.put("mean", statisticalSummary.getMean());
+    map.put("min", statisticalSummary.getMin());
+    map.put("stdev", statisticalSummary.getStandardDeviation());
+    map.put("sum", statisticalSummary.getSum());
+    map.put("N", statisticalSummary.getN());
+    map.put("var", statisticalSummary.getVariance());
+
+    return new EmpiricalDistributionTuple(empiricalDistribution, column1, map);
+  }
+
+  public static class EmpiricalDistributionTuple extends Tuple {
+
+    private EmpiricalDistribution empiricalDistribution;
+    private double[] backingArray;
+
+    public EmpiricalDistributionTuple(EmpiricalDistribution empiricalDistribution, double[] backingArray, Map map) {
+      super(map);
+      this.empiricalDistribution = empiricalDistribution;
+      this.backingArray = backingArray;
+    }
+
+    public double percentile(double d) {
+      int slot = Arrays.binarySearch(backingArray, d);
+
+      if(slot == 0) {
+        return 0.0;
+      }
+
+      if(slot < 0) {
+        if(slot == -1) {
+          return 0.0D;
+        } else {
+          //Not a direct hit
+          slot = Math.abs(slot);
+          --slot;
+          if(slot == backingArray.length) {
+            return 1.0D;
+          } else {
+            return (this.empiricalDistribution.cumulativeProbability(backingArray[slot]));
+          }
+        }
+      } else {
+        return this.empiricalDistribution.cumulativeProbability(backingArray[slot]);
+      }
+    }
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FindDelayEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FindDelayEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FindDelayEvaluator.java
new file mode 100644
index 0000000..c5a9b8a
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FindDelayEvaluator.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 org.apache.commons.math3.util.MathArrays;
+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 FindDelayEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public FindDelayEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public Number evaluate(Tuple tuple) throws IOException {
+    StreamEvaluator colEval1 = subEvaluators.get(0);
+    StreamEvaluator colEval2 = subEvaluators.get(1);
+
+    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
+    List<Number> numbers2 = (List<Number>)colEval2.evaluate(tuple);
+    double[] column1 = new double[numbers1.size()];
+    double[] column2 = new double[numbers2.size()];
+
+    for(int i=0; i<numbers1.size(); i++) {
+      column1[i] = numbers1.get(i).doubleValue();
+    }
+
+    //Reverse the second column.
+    //The convolve function will reverse it back.
+    //This allows correlation to be represented using the convolution math.
+    int rIndex=0;
+    for(int i=numbers2.size()-1; i>=0; i--) {
+      column2[rIndex++] = numbers2.get(i).doubleValue();
+    }
+
+    double[] convolution = MathArrays.convolve(column1, column2);
+    double max = -Double.MAX_VALUE;
+    double maxIndex = -1;
+
+    for(int i=0; i< convolution.length; i++) {
+      double abs = Math.abs(convolution[i]);
+      if(abs > max) {
+        max = abs;
+        maxIndex = i;
+      }
+    }
+
+    return (maxIndex+1)-column2.length;
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LengthEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LengthEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LengthEvaluator.java
new file mode 100644
index 0000000..da55ee4
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LengthEvaluator.java
@@ -0,0 +1,58 @@
+/*
+ * 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 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 LengthEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public LengthEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public Number evaluate(Tuple tuple) throws IOException {
+    StreamEvaluator colEval1 = subEvaluators.get(0);
+    List<Number> numbers = (List<Number>)colEval1.evaluate(tuple);
+    return numbers.size();
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NormalizeEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NormalizeEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NormalizeEvaluator.java
new file mode 100644
index 0000000..c85ac20
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NormalizeEvaluator.java
@@ -0,0 +1,74 @@
+/*
+ * 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.commons.math3.stat.StatUtils;
+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 NormalizeEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public NormalizeEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public List<Number> evaluate(Tuple tuple) throws IOException {
+    StreamEvaluator colEval1 = subEvaluators.get(0);
+
+    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
+    double[] column1 = new double[numbers1.size()];
+
+    for(int i=0; i<numbers1.size(); i++) {
+      column1[i] = numbers1.get(i).doubleValue();
+    }
+
+    double[] normalized = StatUtils.normalize(column1);
+
+    List<Number> normalizeList = new ArrayList(normalized.length);
+    for(double d : normalized) {
+      normalizeList.add(d);
+    }
+
+    return normalizeList;
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PercentileEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PercentileEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PercentileEvaluator.java
new file mode 100644
index 0000000..19d423d
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PercentileEvaluator.java
@@ -0,0 +1,66 @@
+/*
+ * 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.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 PercentileEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public PercentileEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public Number evaluate(Tuple tuple) throws IOException {
+
+    if(subEvaluators.size() != 2) {
+      throw new IOException("Percentile expects 2 parameters: a regression result and a number");
+    }
+
+    StreamEvaluator r = subEvaluators.get(0);
+    StreamEvaluator d = subEvaluators.get(1);
+
+    EmpiricalDistributionEvaluator.EmpiricalDistributionTuple e = (EmpiricalDistributionEvaluator.EmpiricalDistributionTuple)r.evaluate(tuple);
+    Number n = (Number)d.evaluate(tuple);
+    return e.percentile(n.doubleValue());
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PredictEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PredictEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PredictEvaluator.java
new file mode 100644
index 0000000..0d1e763
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PredictEvaluator.java
@@ -0,0 +1,66 @@
+/*
+ * 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.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 PredictEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public PredictEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public Number evaluate(Tuple tuple) throws IOException {
+
+    if(subEvaluators.size() != 2) {
+      throw new IOException("Predict expects 2 parameters: a regression result and a number");
+    }
+
+    StreamEvaluator r = subEvaluators.get(0);
+    StreamEvaluator d = subEvaluators.get(1);
+
+    RegressionEvaluator.RegressionTuple rt= (RegressionEvaluator.RegressionTuple)r.evaluate(tuple);
+    Number n = (Number)d.evaluate(tuple);
+    return rt.predict(n.doubleValue());
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RankEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RankEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RankEvaluator.java
new file mode 100644
index 0000000..8a22e94
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RankEvaluator.java
@@ -0,0 +1,73 @@
+/*
+ * 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.commons.math3.stat.ranking.NaturalRanking;
+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 RankEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public RankEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public List<Number> evaluate(Tuple tuple) throws IOException {
+    StreamEvaluator colEval = subEvaluators.get(0);
+
+    List<Number> numbers = (List<Number>)colEval.evaluate(tuple);
+    double[] values = new double[numbers.size()];
+    for(int i=0; i<numbers.size(); i++) {
+      values[i] = numbers.get(i).doubleValue();
+    }
+
+    NaturalRanking rank = new NaturalRanking();
+    double[] ranked = rank.rank(values);
+    List<Number> rankedList = new ArrayList();
+    for(int i=0; i<numbers.size(); i++) {
+      rankedList.add(ranked[i]);
+    }
+
+    return rankedList;
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RegressionEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RegressionEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RegressionEvaluator.java
new file mode 100644
index 0000000..42a6955
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RegressionEvaluator.java
@@ -0,0 +1,110 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.math3.stat.regression.SimpleRegression;
+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 RegressionEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public RegressionEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public Tuple evaluate(Tuple tuple) throws IOException {
+
+    if(subEvaluators.size() != 2) {
+      throw new IOException("Regress expects 2 columns as parameters");
+    }
+
+    StreamEvaluator colEval1 = subEvaluators.get(0);
+    StreamEvaluator colEval2 = subEvaluators.get(1);
+
+    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
+    List<Number> numbers2 = (List<Number>)colEval2.evaluate(tuple);
+    double[] column1 = new double[numbers1.size()];
+    double[] column2 = new double[numbers2.size()];
+
+    for(int i=0; i<numbers1.size(); i++) {
+      column1[i] = numbers1.get(i).doubleValue();
+    }
+
+    for(int i=0; i<numbers2.size(); i++) {
+      column2[i] = numbers2.get(i).doubleValue();
+    }
+
+    SimpleRegression regression = new SimpleRegression();
+    for(int i=0; i<column1.length; i++) {
+      regression.addData(column1[i], column2[i]);
+    }
+
+    Map map = new HashMap();
+    map.put("slope", regression.getSlope());
+    map.put("intercept", regression.getIntercept());
+    map.put("R", regression.getR());
+    map.put("N", regression.getN());
+    map.put("regressionSumSquares", regression.getRegressionSumSquares());
+    map.put("slopeConfidenceInterval", regression.getSlopeConfidenceInterval());
+    map.put("interceptStdErr", regression.getInterceptStdErr());
+    map.put("totalSumSquares", regression.getTotalSumSquares());
+    map.put("significance", regression.getSignificance());
+    map.put("meanSquareError", regression.getMeanSquareError());
+    return new RegressionTuple(regression, map);
+  }
+
+  public static class RegressionTuple extends Tuple {
+
+    private SimpleRegression simpleRegression;
+
+    public RegressionTuple(SimpleRegression simpleRegression, Map map) {
+      super(map);
+      this.simpleRegression = simpleRegression;
+    }
+
+    public double predict(double d) {
+      return this.simpleRegression.predict(d);
+    }
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ReverseEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ReverseEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ReverseEvaluator.java
new file mode 100644
index 0000000..016e995
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ReverseEvaluator.java
@@ -0,0 +1,65 @@
+/*
+ * 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.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 ReverseEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public ReverseEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public List<Number> evaluate(Tuple tuple) throws IOException {
+    StreamEvaluator colEval1 = subEvaluators.get(0);
+
+    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
+    List<Number> rev = new ArrayList();
+    for(int i=numbers1.size()-1; i>=0; i--) {
+      rev.add(numbers1.get(i));
+    }
+
+    return rev;
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ScaleEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ScaleEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ScaleEvaluator.java
new file mode 100644
index 0000000..8ff2a7c
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ScaleEvaluator.java
@@ -0,0 +1,76 @@
+/*
+ * 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.commons.math3.util.MathArrays;
+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 ScaleEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public ScaleEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public List<Number> evaluate(Tuple tuple) throws IOException {
+    StreamEvaluator numEval = subEvaluators.get(0);
+    StreamEvaluator colEval1 = subEvaluators.get(1);
+
+    Number num = (Number)numEval.evaluate(tuple);
+    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
+    double[] column1 = new double[numbers1.size()];
+
+    for(int i=0; i<numbers1.size(); i++) {
+      column1[i] = numbers1.get(i).doubleValue();
+    }
+
+    double[] scaled = MathArrays.scale(num.doubleValue(), column1);
+
+    List<Number> scaledList = new ArrayList(scaled.length);
+    for(double d : scaled) {
+      scaledList.add(d);
+    }
+
+    return scaledList;
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
+    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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ColumnEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ColumnEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ColumnEvaluator.java
deleted file mode 100644
index 3e56837..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ColumnEvaluator.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.io.stream;
-
-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.eval.SimpleEvaluator;
-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 ColumnEvaluator extends SimpleEvaluator implements Expressible {
-
-  private static final long serialVersionUID = 1;
-  private String name;
-  private String fieldName;
- ;
-  public ColumnEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
-    String name = factory.getValueOperand(expression, 0);
-    String fieldName = factory.getValueOperand(expression, 1);
-    init(name, fieldName);
-  }
-
-  private void init(String name, String fieldName) {
-    this.name = name;
-    this.fieldName = fieldName;
-  }
-
-  public List<Number> evaluate(Tuple tuple) throws IOException {
-    List<Tuple> tuples = (List<Tuple>)tuple.get(name);
-    List<Number> column = new ArrayList(tuples.size());
-    for(Tuple t : tuples) {
-      Object o = t.get(fieldName);
-      if(o instanceof Number) {
-        column.add((Number)o);
-      } else {
-        throw new IOException("Found non-numeric in column:"+o.toString());
-      }
-    }
-    return column;
-  }
-
-  @Override
-  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
-    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
-    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());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ConvolutionEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ConvolutionEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ConvolutionEvaluator.java
deleted file mode 100644
index 6d6e3e3..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ConvolutionEvaluator.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.io.stream;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.ArrayList;
-
-import org.apache.commons.math3.util.MathArrays;
-import org.apache.solr.client.solrj.io.Tuple;
-import org.apache.solr.client.solrj.io.eval.ComplexEvaluator;
-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;
-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 ConvolutionEvaluator extends ComplexEvaluator implements Expressible {
-
-  private static final long serialVersionUID = 1;
-
-  public ConvolutionEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
-    super(expression, factory);
-  }
-
-  public List<Number> evaluate(Tuple tuple) throws IOException {
-    StreamEvaluator colEval1 = subEvaluators.get(0);
-    StreamEvaluator colEval2 = subEvaluators.get(1);
-
-    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
-    List<Number> numbers2 = (List<Number>)colEval2.evaluate(tuple);
-    double[] column1 = new double[numbers1.size()];
-    double[] column2 = new double[numbers2.size()];
-
-    for(int i=0; i<numbers1.size(); i++) {
-      column1[i] = numbers1.get(i).doubleValue();
-    }
-
-    for(int i=0; i<numbers2.size(); i++) {
-      column2[i] = numbers2.get(i).doubleValue();
-    }
-
-    double[] conArray = MathArrays.convolve(column1, column2);
-    List<Number> conList = new ArrayList();
-    for(double d :conArray) {
-      conList.add(d);
-    }
-
-    return conList;
-  }
-
-  @Override
-  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
-    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
-    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());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CopyOfEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CopyOfEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CopyOfEvaluator.java
deleted file mode 100644
index 2380e8f..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CopyOfEvaluator.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.io.stream;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.solr.client.solrj.io.Tuple;
-import org.apache.solr.client.solrj.io.eval.ComplexEvaluator;
-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;
-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 CopyOfEvaluator extends ComplexEvaluator implements Expressible {
-
-  private static final long serialVersionUID = 1;
-
-  public CopyOfEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
-    super(expression, factory);
-  }
-
-  public List<Number> evaluate(Tuple tuple) throws IOException {
-    StreamEvaluator colEval1 = subEvaluators.get(0);
-
-    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
-    double[] vals = new double[numbers1.size()];
-
-    for(int i=0; i<vals.length; i++) {
-      vals[i] = numbers1.get(i).doubleValue();
-    }
-
-    if(subEvaluators.size() == 2) {
-      StreamEvaluator lengthEval = subEvaluators.get(1);
-      Number lengthNum = (Number)lengthEval.evaluate(tuple);
-      int length = lengthNum.intValue();
-      vals = Arrays.copyOf(vals, length);
-    } else {
-      vals = Arrays.copyOf(vals, vals.length);
-    }
-
-    List<Number> copyOf = new ArrayList(vals.length);
-
-    for(int i=0; i<vals.length; i++) {
-      copyOf.add(vals[i]);
-    }
-
-    return copyOf;
-  }
-
-  @Override
-  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
-    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
-    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());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7fef7e33/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CopyOfRangeEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CopyOfRangeEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CopyOfRangeEvaluator.java
deleted file mode 100644
index c4c1e77..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CopyOfRangeEvaluator.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.io.stream;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.solr.client.solrj.io.Tuple;
-import org.apache.solr.client.solrj.io.eval.ComplexEvaluator;
-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;
-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 CopyOfRangeEvaluator extends ComplexEvaluator implements Expressible {
-
-  private static final long serialVersionUID = 1;
-
-  public CopyOfRangeEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
-    super(expression, factory);
-  }
-
-  public List<Number> evaluate(Tuple tuple) throws IOException {
-    StreamEvaluator colEval1 = subEvaluators.get(0);
-
-    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
-    double[] vals = new double[numbers1.size()];
-
-    for(int i=0; i<vals.length; i++) {
-      vals[i] = numbers1.get(i).doubleValue();
-    }
-
-    StreamEvaluator startIndexEval = subEvaluators.get(1);
-    Number startIndexNum = (Number)startIndexEval.evaluate(tuple);
-    int startIndex = startIndexNum.intValue();
-
-    StreamEvaluator endIndexEval = subEvaluators.get(2);
-    Number endIndexNum = (Number)endIndexEval.evaluate(tuple);
-    int endIndex = endIndexNum.intValue();
-
-    vals = Arrays.copyOfRange(vals, startIndex, endIndex);
-
-    List<Number> copyOf = new ArrayList(vals.length);
-
-    for(int i=0; i<vals.length; i++) {
-      copyOf.add(vals[i]);
-    }
-
-    return copyOf;
-  }
-
-  @Override
-  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
-    StreamExpression expression = new StreamExpression(factory.getFunctionName(getClass()));
-    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());
-  }
-}
\ No newline at end of file