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 2016/10/10 19:25:30 UTC

[1/2] lucene-solr:branch_6x: SOLR-9337: Add fetch Streaming Expression

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x e585c84f2 -> 45f2dfe2c


SOLR-9337: Add fetch Streaming Expression


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

Branch: refs/heads/branch_6x
Commit: 5836f4032fac975707c85e260d509ecd06c7f7e1
Parents: 5adb8f1
Author: Joel Bernstein <jb...@apache.org>
Authored: Mon Oct 10 14:20:09 2016 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Mon Oct 10 14:25:05 2016 -0400

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |   1 +
 .../client/solrj/io/stream/FetchStream.java     | 314 +++++++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   | 167 ++++++++++
 3 files changed, 482 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5836f403/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 dfacc1e..7ecaa9b 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -137,6 +137,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
          .withFunctionName("scoreNodes", ScoreNodesStream.class)
          .withFunctionName("model", ModelStream.class)
          .withFunctionName("classify", ClassifyStream.class)
+             .withFunctionName("fetch", FetchStream.class)
 
       // metrics
       .withFunctionName("min", MinMetric.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5836f403/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java
new file mode 100644
index 0000000..463ab4a
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java
@@ -0,0 +1,314 @@
+/*
+ * 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.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.HashMap;
+
+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.StreamExpressionValue;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.apache.solr.common.params.ModifiableSolrParams;
+
+/**
+ *  Iterates over a stream and fetches additional fields from a specified collection.
+ *  Fetches are done in batches.
+ *
+ *  Syntax:
+ *
+ *  fetch(collection, stream, on="a=b", fl="c,d,e", batchSize="50")
+ *
+ **/
+
+public class FetchStream extends TupleStream implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  protected String zkHost;
+  private TupleStream stream;
+  private StreamContext streamContext;
+  private Iterator<Tuple> tuples;
+
+  private String leftKey;
+  private String rightKey;
+  private String fieldList;
+  private String[] fields;
+  private String collection;
+  private int batchSize;
+  private boolean appendVersion = true;
+  private boolean appendKey = true;
+
+  public FetchStream(String zkHost, String collection, TupleStream tupleStream, String on, String fieldList, int batchSize) throws IOException {
+    init(zkHost, collection, tupleStream, on, fieldList, batchSize);
+  }
+
+  public FetchStream(StreamExpression expression, StreamFactory factory) throws IOException {
+    // grab all parameters out
+    String collectionName = factory.getValueOperand(expression, 0);
+    List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
+    StreamExpressionNamedParameter onParam = factory.getNamedOperand(expression, "on");
+    StreamExpressionNamedParameter flParam = factory.getNamedOperand(expression, "fl");
+    StreamExpressionNamedParameter batchSizeParam = factory.getNamedOperand(expression, "batchSize");
+    StreamExpressionNamedParameter zkHostExpression = factory.getNamedOperand(expression, "zkHost");
+
+    String on = null;
+    String fl = null;
+    int batchSize = 50;
+
+    if(onParam == null)  {
+      throw new IOException("on parameter cannot be null for the fetch expression");
+    } else {
+      on = ((StreamExpressionValue)onParam.getParameter()).getValue();
+    }
+
+    if(flParam == null)  {
+      throw new IOException("fl parameter cannot be null for the fetch expression");
+    } else {
+      fl = ((StreamExpressionValue)flParam.getParameter()).getValue();
+    }
+
+    if(batchSizeParam != null)  {
+      batchSize = Integer.parseInt(((StreamExpressionValue)batchSizeParam.getParameter()).getValue());
+    }
+
+    if(1 != streamExpressions.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single stream but found %d",expression, streamExpressions.size()));
+    }
+
+    TupleStream stream = factory.constructStream(streamExpressions.get(0));
+
+    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));
+    }
+
+    init(zkHost, collectionName, stream, on, fl, batchSize);
+  }
+
+  private void init(String zkHost, String collection, TupleStream tupleStream, String on, String fieldList, int batchSize) throws IOException{
+    this.zkHost = zkHost;
+    this.collection = collection;
+    this.stream = tupleStream;
+    this.batchSize = batchSize;
+    this.fields = fieldList.split(",");
+    this.fieldList = fieldList;
+
+    if(on.indexOf("=") > -1) {
+      String[] leftright = on.split("=");
+      leftKey = leftright[0].trim();
+      rightKey = leftright[1].trim();
+    } else {
+      leftKey = rightKey = on;
+    }
+
+    for(int i=0; i<fields.length; i++) {
+      fields[i] = fields[i].trim();
+      if(fields[i].equals("_version_")) {
+        appendVersion = false;
+      }
+
+      if(fields[i].equals(rightKey)) {
+        appendKey = false;
+      }
+    }
+  }
+
+  @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()));
+    expression.addParameter(collection);
+    expression.addParameter(new StreamExpressionNamedParameter("on", leftKey+"="+rightKey));
+    expression.addParameter(new StreamExpressionNamedParameter("fl", fieldList));
+    expression.addParameter(new StreamExpressionNamedParameter("batchSize", Integer.toString(batchSize)));
+
+    // stream
+    if(includeStreams) {
+      if (stream instanceof Expressible) {
+        expression.addParameter(((Expressible) stream).toExpression(factory));
+      } else {
+        throw new IOException("The FetchStream contains a non-expressible TupleStream - it cannot be converted to an expression");
+      }
+    }
+
+    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 streamContext) {
+    this.streamContext = streamContext;
+    this.stream.setStreamContext(streamContext);
+  }
+
+  public List<TupleStream> children() {
+    List<TupleStream> l =  new ArrayList();
+    l.add(stream);
+    return l;
+  }
+
+  public void open() throws IOException {
+    tuples = new ArrayList().iterator();
+    stream.open();
+  }
+
+  private void fetchBatch() throws IOException {
+
+    Tuple EOFTuple = null;
+    List<Tuple> batch = new ArrayList();
+    for(int i=0; i<batchSize; i++) {
+      Tuple tuple = stream.read();
+      if(tuple.EOF) {
+        EOFTuple = tuple;
+        break;
+      } else {
+        batch.add(tuple);
+      }
+    }
+
+    if(batch.size() > 0) {
+      StringBuilder buf = new StringBuilder();
+      buf.append(rightKey);
+      buf.append(":(");
+      for (int i = 0; i < batch.size(); i++) {
+        if (i > 0) {
+          buf.append(" ");
+        }
+        Tuple tuple = batch.get(i);
+        String key = tuple.getString(leftKey);
+        buf.append(key);
+      }
+      buf.append(")");
+
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.add("q", buf.toString());
+      params.add("fl", fieldList+appendFields());
+      params.add("rows", Integer.toString(batchSize));
+      params.add("sort", "_version_ desc");
+
+      CloudSolrStream cloudSolrStream = new CloudSolrStream(zkHost, collection, params);
+      StreamContext newContext = new StreamContext();
+      newContext.setSolrClientCache(streamContext.getSolrClientCache());
+      cloudSolrStream.setStreamContext(newContext);
+      Map<String, Tuple> fetched = new HashMap();
+      try {
+        cloudSolrStream.open();
+        while (true) {
+          Tuple t = cloudSolrStream.read();
+          if (t.EOF) {
+            break;
+          } else {
+            String rightValue = t.getString(rightKey);
+            fetched.put(rightValue, t);
+          }
+        }
+      } finally {
+        cloudSolrStream.close();
+      }
+
+      //Iterate the batch and add the fetched fields to the Tuples
+      for (Tuple batchTuple : batch) {
+        Tuple fetchedTuple = fetched.get(batchTuple.getString(leftKey));
+        if(fetchedTuple !=null) {
+          for (String field : fields) {
+            Object value = fetchedTuple.get(field);
+            if(value != null) {
+              batchTuple.put(field, value);
+            }
+          }
+        }
+      }
+    }
+
+    if(EOFTuple != null) {
+      batch.add(EOFTuple);
+    }
+
+    this.tuples = batch.iterator();
+  }
+
+  public void close() throws IOException {
+    stream.close();
+  }
+
+  public Tuple read() throws IOException {
+    if(!tuples.hasNext()) {
+      fetchBatch();
+    }
+
+    return tuples.next();
+  }
+
+  public StreamComparator getStreamSort(){
+    return stream.getStreamSort();
+  }
+
+  public int getCost() {
+    return 0;
+  }
+
+  private String appendFields() {
+    StringBuffer buf = new StringBuffer();
+    if(appendKey) {
+      buf.append(",");
+      buf.append(rightKey);
+    }
+
+    if(appendVersion) {
+      buf.append(",_version_");
+    }
+    return buf.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5836f403/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 87fc951..4a3db77 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
@@ -718,6 +718,173 @@ public class StreamExpressionTest extends SolrCloudTestCase {
   }
 
   @Test
+  public void testFetchStream() throws Exception {
+
+    SolrClientCache solrClientCache = new SolrClientCache();
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1", "subject", "blah blah blah 0")
+        .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2", "subject", "blah blah blah 2")
+        .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "subject", "blah blah blah 3")
+        .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "subject", "blah blah blah 4")
+        .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5", "subject", "blah blah blah 1")
+        .add(id, "5", "a_s", "hello3", "a_i", "5", "a_f", "6", "subject", "blah blah blah 5")
+        .add(id, "6", "a_s", "hello4", "a_i", "6", "a_f", "7", "subject", "blah blah blah 6")
+        .add(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "8", "subject", "blah blah blah 7")
+        .add(id, "8", "a_s", "hello3", "a_i", "8", "a_f", "9", "subject", "blah blah blah 8")
+        .add(id, "9", "a_s", "hello0", "a_i", "9", "a_f", "10", "subject", "blah blah blah 9")
+        .commit(cluster.getSolrClient(), COLLECTION);
+
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress())
+        .withFunctionName("search", CloudSolrStream.class)
+        .withFunctionName("fetch", FetchStream.class);
+
+    stream = factory.constructStream("fetch("+COLLECTION+",  search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), on=\"id=a_i\", batchSize=\"2\", fl=\"subject\")");
+    StreamContext context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 10);
+    Tuple t = tuples.get(0);
+    assertTrue("blah blah blah 0".equals(t.getString("subject")));
+    t = tuples.get(1);
+    assertTrue("blah blah blah 2".equals(t.getString("subject")));
+    t = tuples.get(2);
+    assertTrue("blah blah blah 3".equals(t.getString("subject")));
+    t = tuples.get(3);
+    assertTrue("blah blah blah 4".equals(t.getString("subject")));
+    t = tuples.get(4);
+    assertTrue("blah blah blah 1".equals(t.getString("subject")));
+    t = tuples.get(5);
+    assertTrue("blah blah blah 5".equals(t.getString("subject")));
+    t = tuples.get(6);
+    assertTrue("blah blah blah 6".equals(t.getString("subject")));
+    t = tuples.get(7);
+    assertTrue("blah blah blah 7".equals(t.getString("subject")));
+    t = tuples.get(8);
+    assertTrue("blah blah blah 8".equals(t.getString("subject")));
+    t = tuples.get(9);
+    assertTrue("blah blah blah 9".equals(t.getString("subject")));
+
+    //Change the batch size
+    stream = factory.constructStream("fetch("+COLLECTION+",  search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), on=\"id=a_i\", batchSize=\"3\", fl=\"subject\")");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 10);
+    t = tuples.get(0);
+    assertTrue("blah blah blah 0".equals(t.getString("subject")));
+    t = tuples.get(1);
+    assertTrue("blah blah blah 2".equals(t.getString("subject")));
+    t = tuples.get(2);
+    assertTrue("blah blah blah 3".equals(t.getString("subject")));
+    t = tuples.get(3);
+    assertTrue("blah blah blah 4".equals(t.getString("subject")));
+    t = tuples.get(4);
+    assertTrue("blah blah blah 1".equals(t.getString("subject")));
+    t = tuples.get(5);
+    assertTrue("blah blah blah 5".equals(t.getString("subject")));
+    t = tuples.get(6);
+    assertTrue("blah blah blah 6".equals(t.getString("subject")));
+    t = tuples.get(7);
+    assertTrue("blah blah blah 7".equals(t.getString("subject")));
+    t = tuples.get(8);
+    assertTrue("blah blah blah 8".equals(t.getString("subject")));
+    t = tuples.get(9);
+    assertTrue("blah blah blah 9".equals(t.getString("subject")));
+    solrClientCache.close();
+  }
+
+  @Test
+  public void testParallelFetchStream() throws Exception {
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1", "subject", "blah blah blah 0")
+        .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2", "subject", "blah blah blah 2")
+        .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "subject", "blah blah blah 3")
+        .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "subject", "blah blah blah 4")
+        .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5", "subject", "blah blah blah 1")
+        .add(id, "5", "a_s", "hello3", "a_i", "5", "a_f", "6", "subject", "blah blah blah 5")
+        .add(id, "6", "a_s", "hello4", "a_i", "6", "a_f", "7", "subject", "blah blah blah 6")
+        .add(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "8", "subject", "blah blah blah 7")
+        .add(id, "8", "a_s", "hello3", "a_i", "8", "a_f", "9", "subject", "blah blah blah 8")
+        .add(id, "9", "a_s", "hello0", "a_i", "9", "a_f", "10", "subject", "blah blah blah 9")
+        .commit(cluster.getSolrClient(), COLLECTION);
+
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress())
+        .withFunctionName("search", CloudSolrStream.class)
+        .withFunctionName("parallel", ParallelStream.class)
+        .withFunctionName("fetch", FetchStream.class);
+
+    stream = factory.constructStream("parallel("+COLLECTION+", workers=2, sort=\"a_f asc\", fetch("+COLLECTION+",  search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=\"id\"), on=\"id=a_i\", batchSize=\"2\", fl=\"subject\"))");
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 10);
+    Tuple t = tuples.get(0);
+    assertTrue("blah blah blah 0".equals(t.getString("subject")));
+    t = tuples.get(1);
+    assertTrue("blah blah blah 2".equals(t.getString("subject")));
+    t = tuples.get(2);
+    assertTrue("blah blah blah 3".equals(t.getString("subject")));
+    t = tuples.get(3);
+    assertTrue("blah blah blah 4".equals(t.getString("subject")));
+    t = tuples.get(4);
+    assertTrue("blah blah blah 1".equals(t.getString("subject")));
+    t = tuples.get(5);
+    assertTrue("blah blah blah 5".equals(t.getString("subject")));
+    t = tuples.get(6);
+    assertTrue("blah blah blah 6".equals(t.getString("subject")));
+    t = tuples.get(7);
+    assertTrue("blah blah blah 7".equals(t.getString("subject")));
+    t = tuples.get(8);
+    assertTrue("blah blah blah 8".equals(t.getString("subject")));
+    t = tuples.get(9);
+    assertTrue("blah blah blah 9".equals(t.getString("subject")));
+
+
+    stream = factory.constructStream("parallel("+COLLECTION+", workers=2, sort=\"a_f asc\", fetch("+COLLECTION+",  search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=\"id\"), on=\"id=a_i\", batchSize=\"3\", fl=\"subject\"))");
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 10);
+    t = tuples.get(0);
+    assertTrue("blah blah blah 0".equals(t.getString("subject")));
+    t = tuples.get(1);
+    assertTrue("blah blah blah 2".equals(t.getString("subject")));
+    t = tuples.get(2);
+    assertTrue("blah blah blah 3".equals(t.getString("subject")));
+    t = tuples.get(3);
+    assertTrue("blah blah blah 4".equals(t.getString("subject")));
+    t = tuples.get(4);
+    assertTrue("blah blah blah 1".equals(t.getString("subject")));
+    t = tuples.get(5);
+    assertTrue("blah blah blah 5".equals(t.getString("subject")));
+    t = tuples.get(6);
+    assertTrue("blah blah blah 6".equals(t.getString("subject")));
+    t = tuples.get(7);
+    assertTrue("blah blah blah 7".equals(t.getString("subject")));
+    t = tuples.get(8);
+    assertTrue("blah blah blah 8".equals(t.getString("subject")));
+    t = tuples.get(9);
+    assertTrue("blah blah blah 9".equals(t.getString("subject")));
+
+  }
+
+
+
+
+
+  @Test
   public void testDaemonStream() throws Exception {
 
     new UpdateRequest()


[2/2] lucene-solr:branch_6x: Merge branch 'branch_6x' of https://git-wip-us.apache.org/repos/asf/lucene-solr into branch_6x

Posted by jb...@apache.org.
Merge branch 'branch_6x' of https://git-wip-us.apache.org/repos/asf/lucene-solr into branch_6x


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

Branch: refs/heads/branch_6x
Commit: 45f2dfe2ce36e79843ee203721d69a7fd9ffda75
Parents: 5836f40 e585c84
Author: Joel Bernstein <jb...@apache.org>
Authored: Mon Oct 10 15:24:43 2016 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Mon Oct 10 15:24:43 2016 -0400

----------------------------------------------------------------------
 build.xml                                       |    2 +
 lucene/CHANGES.txt                              |   24 +
 .../miscellaneous/ASCIIFoldingFilter.java       |   24 +-
 .../miscellaneous/TestASCIIFoldingFilter.java   |   13 +
 .../analysis/ja/JapaneseNumberFilter.java       |   11 +
 .../lucene/analysis/ja/TestFactories.java       |  203 ++++
 .../lucene/codecs/lucene60/Lucene60Codec.java   |    2 +-
 .../lucene50/TestLucene50SegmentInfoFormat.java |   40 +
 .../lucene/codecs/lucene60/Lucene60RWCodec.java |   30 +
 lucene/benchmark/.gitignore                     |    4 +-
 lucene/benchmark/README.enwiki                  |   11 +-
 lucene/benchmark/conf/highlight-profile.alg     |   68 --
 .../conf/highlight-vs-vector-highlight.alg      |   80 --
 lucene/benchmark/conf/highlighters-postings.alg |   65 ++
 lucene/benchmark/conf/highlighters-tv.alg       |   64 ++
 lucene/benchmark/conf/highlights.alg            |   69 ++
 lucene/benchmark/conf/query-phrases.txt         |   10 +
 lucene/benchmark/conf/query-terms.txt           |   10 +
 lucene/benchmark/conf/query-wildcards.txt       |    7 +
 .../benchmark/conf/standard-highlights-notv.alg |   69 --
 .../benchmark/conf/standard-highlights-tv.alg   |   69 --
 .../benchmark/conf/vector-highlight-profile.alg |   68 --
 .../lucene/benchmark/byTask/PerfRunData.java    |    2 +
 .../lucene/benchmark/byTask/feeds/DocMaker.java |    7 +
 .../byTask/tasks/BenchmarkHighlighter.java      |   30 -
 .../lucene/benchmark/byTask/tasks/ReadTask.java |  120 +-
 .../tasks/SearchTravRetHighlightTask.java       |  283 +++--
 .../tasks/SearchTravRetVectorHighlightTask.java |  147 ---
 .../benchmark/byTask/TestPerfTasksLogic.java    |  106 --
 .../tasks/CountingHighlighterTestTask.java      |   68 --
 .../perfield/PerFieldDocValuesFormat.java       |   28 +
 .../codecs/perfield/PerFieldMergeState.java     |  274 +++++
 .../codecs/perfield/PerFieldPostingsFormat.java |   85 +-
 .../lucene/index/DefaultIndexingChain.java      |    4 +-
 .../lucene50/TestLucene50FieldInfoFormat.java   |   33 -
 .../lucene50/TestLucene60FieldInfoFormat.java   |   33 +
 .../perfield/TestPerFieldDocValuesFormat.java   |  122 +++
 .../perfield/TestPerFieldPostingsFormat2.java   |  110 ++
 .../lucene/search/TestBooleanRewrites.java      |   19 +-
 .../uhighlight/AnalysisOffsetStrategy.java      |  190 ++++
 .../uhighlight/DefaultPassageFormatter.java     |  138 +++
 .../search/uhighlight/FieldHighlighter.java     |  276 +++++
 .../search/uhighlight/FieldOffsetStrategy.java  |  122 +++
 .../uhighlight/MultiTermHighlighting.java       |  379 +++++++
 .../uhighlight/MultiValueTokenStream.java       |  148 +++
 .../search/uhighlight/NoOpOffsetStrategy.java   |   50 +
 .../lucene/search/uhighlight/OffsetsEnum.java   |   97 ++
 .../lucene/search/uhighlight/Passage.java       |  161 +++
 .../search/uhighlight/PassageFormatter.java     |   40 +
 .../lucene/search/uhighlight/PassageScorer.java |  113 ++
 .../lucene/search/uhighlight/PhraseHelper.java  |  581 ++++++++++
 .../uhighlight/PostingsOffsetStrategy.java      |   61 ++
 .../PostingsWithTermVectorsOffsetStrategy.java  |   71 ++
 .../uhighlight/SplittingBreakIterator.java      |  244 +++++
 .../TermVectorFilteredLeafReader.java           |  130 +++
 .../uhighlight/TermVectorOffsetStrategy.java    |   68 ++
 .../uhighlight/TokenStreamFromTermVector.java   |  395 +++++++
 .../search/uhighlight/UnifiedHighlighter.java   | 1021 ++++++++++++++++++
 .../lucene/search/uhighlight/package-info.java  |   22 +
 .../search/vectorhighlight/FieldQuery.java      |    7 +
 .../lucene/search/uhighlight/CambridgeMA.utf8   |    1 +
 .../uhighlight/TestSplittingBreakIterator.java  |  192 ++++
 .../uhighlight/TestUnifiedHighlighter.java      |  962 +++++++++++++++++
 .../uhighlight/TestUnifiedHighlighterMTQ.java   |  936 ++++++++++++++++
 .../TestUnifiedHighlighterRanking.java          |  339 ++++++
 .../TestUnifiedHighlighterReanalysis.java       |   74 ++
 .../TestUnifiedHighlighterStrictPhrases.java    |  404 +++++++
 .../TestUnifiedHighlighterTermVec.java          |  182 ++++
 .../lucene/search/uhighlight/UHTestHelper.java  |   69 ++
 .../TestUnifiedHighlighterExtensibility.java    |  182 ++++
 .../FastVectorHighlighterTest.java              |   38 +
 .../classic/MultiFieldQueryParser.java          |    7 +-
 .../classic/TestMultiFieldQueryParser.java      |   21 +
 .../lucene/spatial3d/Geo3DDocValuesField.java   |   60 +
 .../spatial3d/PointInShapeIntersectVisitor.java |   30 +-
 .../apache/lucene/spatial3d/TestGeo3DPoint.java |    4 +-
 .../lucene/index/BasePointsFormatTestCase.java  |    5 +
 solr/CHANGES.txt                                |   42 +
 solr/bin/install_solr_service.sh                |   52 +-
 .../OverseerAutoReplicaFailoverThread.java      |    1 +
 .../org/apache/solr/cloud/ZkController.java     |   52 +-
 .../org/apache/solr/handler/GraphHandler.java   |    7 +-
 .../org/apache/solr/handler/IndexFetcher.java   |    6 +-
 .../org/apache/solr/handler/SQLHandler.java     |   48 +-
 .../org/apache/solr/handler/StreamHandler.java  |   10 +-
 .../solr/handler/admin/LukeRequestHandler.java  |    2 +-
 .../solr/handler/component/ExpandComponent.java |    4 +-
 .../solr/handler/component/FieldFacetStats.java |    2 +-
 .../handler/component/SpellCheckComponent.java  |    2 +-
 .../solr/handler/component/TermsComponent.java  |    2 +-
 .../solr/highlight/DefaultSolrHighlighter.java  |    4 +-
 .../apache/solr/request/DocValuesFacets.java    |    4 +-
 .../org/apache/solr/request/DocValuesStats.java |    4 +-
 .../org/apache/solr/request/NumericFacets.java  |    4 +-
 .../org/apache/solr/request/SimpleFacets.java   |    2 +-
 .../solr/request/macro/MacroExpander.java       |    1 -
 .../solr/response/SortingResponseWriter.java    |    4 +-
 .../transform/SubQueryAugmenterFactory.java     |    1 -
 .../response/transform/TransformerFactory.java  |    2 +-
 .../solr/schema/ManagedIndexSchemaFactory.java  |    1 +
 .../apache/solr/search/AbstractReRankQuery.java |   83 ++
 .../solr/search/CollapsingQParserPlugin.java    |    4 +-
 .../apache/solr/search/JoinQParserPlugin.java   |    8 +-
 .../apache/solr/search/ReRankQParserPlugin.java |   56 +-
 .../apache/solr/search/SolrIndexSearcher.java   |   24 +-
 .../facet/FacetFieldProcessorByArrayUIF.java    |    2 +-
 .../FacetFieldProcessorByEnumTermsStream.java   |    4 +-
 .../facet/FacetFieldProcessorByHashDV.java      |    2 +-
 .../org/apache/solr/search/facet/FieldUtil.java |    4 +-
 .../solr/search/facet/UnInvertedField.java      |    4 +-
 .../solr/search/function/OrdFieldSource.java    |    2 +-
 .../search/function/ReverseOrdFieldSource.java  |    2 +-
 .../join/BlockJoinFieldFacetAccumulator.java    |    4 +-
 .../solr/security/PKIAuthenticationPlugin.java  |    5 +-
 .../org/apache/solr/servlet/HttpSolrCall.java   |    4 +-
 .../apache/solr/update/AddUpdateCommand.java    |    1 -
 .../ClassificationUpdateProcessorFactory.java   |    2 +-
 .../src/java/org/apache/solr/util/SolrCLI.java  |    3 +-
 .../cloud-managed-upgrade/conf/schema.xml       |   27 +
 .../cloud-managed-upgrade/conf/solrconfig.xml   |   50 +
 .../solr/client/solrj/ConnectionReuseTest.java  |  216 ----
 .../client/solrj/impl/ConnectionReuseTest.java  |  224 ++++
 .../apache/solr/cloud/AliasIntegrationTest.java |  245 ++---
 .../AsyncCallRequestStatusResponseTest.java     |   44 +-
 .../solr/cloud/AsyncMigrateRouteKeyTest.java    |  121 ---
 .../apache/solr/cloud/CollectionReloadTest.java |   81 +-
 .../solr/cloud/CollectionStateFormat2Test.java  |   73 +-
 .../solr/cloud/DeleteInactiveReplicaTest.java   |    2 +-
 .../apache/solr/cloud/MigrateRouteKeyTest.java  |  160 ++-
 .../TestLeaderRecoverFromLogOnStartup.java      |   77 ++
 .../TestSolrCloudWithDelegationTokens.java      |   26 +-
 .../org/apache/solr/cloud/rule/RulesTest.java   |  241 ++---
 .../org/apache/solr/handler/TestSQLHandler.java |  235 +++-
 .../org/apache/solr/request/TestFaceting.java   |   22 +-
 .../apache/solr/schema/DocValuesMultiTest.java  |    2 +-
 .../org/apache/solr/schema/DocValuesTest.java   |   12 +-
 .../schema/TestManagedSchemaThreadSafety.java   |  164 +++
 .../solr/search/join/TestScoreJoinQPScore.java  |    2 +-
 .../security/TestAuthorizationFramework.java    |    3 +-
 solr/solrj/ivy.xml                              |    4 -
 .../solr/client/solrj/impl/CloudSolrClient.java |   50 +-
 .../solrj/impl/ConcurrentUpdateSolrClient.java  |    3 +-
 .../impl/DelegationTokenHttpSolrClient.java     |   73 ++
 .../solr/client/solrj/impl/HttpClientUtil.java  |   26 +-
 .../solr/client/solrj/impl/HttpSolrClient.java  |   66 +-
 .../solrj/impl/Krb5HttpClientConfigurer.java    |    3 +-
 .../client/solrj/impl/LBHttpSolrClient.java     |   92 +-
 .../solrj/io/stream/expr/StreamFactory.java     |   44 +-
 .../solrj/request/CollectionAdminRequest.java   |   52 +-
 .../solr/client/solrj/request/CoreStatus.java   |    6 +
 .../solrj/response/DelegationTokenResponse.java |   13 +-
 .../client/solrj/response/LukeResponse.java     |   15 +-
 ...ParamsAllAndReadonlyDigestZkACLProvider.java |    5 +-
 .../solr/common/cloud/rule/ImplicitSnitch.java  |   11 +-
 .../apache/solr/common/cloud/rule/Snitch.java   |    8 +-
 .../org/apache/solr/common/util/RetryUtil.java  |   11 +-
 .../solr/client/solrj/SolrExampleTests.java     |   32 +-
 .../client/solrj/SolrSchemalessExampleTest.java |   21 +-
 .../client/solrj/embedded/JettyWebappTest.java  |    3 +-
 .../solrj/embedded/SolrExampleJettyTest.java    |    9 +-
 .../solrj/impl/BasicHttpSolrClientTest.java     |    2 +-
 .../solrj/impl/HttpSolrClientConPoolTest.java   |  188 ++++
 .../impl/HttpSolrClientSSLAuthConPoolTest.java  |   39 +
 .../java/org/apache/solr/SolrTestCaseJ4.java    |    4 +
 .../apache/solr/cloud/MiniSolrCloudCluster.java |   22 +
 .../apache/solr/cloud/SolrCloudTestCase.java    |   16 +-
 .../org/apache/solr/util/RestTestHarness.java   |   16 +-
 solr/webapp/web/css/angular/dashboard.css       |    8 +-
 .../web/js/angular/controllers/core-overview.js |   83 --
 solr/webapp/web/partials/core_overview.html     |   17 -
 170 files changed, 11374 insertions(+), 2300 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/45f2dfe2/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------