SOLR-9559: Add ExecutorStream to execute stored Streaming Expressions

This commit is contained in:
Joel Bernstein 2016-10-26 17:39:59 -04:00
parent 2ee723140c
commit 727bf559a0
3 changed files with 372 additions and 36 deletions

View File

@ -35,36 +35,7 @@ import org.apache.solr.client.solrj.io.ops.ConcatOperation;
import org.apache.solr.client.solrj.io.ops.DistinctOperation;
import org.apache.solr.client.solrj.io.ops.GroupOperation;
import org.apache.solr.client.solrj.io.ops.ReplaceOperation;
import org.apache.solr.client.solrj.io.stream.CloudSolrStream;
import org.apache.solr.client.solrj.io.stream.CommitStream;
import org.apache.solr.client.solrj.io.stream.ComplementStream;
import org.apache.solr.client.solrj.io.stream.DaemonStream;
import org.apache.solr.client.solrj.io.stream.ExceptionStream;
import org.apache.solr.client.solrj.io.stream.FacetStream;
import org.apache.solr.client.solrj.io.stream.FeaturesSelectionStream;
import org.apache.solr.client.solrj.io.stream.FetchStream;
import org.apache.solr.client.solrj.io.stream.HashJoinStream;
import org.apache.solr.client.solrj.io.stream.InnerJoinStream;
import org.apache.solr.client.solrj.io.stream.IntersectStream;
import org.apache.solr.client.solrj.io.stream.JDBCStream;
import org.apache.solr.client.solrj.io.stream.LeftOuterJoinStream;
import org.apache.solr.client.solrj.io.stream.MergeStream;
import org.apache.solr.client.solrj.io.stream.ModelStream;
import org.apache.solr.client.solrj.io.stream.OuterHashJoinStream;
import org.apache.solr.client.solrj.io.stream.ParallelStream;
import org.apache.solr.client.solrj.io.stream.RankStream;
import org.apache.solr.client.solrj.io.stream.ReducerStream;
import org.apache.solr.client.solrj.io.stream.RollupStream;
import org.apache.solr.client.solrj.io.stream.ScoreNodesStream;
import org.apache.solr.client.solrj.io.stream.SelectStream;
import org.apache.solr.client.solrj.io.stream.SortStream;
import org.apache.solr.client.solrj.io.stream.StatsStream;
import org.apache.solr.client.solrj.io.stream.StreamContext;
import org.apache.solr.client.solrj.io.stream.TextLogitStream;
import org.apache.solr.client.solrj.io.stream.TopicStream;
import org.apache.solr.client.solrj.io.stream.TupleStream;
import org.apache.solr.client.solrj.io.stream.UniqueStream;
import org.apache.solr.client.solrj.io.stream.UpdateStream;
import org.apache.solr.client.solrj.io.stream.*;
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;
@ -167,6 +138,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
.withFunctionName("model", ModelStream.class)
.withFunctionName("classify", ClassifyStream.class)
.withFunctionName("fetch", FetchStream.class)
.withFunctionName("executor", ExecutorStream.class)
// metrics
.withFunctionName("min", MinMetric.class)

View File

