mirror of https://github.com/apache/lucene.git
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:
parent
9d49a76d01
commit
e50f4d94c7
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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";
|
||||
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
||||
|
|
@ -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) {
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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) {
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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;
|
||||
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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();
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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 {
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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()));
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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>
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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\""));
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
Loading…
Reference in New Issue