SOLR-7377: Streaming Expressions

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1678743 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Joel Bernstein 2015-05-11 12:37:18 +00:00
parent 9d49a76d01
commit e50f4d94c7
42 changed files with 3525 additions and 681 deletions

View File

@ -17,32 +17,79 @@
package org.apache.solr.handler;
import java.util.Map.Entry;
import java.net.URLDecoder;
import java.io.ByteArrayInputStream;
import java.io.ObjectInputStream;
import java.net.URLDecoder;
import java.util.Iterator;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.solr.client.solrj.io.SolrClientCache;
import org.apache.solr.client.solrj.io.TupleStream;
import org.apache.solr.client.solrj.io.StreamContext;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.client.solrj.io.stream.CloudSolrStream;
import org.apache.solr.client.solrj.io.stream.ExpressibleStream;
import org.apache.solr.client.solrj.io.stream.MergeStream;
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.StreamContext;
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.expr.StreamFactory;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.core.CloseHook;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.SolrCore;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.util.plugin.SolrCoreAware;
import org.apache.solr.common.util.Base64;
public class StreamHandler extends RequestHandlerBase implements SolrCoreAware {
private SolrClientCache clientCache = new SolrClientCache();
private StreamFactory streamFactory = new StreamFactory();
public void inform(SolrCore core) {
/* The stream factory will always contain the zkUrl for the given collection
* Adds default streams with their corresponding function names. These
* defaults can be overridden or added to in the solrConfig in the stream
* RequestHandler def. Example config override
* <lst name="streamFunctions">
* <str name="group">org.apache.solr.client.solrj.io.stream.ReducerStream</str>
* <str name="count">org.apache.solr.client.solrj.io.stream.CountStream</str>
* </lst>
* */
core.addCloseHook( new CloseHook() {
String defaultCollection = null;
String defaultZkhost = null;
CoreContainer coreContainer = core.getCoreDescriptor().getCoreContainer();
if(coreContainer.isZooKeeperAware()) {
defaultCollection = core.getCoreDescriptor().getCollectionName();
defaultZkhost = core.getCoreDescriptor().getCoreContainer().getZkController().getZkServerAddress();
streamFactory.withCollectionZkHost(defaultCollection, defaultZkhost);
}
streamFactory
.withStreamFunction("search", CloudSolrStream.class)
.withStreamFunction("merge", MergeStream.class)
.withStreamFunction("unique", UniqueStream.class)
.withStreamFunction("top", RankStream.class)
.withStreamFunction("group", ReducerStream.class)
.withStreamFunction("parallel", ParallelStream.class);
// This pulls all the overrides and additions from the config
Object functionMappingsObj = initArgs.get("streamFunctions");
if(null != functionMappingsObj){
NamedList<?> functionMappings = (NamedList<?>)functionMappingsObj;
for(Entry<String,?> functionMapping : functionMappings){
Class<?> clazz = core.getResourceLoader().findClass((String)functionMapping.getValue(), ExpressibleStream.class);
streamFactory.withStreamFunction(functionMapping.getKey(), clazz);
}
}
core.addCloseHook(new CloseHook() {
@Override
public void preClose(SolrCore core) {
//To change body of implemented methods use File | Settings | File Templates.
@ -57,15 +104,23 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware {
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
SolrParams params = req.getParams();
String encodedStream = params.get("stream");
encodedStream = URLDecoder.decode(encodedStream, "UTF-8");
byte[] bytes = Base64.base64ToByteArray(encodedStream);
ByteArrayInputStream byteStream = new ByteArrayInputStream(bytes);
ObjectInputStream objectInputStream = new ObjectInputStream(byteStream);
TupleStream tupleStream = (TupleStream)objectInputStream.readObject();
int worker = params.getInt("workerID");
int numWorkers = params.getInt("numWorkers");
boolean objectSerialize = params.getBool("objectSerialize", false);
TupleStream tupleStream = null;
if(objectSerialize) {
String encodedStream = params.get("stream");
encodedStream = URLDecoder.decode(encodedStream, "UTF-8");
byte[] bytes = Base64.base64ToByteArray(encodedStream);
ByteArrayInputStream byteStream = new ByteArrayInputStream(bytes);
ObjectInputStream objectInputStream = new ObjectInputStream(byteStream);
tupleStream = (TupleStream)objectInputStream.readObject();
} else {
tupleStream = this.streamFactory.constructStream(params.get("stream"));
}
int worker = params.getInt("workerID", 0);
int numWorkers = params.getInt("numWorkers", 1);
StreamContext context = new StreamContext();
context.workerID = worker;
context.numWorkers = numWorkers;
@ -81,4 +136,4 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware {
public String getSource() {
return null;
}
}
}

View File

@ -21,7 +21,7 @@ import java.io.IOException;
import java.io.Writer;
import java.util.*;
import org.apache.solr.client.solrj.io.TupleStream;
import org.apache.solr.client.solrj.io.stream.TupleStream;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.index.StoredDocument;

View File

@ -854,6 +854,37 @@
</lst>
</requestHandler>
<!--
The export request handler is used to export full sorted result sets.
Do not change these defaults.
-->
<requestHandler name="/export" class="solr.SearchHandler">
<lst name="invariants">
<str name="rq">{!xport}</str>
<str name="wt">xsort</str>
<str name="distrib">false</str>
</lst>
<arr name="components">
<str>query</str>
</arr>
</requestHandler>
<!--
Distributed Stream processing.
-->
<requestHandler name="/stream" class="solr.StreamHandler">
<lst name="invariants">
<str name="wt">json</str>
<str name="distrib">false</str>
</lst>
</requestHandler>
<!-- A Robust Example
This example SearchHandler declaration shows off usage of the

View File

@ -1,99 +0,0 @@
/*
* 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;
import java.io.IOException;
import java.util.Comparator;
import java.util.List;
import java.util.ArrayList;
/**
* Unions streamA with streamB ordering the Tuples based on a Comparator.
* Both streams must be sorted by the fields being compared.
**/
public class MergeStream extends TupleStream {
private static final long serialVersionUID = 1;
private PushBackStream streamA;
private PushBackStream streamB;
private Comparator<Tuple> comp;
public MergeStream(TupleStream streamA, TupleStream streamB, Comparator<Tuple> comp) {
this.streamA = new PushBackStream(streamA);
this.streamB = new PushBackStream(streamB);
this.comp = comp;
}
public void setStreamContext(StreamContext context) {
this.streamA.setStreamContext(context);
this.streamB.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(streamA);
l.add(streamB);
return l;
}
public void open() throws IOException {
streamA.open();
streamB.open();
}
public void close() throws IOException {
streamA.close();
streamB.close();
}
public Tuple read() throws IOException {
Tuple a = streamA.read();
Tuple b = streamB.read();
if(a.EOF && b.EOF) {
return a;
}
if(a.EOF) {
streamA.pushBack(a);
return b;
}
if(b.EOF) {
streamB.pushBack(b);
return a;
}
int c = comp.compare(a,b);
if(c < 0) {
streamB.pushBack(b);
return a;
} else {
streamA.pushBack(a);
return b;
}
}
public int getCost() {
return 0;
}
}

View File

@ -1,141 +0,0 @@
/*
* 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;
import java.io.IOException;
import java.io.ObjectOutputStream;
import java.net.URLEncoder;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Map;
import java.io.ByteArrayOutputStream;
import java.util.Random;
import java.util.TreeSet;
import java.util.Iterator;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Base64;
/**
* The ParallelStream decorates a TupleStream implementation and pushes it to N workers for parallel execution.
* Workers are chosen from a SolrCloud collection.
* Tuples that are streamed back from the workers are ordered by a Comparator.
**/
public class ParallelStream extends CloudSolrStream {
private TupleStream tupleStream;
private int workers;
private String encoded;
public ParallelStream(String zkHost,
String collection,
TupleStream tupleStream,
int workers,
Comparator<Tuple> comp) throws IOException {
this.zkHost = zkHost;
this.collection = collection;
this.workers = workers;
this.comp = comp;
this.tupleStream = tupleStream;
ByteArrayOutputStream bout = new ByteArrayOutputStream();
ObjectOutputStream out = new ObjectOutputStream(bout);
out.writeObject(tupleStream);
byte[] bytes = bout.toByteArray();
this.encoded = Base64.byteArrayToBase64(bytes, 0, bytes.length);
this.encoded = URLEncoder.encode(this.encoded, "UTF-8");
this.tuples = new TreeSet();
}
public List<TupleStream> children() {
List l = new ArrayList();
l.add(tupleStream);
return l;
}
public Tuple read() throws IOException {
Tuple tuple = _read();
if(tuple.EOF) {
Map m = new HashMap();
m.put("EOF", true);
Tuple t = new Tuple(m);
t.setMetrics(this.eofTuples);
return t;
}
return tuple;
}
public void setStreamContext(StreamContext streamContext) {
//Note the parallel stream does not set the StreamContext on it's substream.
//This is because the substream is not actually opened by the ParallelStream.
this.streamContext = streamContext;
}
protected void constructStreams() throws IOException {
try {
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
ClusterState clusterState = zkStateReader.getClusterState();
Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
long time = System.currentTimeMillis();
List<Replica> shuffler = new ArrayList();
for(Slice slice : slices) {
Collection<Replica> replicas = slice.getReplicas();
for (Replica replica : replicas) {
shuffler.add(replica);
}
}
if(workers > shuffler.size()) {
throw new IOException("Number of workers exceeds nodes in the worker collection");
}
Collections.shuffle(shuffler, new Random(time));
for(int w=0; w<workers; w++) {
HashMap params = new HashMap();
params.put("distrib","false"); // We are the aggregator.
params.put("numWorkers", workers);
params.put("workerID", w);
params.put("stream", this.encoded);
params.put("qt","/stream");
Replica rep = shuffler.get(w);
ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep);
String url = zkProps.getCoreUrl();
SolrStream solrStream = new SolrStream(url, params);
solrStreams.add(solrStream);
}
assert(solrStreams.size() == workers);
} catch (Exception e) {
throw new IOException(e);
}
}
}

View File

@ -1,116 +0,0 @@
/*
* 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;
import java.io.IOException;
import java.io.Serializable;
import java.util.List;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.PriorityQueue;
import java.util.Comparator;
/**
* Iterates over a TupleStream and Ranks the topN tuples based on a Comparator.
**/
public class RankStream extends TupleStream {
private static final long serialVersionUID = 1;
private TupleStream tupleStream;
private PriorityQueue<Tuple> top;
private Comparator<Tuple> comp;
private boolean finished = false;
private LinkedList<Tuple> topList;
private int size;
public RankStream(TupleStream tupleStream, int size, Comparator<Tuple> comp) {
this.tupleStream = tupleStream;
this.top = new PriorityQueue(size, new ReverseComp(comp));
this.comp = comp;
this.topList = new LinkedList();
this.size = size;
}
public void setStreamContext(StreamContext context) {
this.tupleStream.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(tupleStream);
return l;
}
public void open() throws IOException {
tupleStream.open();
}
public void close() throws IOException {
tupleStream.close();
}
public Tuple read() throws IOException {
if(!finished) {
while(true) {
Tuple tuple = tupleStream.read();
if(tuple.EOF) {
finished = true;
int s = top.size();
for(int i=0; i<s; i++) {
Tuple t = top.poll();
topList.addFirst(t);
}
topList.addLast(tuple);
break;
} else {
Tuple peek = top.peek();
if(top.size() >= size) {
if(comp.compare(tuple, peek) < 0) {
top.poll();
top.add(tuple);
}
} else {
top.add(tuple);
}
}
}
}
return topList.pollFirst();
}
public int getCost() {
return 0;
}
class ReverseComp implements Comparator<Tuple>, Serializable {
private Comparator<Tuple> comp;
public ReverseComp(Comparator<Tuple> comp) {
this.comp = comp;
}
public int compare(Tuple t1, Tuple t2) {
return comp.compare(t1, t2)*(-1);
}
}
}

View File

@ -93,11 +93,11 @@ public class Tuple implements Cloneable {
this.fields.put("_MAPS_", maps);
}
public Map<String,Tuple> getMetrics() {
return (Map<String,Tuple>)this.fields.get("_METRICS_");
public Map<String,Map> getMetrics() {
return (Map<String,Map>)this.fields.get("_METRICS_");
}
public void setMetrics(Map<String, Tuple> metrics) {
public void setMetrics(Map<String, Map> metrics) {
this.fields.put("_METRICS_", metrics);
}

View File

@ -1,93 +0,0 @@
/*
* 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;
import java.io.IOException;
import java.util.Comparator;
import java.util.List;
import java.util.ArrayList;
/**
* The UniqueStream emits a unique stream of Tuples based on a Comparator.
*
* Note: The sort order of the underlying stream must match the Comparator.
**/
public class UniqueStream extends TupleStream {
private static final long serialVersionUID = 1;
private TupleStream tupleStream;
private Comparator<Tuple> comp;
private Tuple currentTuple = null;
public UniqueStream(TupleStream tupleStream, Comparator<Tuple> comp) {
this.tupleStream = tupleStream;
this.comp = comp;
}
public void setStreamContext(StreamContext context) {
this.tupleStream.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(tupleStream);
return l;
}
public void open() throws IOException {
tupleStream.open();
}
public void close() throws IOException {
tupleStream.close();
}
public Tuple read() throws IOException {
Tuple tuple = tupleStream.read();
if(tuple.EOF) {
return tuple;
}
if(currentTuple == null) {
currentTuple = tuple;
return tuple;
} else {
while(true) {
int i = comp.compare(currentTuple, tuple);
if(i == 0) {
//We have duplicate tuple so read the next tuple from the stream.
tuple = tupleStream.read();
if(tuple.EOF) {
return tuple;
}
} else {
//We have a non duplicate
this.currentTuple = tuple;
return tuple;
}
}
}
}
public int getCost() {
return 0;
}
}

View File

@ -0,0 +1,27 @@
package org.apache.solr.client.solrj.io.comp;
import org.apache.solr.client.solrj.io.Tuple;
/*
* 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.
*/
/**
* Interface for use with a comparator lambda
*/
public interface ComparatorLambda {
public int compare(Tuple leftTuple, Tuple rightTuple);
}

View File

@ -0,0 +1,48 @@
package org.apache.solr.client.solrj.io.comp;
import java.util.Locale;
/*
* 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.
*/
/**
* Enum for supported comparator ordering
*/
public enum ComparatorOrder {
ASCENDING, DESCENDING;
public static ComparatorOrder fromString(String order){
switch(order.toLowerCase(Locale.ROOT)){
case "asc":
return ComparatorOrder.ASCENDING;
case "desc":
return ComparatorOrder.DESCENDING;
default:
throw new IllegalArgumentException(String.format(Locale.ROOT,"Unknown order '%s'", order));
}
}
public String toString(){
switch(this){
case DESCENDING:
return "desc";
default:
return "asc";
}
}
}

View File

@ -1,3 +1,13 @@
package org.apache.solr.client.solrj.io.comp;
import java.io.IOException;
import java.io.Serializable;
import java.util.Comparator;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -15,34 +25,9 @@
* limitations under the License.
*/
package org.apache.solr.client.solrj.io;
import java.io.Serializable;
import java.util.Comparator;
/**
* Wraps multiple Comparators to provide sub-sorting.
**/
public class MultiComp implements Comparator<Tuple>, Serializable {
private static final long serialVersionUID = 1;
private Comparator<Tuple>[] comps;
public MultiComp(Comparator<Tuple>... comps) {
this.comps = comps;
}
public int compare(Tuple t1, Tuple t2) {
for(Comparator<Tuple> comp : comps) {
int i = comp.compare(t1, t2);
if(i != 0) {
return i;
}
}
return 0;
}
}
* Defines a comparator that can be expressed in an expression
*/
public interface ExpressibleComparator {
StreamExpressionParameter toExpression(StreamFactory factory) throws IOException;
}

View File

@ -0,0 +1,96 @@
/*
* 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.comp;
import java.io.Serializable;
import java.util.Comparator;
import org.apache.solr.client.solrj.io.Tuple;
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;
/**
* An equality field Comparator which compares a field of two Tuples and determines sort order.
**/
public class FieldComparator extends StreamComparator implements Comparator<Tuple>, ExpressibleComparator, Serializable {
private static final long serialVersionUID = 1;
private ComparatorLambda comparator;
public FieldComparator(String field, ComparatorOrder order) {
super(field, order);
assignComparator();
}
public FieldComparator(String leftField, String rightField, ComparatorOrder order){
super(leftField,rightField,order);
assignComparator();
}
public StreamExpressionParameter toExpression(StreamFactory factory){
StringBuilder sb = new StringBuilder();
sb.append(leftField);
if(!leftField.equals(rightField)){
sb.append("=");
sb.append(rightField);
}
sb.append(" ");
sb.append(order);
return new StreamExpressionValue(sb.toString());
}
/*
* What're we doing here messing around with lambdas for the comparator logic?
* We want the compare(...) function to run as fast as possible because it will be called many many
* times over the lifetime of this object. For that reason we want to limit the number of comparisons
* taking place in the compare(...) function. Because this class supports both ascending and
* descending comparisons and the logic for each is slightly different, we want to do the
* if(ascending){ compare like this } else { compare like this }
* check only once - we can do that in the constructor of this class, create a lambda, and then execute
* that lambda in the compare function. A little bit of branch prediction savings right here.
*/
private void assignComparator(){
if(ComparatorOrder.DESCENDING == order){
// What black magic is this type intersection??
// Because this class is serializable we need to make sure the lambda is also serializable.
// This can be done by providing this type intersection on the definition of the lambda.
// Why not do it in the lambda interface? Functional Interfaces don't allow extends clauses
comparator = (ComparatorLambda & Serializable)(leftTuple, rightTuple) -> {
Comparable leftComp = (Comparable)leftTuple.get(leftField);
Comparable rightComp = (Comparable)rightTuple.get(rightField);
return rightComp.compareTo(leftComp);
};
}
else{
// See above for black magic reasoning.
comparator = (ComparatorLambda & Serializable)(leftTuple, rightTuple) -> {
Comparable leftComp = (Comparable)leftTuple.get(leftField);
Comparable rightComp = (Comparable)rightTuple.get(rightField);
return leftComp.compareTo(rightComp);
};
}
}
public int compare(Tuple leftTuple, Tuple rightTuple) {
return comparator.compare(leftTuple, rightTuple);
}
}

View File

@ -0,0 +1,70 @@
/*
* 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.comp;
import java.io.IOException;
import java.io.Serializable;
import java.util.Comparator;
import org.apache.solr.client.solrj.io.Tuple;
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;
/**
* Wraps multiple Comparators to provide sub-sorting.
**/
public class MultiComp implements Comparator<Tuple>, ExpressibleComparator, Serializable {
private static final long serialVersionUID = 1;
private Comparator<Tuple>[] comps;
public MultiComp(Comparator<Tuple>... comps) {
this.comps = comps;
}
public int compare(Tuple t1, Tuple t2) {
for(Comparator<Tuple> comp : comps) {
int i = comp.compare(t1, t2);
if(i != 0) {
return i;
}
}
return 0;
}
@Override
public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
StringBuilder sb = new StringBuilder();
for(Comparator<Tuple> comp : comps){
if(comp instanceof ExpressibleComparator){
if(sb.length() > 0){ sb.append(","); }
sb.append(((ExpressibleComparator)comp).toExpression(factory));
}
else{
throw new IOException("This MultiComp contains a non-expressible comparator - it cannot be converted to an expression");
}
}
return new StreamExpressionValue(sb.toString());
}
}

View File

@ -0,0 +1,45 @@
package org.apache.solr.client.solrj.io.comp;
import java.io.Serializable;
import java.util.Comparator;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
/*
* 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.
*/
/**
* Defines a comparator that can be expressed in an expression
*/
public abstract class StreamComparator implements Comparator<Tuple>, Serializable {
protected String leftField;
protected String rightField;
protected final ComparatorOrder order;
public StreamComparator(String field, ComparatorOrder order) {
this.leftField = field;
this.rightField = field;
this.order = order;
}
public StreamComparator(String leftField, String rightField, ComparatorOrder order){
this.leftField = leftField;
this.rightField = rightField;
this.order = order;
}
}

View File

@ -0,0 +1,25 @@
/*
* 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.
*/
/**
* Comparators for the Streaming Aggregation API
**/
package org.apache.solr.client.solrj.io.comp;

View File

@ -15,30 +15,41 @@
* limitations under the License.
*/
package org.apache.solr.client.solrj.io;
package org.apache.solr.client.solrj.io.stream;
import java.io.IOException;
import java.util.Map;
import java.util.Comparator;
import java.util.Random;
import java.util.TreeSet;
import java.util.List;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Random;
import java.util.TreeSet;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
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.ComparatorOrder;
import org.apache.solr.client.solrj.io.comp.FieldComparator;
import org.apache.solr.client.solrj.io.comp.MultiComp;
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.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
@ -49,13 +60,13 @@ import org.apache.solr.common.util.SolrjNamedThreadFactory;
* to iterate and merge Tuples from each SolrStream.
**/
public class CloudSolrStream extends TupleStream {
public class CloudSolrStream extends TupleStream implements ExpressibleStream {
private static final long serialVersionUID = 1;
protected String zkHost;
protected String collection;
protected Map params;
protected Map<String,String> params;
private Map<String, String> fieldMappings;
protected Comparator<Tuple> comp;
private int zkConnectTimeout = 10000;
@ -70,23 +81,124 @@ public class CloudSolrStream extends TupleStream {
protected transient TreeSet<TupleWrapper> tuples;
protected transient StreamContext streamContext;
public CloudSolrStream(String zkHost, String collection, Map params) throws IOException {
// Used by parallel stream
protected CloudSolrStream(){
}
public CloudSolrStream(String zkHost, String collectionName, Map params) throws IOException {
init(collectionName, zkHost, params);
}
public CloudSolrStream(StreamExpression expression, StreamFactory factory) throws IOException{
// grab all parameters out
String collectionName = factory.getValueOperand(expression, 0);
List<StreamExpressionNamedParameter> namedParams = factory.getNamedOperands(expression);
StreamExpressionNamedParameter aliasExpression = factory.getNamedOperand(expression, "aliases");
StreamExpressionNamedParameter zkHostExpression = factory.getNamedOperand(expression, "zkHost");
// Validate there are no unknown parameters - zkHost and alias are namedParameter so we don't need to count it twice
if(expression.getParameters().size() != 1 + namedParams.size()){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - unknown operands found",expression));
}
// 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
if(0 == namedParams.size()){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - at least one named parameter expected. eg. 'q=*:*'",expression));
}
Map<String,String> params = new HashMap<String,String>();
for(StreamExpressionNamedParameter namedParam : namedParams){
if(!namedParam.getName().equals("zkHost") && !namedParam.getName().equals("aliases")){
params.put(namedParam.getName(), namedParam.getParameter().toString().trim());
}
}
// Aliases, optional, if provided then need to split
if(null != aliasExpression && aliasExpression.getParameter() instanceof StreamExpressionValue){
fieldMappings = new HashMap<String,String>();
for(String mapping : ((StreamExpressionValue)aliasExpression.getParameter()).getValue().split(",")){
String[] parts = mapping.trim().split("=");
if(2 == parts.length){
fieldMappings.put(parts[0], parts[1]);
}
else{
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - alias expected of the format origName=newName",expression));
}
}
}
// zkHost, optional - if not provided then will look into factory list to get
String zkHost = null;
if(null == zkHostExpression){
zkHost = factory.getCollectionZkHost(collectionName);
}
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(collectionName, zkHost, params);
}
@Override
public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
// functionName(collectionName, param1, param2, ..., paramN, sort="comp", [aliases="field=alias,..."])
// function name
StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
// collection
expression.addParameter(collection);
// parameters
for(Entry<String,String> param : params.entrySet()){
expression.addParameter(new StreamExpressionNamedParameter(param.getKey(), param.getValue()));
}
// zkHost
expression.addParameter(new StreamExpressionNamedParameter("zkHost", zkHost));
// aliases
if(null != fieldMappings && 0 != fieldMappings.size()){
StringBuilder sb = new StringBuilder();
for(Entry<String,String> mapping : fieldMappings.entrySet()){
if(sb.length() > 0){ sb.append(","); }
sb.append(mapping.getKey());
sb.append("=");
sb.append(mapping.getValue());
}
expression.addParameter(new StreamExpressionNamedParameter("aliases", sb.toString()));
}
return expression;
}
private void init(String collectionName, String zkHost, Map params) throws IOException {
this.zkHost = zkHost;
this.collection = collection;
this.collection = collectionName;
this.params = params;
String sort = (String)params.get("sort");
this.comp = parseComp(sort, params);
// If the comparator is null then it was not explicitly set so we will create one using the sort parameter
// of the query. While doing this we will also take into account any aliases such that if we are sorting on
// fieldA but fieldA is aliased to alias.fieldA then the comparater will be against alias.fieldA.
if(!params.containsKey("fl")){
throw new IOException("fl param expected for a stream");
}
if(!params.containsKey("sort")){
throw new IOException("sort param expected for a stream");
}
this.comp = parseComp((String)params.get("sort"), (String)params.get("fl"));
}
//Used by the ParallelStream
protected CloudSolrStream() {
}
public void setComp(Comparator<Tuple> comp) {
this.comp = comp;
}
public void setFieldMappings(Map<String, String> fieldMappings) {
this.fieldMappings = fieldMappings;
}
@ -129,9 +241,8 @@ public class CloudSolrStream extends TupleStream {
return solrStreams;
}
private Comparator<Tuple> parseComp(String sort, Map params) throws IOException {
private Comparator<Tuple> parseComp(String sort, String fl) throws IOException {
String fl = (String)params.get("fl");
String[] fls = fl.split(",");
HashSet fieldSet = new HashSet();
for(String f : fls) {
@ -142,17 +253,22 @@ public class CloudSolrStream extends TupleStream {
Comparator[] comps = new Comparator[sorts.length];
for(int i=0; i<sorts.length; i++) {
String s = sorts[i];
String[] spec = s.trim().split("\\s+"); //This should take into account spaces in the sort spec.
String[] spec = s.trim().split("\\s+"); //This should take into account spaces in the sort spec.
String fieldName = spec[0].trim();
String order = spec[1].trim();
if(!fieldSet.contains(spec[0])) {
throw new IOException("Fields in the sort spec must be included in the field list:"+spec[0]);
}
if(spec[1].trim().equalsIgnoreCase("asc")) {
comps[i] = new AscFieldComp(spec[0]);
} else {
comps[i] = new DescFieldComp(spec[0]);
// if there's an alias for the field then use the alias
if(null != fieldMappings && fieldMappings.containsKey(fieldName)){
fieldName = fieldMappings.get(fieldName);
}
comps[i] = new FieldComparator(fieldName, order.equalsIgnoreCase("asc") ? ComparatorOrder.ASCENDING : ComparatorOrder.DESCENDING);
}
if(comps.length > 1) {

View File

@ -0,0 +1,30 @@
package org.apache.solr.client.solrj.io.stream;
import java.io.IOException;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
/*
* 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.
*/
/**
* Defines a stream that can be expressed in an expression
*/
public interface ExpressibleStream {
StreamExpressionParameter toExpression(StreamFactory factory) throws IOException;
}

View File

@ -1,4 +1,4 @@
package org.apache.solr.client.solrj.io;
package org.apache.solr.client.solrj.io.stream;
import java.io.IOException;
import java.io.InputStream;

View File

@ -0,0 +1,153 @@
/*
* 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.Comparator;
import java.util.List;
import java.util.ArrayList;
import java.util.Locale;
import java.util.Map.Entry;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.comp.FieldComparator;
import org.apache.solr.client.solrj.io.comp.ExpressibleComparator;
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;
/**
* Unions streamA with streamB ordering the Tuples based on a Comparator.
* Both streams must be sorted by the fields being compared.
**/
public class MergeStream extends TupleStream implements ExpressibleStream {
private static final long serialVersionUID = 1;
private PushBackStream streamA;
private PushBackStream streamB;
private Comparator<Tuple> comp;
public MergeStream(TupleStream streamA, TupleStream streamB, Comparator<Tuple> comp) {
this.streamA = new PushBackStream(streamA);
this.streamB = new PushBackStream(streamB);
this.comp = comp;
}
public MergeStream(StreamExpression expression,StreamFactory factory) throws IOException {
// grab all parameters out
List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, ExpressibleStream.class, TupleStream.class);
StreamExpressionNamedParameter onExpression = factory.getNamedOperand(expression, "on");
// validate expression contains only what we want.
if(expression.getParameters().size() != streamExpressions.size() + 1){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - unknown operands found", expression));
}
if(2 != streamExpressions.size()){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting two streams but found %d (must be PushBackStream types)",expression, streamExpressions.size()));
}
this.streamA = new PushBackStream(factory.constructStream(streamExpressions.get(0)));
this.streamB = new PushBackStream(factory.constructStream(streamExpressions.get(1)));
if(null == onExpression || !(onExpression.getParameter() instanceof StreamExpressionValue)){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting single 'on' parameter listing fields to merge on but didn't find one",expression));
}
// Merge is always done over equality, so always use an EqualTo comparator
this.comp = factory.constructComparator(((StreamExpressionValue)onExpression.getParameter()).getValue(), FieldComparator.class);
}
@Override
public StreamExpression toExpression(StreamFactory factory) throws IOException {
// function name
StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
// streams
expression.addParameter(streamA.toExpression(factory));
expression.addParameter(streamB.toExpression(factory));
// on
if(comp instanceof ExpressibleComparator){
expression.addParameter(new StreamExpressionNamedParameter("on",((ExpressibleComparator)comp).toExpression(factory)));
}
else{
throw new IOException("This MergeStream contains a non-expressible comparator - it cannot be converted to an expression");
}
return expression;
}
public void setStreamContext(StreamContext context) {
this.streamA.setStreamContext(context);
this.streamB.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(streamA);
l.add(streamB);
return l;
}
public void open() throws IOException {
streamA.open();
streamB.open();
}
public void close() throws IOException {
streamA.close();
streamB.close();
}
public Tuple read() throws IOException {
Tuple a = streamA.read();
Tuple b = streamB.read();
if(a.EOF && b.EOF) {
return a;
}
if(a.EOF) {
streamA.pushBack(a);
return b;
}
if(b.EOF) {
streamB.pushBack(b);
return a;
}
int c = comp.compare(a,b);
if(c < 0) {
streamB.pushBack(b);
return a;
} else {
streamA.pushBack(a);
return b;
}
}
public int getCost() {
return 0;
}
}

View File

@ -0,0 +1,294 @@
/*
* 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.io.ObjectOutputStream;
import java.net.URLEncoder;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Locale;
import java.util.Map;
import java.util.Map.Entry;
import java.io.ByteArrayOutputStream;
import java.util.Random;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.comp.FieldComparator;
import org.apache.solr.client.solrj.io.comp.ExpressibleComparator;
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.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Base64;
/**
* The ParallelStream decorates a TupleStream implementation and pushes it to N workers for parallel execution.
* Workers are chosen from a SolrCloud collection.
* Tuples that are streamed back from the workers are ordered by a Comparator.
**/
public class ParallelStream extends CloudSolrStream implements ExpressibleStream {
private TupleStream tupleStream;
private int workers;
private boolean objectSerialize = true;
private transient StreamFactory streamFactory;
public ParallelStream(String zkHost,
String collection,
TupleStream tupleStream,
int workers,
Comparator<Tuple> comp) throws IOException {
init(zkHost,collection,tupleStream,workers,comp);
}
public ParallelStream(String zkHost,
String collection,
String expressionString,
int workers,
Comparator<Tuple> comp) throws IOException {
objectSerialize = false;
TupleStream tStream = this.streamFactory.constructStream(expressionString);
init(zkHost,collection, tStream, workers,comp);
}
public ParallelStream(StreamExpression expression, StreamFactory factory) throws IOException {
// grab all parameters out
objectSerialize = false;
String collectionName = factory.getValueOperand(expression, 0);
StreamExpressionNamedParameter workersParam = factory.getNamedOperand(expression, "workers");
List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, ExpressibleStream.class, TupleStream.class);
StreamExpressionNamedParameter sortExpression = factory.getNamedOperand(expression, "sort");
StreamExpressionNamedParameter zkHostExpression = factory.getNamedOperand(expression, "zkHost");
// validate expression contains only what we want.
if(expression.getParameters().size() != streamExpressions.size() + 3 + (null != zkHostExpression ? 1 : 0)){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - unknown operands found", expression));
}
// Collection Name
if(null == collectionName){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - collectionName expected as first operand",expression));
}
// Workers
if(null == workersParam || null == workersParam.getParameter() || !(workersParam.getParameter() instanceof StreamExpressionValue)){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single 'workersParam' parameter of type positive integer but didn't find one",expression));
}
String workersStr = ((StreamExpressionValue)workersParam.getParameter()).getValue();
int workersInt = 0;
try{
workersInt = Integer.parseInt(workersStr);
if(workersInt <= 0){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - workers '%s' must be greater than 0.",expression, workersStr));
}
}
catch(NumberFormatException e){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - workers '%s' is not a valid integer.",expression, workersStr));
}
// Stream
if(1 != streamExpressions.size()){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single stream but found %d",expression, streamExpressions.size()));
}
// Sort
if(null == sortExpression || !(sortExpression.getParameter() instanceof StreamExpressionValue)){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting single 'sort' parameter telling us how to join the parallel streams but didn't find one",expression));
}
// zkHost, optional - if not provided then will look into factory list to get
String zkHost = null;
if(null == zkHostExpression){
zkHost = factory.getCollectionZkHost(collectionName);
}
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
TupleStream stream = factory.constructStream(streamExpressions.get(0));
Comparator<Tuple> comp = factory.constructComparator(((StreamExpressionValue)sortExpression.getParameter()).getValue(), FieldComparator.class);
streamFactory = factory;
init(zkHost,collectionName,stream,workersInt,comp);
}
private void init(String zkHost,String collection,TupleStream tupleStream,int workers,Comparator<Tuple> comp) throws IOException{
this.zkHost = zkHost;
this.collection = collection;
this.workers = workers;
this.comp = comp;
this.tupleStream = tupleStream;
// requires Expressible stream and comparator
if(!objectSerialize && !(tupleStream instanceof ExpressibleStream)){
throw new IOException("Unable to create ParallelStream with a non-expressible TupleStream.");
}
}
@Override
public StreamExpression toExpression(StreamFactory factory) throws IOException {
// function name
StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
// collection
expression.addParameter(collection);
// workers
expression.addParameter(new StreamExpressionNamedParameter("workers", Integer.toString(workers)));
// stream
if(tupleStream instanceof ExpressibleStream){
expression.addParameter(((ExpressibleStream)tupleStream).toExpression(factory));
}
else{
throw new IOException("This ParallelStream contains a non-expressible TupleStream - it cannot be converted to an expression");
}
// sort
if(comp instanceof ExpressibleComparator){
expression.addParameter(new StreamExpressionNamedParameter("sort",((ExpressibleComparator)comp).toExpression(factory)));
}
else{
throw new IOException("This ParallelStream contains a non-expressible comparator - it cannot be converted to an expression");
}
// zkHost
expression.addParameter(new StreamExpressionNamedParameter("zkHost", zkHost));
return expression;
}
public List<TupleStream> children() {
List l = new ArrayList();
l.add(tupleStream);
return l;
}
public Tuple read() throws IOException {
Tuple tuple = _read();
if(tuple.EOF) {
Map m = new HashMap();
m.put("EOF", true);
Tuple t = new Tuple(m);
Map<String, Map> metrics = new HashMap();
Iterator<Entry<String,Tuple>> it = this.eofTuples.entrySet().iterator();
while(it.hasNext()) {
Map.Entry<String, Tuple> entry = it.next();
metrics.put(entry.getKey(), entry.getValue().fields);
}
t.setMetrics(metrics);
return t;
}
return tuple;
}
public void setStreamContext(StreamContext streamContext) {
this.streamContext = streamContext;
if(streamFactory == null) {
this.streamFactory = streamContext.getStreamFactory();
}
this.tupleStream.setStreamContext(streamContext);
}
protected void constructStreams() throws IOException {
try {
Object pushStream = null;
if (objectSerialize) {
ByteArrayOutputStream bout = new ByteArrayOutputStream();
ObjectOutputStream out = new ObjectOutputStream(bout);
out.writeObject(tupleStream);
byte[] bytes = bout.toByteArray();
String encoded = Base64.byteArrayToBase64(bytes, 0, bytes.length);
pushStream = URLEncoder.encode(encoded, "UTF-8");
} else {
pushStream = ((ExpressibleStream) tupleStream).toExpression(streamFactory);
}
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
ClusterState clusterState = zkStateReader.getClusterState();
Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
long time = System.currentTimeMillis();
List<Replica> shuffler = new ArrayList();
for(Slice slice : slices) {
Collection<Replica> replicas = slice.getReplicas();
for (Replica replica : replicas) {
shuffler.add(replica);
}
}
if(workers > shuffler.size()) {
throw new IOException("Number of workers exceeds nodes in the worker collection");
}
Collections.shuffle(shuffler, new Random(time));
for(int w=0; w<workers; w++) {
HashMap params = new HashMap();
params.put("distrib","false"); // We are the aggregator.
params.put("numWorkers", workers);
params.put("workerID", w);
params.put("stream", pushStream);
params.put("qt","/stream");
params.put("objectSerialize", objectSerialize);
Replica rep = shuffler.get(w);
ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep);
String url = zkProps.getCoreUrl();
SolrStream solrStream = new SolrStream(url, params);
solrStreams.add(solrStream);
}
assert(solrStreams.size() == workers);
} catch (Exception e) {
throw new IOException(e);
}
}
public void setObjectSerialize(boolean objectSerialize) {
this.objectSerialize = objectSerialize;
}
public boolean getObjectSerialize() {
return objectSerialize;
}
}

View File

@ -15,12 +15,16 @@
* limitations under the License.
*/
package org.apache.solr.client.solrj.io;
package org.apache.solr.client.solrj.io.stream;
import java.io.IOException;
import java.util.Comparator;
import java.util.List;
import java.util.ArrayList;
import java.util.List;
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.StreamExpressionParameter;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
/**
* A TupleStream that allows a single Tuple to be pushed back onto the stream after it's been read.
@ -37,7 +41,14 @@ public class PushBackStream extends TupleStream {
public PushBackStream(TupleStream stream) {
this.stream = stream;
}
public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException{
if(stream instanceof ExpressibleStream){
return ((ExpressibleStream)stream).toExpression(factory);
}
throw new IOException("This PushBackStream contains a non-expressible TupleStream - it cannot be converted to an expression");
}
public void setStreamContext(StreamContext context) {

View File

@ -0,0 +1,198 @@
/*
* 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.io.Serializable;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.LinkedList;
import java.util.List;
import java.util.Locale;
import java.util.PriorityQueue;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.comp.FieldComparator;
import org.apache.solr.client.solrj.io.comp.ExpressibleComparator;
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;
/**
* Iterates over a TupleStream and Ranks the topN tuples based on a Comparator.
**/
public class RankStream extends TupleStream implements ExpressibleStream {
private static final long serialVersionUID = 1;
private TupleStream tupleStream;
private Comparator<Tuple> comp;
private int size;
private transient PriorityQueue<Tuple> top;
private transient boolean finished = false;
private transient LinkedList<Tuple> topList;
public RankStream(TupleStream tupleStream, int size, Comparator<Tuple> comp) {
init(tupleStream,size,comp);
}
public RankStream(StreamExpression expression, StreamFactory factory) throws IOException {
// grab all parameters out
List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, ExpressibleStream.class, TupleStream.class);
StreamExpressionNamedParameter nParam = factory.getNamedOperand(expression, "n");
StreamExpressionNamedParameter sortExpression = factory.getNamedOperand(expression, "sort");
// validate expression contains only what we want.
if(expression.getParameters().size() != streamExpressions.size() + 2){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - unknown operands found", expression));
}
if(null == nParam || null == nParam.getParameter() || !(nParam.getParameter() instanceof StreamExpressionValue)){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single 'n' parameter of type positive integer but didn't find one",expression));
}
String nStr = ((StreamExpressionValue)nParam.getParameter()).getValue();
int nInt = 0;
try{
nInt = Integer.parseInt(nStr);
if(nInt <= 0){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - topN '%s' must be greater than 0.",expression, nStr));
}
}
catch(NumberFormatException e){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - topN '%s' is not a valid integer.",expression, nStr));
}
if(1 != streamExpressions.size()){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single stream but found %d",expression, streamExpressions.size()));
}
if(null == sortExpression || !(sortExpression.getParameter() instanceof StreamExpressionValue)){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting single 'over' parameter listing fields to unique over but didn't find one",expression));
}
TupleStream stream = factory.constructStream(streamExpressions.get(0));
Comparator<Tuple> comp = factory.constructComparator(((StreamExpressionValue)sortExpression.getParameter()).getValue(), FieldComparator.class);
init(stream,nInt,comp);
}
private void init(TupleStream tupleStream, int size, Comparator<Tuple> comp){
this.tupleStream = tupleStream;
this.comp = comp;
this.size = size;
}
@Override
public StreamExpression toExpression(StreamFactory factory) throws IOException {
// function name
StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
// n
expression.addParameter(new StreamExpressionNamedParameter("n", Integer.toString(size)));
// stream
if(tupleStream instanceof ExpressibleStream){
expression.addParameter(((ExpressibleStream)tupleStream).toExpression(factory));
}
else{
throw new IOException("This RankStream contains a non-expressible TupleStream - it cannot be converted to an expression");
}
// sort
if(comp instanceof ExpressibleComparator){
expression.addParameter(new StreamExpressionNamedParameter("sort",((ExpressibleComparator)comp).toExpression(factory)));
}
else{
throw new IOException("This RankStream contains a non-expressible comparator - it cannot be converted to an expression");
}
return expression;
}
public void setStreamContext(StreamContext context) {
this.tupleStream.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(tupleStream);
return l;
}
public void open() throws IOException {
this.top = new PriorityQueue(size, new ReverseComp(comp));
this.topList = new LinkedList();
tupleStream.open();
}
public void close() throws IOException {
tupleStream.close();
}
public Comparator<Tuple> getComparator(){
return this.comp;
}
public Tuple read() throws IOException {
if(!finished) {
while(true) {
Tuple tuple = tupleStream.read();
if(tuple.EOF) {
finished = true;
int s = top.size();
for(int i=0; i<s; i++) {
Tuple t = top.poll();
topList.addFirst(t);
}
topList.addLast(tuple);
break;
} else {
Tuple peek = top.peek();
if(top.size() >= size) {
if(comp.compare(tuple, peek) < 0) {
top.poll();
top.add(tuple);
}
} else {
top.add(tuple);
}
}
}
}
return topList.pollFirst();
}
public int getCost() {
return 0;
}
class ReverseComp implements Comparator<Tuple>, Serializable {
private Comparator<Tuple> comp;
public ReverseComp(Comparator<Tuple> comp) {
this.comp = comp;
}
public int compare(Tuple t1, Tuple t2) {
return comp.compare(t1, t2)*(-1);
}
}
}

View File

@ -15,15 +15,24 @@
* limitations under the License.
*/
package org.apache.solr.client.solrj.io;
package org.apache.solr.client.solrj.io.stream;
import java.io.IOException;
import java.util.List;
import java.util.ArrayList;
import java.util.Comparator;
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.FieldComparator;
import org.apache.solr.client.solrj.io.comp.ExpressibleComparator;
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;
/**
* Iterates over a TupleStream and buffers Tuples that are equal based on a comparator.
* This allows tuples to be grouped by common field(s).
@ -39,21 +48,63 @@ import java.util.HashMap;
*
**/
public class ReducerStream extends TupleStream {
public class ReducerStream extends TupleStream implements ExpressibleStream {
private static final long serialVersionUID = 1;
private PushBackStream tupleStream;
private Comparator<Tuple> comp;
private Tuple currentGroupHead;
private transient Tuple currentGroupHead;
public ReducerStream(TupleStream tupleStream,
Comparator<Tuple> comp) {
this.tupleStream = new PushBackStream(tupleStream);
this.comp = comp;
}
public ReducerStream(StreamExpression expression, StreamFactory factory) throws IOException{
// grab all parameters out
List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, ExpressibleStream.class, TupleStream.class);
StreamExpressionNamedParameter byExpression = factory.getNamedOperand(expression, "by");
// validate expression contains only what we want.
if(expression.getParameters().size() != streamExpressions.size() + 1){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - unknown operands found", expression));
}
if(1 != streamExpressions.size()){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single stream but found %d",expression, streamExpressions.size()));
}
this.tupleStream = new PushBackStream(factory.constructStream(streamExpressions.get(0)));
if(null == byExpression || !(byExpression.getParameter() instanceof StreamExpressionValue)){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting single 'by' parameter listing fields to group by but didn't find one",expression));
}
// Reducing is always done over equality, so always use an EqualTo comparator
this.comp = factory.constructComparator(((StreamExpressionValue)byExpression.getParameter()).getValue(), FieldComparator.class);
}
@Override
public StreamExpression toExpression(StreamFactory factory) throws IOException {
// function name
StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
// stream
expression.addParameter(tupleStream.toExpression(factory));
// over
if(comp instanceof ExpressibleComparator){
expression.addParameter(new StreamExpressionNamedParameter("by",((ExpressibleComparator)comp).toExpression(factory)));
}
else{
throw new IOException("This ReducerStream contains a non-expressible comparator - it cannot be converted to an expression");
}
return expression;
}
public void setStreamContext(StreamContext context) {
this.tupleStream.setStreamContext(context);
}

View File

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.solr.client.solrj.io;
package org.apache.solr.client.solrj.io.stream;
import java.io.IOException;
import java.util.ArrayList;
@ -25,6 +25,8 @@ import java.util.List;
import java.util.Iterator;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.io.SolrClientCache;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;

View File

@ -15,13 +15,16 @@
* limitations under the License.
*/
package org.apache.solr.client.solrj.io;
package org.apache.solr.client.solrj.io.stream;
import java.io.Serializable;
import java.util.Map;
import java.util.HashMap;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.solr.client.solrj.io.SolrClientCache;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
/**
* The StreamContext is passed to TupleStreams using the TupleStream.setStreamContext() method.
* The StreamContext is used to pass shared context to concentrically wrapped TupleStreams.
@ -37,6 +40,7 @@ public class StreamContext implements Serializable{
public int workerID;
public int numWorkers;
private SolrClientCache clientCache;
private StreamFactory streamFactory;
public Object get(Object key) {
return entries.get(key);
@ -53,4 +57,12 @@ public class StreamContext implements Serializable{
public SolrClientCache getSolrClientCache() {
return this.clientCache;
}
public void setStreamFactory(StreamFactory streamFactory) {
this.streamFactory = streamFactory;
}
public StreamFactory getStreamFactory() {
return this.streamFactory;
}
}

View File

@ -15,12 +15,16 @@
* limitations under the License.
*/
package org.apache.solr.client.solrj.io;
package org.apache.solr.client.solrj.io.stream;
import java.io.IOException;
import java.io.Serializable;
import java.util.List;
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.StreamFactory;
public abstract class TupleStream implements Serializable {
private static final long serialVersionUID = 1;
@ -28,7 +32,7 @@ public abstract class TupleStream implements Serializable {
public TupleStream() {
}
public abstract void setStreamContext(StreamContext context);
public abstract List<TupleStream> children();

View File

@ -0,0 +1,154 @@
/*
* 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.Comparator;
import java.util.List;
import java.util.Locale;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.comp.FieldComparator;
import org.apache.solr.client.solrj.io.comp.ExpressibleComparator;
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;
/**
* The UniqueStream emits a unique stream of Tuples based on a Comparator.
*
* Note: The sort order of the underlying stream must match the Comparator.
**/
public class UniqueStream extends TupleStream implements ExpressibleStream {
private static final long serialVersionUID = 1;
private TupleStream tupleStream;
private Comparator<Tuple> comp;
private transient Tuple currentTuple;
public UniqueStream(TupleStream tupleStream, Comparator<Tuple> comp) {
this.tupleStream = tupleStream;
this.comp = comp;
}
public UniqueStream(StreamExpression expression,StreamFactory factory) throws IOException {
// grab all parameters out
List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, ExpressibleStream.class, TupleStream.class);
StreamExpressionNamedParameter overExpression = factory.getNamedOperand(expression, "over");
// validate expression contains only what we want.
if(expression.getParameters().size() != streamExpressions.size() + 1){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - unknown operands found", expression));
}
if(1 != streamExpressions.size()){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single stream but found %d",expression, streamExpressions.size()));
}
this.tupleStream = factory.constructStream(streamExpressions.get(0));
if(null == overExpression || !(overExpression.getParameter() instanceof StreamExpressionValue)){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting single 'over' parameter listing fields to unique over but didn't find one",expression));
}
// Uniqueness is always done over equality, so always use an EqualTo comparator
this.comp = factory.constructComparator(((StreamExpressionValue)overExpression.getParameter()).getValue(), FieldComparator.class);
}
@Override
public StreamExpression toExpression(StreamFactory factory) throws IOException {
// function name
StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
// streams
if(tupleStream instanceof ExpressibleStream){
expression.addParameter(((ExpressibleStream)tupleStream).toExpression(factory));
}
else{
throw new IOException("This UniqueStream contains a non-expressible TupleStream - it cannot be converted to an expression");
}
// over
if(comp instanceof ExpressibleComparator){
expression.addParameter(new StreamExpressionNamedParameter("over",((ExpressibleComparator)comp).toExpression(factory)));
}
else{
throw new IOException("This UniqueStream contains a non-expressible comparator - it cannot be converted to an expression");
}
return expression;
}
public void setComp(Comparator<Tuple> comp) {
this.comp = comp;
}
public void setStreamContext(StreamContext context) {
this.tupleStream.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList<TupleStream>();
l.add(tupleStream);
return l;
}
public void open() throws IOException {
tupleStream.open();
}
public void close() throws IOException {
tupleStream.close();
}
public Tuple read() throws IOException {
Tuple tuple = tupleStream.read();
if(tuple.EOF) {
return tuple;
}
if(currentTuple == null) {
currentTuple = tuple;
return tuple;
} else {
while(true) {
int i = comp.compare(currentTuple, tuple);
if(i == 0) {
//We have duplicate tuple so read the next tuple from the stream.
tuple = tupleStream.read();
if(tuple.EOF) {
return tuple;
}
} else {
//We have a non duplicate
this.currentTuple = tuple;
return tuple;
}
}
}
}
public int getCost() {
return 0;
}
}

View File

@ -0,0 +1,127 @@
package org.apache.solr.client.solrj.io.stream.expr;
import java.util.ArrayList;
import java.util.List;
/*
* 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.
*/
/**
* Expression containing a function and set of parameters
*/
public class StreamExpression implements StreamExpressionParameter {
private String functionName;
private List<StreamExpressionParameter> parameters;
public StreamExpression(String functionName){
this.functionName = functionName;
parameters = new ArrayList<StreamExpressionParameter>();
}
public String getFunctionName(){
return this.functionName;
}
public void setFunctionName(String functionName){
if(null == functionName){
throw new IllegalArgumentException("Null functionName is not allowed.");
}
this.functionName = functionName;
}
public StreamExpression withFunctionName(String functionName){
setFunctionName(functionName);
return this;
}
public void addParameter(StreamExpressionParameter parameter){
this.parameters.add(parameter);
}
public void addParameter(String parameter){
addParameter(new StreamExpressionValue(parameter));
}
public StreamExpression withParameter(StreamExpressionParameter parameter){
this.parameters.add(parameter);
return this;
}
public StreamExpression withParameter(String parameter){
return withParameter(new StreamExpressionValue(parameter));
}
public List<StreamExpressionParameter> getParameters(){
return this.parameters;
}
public void setParameters(List<StreamExpressionParameter> parameters){
if(null == parameters){
throw new IllegalArgumentException("Null parameter list is not allowed.");
}
this.parameters = parameters;
}
public StreamExpression withParameters(List<StreamExpressionParameter> parameters){
setParameters(parameters);
return this;
}
@Override
public String toString(){
StringBuilder sb = new StringBuilder(this.functionName);
sb.append("(");
for(int idx = 0; idx < parameters.size(); ++idx){
if(0 != idx){ sb.append(","); }
sb.append(parameters.get(idx));
}
sb.append(")");
return sb.toString();
}
@Override
public boolean equals(Object other){
if(other.getClass() != StreamExpression.class){
return false;
}
StreamExpression check = (StreamExpression)other;
if(null == this.functionName && null != check.functionName){
return false;
}
if(null != this.functionName && null == check.functionName){
return false;
}
if(null != this.functionName && null != check.functionName && !this.functionName.equals(check.functionName)){
return false;
}
if(this.parameters.size() != check.parameters.size()){
return false;
}
for(int idx = 0; idx < this.parameters.size(); ++idx){
StreamExpressionParameter left = this.parameters.get(idx);
StreamExpressionParameter right = check.parameters.get(idx);
if(!left.equals(right)){
return false;
}
}
return true;
}
}

View File

@ -0,0 +1,111 @@
package org.apache.solr.client.solrj.io.stream.expr;
import java.util.ArrayList;
import java.util.List;
/*
* 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.
*/
/**
* Provides a named parameter
*/
public class StreamExpressionNamedParameter implements StreamExpressionParameter {
private String name;
private StreamExpressionParameter parameter;
public StreamExpressionNamedParameter(String name){
this.name = name;
}
public StreamExpressionNamedParameter(String name, String parameter){
this.name = name;
setParameter(parameter);
}
public StreamExpressionNamedParameter(String name, StreamExpressionParameter parameter){
this.name = name;
setParameter(parameter);
}
public String getName(){
return this.name;
}
public void setName(String name){
if(null == name || 0 == name.length()){
throw new IllegalArgumentException("Null or empty name is not allowed is not allowed.");
}
this.name = name;
}
public StreamExpressionParameter getParameter(){
return this.parameter;
}
public void setParameter(StreamExpressionParameter parameter){
this.parameter = parameter;
}
public StreamExpressionNamedParameter withParameter(StreamExpressionParameter parameter){
setParameter(parameter);
return this;
}
public void setParameter(String parameter){
this.parameter = new StreamExpressionValue(parameter);
}
public StreamExpressionNamedParameter withParameter(String parameter){
setParameter(parameter);
return this;
}
@Override
public String toString(){
StringBuilder sb = new StringBuilder(name);
sb.append("=");
// check if we require quoting
boolean requiresQuote = false;
if(parameter instanceof StreamExpressionValue){
String value = ((StreamExpressionValue)parameter).getValue();
requiresQuote = !StreamExpressionParser.wordToken(value);
}
if(requiresQuote){ sb.append("\""); }
sb.append(parameter.toString());
if(requiresQuote){ sb.append("\""); }
return sb.toString();
}
@Override
public boolean equals(Object other){
if(other.getClass() != StreamExpressionNamedParameter.class){
return false;
}
StreamExpressionNamedParameter check = (StreamExpressionNamedParameter)other;
if(null == this.name && null != check.name){
return false;
}
if(null != this.name && null == check.name){
return false;
}
if(null != this.name && null != check.name && !this.name.equals(check.name)){
return false;
}
return this.parameter.equals(check.parameter);
}
}

View File

@ -0,0 +1,25 @@
package org.apache.solr.client.solrj.io.stream.expr;
/*
* 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.
*/
/**
* Base interface of a stream parameter
*/
public interface StreamExpressionParameter {
}

View File

@ -0,0 +1,313 @@
package org.apache.solr.client.solrj.io.stream.expr;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
/*
* 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.
*/
/**
* Takes a prefix notation expression and returns a tokenized expression
*/
public class StreamExpressionParser {
static char[] wordChars = {'_','.','-'};
static {
Arrays.sort(wordChars);
}
public static StreamExpression parse(String clause){
StreamExpressionParameter expr = generateStreamExpression(clause);
if(null != expr && expr instanceof StreamExpression){
return (StreamExpression)expr;
}
return null;
}
private static StreamExpressionParameter generateStreamExpression(String clause){
String working = clause.trim();
if(!isExpressionClause(working)){
throw new IllegalArgumentException(String.format(Locale.ROOT,"'%s' is not a proper expression clause", working));
}
// Get functionName
int firstOpenParen = findNextClear(working, 0, '(');
StreamExpression expression = new StreamExpression(working.substring(0, firstOpenParen).trim());
// strip off functionName and ()
working = working.substring(firstOpenParen + 1,working.length() - 1).trim();
List<String> parts = splitOn(working,',');
for(int idx = 0; idx < parts.size(); ++idx){
String part = parts.get(idx).trim();
if(isExpressionClause(part)){
StreamExpressionParameter parameter = generateStreamExpression(part);
if(null != parameter){
expression.addParameter(parameter);
}
}
else if(isNamedParameterClause(part)){
StreamExpressionNamedParameter parameter = generateNamedParameterExpression(part);
if(null != parameter){
expression.addParameter(parameter);
}
}
else{
expression.addParameter(new StreamExpressionValue(part));
}
}
return expression;
}
private static StreamExpressionNamedParameter generateNamedParameterExpression(String clause){
String working = clause.trim();
// might be overkill as the only place this is called from does this check already
if(!isNamedParameterClause(working)){
throw new IllegalArgumentException(String.format(Locale.ROOT,"'%s' is not a proper named parameter clause", working));
}
// Get name
int firstOpenEquals = findNextClear(working, 0, '=');
StreamExpressionNamedParameter namedParameter = new StreamExpressionNamedParameter(working.substring(0, firstOpenEquals).trim());
// we know this is ok because of the check in isNamedParameter
String parameter = working.substring(firstOpenEquals + 1, working.length());
if(isExpressionClause(parameter)){
namedParameter.setParameter(generateStreamExpression(parameter));
}
else{
// if wrapped in quotes, remove them
if(parameter.startsWith("\"") && parameter.endsWith("\"")){
parameter = parameter.substring(1, parameter.length() - 1).trim();
if(0 == parameter.length()){
throw new IllegalArgumentException(String.format(Locale.ROOT,"'%s' is not a proper named parameter clause", working));
}
}
namedParameter.setParameter(new StreamExpressionValue(parameter));
}
return namedParameter;
}
/* Returns true if the clause is a valid expression clause. This is defined to
* mean it begins with ( and ends with )
* Expects that the passed in clause has already been trimmed of leading and
* trailing spaces*/
private static boolean isExpressionClause(String clause){
// operator(.....something.....)
// must be balanced
if(!isBalanced(clause)){ return false; }
// find first (, then check from start to that location and only accept alphanumeric
int firstOpenParen = findNextClear(clause, 0, '(');
if(firstOpenParen <= 0 || firstOpenParen == clause.length() - 1){ return false; }
String functionName = clause.substring(0, firstOpenParen).trim();
if(!wordToken(functionName)){ return false; }
// Must end with )
return clause.endsWith(")");
}
private static boolean isNamedParameterClause(String clause){
// name=thing
// find first = then check from start to that location and only accept alphanumeric
int firstOpenEquals = findNextClear(clause, 0, '=');
if(firstOpenEquals <= 0 || firstOpenEquals == clause.length() - 1){ return false; }
String name = clause.substring(0, firstOpenEquals);
if(!wordToken(name)){ return false; }
return true;
}
/* Finds index of the next char equal to findThis that is not within a quote or set of parens
* Does not work with the following values of findThis: " ' \ ) -- well, it might but wouldn't
* really give you what you want. Don't call with those characters */
private static int findNextClear(String clause, int startingIdx, char findThis){
int openParens = 0;
boolean isDoubleQuote = false;
boolean isSingleQuote = false;
boolean isEscaped = false;
for(int idx = startingIdx; idx < clause.length(); ++idx){
char c = clause.charAt(idx);
// if we're not in a non-escaped quote or paren state, then we've found the space we want
if(c == findThis && !isEscaped && !isSingleQuote && !isDoubleQuote && 0 == openParens){
return idx;
}
switch(c){
case '\\':
// We invert to support situations where \\ exists
isEscaped = !isEscaped;
break;
case '"':
// if we're not in a non-escaped single quote state, then invert the double quote state
if(!isEscaped && !isSingleQuote){
isDoubleQuote = !isDoubleQuote;
}
isEscaped = false;
break;
case '\'':
// if we're not in a non-escaped double quote state, then invert the single quote state
if(!isEscaped && !isDoubleQuote){
isSingleQuote = !isSingleQuote;
}
isEscaped = false;
break;
case '(':
// if we're not in a non-escaped quote state, then increment the # of open parens
if(!isEscaped && !isSingleQuote && !isDoubleQuote){
openParens += 1;
}
isEscaped = false;
break;
case ')':
// if we're not in a non-escaped quote state, then decrement the # of open parens
if(!isEscaped && !isSingleQuote && !isDoubleQuote){
openParens -= 1;
}
isEscaped = false;
break;
default:
isEscaped = false;
}
}
// Not found
return -1;
}
/* Returns a list of the tokens found. Assumed to be of the form
* 'foo bar baz' and not of the for '(foo bar baz)'
* 'foo bar (baz jaz)' is ok and will return three tokens of
* 'foo', 'bar', and '(baz jaz)'
*/
private static List<String> splitOn(String clause, char splitOnThis){
String working = clause.trim();
List<String> parts = new ArrayList<String>();
while(true){ // will break when next splitOnThis isn't found
int nextIdx = findNextClear(working, 0, splitOnThis);
if(nextIdx < 0){
parts.add(working);
break;
}
parts.add(working.substring(0, nextIdx));
// handle ending splitOnThis
if(nextIdx+1 == working.length()){
break;
}
working = working.substring(nextIdx + 1).trim();
}
return parts;
}
/* Returns true if the clause has balanced parenthesis */
private static boolean isBalanced(String clause){
int openParens = 0;
boolean isDoubleQuote = false;
boolean isSingleQuote = false;
boolean isEscaped = false;
for(int idx = 0; idx < clause.length(); ++idx){
char c = clause.charAt(idx);
switch(c){
case '\\':
// We invert to support situations where \\ exists
isEscaped = !isEscaped;
break;
case '"':
// if we're not in a non-escaped single quote state, then invert the double quote state
if(!isEscaped && !isSingleQuote){
isDoubleQuote = !isDoubleQuote;
}
isEscaped = false;
break;
case '\'':
// if we're not in a non-escaped double quote state, then invert the single quote state
if(!isEscaped && !isDoubleQuote){
isSingleQuote = !isSingleQuote;
}
isEscaped = false;
break;
case '(':
// if we're not in a non-escaped quote state, then increment the # of open parens
if(!isEscaped && !isSingleQuote && !isDoubleQuote){
openParens += 1;
}
isEscaped = false;
break;
case ')':
// if we're not in a non-escaped quote state, then decrement the # of open parens
if(!isEscaped && !isSingleQuote && !isDoubleQuote){
openParens -= 1;
// If we're ever < 0 then we know we're not balanced
if(openParens < 0){
return false;
}
}
isEscaped = false;
break;
default:
isEscaped = false;
}
}
return (0 == openParens);
}
public static boolean wordToken(String token) {
for(int i=0; i<token.length(); i++) {
char c = token.charAt(i);
if (!Character.isLetterOrDigit(c) && Arrays.binarySearch(wordChars, c) < 0) {
return false;
}
}
return true;
}
}

View File

@ -0,0 +1,66 @@
package org.apache.solr.client.solrj.io.stream.expr;
/*
* 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.
*/
/**
* Basic string stream expression
*/
public class StreamExpressionValue implements StreamExpressionParameter {
private String value;
public StreamExpressionValue(String value){
this.value = value;
}
public String getValue(){
return this.value;
}
public void setValue(String value){
this.value = value;
}
public StreamExpressionValue withValue(String value){
this.value = value;
return this;
}
@Override
public String toString(){
return this.value;
}
@Override
public boolean equals(Object other){
if(other.getClass() != StreamExpressionValue.class){
return false;
}
StreamExpressionValue check = (StreamExpressionValue)other;
if(null == this.value && null == check.value){
return true;
}
if(null == this.value || null == check.value){
return false;
}
return this.value.equals(((StreamExpressionValue)other).value);
}
}

View File

@ -0,0 +1,223 @@
package org.apache.solr.client.solrj.io.stream.expr;
import java.io.IOException;
import java.io.Serializable;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.comp.ComparatorOrder;
import org.apache.solr.client.solrj.io.comp.ExpressibleComparator;
import org.apache.solr.client.solrj.io.comp.MultiComp;
import org.apache.solr.client.solrj.io.stream.ExpressibleStream;
import org.apache.solr.client.solrj.io.stream.TupleStream;
/*
* 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.
*/
/**
* Used to convert strings into stream expressions
*/
public class StreamFactory implements Serializable {
private transient HashMap<String,String> collectionZkHosts;
private transient HashMap<String,Class> streamFunctions;
public StreamFactory(){
collectionZkHosts = new HashMap<String,String>();
streamFunctions = new HashMap<String,Class>();
}
public StreamFactory withCollectionZkHost(String collectionName, String zkHost){
this.collectionZkHosts.put(collectionName, zkHost);
return this;
}
public String getCollectionZkHost(String collectionName){
return this.collectionZkHosts.getOrDefault(collectionName, null);
}
public Map<String,Class> getStreamFunctions(){
return streamFunctions;
}
public StreamFactory withStreamFunction(String streamFunction, Class clazz){
this.streamFunctions.put(streamFunction, clazz);
return this;
}
public StreamExpressionParameter getOperand(StreamExpression expression, int parameterIndex){
if(null == expression.getParameters() || parameterIndex >= expression.getParameters().size()){
return null;
}
return expression.getParameters().get(parameterIndex);
}
/** Given an expression, will return the value parameter at the given index, or null if doesn't exist */
public String getValueOperand(StreamExpression expression, int parameterIndex){
StreamExpressionParameter parameter = getOperand(expression, parameterIndex);
if(null != parameter){
if(parameter instanceof StreamExpressionValue){
return ((StreamExpressionValue)parameter).getValue();
}
}
return null;
}
public List<StreamExpressionNamedParameter> getNamedOperands(StreamExpression expression){
List<StreamExpressionNamedParameter> namedParameters = new ArrayList<StreamExpressionNamedParameter>();
for(StreamExpressionParameter parameter : getOperandsOfType(expression, StreamExpressionNamedParameter.class)){
namedParameters.add((StreamExpressionNamedParameter)parameter);
}
return namedParameters;
}
public StreamExpressionNamedParameter getNamedOperand(StreamExpression expression, String name){
List<StreamExpressionNamedParameter> namedParameters = getNamedOperands(expression);
for(StreamExpressionNamedParameter param : namedParameters){
if(param.getName().equals(name)){
return param;
}
}
return null;
}
public List<StreamExpression> getExpressionOperands(StreamExpression expression){
List<StreamExpression> namedParameters = new ArrayList<StreamExpression>();
for(StreamExpressionParameter parameter : getOperandsOfType(expression, StreamExpression.class)){
namedParameters.add((StreamExpression)parameter);
}
return namedParameters;
}
public List<StreamExpression> getExpressionOperands(StreamExpression expression, String functionName){
List<StreamExpression> namedParameters = new ArrayList<StreamExpression>();
for(StreamExpressionParameter parameter : getOperandsOfType(expression, StreamExpression.class)){
StreamExpression expressionOperand = (StreamExpression)parameter;
if(expressionOperand.getFunctionName().equals(functionName)){
namedParameters.add(expressionOperand);
}
}
return namedParameters;
}
public List<StreamExpressionParameter> getOperandsOfType(StreamExpression expression, Class ... clazzes){
List<StreamExpressionParameter> parameters = new ArrayList<StreamExpressionParameter>();
parameterLoop:
for(StreamExpressionParameter parameter : expression.getParameters()){
for(Class clazz : clazzes){
if(!clazz.isAssignableFrom(parameter.getClass())){
continue parameterLoop; // go to the next parameter since this parameter cannot be assigned to at least one of the classes
}
}
parameters.add(parameter);
}
return parameters;
}
public List<StreamExpression> getExpressionOperandsRepresentingTypes(StreamExpression expression, Class ... clazzes){
List<StreamExpression> matchingStreamExpressions = new ArrayList<StreamExpression>();
List<StreamExpression> allStreamExpressions = getExpressionOperands(expression);
parameterLoop:
for(StreamExpression streamExpression : allStreamExpressions){
if(streamFunctions.containsKey(streamExpression.getFunctionName())){
for(Class clazz : clazzes){
if(!clazz.isAssignableFrom(streamFunctions.get(streamExpression.getFunctionName()))){
continue parameterLoop;
}
}
matchingStreamExpressions.add(streamExpression);
}
}
return matchingStreamExpressions;
}
public TupleStream constructStream(String expressionClause) throws IOException {
return constructStream(StreamExpressionParser.parse(expressionClause));
}
public TupleStream constructStream(StreamExpression expression) throws IOException{
String function = expression.getFunctionName();
if(streamFunctions.containsKey(function)){
Class clazz = streamFunctions.get(function);
if(ExpressibleStream.class.isAssignableFrom(clazz) && TupleStream.class.isAssignableFrom(clazz)){
TupleStream stream = (TupleStream)createInstance(streamFunctions.get(function), new Class[]{ StreamExpression.class, StreamFactory.class }, new Object[]{ expression, this});
return stream;
}
}
throw new IOException(String.format(Locale.ROOT,"Invalid stream expression %s - function '%s' is unknown (not mapped to a valid TupleStream)", expression, expression.getFunctionName()));
}
public Comparator<Tuple> constructComparator(String comparatorString, Class comparatorType) throws IOException {
if(comparatorString.contains(",")){
String[] parts = comparatorString.split(",");
Comparator[] comps = new Comparator[parts.length];
for(int idx = 0; idx < parts.length; ++idx){
comps[idx] = constructComparator(parts[idx].trim(), comparatorType);
}
return new MultiComp(comps);
}
else{
String[] parts = comparatorString.split(" ");
if(2 != parts.length){
throw new IOException(String.format(Locale.ROOT,"Invalid comparator expression %s - expecting fieldName and order",comparatorString));
}
String fieldName = parts[0].trim();
String order = parts[1].trim();
return (Comparator)createInstance(comparatorType, new Class[]{ String.class, ComparatorOrder.class }, new Object[]{ fieldName, ComparatorOrder.fromString(order) });
}
}
public <T> T createInstance(Class<T> clazz, Class<?>[] paramTypes, Object[] params) throws IOException{
// This should use SolrResourceLoader - TODO
// This is adding a restriction that the class has a public constructor - we may not want to do that
Constructor<T> ctor;
try {
ctor = clazz.getConstructor(paramTypes);
return ctor.newInstance(params);
} catch (NoSuchMethodException | SecurityException | InstantiationException | IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
throw new IOException(String.format(Locale.ROOT,"Unable to construct instance of %s", clazz.getName()),e);
}
}
public String getFunctionName(Class clazz) throws IOException{
for(Entry<String,Class> entry : streamFunctions.entrySet()){
if(entry.getValue() == clazz){
return entry.getKey();
}
}
throw new IOException(String.format(Locale.ROOT, "Unable to find function name for class '%s'", clazz.getName()));
}
}

View File

@ -15,30 +15,13 @@
* limitations under the License.
*/
package org.apache.solr.client.solrj.io;
import java.io.Serializable;
import java.util.Comparator;
/**
* An ascending field Comparator which compares a field of two Tuples and determines sort order.
* Expression language for the Streaming Aggregation API
**/
package org.apache.solr.client.solrj.io.stream.expr;
public class AscFieldComp implements Comparator<Tuple>, Serializable {
private static final long serialVersionUID = 1;
private String field;
public AscFieldComp(String field) {
this.field = field;
}
public int compare(Tuple t1, Tuple t2) {
Comparable o1 = (Comparable)t1.get(field);
Comparable o2 = (Comparable)t2.get(field);
return o1.compareTo(o2);
}
}

View File

@ -15,34 +15,13 @@
* limitations under the License.
*/
package org.apache.solr.client.solrj.io;
import java.io.Serializable;
import java.util.Comparator;
/**
* An descending field Comparator which compares a field of two Tuples and determines sort order.
* Stream implementations for the Streaming Aggregation API
**/
package org.apache.solr.client.solrj.io.stream;
public class DescFieldComp implements Comparator<Tuple>, Serializable {
private static final long serialVersionUID = 1;
private String field;
public DescFieldComp(String field) {
this.field = field;
}
public int compare(Tuple t1, Tuple t2) {
Comparable o1 = (Comparable)t1.get(field);
Comparable o2 = (Comparable)t2.get(field);
int c = o1.compareTo(o2);
if(c == 0) {
return 0;
} else {
return -c;
}
}
}

View File

@ -52,8 +52,11 @@
<str name="wt">json</str>
<str name="distrib">false</str>
</lst>
<lst name="streamFunctions">
<str name="count">org.apache.solr.client.solrj.io.stream.CountStream</str>
</lst>
</requestHandler>
<requestDispatcher handleSelect="true" >
<requestParsers enableRemoteStreaming="false" multipartUploadLimitInKB="2048" />
</requestDispatcher>

View File

@ -1,61 +0,0 @@
package org.apache.solr.client.solrj.io;
/*
* 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.
*/
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
public class CountStream extends TupleStream implements Serializable {
private TupleStream stream;
private int count;
public CountStream(TupleStream stream) {
this.stream = stream;
}
public void close() throws IOException {
this.stream.close();
}
public void open() throws IOException {
this.stream.open();
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(stream);
return l;
}
public void setStreamContext(StreamContext streamContext) {
stream.setStreamContext(streamContext);
}
public Tuple read() throws IOException {
Tuple t = stream.read();
if(t.EOF) {
t.put("count", count);
return t;
} else {
++count;
return t;
}
}
}

View File

@ -0,0 +1,98 @@
package org.apache.solr.client.solrj.io.stream;
/*
* 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.
*/
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import java.util.Locale;
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.StreamExpressionParameter;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
public class CountStream extends TupleStream implements ExpressibleStream, Serializable {
private TupleStream stream;
private int count;
public CountStream(TupleStream stream) {
this.stream = stream;
}
public CountStream(StreamExpression expression, StreamFactory factory) throws IOException{
List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, ExpressibleStream.class, TupleStream.class);
// validate expression contains only what we want.
if(expression.getParameters().size() != streamExpressions.size()){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - unknown operands found", expression));
}
if(1 != streamExpressions.size()){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single stream but found %d",expression, streamExpressions.size()));
}
stream = factory.constructStream(streamExpressions.get(0));
}
@Override
public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
// function name
StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
// stream
if(stream instanceof ExpressibleStream){
expression.addParameter(((ExpressibleStream)stream).toExpression(factory));
}
else{
throw new IOException("This CountStream contains a non-expressible TupleStream - it cannot be converted to an expression");
}
return expression;
}
public void close() throws IOException {
this.stream.close();
}
public void open() throws IOException {
this.stream.open();
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(stream);
return l;
}
public void setStreamContext(StreamContext streamContext) {
stream.setStreamContext(streamContext);
}
public Tuple read() throws IOException {
Tuple t = stream.read();
if(t.EOF) {
t.put("count", count);
return t;
} else {
++count;
return t;
}
}
}

View File

@ -0,0 +1,745 @@
package org.apache.solr.client.solrj.io.stream;
/*
* 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.
*/
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.Slow;
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.StreamExpressionParser;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
import org.apache.solr.cloud.AbstractZkTestCase;
import org.apache.solr.common.SolrInputDocument;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* All base tests will be done with CloudSolrStream. Under the covers CloudSolrStream uses SolrStream so
* SolrStream will get fully exercised through these tests.
*
**/
@Slow
@LuceneTestCase.SuppressCodecs({"Lucene3x", "Lucene40","Lucene41","Lucene42","Lucene45"})
public class StreamExpressionTest extends AbstractFullDistribZkTestBase {
private static final String SOLR_HOME = getFile("solrj" + File.separator + "solr").getAbsolutePath();
static {
schemaString = "schema-streaming.xml";
}
@BeforeClass
public static void beforeSuperClass() {
AbstractZkTestCase.SOLRHOME = new File(SOLR_HOME());
}
@AfterClass
public static void afterSuperClass() {
}
protected String getCloudSolrConfig() {
return "solrconfig-streaming.xml";
}
@Override
public String getSolrHome() {
return SOLR_HOME;
}
public static String SOLR_HOME() {
return SOLR_HOME;
}
@Before
@Override
public void setUp() throws Exception {
super.setUp();
// we expect this time of exception as shards go up and down...
//ignoreException(".*");
System.setProperty("numShards", Integer.toString(sliceCount));
}
@Override
@After
public void tearDown() throws Exception {
super.tearDown();
resetExceptionIgnores();
}
public StreamExpressionTest() {
super();
sliceCount = 2;
}
@Test
public void testAll() throws Exception{
assertNotNull(cloudClient);
handle.clear();
handle.put("timestamp", SKIPVAL);
waitForThingsToLevelOut(30);
del("*:*");
commit();
testCloudSolrStream();
testCloudSolrStreamWithZkHost();
testMergeStream();
testRankStream();
testReducerStream();
testUniqueStream();
testParallelUniqueStream();
testParallelReducerStream();
testParallelRankStream();
testParallelMergeStream();
}
private void testCloudSolrStream() throws Exception {
indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0");
indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0");
indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3");
indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4");
indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1");
commit();
StreamFactory factory = new StreamFactory().withCollectionZkHost("collection1", zkServer.getZkAddress());
StreamExpression expression;
CloudSolrStream stream;
List<Tuple> tuples;
// Basic test
expression = StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")");
stream = new CloudSolrStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 5);
assertOrder(tuples, 0,2,1,3,4);
assertLong(tuples.get(0),"a_i", 0);
// Basic w/aliases
expression = StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", aliases=\"a_i=alias.a_i, a_s=name\")");
stream = new CloudSolrStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 5);
assertOrder(tuples, 0,2,1,3,4);
assertLong(tuples.get(0),"alias.a_i", 0);
assertString(tuples.get(0),"name", "hello0");
// Basic filtered test
expression = StreamExpressionParser.parse("search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")");
stream = new CloudSolrStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 3);
assertOrder(tuples, 0,3,4);
assertLong(tuples.get(1),"a_i", 3);
del("*:*");
commit();
}
private void testCloudSolrStreamWithZkHost() throws Exception {
indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0");
indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0");
indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3");
indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4");
indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1");
commit();
StreamFactory factory = new StreamFactory();
StreamExpression expression;
CloudSolrStream stream;
List<Tuple> tuples;
// Basic test
expression = StreamExpressionParser.parse("search(collection1, zkHost=" + zkServer.getZkAddress() + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")");
stream = new CloudSolrStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 5);
assertOrder(tuples, 0,2,1,3,4);
assertLong(tuples.get(0),"a_i", 0);
// Basic w/aliases
expression = StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", aliases=\"a_i=alias.a_i, a_s=name\", zkHost=" + zkServer.getZkAddress() + ")");
stream = new CloudSolrStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 5);
assertOrder(tuples, 0,2,1,3,4);
assertLong(tuples.get(0),"alias.a_i", 0);
assertString(tuples.get(0),"name", "hello0");
// Basic filtered test
expression = StreamExpressionParser.parse("search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", zkHost=" + zkServer.getZkAddress() + ", sort=\"a_f asc, a_i asc\")");
stream = new CloudSolrStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 3);
assertOrder(tuples, 0,3,4);
assertLong(tuples.get(1),"a_i", 3);
del("*:*");
commit();
}
private void testUniqueStream() throws Exception {
indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0");
indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0");
indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3");
indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4");
indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1");
commit();
StreamExpression expression;
TupleStream stream;
List<Tuple> tuples;
StreamFactory factory = new StreamFactory()
.withCollectionZkHost("collection1", zkServer.getZkAddress())
.withStreamFunction("search", CloudSolrStream.class)
.withStreamFunction("unique", UniqueStream.class);
// Basic test
expression = StreamExpressionParser.parse("unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f asc\")");
stream = new UniqueStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 4);
assertOrder(tuples, 0, 1, 3, 4);
// Basic test desc
expression = StreamExpressionParser.parse("unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f desc, a_i desc\"), over=\"a_f desc\")");
stream = new UniqueStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 4);
assertOrder(tuples, 4,3,1,2);
// Basic w/multi comp
expression = StreamExpressionParser.parse("unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f asc, a_i asc\")");
stream = new UniqueStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 5);
assertOrder(tuples, 0,2,1,3,4);
// full factory w/multi comp
stream = factory.constructStream("unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f asc, a_i asc\")");
tuples = getTuples(stream);
assert(tuples.size() == 5);
assertOrder(tuples, 0, 2, 1, 3, 4);
del("*:*");
commit();
}
private void testMergeStream() throws Exception {
indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0");
indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0");
indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3");
indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4");
indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1");
commit();
StreamExpression expression;
TupleStream stream;
List<Tuple> tuples;
StreamFactory factory = new StreamFactory()
.withCollectionZkHost("collection1", zkServer.getZkAddress())
.withStreamFunction("search", CloudSolrStream.class)
.withStreamFunction("unique", UniqueStream.class)
.withStreamFunction("merge", MergeStream.class);
// Basic test
expression = StreamExpressionParser.parse("merge("
+ "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"),"
+ "search(collection1, q=\"id:(1)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"),"
+ "on=\"a_f asc\")");
stream = new MergeStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 4);
assertOrder(tuples, 0,1,3,4);
// Basic test desc
expression = StreamExpressionParser.parse("merge("
+ "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f desc\"),"
+ "search(collection1, q=\"id:(1)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f desc\"),"
+ "on=\"a_f desc\")");
stream = new MergeStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 4);
assertOrder(tuples, 4,3,1,0);
// Basic w/multi comp
expression = StreamExpressionParser.parse("merge("
+ "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
+ "search(collection1, q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
+ "on=\"a_f asc, a_s asc\")");
stream = new MergeStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 5);
assertOrder(tuples, 0,2,1,3,4);
// full factory w/multi comp
stream = factory.constructStream("merge("
+ "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
+ "search(collection1, q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
+ "on=\"a_f asc, a_s asc\")");
tuples = getTuples(stream);
assert(tuples.size() == 5);
assertOrder(tuples, 0,2,1,3,4);
del("*:*");
commit();
}
private void testRankStream() throws Exception {
indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0");
indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0");
indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3");
indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4");
indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1");
commit();
StreamExpression expression;
TupleStream stream;
List<Tuple> tuples;
StreamFactory factory = new StreamFactory()
.withCollectionZkHost("collection1", zkServer.getZkAddress())
.withStreamFunction("search", CloudSolrStream.class)
.withStreamFunction("unique", UniqueStream.class)
.withStreamFunction("top", RankStream.class);
// Basic test
expression = StreamExpressionParser.parse("top("
+ "n=3,"
+ "search(collection1, 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);
// Basic test desc
expression = StreamExpressionParser.parse("top("
+ "n=2,"
+ "unique("
+ "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f desc\"),"
+ "over=\"a_f desc\"),"
+ "sort=\"a_f desc\")");
stream = new RankStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 2);
assertOrder(tuples, 4,3);
// full factory
stream = factory.constructStream("top("
+ "n=4,"
+ "unique("
+ "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"),"
+ "over=\"a_f asc\"),"
+ "sort=\"a_f asc\")");
tuples = getTuples(stream);
assert(tuples.size() == 4);
assertOrder(tuples, 0,1,3,4);
del("*:*");
commit();
}
private void testReducerStream() throws Exception{
indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1");
indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2");
indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3");
indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4");
indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5");
indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6");
indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7");
indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8");
indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9");
indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10");
commit();
StreamExpression expression;
TupleStream stream;
List<Tuple> tuples;
Tuple t0, t1, t2;
List<Map> maps0, maps1, maps2;
StreamFactory factory = new StreamFactory()
.withCollectionZkHost("collection1", zkServer.getZkAddress())
.withStreamFunction("search", CloudSolrStream.class)
.withStreamFunction("unique", UniqueStream.class)
.withStreamFunction("top", RankStream.class)
.withStreamFunction("group", ReducerStream.class);
// basic
expression = StreamExpressionParser.parse("group("
+ "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_s asc, a_f asc\"),"
+ "by=\"a_s asc\")");
stream = new ReducerStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 3);
assertOrder(tuples, 0,3,4);
t0 = tuples.get(0);
maps0 = t0.getMaps();
assertMaps(maps0, 0, 2,1, 9);
t1 = tuples.get(1);
maps1 = t1.getMaps();
assertMaps(maps1, 3, 5, 7, 8);
t2 = tuples.get(2);
maps2 = t2.getMaps();
assertMaps(maps2, 4, 6);
// basic w/spaces
expression = StreamExpressionParser.parse("group("
+ "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_s asc, a_f asc\"),"
+ "by=\"a_s asc\")");
stream = new ReducerStream(expression, factory);
tuples = getTuples(stream);
assert(tuples.size() == 3);
assertOrder(tuples, 0,3,4);
t0 = tuples.get(0);
maps0 = t0.getMaps();
assertMaps(maps0, 0, 2,1, 9);
t1 = tuples.get(1);
maps1 = t1.getMaps();
assertMaps(maps1, 3, 5, 7, 8);
t2 = tuples.get(2);
maps2 = t2.getMaps();
assertMaps(maps2, 4, 6);
del("*:*");
commit();
}
private void testParallelUniqueStream() throws Exception {
indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0");
indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0");
indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3");
indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4");
indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1");
indexr(id, "5", "a_s", "hello1", "a_i", "10", "a_f", "1");
indexr(id, "6", "a_s", "hello1", "a_i", "11", "a_f", "5");
indexr(id, "7", "a_s", "hello1", "a_i", "12", "a_f", "5");
indexr(id, "8", "a_s", "hello1", "a_i", "13", "a_f", "4");
commit();
String zkHost = zkServer.getZkAddress();
StreamFactory streamFactory = new StreamFactory().withCollectionZkHost("collection1", zkServer.getZkAddress())
.withStreamFunction("search", CloudSolrStream.class)
.withStreamFunction("unique", UniqueStream.class)
.withStreamFunction("top", RankStream.class)
.withStreamFunction("group", ReducerStream.class)
.withStreamFunction("parallel", ParallelStream.class);
ParallelStream pstream = (ParallelStream)streamFactory.constructStream("parallel(collection1, unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", partitionKeys=\"a_f\"), over=\"a_f asc\"), workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_f asc\")");
List<Tuple> tuples = getTuples(pstream);
assert(tuples.size() == 5);
assertOrder(tuples, 0,1,3,4,6);
//Test the eofTuples
Map<String,Tuple> eofTuples = pstream.getEofTuples();
assert(eofTuples.size() == 2); //There should be an EOF tuple for each worker.
del("*:*");
commit();
}
private void testParallelReducerStream() throws Exception {
indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1");
indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2");
indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3");
indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4");
indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5");
indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6");
indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7");
indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8");
indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9");
indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10");
commit();
String zkHost = zkServer.getZkAddress();
StreamFactory streamFactory = new StreamFactory().withCollectionZkHost("collection1", zkServer.getZkAddress())
.withStreamFunction("search", CloudSolrStream.class)
.withStreamFunction("unique", UniqueStream.class)
.withStreamFunction("top", RankStream.class)
.withStreamFunction("group", ReducerStream.class)
.withStreamFunction("parallel", ParallelStream.class);
ParallelStream pstream = (ParallelStream)streamFactory.constructStream("parallel(collection1, group(search(collection1, q=\"*:*\", fl=\"id,a_s,a_i,a_f\", sort=\"a_s asc,a_f asc\", partitionKeys=\"a_s\"), by=\"a_s asc\"), workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_s asc\")");
List<Tuple> tuples = getTuples(pstream);
assert(tuples.size() == 3);
assertOrder(tuples, 0,3,4);
Tuple t0 = tuples.get(0);
List<Map> maps0 = t0.getMaps();
assertMaps(maps0, 0, 2, 1, 9);
Tuple t1 = tuples.get(1);
List<Map> maps1 = t1.getMaps();
assertMaps(maps1, 3, 5, 7, 8);
Tuple t2 = tuples.get(2);
List<Map> maps2 = t2.getMaps();
assertMaps(maps2, 4, 6);
//Test Descending with Ascending subsort
pstream = (ParallelStream)streamFactory.constructStream("parallel(collection1, group(search(collection1, q=\"*:*\", fl=\"id,a_s,a_i,a_f\", sort=\"a_s desc,a_f asc\", partitionKeys=\"a_s\"), by=\"a_s desc\"), workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_s desc\")");
tuples = getTuples(pstream);
assert(tuples.size() == 3);
assertOrder(tuples, 4,3,0);
t0 = tuples.get(0);
maps0 = t0.getMaps();
assertMaps(maps0, 4, 6);
t1 = tuples.get(1);
maps1 = t1.getMaps();
assertMaps(maps1, 3, 5, 7, 8);
t2 = tuples.get(2);
maps2 = t2.getMaps();
assertMaps(maps2, 0, 2, 1, 9);
del("*:*");
commit();
}
private void testParallelRankStream() throws Exception {
indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0");
indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0");
indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3");
indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4");
indexr(id, "5", "a_s", "hello1", "a_i", "5", "a_f", "1");
indexr(id, "6", "a_s", "hello1", "a_i", "6", "a_f", "1");
indexr(id, "7", "a_s", "hello1", "a_i", "7", "a_f", "1");
indexr(id, "8", "a_s", "hello1", "a_i", "8", "a_f", "1");
indexr(id, "9", "a_s", "hello1", "a_i", "9", "a_f", "1");
indexr(id, "10", "a_s", "hello1", "a_i", "10", "a_f", "1");
commit();
String zkHost = zkServer.getZkAddress();
StreamFactory streamFactory = new StreamFactory().withCollectionZkHost("collection1", zkServer.getZkAddress())
.withStreamFunction("search", CloudSolrStream.class)
.withStreamFunction("unique", UniqueStream.class)
.withStreamFunction("top", RankStream.class)
.withStreamFunction("group", ReducerStream.class)
.withStreamFunction("parallel", ParallelStream.class);
ParallelStream pstream = (ParallelStream)streamFactory.constructStream("parallel(collection1, top(search(collection1, q=\"*:*\", fl=\"id,a_s,a_i\", sort=\"a_i asc\", partitionKeys=\"a_i\"), n=\"11\", sort=\"a_i desc\"), workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_i desc\")");
List<Tuple> tuples = getTuples(pstream);
assert(tuples.size() == 10);
assertOrder(tuples, 10,9,8,7,6,5,4,3,2,0);
del("*:*");
commit();
}
private void testParallelMergeStream() throws Exception {
indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0");
indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0");
indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3");
indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4");
indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1");
indexr(id, "5", "a_s", "hello0", "a_i", "10", "a_f", "0");
indexr(id, "6", "a_s", "hello2", "a_i", "8", "a_f", "0");
indexr(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "3");
indexr(id, "8", "a_s", "hello4", "a_i", "11", "a_f", "4");
indexr(id, "9", "a_s", "hello1", "a_i", "100", "a_f", "1");
commit();
String zkHost = zkServer.getZkAddress();
StreamFactory streamFactory = new StreamFactory().withCollectionZkHost("collection1", zkServer.getZkAddress())
.withStreamFunction("search", CloudSolrStream.class)
.withStreamFunction("unique", UniqueStream.class)
.withStreamFunction("top", RankStream.class)
.withStreamFunction("group", ReducerStream.class)
.withStreamFunction("merge", MergeStream.class)
.withStreamFunction("parallel", ParallelStream.class);
//Test ascending
ParallelStream pstream = (ParallelStream)streamFactory.constructStream("parallel(collection1, merge(search(collection1, q=\"id:(4 1 8 7 9)\", fl=\"id,a_s,a_i\", sort=\"a_i asc\", partitionKeys=\"a_i\"), search(collection1, q=\"id:(0 2 3 6)\", fl=\"id,a_s,a_i\", sort=\"a_i asc\", partitionKeys=\"a_i\"), on=\"a_i asc\"), workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_i asc\")");
List<Tuple> tuples = getTuples(pstream);
assert(tuples.size() == 9);
assertOrder(tuples, 0,1,2,3,4,7,6,8,9);
//Test descending
pstream = (ParallelStream)streamFactory.constructStream("parallel(collection1, merge(search(collection1, q=\"id:(4 1 8 9)\", fl=\"id,a_s,a_i\", sort=\"a_i desc\", partitionKeys=\"a_i\"), search(collection1, q=\"id:(0 2 3 6)\", fl=\"id,a_s,a_i\", sort=\"a_i desc\", partitionKeys=\"a_i\"), on=\"a_i desc\"), workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_i desc\")");
tuples = getTuples(pstream);
assert(tuples.size() == 8);
assertOrder(tuples, 9,8,6,4,3,2,1,0);
del("*:*");
commit();
}
protected List<Tuple> getTuples(TupleStream tupleStream) throws IOException {
tupleStream.open();
List<Tuple> tuples = new ArrayList<Tuple>();
for(Tuple t = tupleStream.read(); !t.EOF; t = tupleStream.read()) {
tuples.add(t);
}
tupleStream.close();
return tuples;
}
protected boolean assertOrder(List<Tuple> tuples, int... ids) throws Exception {
return assertOrderOf(tuples, "id", ids);
}
protected boolean assertOrderOf(List<Tuple> tuples, String fieldName, int... ids) throws Exception {
int i = 0;
for(int val : ids) {
Tuple t = tuples.get(i);
Long tip = (Long)t.get(fieldName);
if(tip.intValue() != val) {
throw new Exception("Found value:"+tip.intValue()+" expecting:"+val);
}
++i;
}
return true;
}
protected boolean assertGroupOrder(Tuple tuple, int... ids) throws Exception {
List<?> group = (List<?>)tuple.get("tuples");
int i=0;
for(int val : ids) {
Map<?,?> t = (Map<?,?>)group.get(i);
Long tip = (Long)t.get("id");
if(tip.intValue() != val) {
throw new Exception("Found value:"+tip.intValue()+" expecting:"+val);
}
++i;
}
return true;
}
public boolean assertLong(Tuple tuple, String fieldName, long l) throws Exception {
long lv = (long)tuple.get(fieldName);
if(lv != l) {
throw new Exception("Longs not equal:"+l+" : "+lv);
}
return true;
}
public boolean assertString(Tuple tuple, String fieldName, String expected) throws Exception {
String actual = (String)tuple.get(fieldName);
if( (null == expected && null != actual) ||
(null != expected && null == actual) ||
(null != expected && !expected.equals(actual))){
throw new Exception("Longs not equal:"+expected+" : "+actual);
}
return true;
}
protected boolean assertMaps(List<Map> maps, int... ids) throws Exception {
if(maps.size() != ids.length) {
throw new Exception("Expected id count != actual map count:"+ids.length+":"+maps.size());
}
int i=0;
for(int val : ids) {
Map t = maps.get(i);
Long tip = (Long)t.get("id");
if(tip.intValue() != val) {
throw new Exception("Found value:"+tip.intValue()+" expecting:"+val);
}
++i;
}
return true;
}
@Override
protected void indexr(Object... fields) throws Exception {
SolrInputDocument doc = getDoc(fields);
indexDoc(doc);
}
}

