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/30 11:28:11 UTC

[02/25] lucene-solr:jira/solr-8668: SOLR-10753: Add array Stream Evaluator

SOLR-10753: Add array Stream Evaluator


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

Branch: refs/heads/jira/solr-8668
Commit: 81821b29ae4c507f02d09b29fcd0196db8647865
Parents: e7099e4
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri May 26 10:23:21 2017 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Fri May 26 10:23:46 2017 -0400

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |  1 +
 .../client/solrj/io/eval/ArrayEvaluator.java    | 63 ++++++++++++++++++++
 .../solrj/io/stream/expr/StreamFactory.java     |  2 +-
 .../solrj/io/stream/StreamExpressionTest.java   | 40 +++++++++++++
 4 files changed, 105 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81821b29/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 a35f0ec..dc097be 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -190,6 +190,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("describe", DescribeEvaluator.class)
       .withFunctionName("finddelay", FindDelayEvaluator.class)
       .withFunctionName("sequence", SequenceEvaluator.class)
+      .withFunctionName("array", ArrayEvaluator.class)
 
       // metrics
          .withFunctionName("min", MinMetric.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81821b29/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArrayEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArrayEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArrayEvaluator.java
new file mode 100644
index 0000000..31d89a2
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArrayEvaluator.java
@@ -0,0 +1,63 @@
+/*
+ * 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 ArrayEvaluator extends ComplexEvaluator implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  public ArrayEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public List<Number> evaluate(Tuple tuple) throws IOException {
+    List<Number> list = new ArrayList();
+    for(StreamEvaluator subEvaluator : subEvaluators) {
+      Number num = (Number)subEvaluator.evaluate(tuple);
+      list.add(num);
+    }
+
+    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/81821b29/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java
index 703acf4..74e1de8 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamFactory.java
@@ -433,7 +433,7 @@ public class StreamFactory implements Serializable {
     if("null".equals(lower)){ return null; }
     if("true".equals(lower) || "false".equals(lower)){ return Boolean.parseBoolean(lower); }
     try{ return Long.valueOf(original); } catch(Exception ignored){};
-    try{ if (original.matches(".{1,8}")){ return Float.valueOf(original); }} catch(Exception ignored){};
+    try{ if (original.matches(".{1,8}")){ return Double.valueOf(original); }} catch(Exception ignored){};
     try{ if (original.matches(".{1,17}")){ return Double.valueOf(original); }} catch(Exception ignored){};
     
     // is a string

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81821b29/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 a2e8ca3..cbd57b8 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
@@ -5734,6 +5734,46 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     assertTrue(ranked.get(3).doubleValue() == 3D);
   }
 
+
+  @Test
+  public void testArray() throws Exception {
+    String cexpr = "array(1, 2, 3, 300, 2, 500)";
+    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);
+    List<Number> out = (List<Number>)tuples.get(0).get("out");
+    assertTrue(out.size() == 6);
+    assertTrue(out.get(0).intValue() == 1);
+    assertTrue(out.get(1).intValue() == 2);
+    assertTrue(out.get(2).intValue() == 3);
+    assertTrue(out.get(3).intValue() == 300);
+    assertTrue(out.get(4).intValue() == 2);
+    assertTrue(out.get(5).intValue() == 500);
+
+    cexpr = "array(1.122, 2.222, 3.333, 300.1, 2.13, 500.23)";
+    paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+    solrStream = new SolrStream(url, paramsLoc);
+    solrStream.setStreamContext(context);
+    tuples = getTuples(solrStream);
+    assertTrue(tuples.size() == 1);
+    out = (List<Number>)tuples.get(0).get("out");
+    assertTrue(out.size() == 6);
+    assertTrue(out.get(0).doubleValue() == 1.122D);
+    assertTrue(out.get(1).doubleValue() == 2.222D);
+    assertTrue(out.get(2).doubleValue() == 3.333D);
+    assertTrue(out.get(3).doubleValue() == 300.1D);
+    assertTrue(out.get(4).doubleValue() == 2.13D);
+    assertTrue(out.get(5).doubleValue() == 500.23D);
+  }
+
   @Test
   public void testScale() throws Exception {
     UpdateRequest updateRequest = new UpdateRequest();