@ -0,0 +1,224 @@
/*
* 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.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.List;
import java.util.Locale;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
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.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* The executor function wraps a stream with Tuples containing Streaming Expressions
* and executes them in parallel. Sample syntax:
*
* executor(thread=10, topic(storedExpressions, q="*:*", fl="expr_s, id", id="topic1"))
*
* The Streaming Expression to execute is taken from the expr field in the Tuples.
*/
public class ExecutorStream extends TupleStream implements Expressible {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private TupleStream stream;
private int threads;
private ExecutorService executorService;
private StreamFactory streamFactory;
private StreamContext streamContext;
public ExecutorStream(StreamExpression expression, StreamFactory factory) throws IOException {
// grab all parameters out
List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
StreamExpressionNamedParameter threadsParam = factory.getNamedOperand(expression, "threads");
int threads = 6;
if(threadsParam != null) {
threads = Integer.parseInt(((StreamExpressionValue)threadsParam.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));
init(stream, threads, factory);
}
private void init(TupleStream tupleStream, int threads, StreamFactory factory) throws IOException{
this.threads = threads;
this.stream = tupleStream;
this.streamFactory = factory;
}
@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(new StreamExpressionNamedParameter("threads", Integer.toString(threads)));
// stream
if(includeStreams) {
if (stream instanceof Expressible) {
expression.addParameter(((Expressible) stream).toExpression(factory));
} else {
throw new IOException("The ExecuteStream 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 {
executorService = ExecutorUtil.newMDCAwareFixedThreadPool(threads, new SolrjNamedThreadFactory("ExecutorStream"));
stream.open();
}
public void close() throws IOException {
stream.close();
executorService.shutdown();
try {
executorService.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS);
} catch(InterruptedException e) {
logger.error("Interrupted while waiting for termination", e);
}
}
public Tuple read() throws IOException {
ArrayBlockingQueue<Tuple> queue = new ArrayBlockingQueue(10000);
while(true) {
Tuple tuple = stream.read();
if (!tuple.EOF) {
try {
queue.put(tuple);
} catch (InterruptedException e) {
throw new IOException(e);
}
executorService.execute(new StreamTask(queue, streamFactory, streamContext));
} else {
return tuple;
}
}
}
public StreamComparator getStreamSort(){
return stream.getStreamSort();
}
public int getCost() {
return 0;
}
public static class StreamTask implements Runnable {
private ArrayBlockingQueue<Tuple> queue;
private StreamFactory streamFactory;
private StreamContext streamContext;
public StreamTask(ArrayBlockingQueue queue, StreamFactory streamFactory, StreamContext streamContext) {
this.queue = queue;
this.streamFactory = streamFactory;
this.streamContext = new StreamContext();
this.streamContext.setSolrClientCache(streamContext.getSolrClientCache());
this.streamContext.setModelCache(streamContext.getModelCache());
}
public void run() {
Tuple tuple = null;
try {
tuple = queue.take();
} catch (Exception e) {
throw new RuntimeException(e);
}
String expr = tuple.getString("expr_s");
Object id = tuple.get("id");
TupleStream stream = null;
try {
stream = streamFactory.constructStream(expr);
stream.setStreamContext(streamContext);
stream.open();
while (true) {
Tuple t = stream.read();
if (t.EOF) {
break;
}
}
} catch (Exception e) {
logger.error("Executor Error: id="+id+" expr_s="+expr, e);
} finally {
try {
stream.close();
} catch (Exception e1) {
logger.error("Executor Error", e1);
}
}
}
}
}

View File

@ -256,7 +256,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
tuples = getTuples(stream);
assertEquals(4, tuples.size());
assertOrder(tuples, 4,3,1,0);
assertOrder(tuples, 4, 3, 1, 0);
// Basic w/ multi comp
sParams.set("q2", "search(" + COLLECTION + ", q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=${mySort})");
@ -522,14 +522,14 @@ public class StreamExpressionTest extends SolrCloudTestCase {
// Basic test
expression = StreamExpressionParser.parse("top("
+ "n=3,"
+ "search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"),"
+ "sort=\"a_f asc, a_i asc\")");
+ "n=3,"
+ "search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"),"
+ "sort=\"a_f asc, a_i asc\")");
stream = new RankStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 3);
assertOrder(tuples, 0,2,1);
assertOrder(tuples, 0, 2, 1);
// Basic test desc
expression = StreamExpressionParser.parse("top("
@ -3794,7 +3794,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
paramsLoc = new ModifiableSolrParams();
paramsLoc.set("expr", expr);
paramsLoc.set("qt","/stream");
paramsLoc.set("qt", "/stream");
SolrStream classifyStream = new SolrStream(url, paramsLoc);
Map<String, Double> idToLabel = getIdToLabel(classifyStream, "probability_d");
assertEquals(idToLabel.size(), 2);
@ -3866,6 +3866,146 @@ public class StreamExpressionTest extends SolrCloudTestCase {
CollectionAdminRequest.deleteCollection("checkpointCollection").process(cluster.getSolrClient());
}
@Test
public void testExecutorStream() throws Exception {
CollectionAdminRequest.createCollection("workQueue", "conf", 2, 1).process(cluster.getSolrClient());
AbstractDistribZkTestBase.waitForRecoveriesToFinish("workQueue", cluster.getSolrClient().getZkStateReader(),
false, true, TIMEOUT);
CollectionAdminRequest.createCollection("mainCorpus", "conf", 2, 1).process(cluster.getSolrClient());
AbstractDistribZkTestBase.waitForRecoveriesToFinish("mainCorpus", cluster.getSolrClient().getZkStateReader(),
false, true, TIMEOUT);
CollectionAdminRequest.createCollection("destination", "conf", 2, 1).process(cluster.getSolrClient());
AbstractDistribZkTestBase.waitForRecoveriesToFinish("destination", cluster.getSolrClient().getZkStateReader(),
false, true, TIMEOUT);
UpdateRequest workRequest = new UpdateRequest();
UpdateRequest dataRequest = new UpdateRequest();
for (int i = 0; i < 500; i++) {
workRequest.add(id, String.valueOf(i), "expr_s", "update(destination, batchSize=50, search(mainCorpus, q=id:"+i+", rows=1, sort=\"id asc\", fl=\"id, body_t, field_i\"))");
dataRequest.add(id, String.valueOf(i), "body_t", "hello world "+i, "field_i", Integer.toString(i));
}
workRequest.commit(cluster.getSolrClient(), "workQueue");
dataRequest.commit(cluster.getSolrClient(), "mainCorpus");
String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString() + "/destination";
TupleStream executorStream;
ModifiableSolrParams paramsLoc;
StreamFactory factory = new StreamFactory()
.withCollectionZkHost("workQueue", cluster.getZkServer().getZkAddress())
.withCollectionZkHost("mainCorpus", cluster.getZkServer().getZkAddress())
.withCollectionZkHost("destination", cluster.getZkServer().getZkAddress())
.withFunctionName("search", CloudSolrStream.class)
.withFunctionName("executor", ExecutorStream.class)
.withFunctionName("update", UpdateStream.class);
String executorExpression = "executor(threads=3, search(workQueue, q=\"*:*\", fl=\"id, expr_s\", rows=1000, sort=\"id desc\"))";
executorStream = factory.constructStream(executorExpression);
StreamContext context = new StreamContext();
SolrClientCache clientCache = new SolrClientCache();
context.setSolrClientCache(clientCache);
executorStream.setStreamContext(context);
getTuples(executorStream);
//Destination collection should now contain all the records in the main corpus.
cluster.getSolrClient().commit("destination");
paramsLoc = new ModifiableSolrParams();
paramsLoc.set("expr", "search(destination, q=\"*:*\", fl=\"id, body_t, field_i\", rows=1000, sort=\"field_i asc\")");
paramsLoc.set("qt","/stream");
SolrStream solrStream = new SolrStream(url, paramsLoc);
List<Tuple> tuples = getTuples(solrStream);
assertTrue(tuples.size() == 500);
for(int i=0; i<500; i++) {
Tuple tuple = tuples.get(i);
long ivalue = tuple.getLong("field_i");
String body = tuple.getString("body_t");
assertTrue(ivalue == i);
assertTrue(body.equals("hello world "+i));
}
solrStream.close();
clientCache.close();
CollectionAdminRequest.deleteCollection("workQueue").process(cluster.getSolrClient());
CollectionAdminRequest.deleteCollection("mainCorpus").process(cluster.getSolrClient());
CollectionAdminRequest.deleteCollection("destination").process(cluster.getSolrClient());
}
@Test
public void testParallelExecutorStream() throws Exception {
CollectionAdminRequest.createCollection("workQueue", "conf", 2, 1).process(cluster.getSolrClient());
AbstractDistribZkTestBase.waitForRecoveriesToFinish("workQueue", cluster.getSolrClient().getZkStateReader(),
false, true, TIMEOUT);
CollectionAdminRequest.createCollection("mainCorpus", "conf", 2, 1).process(cluster.getSolrClient());
AbstractDistribZkTestBase.waitForRecoveriesToFinish("mainCorpus", cluster.getSolrClient().getZkStateReader(),
false, true, TIMEOUT);
CollectionAdminRequest.createCollection("destination", "conf", 2, 1).process(cluster.getSolrClient());
AbstractDistribZkTestBase.waitForRecoveriesToFinish("destination", cluster.getSolrClient().getZkStateReader(),
false, true, TIMEOUT);
UpdateRequest workRequest = new UpdateRequest();
UpdateRequest dataRequest = new UpdateRequest();
for (int i = 0; i < 500; i++) {
workRequest.add(id, String.valueOf(i), "expr_s", "update(destination, batchSize=50, search(mainCorpus, q=id:"+i+", rows=1, sort=\"id asc\", fl=\"id, body_t, field_i\"))");
dataRequest.add(id, String.valueOf(i), "body_t", "hello world "+i, "field_i", Integer.toString(i));
}
workRequest.commit(cluster.getSolrClient(), "workQueue");
dataRequest.commit(cluster.getSolrClient(), "mainCorpus");
String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString() + "/destination";
TupleStream executorStream;
ModifiableSolrParams paramsLoc;
StreamFactory factory = new StreamFactory()
.withCollectionZkHost("workQueue", cluster.getZkServer().getZkAddress())
.withCollectionZkHost("mainCorpus", cluster.getZkServer().getZkAddress())
.withCollectionZkHost("destination", cluster.getZkServer().getZkAddress())
.withFunctionName("search", CloudSolrStream.class)
.withFunctionName("executor", ExecutorStream.class)
.withFunctionName("parallel", ParallelStream.class)
.withFunctionName("update", UpdateStream.class);
String executorExpression = "parallel(workQueue, workers=2, sort=\"EOF asc\", executor(threads=3, queueSize=100, search(workQueue, q=\"*:*\", fl=\"id, expr_s\", rows=1000, partitionKeys=id, sort=\"id desc\")))";
executorStream = factory.constructStream(executorExpression);
StreamContext context = new StreamContext();
SolrClientCache clientCache = new SolrClientCache();
context.setSolrClientCache(clientCache);
executorStream.setStreamContext(context);
getTuples(executorStream);
//Destination collection should now contain all the records in the main corpus.
cluster.getSolrClient().commit("destination");
paramsLoc = new ModifiableSolrParams();
paramsLoc.set("expr", "search(destination, q=\"*:*\", fl=\"id, body_t, field_i\", rows=1000, sort=\"field_i asc\")");
paramsLoc.set("qt","/stream");
SolrStream solrStream = new SolrStream(url, paramsLoc);
List<Tuple> tuples = getTuples(solrStream);
assertTrue(tuples.size() == 500);
for(int i=0; i<500; i++) {
Tuple tuple = tuples.get(i);
long ivalue = tuple.getLong("field_i");
String body = tuple.getString("body_t");
assertTrue(ivalue == i);
assertTrue(body.equals("hello world " + i));
}
solrStream.close();
clientCache.close();
CollectionAdminRequest.deleteCollection("workQueue").process(cluster.getSolrClient());
CollectionAdminRequest.deleteCollection("mainCorpus").process(cluster.getSolrClient());
CollectionAdminRequest.deleteCollection("destination").process(cluster.getSolrClient());
}
private Map<String,Double> getIdToLabel(TupleStream stream, String outField) throws IOException {
Map<String, Double> idToLabel = new HashMap<>();
List<Tuple> tuples = getTuples(stream);