View File

@ -0,0 +1,137 @@
package org.apache.solr.client.solrj.io.stream;
/*
* 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.
*/
import java.util.List;
import org.apache.lucene.util.LuceneTestCase;
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.StreamExpressionParser;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
import org.junit.Test;
/**
**/
public class StreamExpressionToExpessionTest extends LuceneTestCase {
private StreamFactory factory;
public StreamExpressionToExpessionTest() {
super();
factory = new StreamFactory()
.withCollectionZkHost("collection1", "testhost:1234")
.withStreamFunction("search", CloudSolrStream.class)
.withStreamFunction("merge", MergeStream.class)
.withStreamFunction("unique", UniqueStream.class)
.withStreamFunction("top", RankStream.class)
.withStreamFunction("group", ReducerStream.class)
;
}
@Test
public void testCloudSolrStream() throws Exception {
CloudSolrStream stream;
String expressionString;
// Basic test
stream = new CloudSolrStream(StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")"), factory);
expressionString = stream.toExpression(factory).toString();
assertTrue(expressionString.contains("search(collection1,"));
assertTrue(expressionString.contains("q=\"*:*\""));
assertTrue(expressionString.contains("fl=\"id,a_s,a_i,a_f\""));
assertTrue(expressionString.contains("sort=\"a_f asc, a_i asc\""));
// Basic w/aliases
stream = new CloudSolrStream(StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", aliases=\"id=izzy,a_s=kayden\")"), factory);
expressionString = stream.toExpression(factory).toString();
assertTrue(expressionString.contains("id=izzy"));
assertTrue(expressionString.contains("a_s=kayden"));
}
@Test
public void testUniqueStream() throws Exception {
UniqueStream stream;
String expressionString;
// Basic test
stream = new UniqueStream(StreamExpressionParser.parse("unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f asc\")"), factory);
expressionString = stream.toExpression(factory).toString();
assertTrue(expressionString.contains("unique(search(collection1"));
assertTrue(expressionString.contains("over=\"a_f asc\""));
}
@Test
public void testMergeStream() throws Exception {
MergeStream stream;
String expressionString;
// Basic test
stream = new MergeStream(StreamExpressionParser.parse("merge("
+ "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
+ "search(collection1, q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
+ "on=\"a_f asc, a_s asc\")"), factory);
expressionString = stream.toExpression(factory).toString();
assertTrue(expressionString.contains("q=\"id:(0 3 4)\""));
assertTrue(expressionString.contains("q=\"id:(1 2)\""));
assertTrue(expressionString.contains("on=\"a_f asc,a_s asc\""));
}
@Test
public void testRankStream() throws Exception {
RankStream stream;
String expressionString;
// Basic test
stream = new RankStream(StreamExpressionParser.parse("top("
+ "n=3,"
+ "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc,a_i asc\"),"
+ "sort=\"a_f asc, a_i asc\")"), factory);
expressionString = stream.toExpression(factory).toString();
assertTrue(expressionString.contains("top(n=3,search(collection1"));
assertTrue(expressionString.contains("sort=\"a_f asc,a_i asc\""));
// find 2nd instance of sort
assertTrue(expressionString.substring(expressionString.indexOf("sort=\"a_f asc,a_i asc\"") + 1).contains("sort=\"a_f asc,a_i asc\""));
}
@Test
public void testReducerStream() throws Exception {
ReducerStream stream;
String expressionString;
// Basic test
stream = new ReducerStream(StreamExpressionParser.parse("group("
+ "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_s desc, a_f asc\"),"
+ "by=\"a_s desc\")"), factory);
expressionString = stream.toExpression(factory).toString();
assertTrue(expressionString.contains("group(search(collection1"));
assertTrue(expressionString.contains("by=\"a_s desc\""));
}
}

