You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by tf...@apache.org on 2017/05/08 22:07:51 UTC

[37/50] [abbrv] lucene-solr:jira/solr-10233: SOLR-10622: Add regress and predict Stream Evaluators

SOLR-10622: Add regress and predict Stream Evaluators


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

Branch: refs/heads/jira/solr-10233
Commit: d724983c03249acd8a26e1c00b77b5573f6eefc0
Parents: 0ed39b2
Author: Joel Bernstein <jb...@apache.org>
Authored: Sun May 7 15:23:56 2017 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Sun May 7 15:24:21 2017 -0400

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |   2 +
 .../solrj/io/stream/PredictEvaluator.java       |  68 +++++++++++
 .../solrj/io/stream/RegressionEvaluator.java    | 112 +++++++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   |  50 ++++++++-
 4 files changed, 231 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d724983c/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index 474f5b0..f0282d0 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -168,6 +168,8 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("timeseries", TimeSeriesStream.class)
       .withFunctionName("tuple", TupStream.class)
       .withFunctionName("col", ColumnEvaluator.class)
+      .withFunctionName("predict", PredictEvaluator.class)
+      .withFunctionName("regress", RegressionEvaluator.class)
       // metrics
          .withFunctionName("min", MinMetric.class)
       .withFunctionName("max", MaxMetric.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d724983c/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/PredictEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/PredictEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/PredictEvaluator.java
new file mode 100644
index 0000000..3c3ab84
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/PredictEvaluator.java
@@ -0,0 +1,68 @@
+/*
+ * 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 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 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/d724983c/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/RegressionEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/RegressionEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/RegressionEvaluator.java
new file mode 100644
index 0000000..5306193
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/RegressionEvaluator.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.client.solrj.io.stream;
+
+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.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 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/d724983c/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 0af7935..41ffe09 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
@@ -5208,7 +5208,6 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
   }
 
-
   @Test
   public void testCorrelationStream() throws Exception {
     UpdateRequest updateRequest = new UpdateRequest();
@@ -5256,6 +5255,55 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
 
 
+
+  @Test
+  public void testRegressAndPredict() throws Exception {
+    UpdateRequest updateRequest = new UpdateRequest();
+
+
+    updateRequest.add(id, "1", "price_f", "100.0", "col_s", "a", "order_i", "1");
+    updateRequest.add(id, "2", "price_f", "200.0", "col_s", "a", "order_i", "2");
+    updateRequest.add(id, "3", "price_f", "300.0", "col_s", "a", "order_i", "3");
+    updateRequest.add(id, "4", "price_f", "100.0", "col_s", "a", "order_i", "4");
+    updateRequest.add(id, "5", "price_f", "200.0", "col_s", "a", "order_i", "5");
+    updateRequest.add(id, "6", "price_f", "400.0", "col_s", "a", "order_i", "6");
+    updateRequest.add(id, "7", "price_f", "600.0", "col_s", "a", "order_i", "7");
+
+    updateRequest.add(id, "8", "price_f", "200.0", "col_s", "b", "order_i", "1");
+    updateRequest.add(id, "9", "price_f", "400.0", "col_s", "b", "order_i", "2");
+    updateRequest.add(id, "10", "price_f", "600.0", "col_s", "b", "order_i", "3");
+    updateRequest.add(id, "11", "price_f", "200.0", "col_s", "b", "order_i", "4");
+    updateRequest.add(id, "12", "price_f", "400.0", "col_s", "b", "order_i", "5");
+    updateRequest.add(id, "13", "price_f", "800.0", "col_s", "b", "order_i", "6");
+    updateRequest.add(id, "14", "price_f", "1200.0", "col_s", "b", "order_i", "7");
+    updateRequest.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    String expr1 = "search("+COLLECTIONORALIAS+", q=\"col_s:a\", fl=\"price_f, order_i\", sort=\"order_i asc\")";
+    String expr2 = "search("+COLLECTIONORALIAS+", q=\"col_s:b\", fl=\"price_f, order_i\", sort=\"order_i asc\")";
+
+    String cexpr = "let(a="+expr1+", b="+expr2+", c=col(a, price_f), d=col(b, price_f), e=regress(c, d), tuple(regress=e, p=predict(e, 300)))";
+
+    ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+
+    String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString()+"/"+COLLECTIONORALIAS;
+    TupleStream solrStream = new SolrStream(url, paramsLoc);
+
+    StreamContext context = new StreamContext();
+    solrStream.setStreamContext(context);
+    List<Tuple> tuples = getTuples(solrStream);
+    assertTrue(tuples.size() == 1);
+    Tuple tuple = tuples.get(0);
+    Map regression = (Map)tuple.get("regress");
+    double slope = (double)regression.get("slope");
+    double intercept= (double) regression.get("intercept");
+    assertTrue(slope == 2.0D);
+    assertTrue(intercept == 0.0D);
+    double prediction = tuple.getDouble("p");
+    assertTrue(prediction == 600.0D);
+  }
+
   @Test
   public void testListStream() throws Exception {
     UpdateRequest updateRequest = new UpdateRequest();