You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jb...@apache.org on 2017/07/31 20:01:00 UTC

[1/2] lucene-solr:master: SOLR-11160: Add normalDistribution, uniformDistribution, sample and kolmogorovSmirnov Stream Evaluators

Repository: lucene-solr
Updated Branches:
  refs/heads/master 88614dd15 -> b058818a3


SOLR-11160: Add normalDistribution, uniformDistribution, sample and kolmogorovSmirnov 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/71517bc2
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/71517bc2
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/71517bc2

Branch: refs/heads/master
Commit: 71517bc29258e0f1640f30f37f0a0aad1ed98f7a
Parents: 88614dd
Author: Joel Bernstein <jb...@apache.org>
Authored: Mon Jul 31 15:39:55 2017 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Mon Jul 31 15:40:21 2017 -0400

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |   7 +-
 .../solr/client/solrj/io/eval/AscEvaluator.java |  80 +++++++++++++
 .../io/eval/CumulativeProbabilityEvaluator.java |  67 -----------
 .../io/eval/EmpiricalDistributionEvaluator.java |  53 +--------
 .../solrj/io/eval/HistogramEvaluator.java       |   6 +-
 .../io/eval/KolmogorovSmirnovEvaluator.java     | 102 ++++++++++++++++
 .../io/eval/NormalDistributionEvaluator.java    |  69 +++++++++++
 .../client/solrj/io/eval/SampleEvaluator.java   |  75 ++++++++++++
 .../io/eval/UniformDistributionEvaluator.java   |  69 +++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   | 116 +++++++++++--------
 10 files changed, 474 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71517bc2/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 d52972d..687eb41 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -200,7 +200,6 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
         .withFunctionName("copyOf", CopyOfEvaluator.class)
         .withFunctionName("copyOfRange", CopyOfRangeEvaluator.class)
         .withFunctionName("cov", CovarianceEvaluator.class)
-        .withFunctionName("cumulativeProbability", CumulativeProbabilityEvaluator.class)
         .withFunctionName("describe", DescribeEvaluator.class)
         .withFunctionName("distance", DistanceEvaluator.class)
         .withFunctionName("empiricalDistribution", EmpiricalDistributionEvaluator.class)
@@ -219,6 +218,12 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
         .withFunctionName("addAll", AddAllEvaluator.class)
         .withFunctionName("residuals", ResidualsEvaluator.class)
         .withFunctionName("plot", PlotStream.class)
+        .withFunctionName("normalDistribution", NormalDistributionEvaluator.class)
+        .withFunctionName("uniformDistribution", UniformDistributionEvaluator.class)
+        .withFunctionName("sample", SampleEvaluator.class)
+        .withFunctionName("kolmogorovSmirnov", KolmogorovSmirnovEvaluator.class)
+        .withFunctionName("ks", KolmogorovSmirnovEvaluator.class)
+        .withFunctionName("asc", AscEvaluator.class)
 
         // Boolean Stream Evaluators
         .withFunctionName("and", AndEvaluator.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71517bc2/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AscEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AscEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AscEvaluator.java
new file mode 100644
index 0000000..8e5e4a1
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/AscEvaluator.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.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.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 AscEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public AscEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+
+    if(1 != subEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size()));
+    }
+
+  }
+
+  public List<Number> evaluate(Tuple tuple) throws IOException {
+
+    StreamEvaluator colEval1 = subEvaluators.get(0);
+
+    List<Number> numbers1 = (List<Number>)colEval1.evaluate(tuple);
+    List<Number> asc = new ArrayList();
+    asc.addAll(numbers1);
+    Collections.sort(asc, new Comparator<Number>() {
+      @Override
+      public int compare(Number a, Number b) {
+        return new BigDecimal(a.toString()).compareTo(new BigDecimal(b.toString()));
+
+      }
+    });
+
+    return asc;
+  }
+
+  @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/71517bc2/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CumulativeProbabilityEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CumulativeProbabilityEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CumulativeProbabilityEvaluator.java
deleted file mode 100644
index 9e2bbaf..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CumulativeProbabilityEvaluator.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.client.solrj.io.eval;
-
-import java.io.IOException;
-import java.util.Locale;
-
-import org.apache.solr.client.solrj.io.Tuple;
-import org.apache.solr.client.solrj.io.stream.expr.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 CumulativeProbabilityEvaluator extends ComplexEvaluator implements Expressible {
-
-  private static final long serialVersionUID = 1;
-
-  public CumulativeProbabilityEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
-    super(expression, factory);
-    
-    if(2 != subEvaluators.size()){
-      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting two values (emperical distribution and a number) but found %d",expression,subEvaluators.size()));
-    }
-  }
-
-  public Number evaluate(Tuple tuple) throws IOException {
-
-    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/71517bc2/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
index 8456b4d..5a50702 100644
--- 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
@@ -17,14 +17,12 @@
 package org.apache.solr.client.solrj.io.eval;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Locale;
 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;
@@ -45,7 +43,7 @@ public class EmpiricalDistributionEvaluator extends ComplexEvaluator implements
     }
   }
 