View File

@ -1,4 +1,4 @@
package org.apache.solr.client.solrj.io;
package org.apache.solr.client.solrj.io.stream;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -27,6 +27,18 @@ import java.util.HashMap;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.comp.ComparatorOrder;
import org.apache.solr.client.solrj.io.comp.FieldComparator;
import org.apache.solr.client.solrj.io.comp.MultiComp;
import org.apache.solr.client.solrj.io.stream.CloudSolrStream;
import org.apache.solr.client.solrj.io.stream.MergeStream;
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.TupleStream;
import org.apache.solr.client.solrj.io.stream.UniqueStream;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
import org.apache.solr.cloud.AbstractZkTestCase;
import org.apache.solr.common.SolrInputDocument;
@ -50,6 +62,7 @@ import java.util.ArrayList;
public class StreamingTest extends AbstractFullDistribZkTestBase {
private static final String SOLR_HOME = getFile("solrj" + File.separator + "solr").getAbsolutePath();
private StreamFactory streamFactory;
static {
schemaString = "schema-streaming.xml";
@ -99,6 +112,15 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
public StreamingTest() {
super();
sliceCount = 2;
streamFactory = new StreamFactory()
.withStreamFunction("search", CloudSolrStream.class)
.withStreamFunction("merge", MergeStream.class)
.withStreamFunction("unique", UniqueStream.class)
.withStreamFunction("top", RankStream.class)
.withStreamFunction("group", ReducerStream.class)
.withStreamFunction("count", CountStream.class)
;
}
private void testUniqueStream() throws Exception {
@ -115,10 +137,11 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
Map params = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params);
UniqueStream ustream = new UniqueStream(stream, new AscFieldComp("a_f"));
UniqueStream ustream = new UniqueStream(stream, new FieldComparator("a_f",ComparatorOrder.ASCENDING));
List<Tuple> tuples = getTuples(ustream);
assert(tuples.size() == 4);
assertOrder(tuples, 0,1,3,4);
@ -132,6 +155,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
private void testSpacesInParams() throws Exception {
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
Map params = mapParams("q","*:*","fl","id , a_s , a_i , a_f","sort", "a_f asc , a_i asc");
@ -162,10 +186,11 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
commit();
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
Map paramsA = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_s asc,a_f asc", "partitionKeys", "none");
CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA);
ParallelStream pstream = new ParallelStream(zkHost, "collection1", stream, 2, new AscFieldComp("a_s"));
ParallelStream pstream = new ParallelStream(zkHost, "collection1", stream, 2, new FieldComparator("a_s",ComparatorOrder.ASCENDING));
List<Tuple> tuples = getTuples(pstream);
@ -194,11 +219,12 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
commit();
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
Map params = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i asc", "partitionKeys", "a_f");
CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params);
UniqueStream ustream = new UniqueStream(stream, new AscFieldComp("a_f"));
ParallelStream pstream = new ParallelStream(zkHost, "collection1", ustream, 2, new AscFieldComp("a_f"));
UniqueStream ustream = new UniqueStream(stream, new FieldComparator("a_f",ComparatorOrder.ASCENDING));
ParallelStream pstream = new ParallelStream(zkHost, "collection1", ustream, 2, new FieldComparator("a_f",ComparatorOrder.ASCENDING));
List<Tuple> tuples = getTuples(pstream);
assert(tuples.size() == 5);
assertOrder(tuples, 0,1,3,4,6);
@ -227,10 +253,11 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
commit();
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
Map params = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params);
RankStream rstream = new RankStream(stream, 3, new DescFieldComp("a_i"));
RankStream rstream = new RankStream(stream, 3, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
List<Tuple> tuples = getTuples(rstream);
@ -258,11 +285,12 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
commit();
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
Map params = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc", "partitionKeys", "a_i");
CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params);
RankStream rstream = new RankStream(stream, 11, new DescFieldComp("a_i"));
ParallelStream pstream = new ParallelStream(zkHost, "collection1", rstream, 2, new DescFieldComp("a_i"));
RankStream rstream = new RankStream(stream, 11, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
ParallelStream pstream = new ParallelStream(zkHost, "collection1", rstream, 2, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
List<Tuple> tuples = getTuples(pstream);
assert(tuples.size() == 10);
@ -288,6 +316,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
commit();
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
//Test with spaces in the parameter lists.
Map paramsA = mapParams("q","*:*","fl","id,a_s, a_i, a_f","sort", "a_s asc , a_f asc");
@ -322,11 +351,12 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
commit();
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
//Test with spaces in the parameter lists.
Map paramsA = mapParams("q","*:*","fl","id,a_s, a_i, a_f","sort", "a_s asc , a_f asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA);
ReducerStream rstream = new ReducerStream(stream, new AscFieldComp("a_s"));
ReducerStream rstream = new ReducerStream(stream, new FieldComparator("a_s",ComparatorOrder.ASCENDING));
List<Tuple> tuples = getTuples(rstream);
@ -368,11 +398,12 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
commit();
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
//Test with spaces in the parameter lists.
Map paramsA = mapParams("q", "blah", "fl", "id,a_s, a_i, a_f", "sort", "a_s asc , a_f asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA);
ReducerStream rstream = new ReducerStream(stream, new AscFieldComp("a_s"));
ReducerStream rstream = new ReducerStream(stream, new FieldComparator("a_s", ComparatorOrder.ASCENDING));
List<Tuple> tuples = getTuples(rstream);
@ -399,11 +430,12 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
commit();
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
Map paramsA = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_s asc,a_f asc", "partitionKeys", "a_s");
CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA);
ReducerStream rstream = new ReducerStream(stream, new AscFieldComp("a_s"));
ParallelStream pstream = new ParallelStream(zkHost, "collection1", rstream, 2, new AscFieldComp("a_s"));
ReducerStream rstream = new ReducerStream(stream, new FieldComparator("a_s",ComparatorOrder.ASCENDING));
ParallelStream pstream = new ParallelStream(zkHost, "collection1", rstream, 2, new FieldComparator("a_s",ComparatorOrder.ASCENDING));
List<Tuple> tuples = getTuples(pstream);
@ -426,8 +458,8 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
paramsA = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_s desc,a_f asc", "partitionKeys", "a_s");
stream = new CloudSolrStream(zkHost, "collection1", paramsA);
rstream = new ReducerStream(stream, new DescFieldComp("a_s"));
pstream = new ParallelStream(zkHost, "collection1", rstream, 2, new DescFieldComp("a_s"));
rstream = new ReducerStream(stream, new FieldComparator("a_s",ComparatorOrder.DESCENDING));
pstream = new ParallelStream(zkHost, "collection1", rstream, 2, new FieldComparator("a_s",ComparatorOrder.DESCENDING));
tuples = getTuples(pstream);
@ -470,11 +502,12 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
commit();
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
Map paramsA = mapParams("q","blah","fl","id,a_s,a_i,a_f","sort", "a_s asc,a_f asc", "partitionKeys", "a_s");
CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA);
ReducerStream rstream = new ReducerStream(stream, new AscFieldComp("a_s"));
ParallelStream pstream = new ParallelStream(zkHost, "collection1", rstream, 2, new AscFieldComp("a_s"));
ReducerStream rstream = new ReducerStream(stream, new FieldComparator("a_s", ComparatorOrder.ASCENDING));
ParallelStream pstream = new ParallelStream(zkHost, "collection1", rstream, 2, new FieldComparator("a_s", ComparatorOrder.ASCENDING));
List<Tuple> tuples = getTuples(pstream);
assert(tuples.size() == 0);
@ -490,6 +523,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
commit();
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
Map params = mapParams("q","*:*","fl","id,a_s,a_i,a_f,s_multi,i_multi,f_multi","sort", "a_s asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params);
@ -533,6 +567,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
commit();
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
//Test ascending
Map paramsA = mapParams("q","id:(4 1)","fl","id,a_s,a_i","sort", "a_i asc");
@ -541,7 +576,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
Map paramsB = mapParams("q","id:(0 2 3)","fl","id,a_s,a_i","sort", "a_i asc");
CloudSolrStream streamB = new CloudSolrStream(zkHost, "collection1", paramsB);
MergeStream mstream = new MergeStream(streamA, streamB, new AscFieldComp("a_i"));
MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.ASCENDING));
List<Tuple> tuples = getTuples(mstream);
assert(tuples.size() == 5);
@ -554,7 +589,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
paramsB = mapParams("q","id:(0 2 3)","fl","id,a_s,a_i","sort", "a_i desc");
streamB = new CloudSolrStream(zkHost, "collection1", paramsB);
mstream = new MergeStream(streamA, streamB, new DescFieldComp("a_i"));
mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
tuples = getTuples(mstream);
assert(tuples.size() == 5);
@ -568,7 +603,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
paramsB = mapParams("q","id:(0 3)","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i asc");
streamB = new CloudSolrStream(zkHost, "collection1", paramsB);
mstream = new MergeStream(streamA, streamB, new MultiComp(new AscFieldComp("a_f"),new AscFieldComp("a_i")));
mstream = new MergeStream(streamA, streamB, new MultiComp(new FieldComparator("a_f",ComparatorOrder.ASCENDING),new FieldComparator("a_i",ComparatorOrder.ASCENDING)));
tuples = getTuples(mstream);
assert(tuples.size() == 5);
@ -580,7 +615,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
paramsB = mapParams("q","id:(0 3)","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i desc");
streamB = new CloudSolrStream(zkHost, "collection1", paramsB);
mstream = new MergeStream(streamA, streamB, new MultiComp(new AscFieldComp("a_f"),new DescFieldComp("a_i")));
mstream = new MergeStream(streamA, streamB, new MultiComp(new FieldComparator("a_f",ComparatorOrder.ASCENDING),new FieldComparator("a_i",ComparatorOrder.DESCENDING)));
tuples = getTuples(mstream);
assert(tuples.size() == 5);
@ -607,6 +642,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
commit();
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
//Test ascending
Map paramsA = mapParams("q","id:(4 1 8 7 9)","fl","id,a_s,a_i","sort", "a_i asc", "partitionKeys", "a_i");
@ -615,8 +651,8 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
Map paramsB = mapParams("q","id:(0 2 3 6)","fl","id,a_s,a_i","sort", "a_i asc", "partitionKeys", "a_i");
CloudSolrStream streamB = new CloudSolrStream(zkHost, "collection1", paramsB);
MergeStream mstream = new MergeStream(streamA, streamB, new AscFieldComp("a_i"));
ParallelStream pstream = new ParallelStream(zkHost, "collection1", mstream, 2, new AscFieldComp("a_i"));
MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.ASCENDING));
ParallelStream pstream = new ParallelStream(zkHost, "collection1", mstream, 2, new FieldComparator("a_i",ComparatorOrder.ASCENDING));
List<Tuple> tuples = getTuples(pstream);
assert(tuples.size() == 9);
@ -629,8 +665,8 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
paramsB = mapParams("q","id:(0 2 3 6)","fl","id,a_s,a_i","sort", "a_i desc", "partitionKeys", "a_i");
streamB = new CloudSolrStream(zkHost, "collection1", paramsB);
mstream = new MergeStream(streamA, streamB, new DescFieldComp("a_i"));
pstream = new ParallelStream(zkHost, "collection1", mstream, 2, new DescFieldComp("a_i"));
mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
pstream = new ParallelStream(zkHost, "collection1", mstream, 2, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
tuples = getTuples(pstream);
assert(tuples.size() == 8);
@ -656,6 +692,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
commit();
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
//Test ascending
Map paramsA = mapParams("q","id:(4 1 8 7 9)","fl","id,a_s,a_i","sort", "a_i asc", "partitionKeys", "a_i");
@ -664,9 +701,9 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
Map paramsB = mapParams("q","id:(0 2 3 6)","fl","id,a_s,a_i","sort", "a_i asc", "partitionKeys", "a_i");
CloudSolrStream streamB = new CloudSolrStream(zkHost, "collection1", paramsB);
MergeStream mstream = new MergeStream(streamA, streamB, new AscFieldComp("a_i"));
MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.ASCENDING));
CountStream cstream = new CountStream(mstream);
ParallelStream pstream = new ParallelStream(zkHost, "collection1", cstream, 2, new AscFieldComp("a_i"));
ParallelStream pstream = new ParallelStream(zkHost, "collection1", cstream, 2, new FieldComparator("a_i",ComparatorOrder.ASCENDING));
List<Tuple> tuples = getTuples(pstream);
assert(tuples.size() == 9);
@ -711,6 +748,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
commit();
String zkHost = zkServer.getZkAddress();
streamFactory.withCollectionZkHost("collection1", zkHost);
Map params = null;
//Basic CloudSolrStream Test with Descending Sort
@ -788,10 +826,6 @@ public class StreamingTest extends AbstractFullDistribZkTestBase {
for(;;) {
Tuple t = tupleStream.read();
if(t.EOF) {
if(tupleStream instanceof ParallelStream) {
ParallelStream p = (ParallelStream) tupleStream;
assert(t.getMetrics() == p.getEofTuples()); // Make sure the EOF tuples are properly set on the final EOF tuple
}
break;
} else {
tuples.add(t);

View File

@ -0,0 +1,103 @@
package org.apache.solr.client.solrj.io.stream.expr;
/*
* 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.
*/
import org.apache.lucene.util.LuceneTestCase;
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.StreamExpressionParser;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
import org.junit.Test;
/**
**/
public class StreamExpressionParserTest extends LuceneTestCase {
public StreamExpressionParserTest() {
super();
}
@Test
public void testParsing() throws Exception{
StreamExpression actual, expected;
actual = StreamExpressionParser.parse("aliases(a_i=alias.a_i)");
expected = new StreamExpression("aliases")
.withParameter(new StreamExpressionNamedParameter("a_i", "alias.a_i"));
assertEquals(expected,actual);
actual = StreamExpressionParser.parse("search(a,b)");
expected = new StreamExpression("search").withParameter("a").withParameter("b");
assertEquals(expected, actual);
actual = StreamExpressionParser.parse("search(collection1, q=*:*, sort=\"fieldA desc, fieldB asc, fieldC asc\")");
expected = new StreamExpression("search")
.withParameter(new StreamExpressionValue("collection1"))
.withParameter(new StreamExpressionNamedParameter("q").withParameter("*:*"))
.withParameter(new StreamExpressionNamedParameter("sort").withParameter("fieldA desc, fieldB asc, fieldC asc"));
assertEquals(expected,actual);
actual = StreamExpressionParser.parse("unique(search(collection1, q=*:*, sort=\"fieldA desc, fieldB asc, fieldC asc\"))");
expected = new StreamExpression("unique")
.withParameter(new StreamExpression("search")
.withParameter(new StreamExpressionValue("collection1"))
.withParameter(new StreamExpressionNamedParameter("q").withParameter("*:*"))
.withParameter(new StreamExpressionNamedParameter("sort").withParameter("fieldA desc, fieldB asc, fieldC asc"))
);
assertEquals(expected,actual);
actual = StreamExpressionParser.parse("unique(search(collection1, q=*:*, sort=\"fieldA desc, fieldB asc, fieldC asc\"), alt=search(collection1, foo=bar))");
expected = new StreamExpression("unique")
.withParameter(new StreamExpression("search")
.withParameter(new StreamExpressionValue("collection1"))
.withParameter(new StreamExpressionNamedParameter("q").withParameter("*:*"))
.withParameter(new StreamExpressionNamedParameter("sort").withParameter("fieldA desc, fieldB asc, fieldC asc")))
.withParameter(new StreamExpressionNamedParameter("alt")
.withParameter(new StreamExpression("search")
.withParameter("collection1")
.withParameter(new StreamExpressionNamedParameter("foo")
.withParameter("bar"))));
assertEquals(expected,actual);
actual = StreamExpressionParser.parse("innerJoin("
+ "left=search(collection1, q=*:*, fl=\"fieldA,fieldB,fieldC\", sort=\"fieldA asc, fieldB asc\"),"
+ "right=search(collection2, q=*:*, fl=\"fieldA,fieldD\", sort=fieldA asc),"
+ "on(equals(fieldA), notEquals(fieldC,fieldD))"
+ ")");
expected = new StreamExpression("innerJoin")
.withParameter(new StreamExpressionNamedParameter("left")
.withParameter(new StreamExpression("search")
.withParameter("collection1")
.withParameter(new StreamExpressionNamedParameter("q","*:*"))
.withParameter(new StreamExpressionNamedParameter("fl","fieldA,fieldB,fieldC"))
.withParameter(new StreamExpressionNamedParameter("sort","fieldA asc, fieldB asc"))))
.withParameter(new StreamExpressionNamedParameter("right")
.withParameter(new StreamExpression("search")
.withParameter("collection2")
.withParameter(new StreamExpressionNamedParameter("q","*:*"))
.withParameter(new StreamExpressionNamedParameter("fl","fieldA,fieldD"))
.withParameter(new StreamExpressionNamedParameter("sort","fieldA asc"))))
.withParameter(new StreamExpression("on")
.withParameter(new StreamExpression("equals").withParameter("fieldA"))
.withParameter(new StreamExpression("notEquals").withParameter("fieldC").withParameter("fieldD")));
assertEquals(expected,actual);
}
}