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/25 17:55:32 UTC

[23/44] lucene-solr:jira/solr-8668: SOLR-10731: Add knn Streaming Expression WIP

SOLR-10731: Add knn Streaming Expression WIP


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

Branch: refs/heads/jira/solr-8668
Commit: 89d48dfd9cfd63dfd54595551ec07e042eba432e
Parents: 65bfa48
Author: Joel Bernstein <jb...@apache.org>
Authored: Tue May 23 19:07:39 2017 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed May 24 07:59:01 2017 -0400

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |   1 +
 .../solr/client/solrj/io/stream/KnnStream.java  | 249 +++++++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   |  40 +++
 3 files changed, 290 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89d48dfd/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 9fd7c8b..c045f20 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -121,6 +121,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("topic", TopicStream.class)
       .withFunctionName("commit", CommitStream.class)
       .withFunctionName("random", RandomStream.class)
+      .withFunctionName("knn", KnnStream.class)
       
       // decorator streams
       .withFunctionName("merge", MergeStream.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89d48dfd/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/KnnStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/KnnStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/KnnStream.java
new file mode 100644
index 0000000..f0a188a
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/KnnStream.java
@@ -0,0 +1,249 @@
+/*
+ * 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.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.io.SolrClientCache;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.comp.StreamComparator;
+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.StreamExplanation;
+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.StreamExpressionParameter;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.params.ModifiableSolrParams;
+
+import static org.apache.solr.common.params.CommonParams.Q;
+
+public class KnnStream extends TupleStream implements Expressible  {
+
+  private static String[] mltParams = {"qf", "mintf", "mindf", "maxdf", "minwl", "maxwl", "maxqt", "maxntp", "boost"};
+
+  private String zkHost;
+  private Map<String, String> props;
+  private String collection;
+  protected transient SolrClientCache cache;
+  protected transient CloudSolrClient cloudSolrClient;
+  private Iterator<SolrDocument> documentIterator;
+  private String id;
+
+  public KnnStream(String zkHost,
+                   String collection,
+                   String id,
+                   Map<String, String> props) throws IOException {
+    init(zkHost, collection, id, props);
+  }
+
+  public KnnStream(StreamExpression expression, StreamFactory factory) throws IOException{
+    // grab all parameters out
+    String collectionName = factory.getValueOperand(expression, 0);
+    List<StreamExpressionNamedParameter> namedParams = factory.getNamedOperands(expression);
+    StreamExpressionNamedParameter zkHostExpression = factory.getNamedOperand(expression, "zkHost");
+    StreamExpressionNamedParameter idExpression = factory.getNamedOperand(expression, "id");
+    StreamExpressionNamedParameter qfExpression = factory.getNamedOperand(expression, "qf");
+
+
+    // Collection Name
+    if(null == collectionName){
+      throw new IOException(String.format(Locale.ROOT,"invalid expression %s - collectionName expected as first operand",expression));
+    }
+
+    // Named parameters - passed directly to solr as solrparams
+    System.out.println("####Params:"+namedParams.size());
+    if(namedParams.size() < 2){
+      throw new IOException(String.format(Locale.ROOT,"invalid expression %s - at least two named parameters expected. eg. 'id' and 'qf'",expression));
+    }
+
+    // pull out known named params
+    Map<String,String> params = new HashMap<String,String>();
+    for(StreamExpressionNamedParameter namedParam : namedParams){
+      if(!namedParam.getName().equals("zkHost") && !namedParam.getName().equals("id")){
+        params.put(namedParam.getName(), namedParam.getParameter().toString().trim());
+      }
+    }
+
+    String id = null;
+    if(idExpression != null) {
+      id = ((StreamExpressionValue)idExpression.getParameter()).getValue();
+    } else {
+      throw new IOException("id parameter is expected for KnnStream");
+    }
+
+    if(qfExpression == null) {
+      throw new IOException("qf parameter is expected for KnnStream");
+    }
+
+    // zkHost, optional - if not provided then will look into factory list to get
+    String zkHost = null;
+    if(null == zkHostExpression){
+      zkHost = factory.getCollectionZkHost(collectionName);
+      if(zkHost == null) {
+        zkHost = factory.getDefaultZkHost();
+      }
+    }
+    else if(zkHostExpression.getParameter() instanceof StreamExpressionValue){
+      zkHost = ((StreamExpressionValue)zkHostExpression.getParameter()).getValue();
+    }
+    if(null == zkHost){
+      throw new IOException(String.format(Locale.ROOT,"invalid expression %s - zkHost not found for collection '%s'",expression,collectionName));
+    }
+
+    // We've got all the required items
+    init(zkHost, collectionName, id,  params);
+  }
+
+  private void init(String zkHost, String collection, String id, Map<String, String> props) throws IOException {
+    this.zkHost  = zkHost;
+    this.props   = props;
+    this.collection = collection;
+    this.id = id;
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    // function name
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+
+    // collection
+    expression.addParameter(collection);
+
+    // parameters
+    for(Entry<String,String> param : props.entrySet()){
+      expression.addParameter(new StreamExpressionNamedParameter(param.getKey(), param.getValue()));
+    }
+
+    // zkHost
+    expression.addParameter(new StreamExpressionNamedParameter("zkHost", zkHost));
+
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+
+    StreamExplanation explanation = new StreamExplanation(getStreamNodeId().toString());
+
+    explanation.setFunctionName(factory.getFunctionName(this.getClass()));
+    explanation.setImplementingClass(this.getClass().getName());
+    explanation.setExpressionType(ExpressionType.STREAM_SOURCE);
+    explanation.setExpression(toExpression(factory).toString());
+
+    // child is a datastore so add it at this point
+    StreamExplanation child = new StreamExplanation(getStreamNodeId() + "-datastore");
+    child.setFunctionName(String.format(Locale.ROOT, "solr (%s)", collection));
+    child.setImplementingClass("Solr/Lucene");
+    child.setExpressionType(ExpressionType.DATASTORE);
+    if(null != props){
+      child.setExpression(props.entrySet().stream().map(e -> String.format(Locale.ROOT, "%s=%s", e.getKey(), e.getValue())).collect(Collectors.joining(",")));
+    }
+    explanation.addChild(child);
+
+    return explanation;
+  }
+
+  public void setStreamContext(StreamContext context) {
+    cache = context.getSolrClientCache();
+  }
+
+  public List<TupleStream> children() {
+    List<TupleStream> l =  new ArrayList();
+    return l;
+  }
+
+  public void open() throws IOException {
+    cloudSolrClient = cache.getCloudSolrClient(zkHost);
+    ModifiableSolrParams params = getParams(this.props);
+
+    StringBuilder builder = new StringBuilder();
+
+    for(String key : mltParams) {
+      if(params.get(key) != null) {
+        builder.append(" " + key + "=" + params.get(key));
+        params.remove(key);
+      }
+    }
+
+    params.add(Q, "{!mlt"+builder.toString()+"}"+id);
+
+    QueryRequest request = new QueryRequest(params);
+    try {
+      QueryResponse response = request.process(cloudSolrClient, collection);
+      SolrDocumentList docs = response.getResults();
+      documentIterator = docs.iterator();
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  public void close() throws IOException {
+
+  }
+
+  public Tuple read() throws IOException {
+    if(documentIterator.hasNext()) {
+      Map map = new HashMap();
+      SolrDocument doc = documentIterator.next();
+      for(String key  : doc.keySet()) {
+        map.put(key, doc.get(key));
+      }
+      return new Tuple(map);
+    } else {
+      Map fields = new HashMap();
+      fields.put("EOF", true);
+      Tuple tuple = new Tuple(fields);
+      return tuple;
+    }
+  }
+
+  private ModifiableSolrParams getParams(Map<String, String> props) {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    for(String key : props.keySet()) {
+      String value = props.get(key);
+      params.add(key, value);
+    }
+    return params;
+  }
+
+  public int getCost() {
+    return 0;
+  }
+
+  @Override
+  public StreamComparator getStreamSort() {
+    return null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89d48dfd/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 a881e53..d464989 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
@@ -920,6 +920,46 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     }
   }
 
+
+  @Test
+  public void testKnnStream() throws Exception {
+
+    UpdateRequest update = new UpdateRequest();
+
+    update.add(id, "1", "a_t", "hello world have a very nice day blah");
+    update.add(id, "2", "a_t", "hello world have a very nice day fancy sky");
+    update.add(id, "3", "a_t", "hello world have a very nice bug out");
+    update.add(id, "4", "a_t", "hello world have a very streaming is fun");
+
+
+    update.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    StreamExpression expression;
+    TupleStream stream;
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
+        .withFunctionName("random", RandomStream.class);
+
+
+    StreamContext context = new StreamContext();
+    SolrClientCache cache = new SolrClientCache();
+    try {
+      context.setSolrClientCache(cache);
+
+      ModifiableSolrParams sParams = new ModifiableSolrParams(StreamingTest.mapParams(CommonParams.QT, "/stream"));
+      sParams.add("expr", "knn(" + COLLECTIONORALIAS + ", id=\"1\", qf=\"a_t\", rows=\"4\", fl=\"id\")");
+      JettySolrRunner jetty = cluster.getJettySolrRunner(0);
+      SolrStream solrStream = new SolrStream(jetty.getBaseUrl().toString() + "/collection1", sParams);
+      List<Tuple> tuples = getTuples(solrStream);
+      System.out.println("## Tuples:"+tuples.size());
+      assertTrue(tuples.size() == 4);
+
+    } finally {
+      cache.close();
+    }
+  }
+
   @Test
   public void testReducerStream() throws Exception {