-  public Tuple evaluate(Tuple tuple) throws IOException {
+  public Object evaluate(Tuple tuple) throws IOException {
 
     StreamEvaluator colEval1 = subEvaluators.get(0);
 
@@ -60,56 +58,9 @@ public class EmpiricalDistributionEvaluator extends ComplexEvaluator implements
     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);
+    return empiricalDistribution;
   }
 
-  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 {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71517bc2/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HistogramEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HistogramEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HistogramEvaluator.java
index 0217bae..beabe3a 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HistogramEvaluator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HistogramEvaluator.java
@@ -45,7 +45,7 @@ public class HistogramEvaluator extends ComplexEvaluator implements Expressible
     }
   }
 
-  public List<Map> evaluate(Tuple tuple) throws IOException {
+  public List<Tuple> evaluate(Tuple tuple) throws IOException {
 
     StreamEvaluator colEval1 = subEvaluators.get(0);
 
@@ -66,7 +66,7 @@ public class HistogramEvaluator extends ComplexEvaluator implements Expressible
     EmpiricalDistribution empiricalDistribution = new EmpiricalDistribution(bins);
     empiricalDistribution.load(column1);
 
-    List<Map> binList = new ArrayList();
+    List<Tuple> binList = new ArrayList();
 
     List<SummaryStatistics> summaries = empiricalDistribution.getBinStats();
     for(SummaryStatistics statisticalSummary : summaries) {
@@ -78,7 +78,7 @@ public class HistogramEvaluator extends ComplexEvaluator implements Expressible
       map.put("sum", statisticalSummary.getSum());
       map.put("N", statisticalSummary.getN());
       map.put("var", statisticalSummary.getVariance());
-      binList.add(map);
+      binList.add(new Tuple(map));
     }
 
     return binList;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/71517bc2/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/KolmogorovSmirnovEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/KolmogorovSmirnovEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/KolmogorovSmirnovEvaluator.java
new file mode 100644
index 0000000..aa7c537
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/KolmogorovSmirnovEvaluator.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.client.solrj.io.eval;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Locale;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.math3.distribution.RealDistribution;
+import org.apache.commons.math3.stat.inference.KolmogorovSmirnovTest;
+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 KolmogorovSmirnovEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public KolmogorovSmirnovEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+
+    if(subEvaluators.size() != 2){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at least two values but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  public Tuple evaluate(Tuple tuple) throws IOException {
+
+    StreamEvaluator se1 = subEvaluators.get(0);
+    StreamEvaluator se2 = subEvaluators.get(1);
+
+    KolmogorovSmirnovTest ks = new KolmogorovSmirnovTest();
+    List<Number> sample = (List<Number>)se2.evaluate(tuple);
+    double[] data = new double[sample.size()];
+
+    for(int i=0; i<data.length; i++) {
+      data[i] = sample.get(i).doubleValue();
+    }
+
+    Object o = se1.evaluate(tuple);
+
+    if(o instanceof RealDistribution) {
+      RealDistribution realDistribution = (RealDistribution)o;
+      double d = ks.kolmogorovSmirnovStatistic(realDistribution, data);
+      double p = ks.kolmogorovSmirnovTest(realDistribution, data);
+
+
+      Map m = new HashMap();
+      m.put("p-value", p);
+      m.put("d-statistic", d);
+      return new Tuple(m);
+    } else {
+      List<Number> sample2 = (List<Number>)o;
+      double[] data2 = new double[sample2.size()];
+      for(int i=0; i<data2.length; i++) {
+        data2[i] = sample2.get(i).doubleValue();
+      }
+
+      double d = ks.kolmogorovSmirnovStatistic(data, data2);
+      //double p = ks.(data, data2);
+      Map m = new HashMap();
+      //m.put("p-value", p);
+      m.put("d-statistic", d);
+      return new Tuple(m);
+    }
+  }
+
+  @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/71517bc2/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NormalDistributionEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NormalDistributionEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NormalDistributionEvaluator.java
new file mode 100644
index 0000000..698402c
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NormalDistributionEvaluator.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.client.solrj.io.eval;
+
+import java.io.IOException;
+import java.util.Locale;
+
+
+import org.apache.commons.math3.distribution.NormalDistribution;
+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 NormalDistributionEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public NormalDistributionEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+
+    if(2 != subEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one column but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  public Object evaluate(Tuple tuple) throws IOException {
+
+    StreamEvaluator numEval1 = subEvaluators.get(0);
+    StreamEvaluator numEval2 = subEvaluators.get(1);
+
+    Number mean = (Number)numEval1.evaluate(tuple);
+    Number stdDev = (Number)numEval2.evaluate(tuple);
+
+    return new NormalDistribution(mean.doubleValue(), stdDev.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/71517bc2/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.java
new file mode 100644
index 0000000..7c59ed4
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.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.Locale;
+
+import org.apache.commons.math3.distribution.NormalDistribution;
+import org.apache.commons.math3.distribution.RealDistribution;
+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;
+
+import java.util.List;
+import java.util.ArrayList;
+
+public class SampleEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public SampleEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+
+    if(2 != subEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting two values (regression result and a number) but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  public List<Number> evaluate(Tuple tuple) throws IOException {
+    StreamEvaluator r = subEvaluators.get(0);
+    StreamEvaluator d = subEvaluators.get(1);
+    Number number = (Number)d.evaluate(tuple);
+    RealDistribution rd= (RealDistribution)r.evaluate(tuple);
+    double[] sample = rd.sample(number.intValue());
+    List<Number> list = new ArrayList();
+    for(double n : sample) {
+      list.add(n);
+    }
+    return list;
+  }
+
+  @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/71517bc2/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/UniformDistributionEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/UniformDistributionEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/UniformDistributionEvaluator.java
new file mode 100644
index 0000000..dec6aa4
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/UniformDistributionEvaluator.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.client.solrj.io.eval;
+
+import java.io.IOException;
+import java.util.Locale;
+
+
+import org.apache.commons.math3.distribution.UniformRealDistribution;
+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 UniformDistributionEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public UniformDistributionEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+
+    if(2 != subEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one column but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  public Object evaluate(Tuple tuple) throws IOException {
+
+    StreamEvaluator numEval1 = subEvaluators.get(0);
+    StreamEvaluator numEval2 = subEvaluators.get(1);
+
+    Number lower = (Number)numEval1.evaluate(tuple);
+    Number upper = (Number)numEval2.evaluate(tuple);
+
+    return new UniformRealDistribution(lower.doubleValue(), upper.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/71517bc2/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 58a5385..94f4106 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
@@ -5825,10 +5825,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     assertEquals(p, 2.4, 0.001);
   }
 
-  /*
+
   @Test
-  public void testArraySort() throws Exception {
-    String cexpr = "arraySort(array(11.5, 12.3, 4, 3, 1, 0))";
+  public void testAscend() throws Exception {
+    String cexpr = "asc(array(11.5, 12.3, 4, 3, 1, 0))";
     ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
     paramsLoc.set("expr", cexpr);
     paramsLoc.set("qt", "/stream");
@@ -5851,52 +5851,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     assertEquals(asort.get(5).doubleValue(), 12.3, 0.0);
   }
 
-*/
-  @Test
-  public void testCumulativeProbability() throws Exception {
-    UpdateRequest updateRequest = new UpdateRequest();
-
-    int i=0;
-    while(i<100) {
-      i=i+2;
-      updateRequest.add(id, "id_"+(i), "price_f", Integer.toString(i));
-    }
-
-    updateRequest.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
 
-    String expr = "search("+COLLECTIONORALIAS+", q=\"*:*\", fl=\"price_f\", sort=\"price_f asc\", rows=\"200\")";
-    String cexpr = "let(a="+expr+", c=col(a, price_f), e=empiricalDistribution(c), " +
-        "tuple(p1=cumulativeProbability(e, 88), " +
-        "p2=cumulativeProbability(e, 2), " +
-        "p3=cumulativeProbability(e, 99), " +
-        "p4=cumulativeProbability(e, 77), " +
-        "p5=cumulativeProbability(e, 98)))";
-
-    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);
-    double percentile1 = tuples.get(0).getDouble("p1");
-    double percentile2 = tuples.get(0).getDouble("p2");
-    double percentile3 = tuples.get(0).getDouble("p3");
-    double percentile4 = tuples.get(0).getDouble("p4");
-    double percentile5 = tuples.get(0).getDouble("p5");
-
-
-    assertEquals(.88D, percentile1, 0.001);
-    assertEquals(.0D, percentile2, 0.001);
-    assertEquals(1.0D, percentile3, 0.001);
-    assertEquals(.78D, percentile4, 0.001);
-    assertEquals(.98D, percentile5, 0.001);
-
-  }
 
   @Test
   public void testRankTransform() throws Exception {
@@ -6023,6 +5978,71 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
 
   @Test
+  public void testDistributions() throws Exception {
+    String cexpr = "let(a=normalDistribution(10, 2), " +
+                       "b=sample(a, 250), " +
+                       "c=normalDistribution(100, 6), " +
+                       "d=sample(c, 250), " +
+                       "u=uniformDistribution(1, 6),"+
+                       "t=sample(u, 250),"+
+                       "e=empiricalDistribution(d),"+
+                       "f=sample(e, 250),"+
+                       "tuple(sample=b, ks=ks(a,b), ks2=ks(a, d), ks3=ks(u, t)))";
+    ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+    String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString()+"/"+COLLECTIONORALIAS;
+    try {
+      TupleStream solrStream = new SolrStream(url, paramsLoc);
+      StreamContext context = new StreamContext();
+      solrStream.setStreamContext(context);
+      List<Tuple> tuples = getTuples(solrStream);
+      assertTrue(tuples.size() == 1);
+      List<Number> out = (List<Number>) tuples.get(0).get("sample");
+
+      Map ks = (Map) tuples.get(0).get("ks");
+      Map ks2 = (Map) tuples.get(0).get("ks2");
+      Map ks3 = (Map) tuples.get(0).get("ks3");
+
+      assertTrue(out.size() == 250);
+      Number pvalue = (Number) ks.get("p-value");
+      Number pvalue2 = (Number) ks2.get("p-value");
+      Number pvalue3 = (Number) ks3.get("p-value");
+
+      assertTrue(pvalue.doubleValue() > .05D);
+      assertTrue(pvalue2.doubleValue() == 0);
+      assertTrue(pvalue3.doubleValue() > .05D);
+
+    } catch(AssertionError e) {
+
+      //This test will have random failures do to the random sampling. So if it fails try it again.
+      //If it fails twice in a row, we probably broke some code.
+
+      TupleStream solrStream = new SolrStream(url, paramsLoc);
+      StreamContext context = new StreamContext();
+      solrStream.setStreamContext(context);
+      List<Tuple> tuples = getTuples(solrStream);
+      assertTrue(tuples.size() == 1);
+      List<Number> out = (List<Number>) tuples.get(0).get("sample");
+
+      Map ks = (Map) tuples.get(0).get("ks");
+      Map ks2 = (Map) tuples.get(0).get("ks2");
+      Map ks3 = (Map) tuples.get(0).get("ks3");
+
+      assertTrue(out.size() == 250);
+      Number pvalue = (Number) ks.get("p-value");
+      Number pvalue2 = (Number) ks2.get("p-value");
+      Number pvalue3 = (Number) ks3.get("p-value");
+
+      assertTrue(pvalue.doubleValue() > .05D);
+      assertTrue(pvalue2.doubleValue() == 0);
+      assertTrue(pvalue3.doubleValue() > .05D);
+    }
+  }
+
+
+
+  @Test
   public void testResiduals() throws Exception {
     String cexpr = "let(a=array(1,2,3,4,5,6), b=array(2,4,6,8,10,12), c=regress(a,b), tuple(res=residuals(c,a,a)))";
     ModifiableSolrParams paramsLoc = new ModifiableSolrParams();


[2/2] lucene-solr:master: SOLR-11160: Fix precommit

Posted by jb...@apache.org.
SOLR-11160: Fix precommit


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

Branch: refs/heads/master
Commit: b058818a3edf5f7af94107dba2d208a68db0758b
Parents: 71517bc
Author: Joel Bernstein <jb...@apache.org>
Authored: Mon Jul 31 15:57:30 2017 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Mon Jul 31 15:57:30 2017 -0400

----------------------------------------------------------------------
 .../solr/client/solrj/io/eval/EmpiricalDistributionEvaluator.java   | 1 -
 .../java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.java  | 1 -
 2 files changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b058818a/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
index 5a50702..e57c225 100644
--- 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
@@ -19,7 +19,6 @@ package org.apache.solr.client.solrj.io.eval;
 import java.io.IOException;
 import java.util.List;
 import java.util.Locale;
-import java.util.Map;
 import java.util.Arrays;
 
 import org.apache.commons.math3.random.EmpiricalDistribution;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b058818a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.java
index 7c59ed4..8b725cf 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.java
@@ -20,7 +20,6 @@ package org.apache.solr.client.solrj.io.eval;
 import java.io.IOException;
 import java.util.Locale;
 
-import org.apache.commons.math3.distribution.NormalDistribution;
 import org.apache.commons.math3.distribution.RealDistribution;
 import org.apache.solr.client.solrj.io.Tuple;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation;