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 2017/01/02 10:18:23 UTC

[02/19] lucene-solr:jira/solr-9854: SOLR-9905: Add NullStream to isolate the performance of the ExportWriter

SOLR-9905: Add NullStream to isolate the performance of the ExportWriter


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

Branch: refs/heads/jira/solr-9854
Commit: 7dcb557ab73da7fb7af0e8f698895e28dde4bbca
Parents: db9190d
Author: Joel Bernstein <jb...@apache.org>
Authored: Thu Dec 29 13:46:04 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Thu Dec 29 14:39:16 2016 -0500

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |   2 +-
 .../solr/client/solrj/io/stream/NullStream.java | 155 +++++++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   |  65 ++++++++
 3 files changed, 221 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dcb557a/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 c6f3c62..13ce636 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -139,7 +139,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("classify", ClassifyStream.class)
       .withFunctionName("fetch", FetchStream.class)
       .withFunctionName("executor", ExecutorStream.class)
-
+      .withFunctionName("null", NullStream.class)
       // metrics
       .withFunctionName("min", MinMetric.class)
       .withFunctionName("max", MaxMetric.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dcb557a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NullStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NullStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NullStream.java
new file mode 100644
index 0000000..bef3b1c
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NullStream.java
@@ -0,0 +1,155 @@
+/*
+ * 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.List;
+import java.util.Date;
+
+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.StreamFactory;
+
+
+/**
+ *  The NullStream Iterates over a TupleStream and eats the tuples. It returns the tuple count in the EOF tuple.
+ *  Because the NullStreaam eats all the Tuples it see's it can be used as a simple tool for performance analysis of
+ *  underlying streams.
+ **/
+
+public class NullStream extends TupleStream implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  private TupleStream stream;
+  private long count;
+  private long start;
+  private Tuple eof;
+
+  public NullStream(TupleStream tupleStream) throws IOException {
+    init(tupleStream);
+  }
+
+  public NullStream(StreamExpression expression, StreamFactory factory) throws IOException {
+    // grab all parameters out
+    List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
+    TupleStream stream = factory.constructStream(streamExpressions.get(0));
+
+    init(stream);
+  }
+
+  private void init(TupleStream tupleStream) throws IOException{
+    this.stream = tupleStream;
+  }
+
+  @Override
+  public StreamExpression toExpression(StreamFactory factory) throws IOException{
+    return toExpression(factory, true);
+  }
+
+  private StreamExpression toExpression(StreamFactory factory, boolean includeStreams) throws IOException {
+    // function name
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+
+    if(includeStreams){
+      // stream
+      if(stream instanceof Expressible){
+        expression.addParameter(((Expressible)stream).toExpression(factory));
+      }
+      else{
+        throw new IOException("This RankStream contains a non-expressible TupleStream - it cannot be converted to an expression");
+      }
+    }
+    else{
+      expression.addParameter("<stream>");
+    }
+
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+
+    return new StreamExplanation(getStreamNodeId().toString())
+        .withChildren(new Explanation[]{
+            stream.toExplanation(factory)
+        })
+        .withFunctionName(factory.getFunctionName(this.getClass()))
+        .withImplementingClass(this.getClass().getName())
+        .withExpressionType(ExpressionType.STREAM_DECORATOR)
+        .withExpression(toExpression(factory, false).toString());
+  }
+
+  public void setStreamContext(StreamContext context) {
+    this.stream.setStreamContext(context);
+  }
+
+  public List<TupleStream> children() {
+    List<TupleStream> l =  new ArrayList<TupleStream>();
+    l.add(stream);
+    return l;
+  }
+
+  public void open() throws IOException {
+    start = new Date().getTime();
+    count = 0;
+    stream.open();
+  }
+
+  public void close() throws IOException {
+    stream.close();
+  }
+
+  public Tuple read() throws IOException {
+
+    if(eof != null) {
+      return eof;
+    }
+
+    while(true) {
+      Tuple tuple  = stream.read();
+      if(tuple.EOF) {
+        eof = tuple;
+        long end = new Date().getTime();
+        Tuple t = new Tuple(new HashMap());
+        t.put("nullCount", count);
+        t.put("timer", end-start);
+        return t;
+      } else {
+        ++count;
+      }
+    }
+  }
+
+  /** Return the stream sort - ie, the order in which records are returned */
+  public StreamComparator getStreamSort(){
+    return stream.getStreamSort();
+  }
+
+  public int getCost() {
+    return 0;
+  }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dcb557a/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 7d48c0e..0c9d5b3 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
@@ -373,6 +373,71 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
   }
 
+
+  @Test
+  public void testNullStream() throws Exception {
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0")
+        .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0")
+        .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3")
+        .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4")
+        .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
+        .add(id, "5", "a_s", "hello1", "a_i", "1", "a_f", "2")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    StreamExpression expression;
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
+        .withFunctionName("search", CloudSolrStream.class)
+        .withFunctionName("null", NullStream.class);
+
+    // Basic test
+    stream = factory.constructStream("null(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), by=\"a_i asc\")");
+    tuples = getTuples(stream);
+    assertTrue(tuples.size() == 1);
+    assertTrue(tuples.get(0).getLong("nullCount") == 6);
+  }
+
+
+  @Test
+  public void testParallelNullStream() throws Exception {
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0")
+        .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0")
+        .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3")
+        .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4")
+        .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
+        .add(id, "5", "a_s", "hello1", "a_i", "1", "a_f", "2")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    StreamExpression expression;
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
+        .withFunctionName("search", CloudSolrStream.class)
+        .withFunctionName("null", NullStream.class)
+        .withFunctionName("parallel", ParallelStream.class);
+
+    // Basic test
+    stream = factory.constructStream("parallel(" + COLLECTIONORALIAS + ", workers=2, sort=\"nullCount desc\", null(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=id), by=\"a_i asc\"))");
+    tuples = getTuples(stream);
+    assertTrue(tuples.size() == 2);
+    long nullCount = 0;
+    for(Tuple t : tuples) {
+      nullCount += t.getLong("nullCount");
+    }
+
+    assertEquals(nullCount, 6L);
+  }
+
+
   @Test
   public void testNulls() throws Exception {