You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2018/09/17 09:44:15 UTC

[29/47] lucene-solr:jira/solr-12709: SOLR-11943: Add machine learning functions for location data

SOLR-11943: Add machine learning functions for location data


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

Branch: refs/heads/jira/solr-12709
Commit: b8e87a101017711d634733242d5563eef836365e
Parents: 597bd5d
Author: Joel Bernstein <jb...@apache.org>
Authored: Thu Sep 6 14:00:38 2018 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Thu Sep 6 14:01:09 2018 -0400

----------------------------------------------------------------------
 .../solr/handler/HaversineMetersEvaluator.java  |  59 +++++++++++
 .../solr/handler/SolrDefaultStreamFactory.java  |   1 +
 .../org/apache/solr/client/solrj/io/Lang.java   |   1 +
 .../solrj/io/eval/LocationVectorsEvaluator.java | 105 +++++++++++++++++++
 .../apache/solr/client/solrj/io/TestLang.java   |   2 +-
 .../solrj/io/stream/MathExpressionTest.java     |  46 ++++++++
 6 files changed, 213 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8e87a10/solr/core/src/java/org/apache/solr/handler/HaversineMetersEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/HaversineMetersEvaluator.java b/solr/core/src/java/org/apache/solr/handler/HaversineMetersEvaluator.java
new file mode 100644
index 0000000..2e30555
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/HaversineMetersEvaluator.java
@@ -0,0 +1,59 @@
+/*
+ * 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.handler;
+
+import java.io.IOException;
+
+import org.apache.commons.math3.exception.DimensionMismatchException;
+import org.apache.commons.math3.ml.distance.DistanceMeasure;
+import org.apache.lucene.util.SloppyMath;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.eval.RecursiveEvaluator;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class HaversineMetersEvaluator extends RecursiveEvaluator {
+  protected static final long serialVersionUID = 1L;
+
+  public HaversineMetersEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+  }
+
+
+  @Override
+  public Object evaluate(Tuple tuple) throws IOException {
+    return new HaversineDistance();
+  }
+
+  @Override
+  public Object doWork(Object... values) throws IOException {
+    // Nothing to do here
+    throw new IOException("This call should never occur");
+  }
+
+  public static class HaversineDistance implements DistanceMeasure {
+    private static final long serialVersionUID = -9108154600539125566L;
+
+    public HaversineDistance() {
+    }
+
+    public double compute(double[] a, double[] b) throws DimensionMismatchException {
+      return SloppyMath.haversinMeters(a[0], a[1], b[0], b[1]);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8e87a10/solr/core/src/java/org/apache/solr/handler/SolrDefaultStreamFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SolrDefaultStreamFactory.java b/solr/core/src/java/org/apache/solr/handler/SolrDefaultStreamFactory.java
index 0b375f4..c072f0b 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrDefaultStreamFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrDefaultStreamFactory.java
@@ -35,6 +35,7 @@ public class SolrDefaultStreamFactory extends DefaultStreamFactory {
     super();
     this.withFunctionName("analyze",  AnalyzeEvaluator.class);
     this.withFunctionName("classify", ClassifyStream.class);
+    this.withFunctionName("haversineMeters", HaversineMetersEvaluator.class);
   }
 
   public SolrDefaultStreamFactory withSolrResourceLoader(SolrResourceLoader solrResourceLoader) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8e87a10/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
index 32ee6fc..9568ecb 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
@@ -255,6 +255,7 @@ public class Lang {
         .withFunctionName("removeCache", RemoveCacheEvaluator.class)
         .withFunctionName("listCache", ListCacheEvaluator.class)
         .withFunctionName("zscores", NormalizeEvaluator.class)
+        .withFunctionName("locationVectors", LocationVectorsEvaluator.class)
 
         // Boolean Stream Evaluators
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8e87a10/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LocationVectorsEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LocationVectorsEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LocationVectorsEvaluator.java
new file mode 100644
index 0000000..0c1ba99
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/LocationVectorsEvaluator.java
@@ -0,0 +1,105 @@
+/*
+ * 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.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParameter;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class LocationVectorsEvaluator extends RecursiveObjectEvaluator implements ManyValueWorker {
+  protected static final long serialVersionUID = 1L;
+
+  private String field;
+
+  public LocationVectorsEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+
+    List<StreamExpressionNamedParameter> namedParams = factory.getNamedOperands(expression);
+
+    for (StreamExpressionNamedParameter namedParam : namedParams) {
+      if(namedParam.getName().equals("field")) {
+        this.field = namedParam.getParameter().toString();
+      } else {
+        throw new IOException("Unexpected named parameter:" + namedParam.getName());
+      }
+    }
+
+    if(field == null) {
+      throw new IOException("The named parameter \"field\" must be set for the locationVectors function.");
+    }
+  }
+
+  @Override
+  public Object doWork(Object... objects) throws IOException {
+
+    if (objects.length == 1) {
+      //Just docs
+      if(!(objects[0] instanceof List)) {
+        throw new IOException("The locationVectors function expects a list of Tuples as a parameter.");
+      } else {
+        List list = (List)objects[0];
+        if(list.size() > 0) {
+          Object o = list.get(0);
+          if(!(o instanceof Tuple)) {
+            throw new IOException("The locationVectors function expects a list of Tuples as a parameter.");
+          }
+        } else {
+          throw new IOException("Empty list was passed as a parameter to termVectors function.");
+        }
+      }
+
+      List<Tuple> tuples = (List<Tuple>) objects[0];
+
+      double[][] locationVectors = new double[tuples.size()][2];
+      List<String> features = new ArrayList();
+      features.add("lat");
+      features.add("long");
+
+      List<String> rowLabels = new ArrayList();
+
+      for(int i=0; i< tuples.size(); i++) {
+        Tuple tuple = tuples.get(i);
+        String value = tuple.getString(field);
+        String[] latLong = null;
+        if(value.contains(",")) {
+          latLong = value.split(",");
+        } else {
+          latLong = value.split(" ");
+        }
+
+        locationVectors[i][0] = Double.parseDouble(latLong[0].trim());
+        locationVectors[i][1] = Double.parseDouble(latLong[1].trim());
+        if(tuple.get("id") != null) {
+          rowLabels.add(tuple.get("id").toString());
+        }
+      }
+
+      Matrix matrix = new Matrix(locationVectors);
+      matrix.setColumnLabels(features);
+      matrix.setRowLabels(rowLabels);
+      return matrix;
+    } else {
+      throw new IOException("The termVectors function takes a single positional parameter.");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8e87a10/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
index 8c2cb65..ee8c1e1 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
@@ -70,7 +70,7 @@ public class TestLang extends LuceneTestCase {
       "mod", "ceil", "floor", "sin", "asin", "sinh", "cos", "acos", "cosh", "tan", "atan", "tanh", "round", "sqrt",
       "cbrt", "coalesce", "uuid", "if", "convert", "valueAt", "memset", "fft", "ifft", "euclidean","manhattan",
       "earthMovers", "canberra", "chebyshev", "ones", "zeros", "setValue", "getValue", "knnRegress", "gaussfit",
-      "outliers", "stream", "getCache", "putCache", "listCache", "removeCache", "zscores"};
+      "outliers", "stream", "getCache", "putCache", "listCache", "removeCache", "zscores", "locationVectors"};
 
   @Test
   public void testLang() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8e87a10/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
index 229e9eb..4bcf50d 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
@@ -310,6 +310,52 @@ public class MathExpressionTest extends SolrCloudTestCase {
     assertEquals(array.get(2).intValue(), 50);
     assertEquals(array.get(3).intValue(), 50);
   }
+
+  @Test
+  public void testLocationFunctions() throws Exception {
+    UpdateRequest updateRequest = new UpdateRequest();
+
+    int i=0;
+    while(i<5) {
+      updateRequest.add(id, "id_"+(++i),"test_dt", getDateString("2016", "5", "1"),
+          "price_i",  Integer.toString(i), "loc_p", (42.906797030808235+i)+","+(76.69455762489834+i));
+    }
+
+
+    updateRequest.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    String expr = "let(echo=true," +
+        "              a=search("+COLLECTIONORALIAS+", q=*:*, fl=\"id, loc_p, price_i\",rows=100, sort=\"price_i asc\"),"+
+        "              b=locationVectors(a, field=loc_p)," +
+        "              c=distance(array(40.7128, 74.0060), array(45.7128, 74.0060), haversineMeters()))";
+
+
+    ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", expr);
+    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>>locVectors = (List<List<Number>>)tuples.get(0).get("b");
+    System.out.println(locVectors);
+    int v=1;
+    for(List<Number> row : locVectors) {
+     double lat = row.get(0).doubleValue();
+     double lon = row.get(1).doubleValue();
+     assertEquals(lat, 42.906797030808235+v, 0);
+     assertEquals(lon, 76.69455762489834+v, 0);
+     ++v;
+    }
+
+    double distance = tuples.get(0).getDouble("c");
+    assertEquals(distance, 555975.3986718428, 1.0);
+
+  }
   
   @Test
   public void testHist() throws Exception {