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 2018/02/20 01:06:17 UTC

lucene-solr:branch_7x: SOLR-11588: Add matrixMult Stream Evaluator to support matrix multiplication

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x aac11fc12 -> 31d2a8eb6


SOLR-11588: Add matrixMult Stream Evaluator to support matrix multiplication


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

Branch: refs/heads/branch_7x
Commit: 31d2a8eb631d867d94c05da1f949fd35be05b60e
Parents: aac11fc
Author: Joel Bernstein <jb...@apache.org>
Authored: Mon Feb 19 19:27:13 2018 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Mon Feb 19 20:03:29 2018 -0500

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |  1 +
 .../solrj/io/eval/MatrixMultiplyEvaluator.java  | 67 +++++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   | 78 ++++++++++++++++++++
 3 files changed, 146 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/31d2a8eb/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 f59c8ae..d1c1422 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -316,6 +316,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
         .withFunctionName("l2norm", NormEvaluator.class)
         .withFunctionName("l1norm", L1NormEvaluator.class)
         .withFunctionName("linfnorm", LInfNormEvaluator.class)
+        .withFunctionName("matrixMult", MatrixMultiplyEvaluator.class)
 
         // Boolean Stream Evaluators
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/31d2a8eb/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/MatrixMultiplyEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/MatrixMultiplyEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/MatrixMultiplyEvaluator.java
new file mode 100644
index 0000000..5f4f0d1
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/MatrixMultiplyEvaluator.java
@@ -0,0 +1,67 @@
+/*
+ * 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 org.apache.commons.math3.linear.Array2DRowRealMatrix;
+import org.apache.commons.math3.linear.RealMatrix;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class MatrixMultiplyEvaluator extends RecursiveObjectEvaluator implements TwoValueWorker {
+  protected static final long serialVersionUID = 1L;
+
+  public MatrixMultiplyEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+  }
+
+  @Override
+  public Object doWork(Object first, Object second) throws IOException {
+    if(null == first){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - null found for the first value",toExpression(constructingFactory)));
+    }
+    if(null == second){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - null found for the second value",toExpression(constructingFactory)));
+    }
+
+    RealMatrix realMatrix1 = getMatrix(first);
+    RealMatrix realMatrix2 = getMatrix(second);
+    RealMatrix realMatrix3 = realMatrix1.multiply(realMatrix2);
+    return new Matrix(realMatrix3.getData());
+
+  }
+
+  private RealMatrix getMatrix(Object o) throws IOException {
+    if(o instanceof Matrix) {
+      Matrix matrix = (Matrix)o;
+      return new Array2DRowRealMatrix(matrix.getData());
+    } else if(o instanceof List) {
+      List<Number> vec = (List<Number>)o;
+      double[][] data1 = new double[1][vec.size()];
+      for(int i=0; i<vec.size(); i++) {
+        data1[0][i] = vec.get(i).doubleValue();
+      }
+      return new Array2DRowRealMatrix(data1);
+    } else {
+      throw new IOException("The matrixMult function can only be applied to numeric arrays and matrices:"+o.getClass().toString());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/31d2a8eb/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 70e5992..5c12645 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
@@ -6997,6 +6997,84 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     assertTrue(out.get(5).intValue() == 6);
   }
 
+
+
+  @Test
+  public void testMatrixMult() throws Exception {
+    String cexpr = "let(echo=true," +
+        "               a=array(1,2,3)," +
+        "               b=matrix(array(4), array(5), array(6))," +
+        "               c=matrixMult(a, b)," +
+        "               d=matrix(array(3, 4), array(10,11), array(30, 40))," +
+        "               e=matrixMult(a, d)," +
+        "               f=array(4,8,10)," +
+        "               g=matrix(a, f)," +
+        "               h=matrixMult(d, g)," +
+        "               i=matrixMult(b, a))";
+    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<List<Number>> matrix = (List<List<Number>>)tuples.get(0).get("c");
+    assertEquals(matrix.size(), 1);
+    List<Number> row = matrix.get(0);
+    assertEquals(row.size(), 1);
+    assertEquals(row.get(0).doubleValue(), 32.0, 0.0);
+
+    matrix = (List<List<Number>>)tuples.get(0).get("e");
+    assertEquals(matrix.size(), 1);
+    row = matrix.get(0);
+    assertEquals(row.size(), 2);
+    assertEquals(row.get(0).doubleValue(), 113.0, 0.0);
+    assertEquals(row.get(1).doubleValue(), 146.0, 0.0);
+
+    matrix = (List<List<Number>>)tuples.get(0).get("h");
+    assertEquals(matrix.size(), 3);
+    row = matrix.get(0);
+    assertEquals(row.size(), 3);
+    assertEquals(row.get(0).doubleValue(), 19.0, 0.0);
+    assertEquals(row.get(1).doubleValue(), 38.0, 0.0);
+    assertEquals(row.get(2).doubleValue(), 49.0, 0.0);
+
+    row = matrix.get(1);
+    assertEquals(row.size(), 3);
+    assertEquals(row.get(0).doubleValue(), 54.0, 0.0);
+    assertEquals(row.get(1).doubleValue(), 108.0, 0.0);
+    assertEquals(row.get(2).doubleValue(), 140.0, 0.0);
+
+    row = matrix.get(2);
+    assertEquals(row.size(), 3);
+    assertEquals(row.get(0).doubleValue(), 190.0, 0.0);
+    assertEquals(row.get(1).doubleValue(), 380.0, 0.0);
+    assertEquals(row.get(2).doubleValue(), 490.0, 0.0);
+
+    matrix = (List<List<Number>>)tuples.get(0).get("i");
+
+    assertEquals(matrix.size(), 3);
+    row = matrix.get(0);
+    assertEquals(row.size(), 3);
+    assertEquals(row.get(0).doubleValue(), 4.0, 0.0);
+    assertEquals(row.get(1).doubleValue(), 8.0, 0.0);
+    assertEquals(row.get(2).doubleValue(), 12.0, 0.0);
+
+    row = matrix.get(1);
+    assertEquals(row.size(), 3);
+    assertEquals(row.get(0).doubleValue(), 5.0, 0.0);
+    assertEquals(row.get(1).doubleValue(), 10.0, 0.0);
+    assertEquals(row.get(2).doubleValue(), 15.0, 0.0);
+
+    row = matrix.get(2);
+    assertEquals(row.size(), 3);
+    assertEquals(row.get(0).doubleValue(), 6.0, 0.0);
+    assertEquals(row.get(1).doubleValue(), 12.0, 0.0);
+    assertEquals(row.get(2).doubleValue(), 18.0, 0.0);
+  }
+
   @Test
   public void testKmeans() throws Exception {
     String cexpr = "let(echo=true," +