diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java new file mode 100644 index 00000000000..98a7f8f9463 --- /dev/null +++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java @@ -0,0 +1,84 @@ +/* + * 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.handler; + +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.core.CloseHook; +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(); + + public void inform(SolrCore core) { + + core.addCloseHook( new CloseHook() { + @Override + public void preClose(SolrCore core) { + //To change body of implemented methods use File | Settings | File Templates. + } + + @Override + public void postClose(SolrCore core) { + clientCache.close(); + } + }); + } + + 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"); + StreamContext context = new StreamContext(); + context.workerID = worker; + context.numWorkers = numWorkers; + context.clientCache = clientCache; + tupleStream.setStreamContext(context); + rsp.add("tuples", tupleStream); + } + + public String getDescription() { + return "StreamHandler"; + } + + public String getSource() { + return null; + } +} \ No newline at end of file diff --git a/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java b/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java index f03d25cedf5..e2b94d76f6d 100644 --- a/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java +++ b/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java @@ -23,6 +23,8 @@ import java.util.*; import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexableField; +import org.apache.solr.client.solrj.io.TupleStream; +import org.apache.solr.client.solrj.io.Tuple; import org.apache.lucene.index.StorableField; import org.apache.lucene.index.StoredDocument; import org.apache.lucene.util.BytesRef; @@ -186,12 +188,14 @@ public abstract class TextResponseWriter { writeMap(name, (Map)val, false, true); } else if (val instanceof NamedList) { writeNamedList(name, (NamedList)val); + } else if (val instanceof TupleStream) { + writeTupleStream((TupleStream) val); } else if (val instanceof Iterable) { writeArray(name,((Iterable)val).iterator()); } else if (val instanceof Object[]) { writeArray(name,(Object[])val); } else if (val instanceof Iterator) { - writeArray(name,(Iterator)val); + writeArray(name, (Iterator) val); } else if (val instanceof byte[]) { byte[] arr = (byte[])val; writeByteArr(name, arr, 0, arr.length); @@ -309,6 +313,26 @@ public abstract class TextResponseWriter { } } + public void writeTupleStream(TupleStream tupleStream) throws IOException { + tupleStream.open(); + writeStartDocumentList("response", -1, -1, -1, null); + boolean isFirst = true; + while(true) { + Tuple tuple = tupleStream.read(); + if(!isFirst) { + writer.write(","); + } + writeMap(null, tuple.fields, false, true); + isFirst = false; + if(tuple.EOF) { + break; + } + } + writeEndDocumentList(); + tupleStream.close(); + } + + /** if this form of the method is called, val is the Java string form of a double */ public abstract void writeDouble(String name, String val) throws IOException; diff --git a/solr/core/src/java/org/apache/solr/search/HashQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/HashQParserPlugin.java new file mode 100644 index 00000000000..8fdbf11d894 --- /dev/null +++ b/solr/core/src/java/org/apache/solr/search/HashQParserPlugin.java @@ -0,0 +1,394 @@ +/* + * 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.search; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Semaphore; +import java.util.concurrent.Future; +import com.google.common.primitives.Longs; + +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.IndexReaderContext; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.util.BitDocIdSet; +import org.apache.lucene.util.CharsRef; +import org.apache.lucene.util.CharsRefBuilder; +import org.apache.lucene.util.FixedBitSet; + +import org.apache.lucene.util.Bits; +import org.apache.lucene.search.BitsFilteredDocIdSet; +import org.apache.solr.common.params.SolrParams; +import org.apache.solr.common.util.SolrjNamedThreadFactory; +import org.apache.solr.core.CloseHook; +import org.apache.solr.request.SolrQueryRequest; +import org.apache.solr.schema.IndexSchema; +import org.apache.solr.schema.FieldType; +import org.apache.solr.schema.StrField; +import org.apache.solr.schema.TrieField; +import org.apache.solr.core.SolrCore; + +import org.apache.lucene.search.Query; +import org.apache.lucene.search.ConstantScoreQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Weight; +import org.apache.lucene.search.DocIdSet; +import org.apache.lucene.search.Filter; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.util.BytesRef; + +import org.apache.solr.common.util.NamedList; + +/** +* syntax fq={!hash workers=11 worker=4 keys=field1,field2} +* */ + +public class HashQParserPlugin extends QParserPlugin { + + public static final String NAME = "hash"; + private static Semaphore semaphore = new Semaphore(8,true); + private static ExecutorService threadPool = Executors.newCachedThreadPool(new SolrjNamedThreadFactory("HashQParserPlugin")); + private static boolean init = true; + + private static synchronized void closeHook(SolrCore core) { + if(init) { + init = false; + core.addCloseHook(new CloseHook() { + @Override + public void preClose(SolrCore core) { + threadPool.shutdown(); + //To change body of implemented methods use File | Settings | File Templates. + } + + @Override + public void postClose(SolrCore core) { + //To change body of implemented methods use File | Settings | File Templates. + } + }); + } + } + + public void init(NamedList params) { + + } + + public QParser createParser(String query, SolrParams localParams, SolrParams params, SolrQueryRequest request) { + closeHook(request.getSearcher().getCore()); + return new HashQParser(query, localParams, params, request); + } + + private class HashQParser extends QParser { + + public HashQParser(String query, SolrParams localParams, SolrParams params, SolrQueryRequest request) { + super(query, localParams, params, request); + } + + public Query parse() { + int workers = localParams.getInt("workers"); + int worker = localParams.getInt("worker"); + String keys = params.get("partitionKeys"); + return new HashQuery(keys, workers, worker); + } + } + + private class HashQuery extends ExtendedQueryBase implements PostFilter { + + private String keysParam; + private int workers; + private int worker; + + public boolean getCache() { + if(getCost() > 99) { + return false; + } else { + return super.getCache(); + } + } + + public int hashCode() { + return keysParam.hashCode()+workers+worker+(int)getBoost(); + } + + public boolean equals(Object o) { + if (o instanceof HashQuery) { + HashQuery h = (HashQuery)o; + if(keysParam.equals(h.keysParam) && workers == h.workers && worker == h.worker && getBoost() == h.getBoost()) { + return true; + } + } + + return false; + } + + public HashQuery(String keysParam, int workers, int worker) { + this.keysParam = keysParam; + this.workers = workers; + this.worker = worker; + } + + public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException { + + String[] keys = keysParam.split(","); + SolrIndexSearcher solrIndexSearcher = (SolrIndexSearcher)searcher; + IndexReaderContext context = solrIndexSearcher.getTopReaderContext(); + + List leaves = context.leaves(); + ArrayBlockingQueue queue = new ArrayBlockingQueue(leaves.size()); + + + for(LeafReaderContext leaf : leaves) { + try { + semaphore.acquire(); + SegmentPartitioner segmentPartitioner = new SegmentPartitioner(leaf,worker,workers, keys, solrIndexSearcher, queue,semaphore); + threadPool.execute(segmentPartitioner); + } catch(Exception e) { + throw new IOException(e); + } + } + + FixedBitSet[] fixedBitSets = new FixedBitSet[leaves.size()]; + for(int i=0; i 1) ? new CompositeHash(hashKeys) : hashKeys[0]; + } + + public void run() { + LeafReader reader = context.reader(); + + try { + k.setNextReader(context); + this.docs = new FixedBitSet(reader.maxDoc()); + int maxDoc = reader.maxDoc(); + for(int i=0; i 1) ? new CompositeHash(hashKeys) : hashKeys[0]; + return new HashCollector(k, workers, worker); + } + } + + private class HashCollector extends DelegatingCollector { + private int worker; + private int workers; + private HashKey hashKey; + private LeafCollector leafCollector; + + public HashCollector(HashKey hashKey, int workers, int worker) { + this.hashKey = hashKey; + this.workers = workers; + this.worker = worker; + } + + public void setScorer(Scorer scorer) throws IOException{ + leafCollector.setScorer(scorer); + } + + public void doSetNextReader(LeafReaderContext context) throws IOException { + this.hashKey.setNextReader(context); + this.leafCollector = delegate.getLeafCollector(context); + } + + public void collect(int doc) throws IOException { + if((hashKey.hashCode(doc) & 0x7FFFFFFF) % workers == worker) { + leafCollector.collect(doc); + } + } + } + + private interface HashKey { + public void setNextReader(LeafReaderContext reader) throws IOException; + public long hashCode(int doc); + } + + private class BytesHash implements HashKey { + + private SortedDocValues values; + private String field; + private FieldType fieldType; + private CharsRefBuilder charsRefBuilder = new CharsRefBuilder(); + + public BytesHash(String field, FieldType fieldType) { + this.field = field; + this.fieldType = fieldType; + } + + public void setNextReader(LeafReaderContext context) throws IOException { + values = context.reader().getSortedDocValues(field); + } + + public long hashCode(int doc) { + BytesRef ref = values.get(doc); + this.fieldType.indexedToReadable(ref, charsRefBuilder); + CharsRef charsRef = charsRefBuilder.get(); + return charsRef.hashCode(); + } + } + + private class NumericHash implements HashKey { + + private NumericDocValues values; + private String field; + + public NumericHash(String field) { + this.field = field; + } + + public void setNextReader(LeafReaderContext context) throws IOException { + values = context.reader().getNumericDocValues(field); + } + + public long hashCode(int doc) { + long l = values.get(doc); + return Longs.hashCode(l); + } + } + + private class ZeroHash implements HashKey { + + public long hashCode(int doc) { + return 0; + } + + public void setNextReader(LeafReaderContext context) { + + } + } + + private class CompositeHash implements HashKey { + + private HashKey key1; + private HashKey key2; + private HashKey key3; + private HashKey key4; + + public CompositeHash(HashKey[] hashKeys) { + key1 = hashKeys[0]; + key2 = hashKeys[1]; + key3 = (hashKeys.length > 2) ? hashKeys[2] : new ZeroHash(); + key4 = (hashKeys.length > 3) ? hashKeys[3] : new ZeroHash(); + } + + public void setNextReader(LeafReaderContext context) throws IOException { + key1.setNextReader(context); + key2.setNextReader(context); + key3.setNextReader(context); + key4.setNextReader(context); + } + + public long hashCode(int doc) { + return key1.hashCode(doc)+key2.hashCode(doc)+key3.hashCode(doc)+key4.hashCode(doc); + } + } +} diff --git a/solr/core/src/java/org/apache/solr/search/QParserPlugin.java b/solr/core/src/java/org/apache/solr/search/QParserPlugin.java index e8da6729c85..aa4ec36dd98 100644 --- a/solr/core/src/java/org/apache/solr/search/QParserPlugin.java +++ b/solr/core/src/java/org/apache/solr/search/QParserPlugin.java @@ -72,6 +72,7 @@ public abstract class QParserPlugin implements NamedListInitializedPlugin, SolrI map.put(ReRankQParserPlugin.NAME, ReRankQParserPlugin.class); map.put(ExportQParserPlugin.NAME, ExportQParserPlugin.class); map.put(MLTQParserPlugin.NAME, MLTQParserPlugin.class); + map.put(HashQParserPlugin.NAME, HashQParserPlugin.class); standardPlugins = Collections.unmodifiableMap(map); } diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-hash.xml b/solr/core/src/test-files/solr/collection1/conf/schema-hash.xml new file mode 100644 index 00000000000..6b0495281cc --- /dev/null +++ b/solr/core/src/test-files/solr/collection1/conf/schema-hash.xml @@ -0,0 +1,587 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + text + id + + + + + + + + + + + + + + + + + + + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-hash.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-hash.xml new file mode 100644 index 00000000000..ecd0f48bdfc --- /dev/null +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-hash.xml @@ -0,0 +1,83 @@ + + + + + + ${tests.luceneMatchVersion:LUCENE_CURRENT} + + ${useCompoundFile:false} + + ${solr.data.dir:} + + + + + ${solr.data.dir:} + + + + + + + true + + + + + + + + json + false + + + + + + + + + + + + + + + + *:* + + + all + + server-enabled.txt + + + + + solr + + + + diff --git a/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java b/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java index e87b8c7ab54..338112cd257 100644 --- a/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java +++ b/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java @@ -268,6 +268,19 @@ public class QueryEqualityTest extends SolrTestCaseJ4 { } } + + public void testHash() throws Exception { + SolrQueryRequest req = req("partitionKeys","foo_s"); + + try { + assertQueryEquals("hash", req, + "{!hash workers=3 worker=0}"); + + } finally { + req.close(); + } + } + public void testQueryNested() throws Exception { SolrQueryRequest req = req("df", "foo_s"); try { diff --git a/solr/core/src/test/org/apache/solr/search/TestHashQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/TestHashQParserPlugin.java new file mode 100644 index 00000000000..877a28eb345 --- /dev/null +++ b/solr/core/src/test/org/apache/solr/search/TestHashQParserPlugin.java @@ -0,0 +1,252 @@ +/* + * 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.search; + +import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.SolrTestCaseJ4; +import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.common.util.NamedList; +import org.apache.solr.response.SolrQueryResponse; +import org.apache.lucene.util.BytesRef; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; +import java.util.*; +import java.util.Random; + +@LuceneTestCase.SuppressCodecs({"Lucene3x", "Lucene40","Lucene41","Lucene42","Lucene45"}) +public class TestHashQParserPlugin extends SolrTestCaseJ4 { + + @BeforeClass + public static void beforeClass() throws Exception { + initCore("solrconfig-hash.xml", "schema-hash.xml"); + } + + @Override + @Before + public void setUp() throws Exception { + // if you override setUp or tearDown, you better call + // the super classes version + super.setUp(); + clearIndex(); + assertU(commit()); + } + + + public int getCost(Random random) { + int i = random.nextInt(2); + if(i == 0) { + return 200; + } else { + return 1; + } + } + + + @Test + public void testHashPartition() throws Exception { + + + Random random = random(); + HashSet set = new HashSet(); + + for(int i=0; i<50; i++) { + int v = random.nextInt(1000000); + String val = Integer.toString(v); + if(!set.contains(val)){ + set.add(val); + String[] doc = {"id", val, "a_s", val, "a_i", val, "a_l", val}; + assertU(adoc(doc)); + if(i % 10 == 0) + assertU(commit()); + + } + } + assertU(commit()); + + + //Test with 3 worker and String hash ID. + + ModifiableSolrParams params = new ModifiableSolrParams(); + params.add("q", "*:*"); + params.add("fq", "{!hash worker=0 workers=3 cost="+getCost(random)+"}"); + params.add("partitionKeys", "a_s"); + params.add("rows","50"); + HashSet set1 = new HashSet(); + String response = h.query(req(params)); + + Iterator it = set.iterator(); + + while(it.hasNext()) { + String s = it.next(); + String results = h.validateXPath(response, "*[count(//int[@name='id'][.='"+s+"'])=1]"); + if(results == null) { + set1.add(s); + } + } + + params = new ModifiableSolrParams(); + params.add("q", "*:*"); + params.add("fq", "{!hash worker=1 workers=3 cost="+getCost(random)+"}"); + params.add("partitionKeys", "a_s"); + params.add("rows","50"); + HashSet set2 = new HashSet(); + response = h.query(req(params)); + + it = set.iterator(); + + while(it.hasNext()) { + String s = it.next(); + String results = h.validateXPath(response, "*[count(//int[@name='id'][.='"+s+"'])=1]"); + if(results == null) { + set2.add(s); + } + } + + + params = new ModifiableSolrParams(); + params.add("q", "*:*"); + params.add("fq", "{!hash worker=2 workers=3 cost="+getCost(random)+"}"); + params.add("partitionKeys", "a_s"); + params.add("rows","50"); + HashSet set3 = new HashSet(); + response = h.query(req(params)); + + it = set.iterator(); + + while(it.hasNext()) { + String s = it.next(); + String results = h.validateXPath(response, "*[count(//int[@name='id'][.='"+s+"'])=1]"); + if(results == null) { + set3.add(s); + } + } + + assert(set1.size() > 0); + assert(set2.size() > 0); + assert(set3.size() > 0); + assert(set1.size()+set2.size()+set3.size()==set.size()); + assertNoOverLap(set1, set2); + assertNoOverLap(set1, set3); + assertNoOverLap(set2, set3); + + + //Test with 2 workers and int partition Key + + + params = new ModifiableSolrParams(); + params.add("q", "*:*"); + params.add("fq", "{!hash worker=0 workers=2 cost="+getCost(random)+"}"); + params.add("partitionKeys", "a_i"); + params.add("rows","50"); + set1 = new HashSet(); + response = h.query(req(params)); + + it = set.iterator(); + + while(it.hasNext()) { + String s = it.next(); + String results = h.validateXPath(response, "*[count(//int[@name='id'][.='"+s+"'])=1]"); + if(results == null) { + set1.add(s); + } + } + + params = new ModifiableSolrParams(); + params.add("q", "*:*"); + params.add("fq", "{!hash worker=1 workers=2 cost="+getCost(random)+"}"); + params.add("partitionKeys", "a_i"); + params.add("rows","50"); + set2 = new HashSet(); + response = h.query(req(params)); + + it = set.iterator(); + + while(it.hasNext()) { + String s = it.next(); + String results = h.validateXPath(response, "*[count(//int[@name='id'][.='"+s+"'])=1]"); + if(results == null) { + set2.add(s); + } + } + + assert(set1.size() > 0); + assert(set2.size() > 0); + assert(set1.size()+set2.size()==set.size()); + assertNoOverLap(set1, set2); + + + //Test with 2 workers and compound partition Key + + + params = new ModifiableSolrParams(); + params.add("q", "*:*"); + params.add("fq", "{!hash worker=0 workers=2 cost="+getCost(random)+"}"); + params.add("partitionKeys", "a_s,a_i,a_l"); + params.add("rows","50"); + set1 = new HashSet(); + response = h.query(req(params)); + + it = set.iterator(); + + while(it.hasNext()) { + String s = it.next(); + String results = h.validateXPath(response, "*[count(//int[@name='id'][.='"+s+"'])=1]"); + if(results == null) { + set1.add(s); + } + } + + params = new ModifiableSolrParams(); + params.add("q", "*:*"); + params.add("fq", "{!hash worker=1 workers=2 cost="+getCost(random)+"}"); + params.add("partitionKeys", "a_s,a_i,a_l"); + params.add("rows","50"); + set2 = new HashSet(); + response = h.query(req(params)); + + it = set.iterator(); + + while(it.hasNext()) { + String s = it.next(); + String results = h.validateXPath(response, "*[count(//int[@name='id'][.='"+s+"'])=1]"); + if(results == null) { + set2.add(s); + } + } + + assert(set1.size() > 0); + assert(set2.size() > 0); + assert(set1.size()+set2.size()==set.size()); + assertNoOverLap(set1, set2); + } + + + private void assertNoOverLap(Set setA, Set setB) throws Exception { + Iterator it = setA.iterator(); + while(it.hasNext()) { + Object o = it.next(); + if(setB.contains(o)) { + throw new Exception("Overlapping sets for value:"+o.toString()); + } + } + } +} diff --git a/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml b/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml index d531a5575fa..12f05dbdb2a 100755 --- a/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml +++ b/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml @@ -893,6 +893,36 @@ + + + + + {!xport} + xsort + false + + + + query + + + + + + + + + json + false + + + + + + + + json + false + + + + diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java index 80008c8fdae..c07fa75f7da 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java @@ -520,7 +520,7 @@ public class HttpSolrClient extends SolrClient { null); } } - if (processor == null) { + if (processor == null || processor instanceof InputStreamResponseParser) { // no processor specified, return raw stream NamedList rsp = new NamedList<>(); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/InputStreamResponseParser.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/InputStreamResponseParser.java new file mode 100644 index 00000000000..2f181525b32 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/InputStreamResponseParser.java @@ -0,0 +1,57 @@ +package org.apache.solr.client.solrj.impl; + +/* + * 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.InputStream; +import java.io.Reader; +import java.io.StringWriter; + +import org.apache.commons.io.IOUtils; +import org.apache.solr.client.solrj.ResponseParser; +import org.apache.solr.common.SolrException; +import org.apache.solr.common.util.NamedList; + +/** + * Simply puts the InputStream into an entry in a NamedList named "stream". + */ +public class InputStreamResponseParser extends ResponseParser { + + private final String writerType; + + public InputStreamResponseParser(String writerType) { + this.writerType = writerType; + } + + @Override + public String getWriterType() { + return writerType; + } + + @Override + public NamedList processResponse(Reader reader) { + throw new UnsupportedOperationException(); + } + + @Override + public NamedList processResponse(InputStream body, String encoding) { + throw new UnsupportedOperationException(); + } + +} + diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscBucketComp.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscBucketComp.java new file mode 100644 index 00000000000..5428bc3eb5b --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscBucketComp.java @@ -0,0 +1,42 @@ +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.Serializable; +import java.util.Comparator; + +public class AscBucketComp implements Comparator, Serializable { + + private int ord; + + public AscBucketComp(int ord) { + this.ord = ord; + } + + public int compare(BucketMetrics b1, BucketMetrics b2) { + double d1 = b1.getMetrics()[ord].getValue(); + double d2 = b2.getMetrics()[ord].getValue(); + if(d1 > d2) { + return 1; + } else if(d1 < d2) { + return -1; + } else { + return 0; + } + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscFieldComp.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscFieldComp.java new file mode 100644 index 00000000000..802d7e6844e --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscFieldComp.java @@ -0,0 +1,38 @@ +/* + * 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.Serializable; +import java.util.Comparator; + +public class AscFieldComp implements Comparator, 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); + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscMetricComp.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscMetricComp.java new file mode 100644 index 00000000000..c6d5a1afc11 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscMetricComp.java @@ -0,0 +1,39 @@ +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.Serializable; +import java.util.Comparator; +import java.util.List; + +public class AscMetricComp implements Comparator, Serializable { + + private static final long serialVersionUID = 1; + + private int ord; + + public AscMetricComp(int ord) { + this.ord = ord; + } + + public int compare(Tuple t1, Tuple t2) { + List values1 = (List)t1.get("metricValues"); + List values2 = (List)t2.get("metricValues"); + return values1.get(ord).compareTo(values2.get(ord)); + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Bucket.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Bucket.java new file mode 100644 index 00000000000..b6cd02f8391 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Bucket.java @@ -0,0 +1,43 @@ +/* + * 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.Serializable; + +public class Bucket implements Serializable { + + private static final long serialVersionUID = 1; + + private String bucketKey; + + public Bucket() { + + } + + public Bucket(String bucketKey) { + this.bucketKey = bucketKey; + } + + public String getBucketValue(Tuple tuple) { + return tuple.get(bucketKey).toString(); + } +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/BucketMetrics.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/BucketMetrics.java new file mode 100644 index 00000000000..f0818b23d80 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/BucketMetrics.java @@ -0,0 +1,43 @@ +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.Serializable; + +public class BucketMetrics implements Serializable { + + private static final long serialVersionUID = 1; + + private HashKey key; + private Metric[] metrics; + + public BucketMetrics(HashKey key, Metric[] metrics) { + this.key = key; + this.metrics = metrics; + } + + public Metric[] getMetrics() { + return metrics; + } + + public HashKey getKey() { + return key; + } + + +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/CloudSolrStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/CloudSolrStream.java new file mode 100644 index 00000000000..62bb394894a --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/CloudSolrStream.java @@ -0,0 +1,289 @@ +/* + * 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.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.HashMap; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeoutException; + +import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.common.SolrException; +import org.apache.solr.common.cloud.Replica; +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.ZooKeeperException; +import org.apache.solr.common.cloud.Slice; +import org.apache.solr.common.util.SolrjNamedThreadFactory; +import org.apache.zookeeper.KeeperException; + +/** +* Connects to Zookeeper to pick replicas from a specific collection to send the query to. +* SolrStream instances are used to send the query to the replicas. +* SolrStreams are opened using a Thread pool, but a single thread is used to iterate through each stream's tuples.* * +**/ + +public class CloudSolrStream extends TupleStream { + + private static final long serialVersionUID = 1; + + protected String zkHost; + protected String collection; + protected Map params; + private Map fieldMappings; + protected TreeSet tuples; + protected Comparator comp; + protected List solrStreams = new ArrayList(); + private int zkConnectTimeout = 10000; + private int zkClientTimeout = 10000; + protected transient SolrClientCache cache; + protected transient CloudSolrClient cloudSolrClient; + private int numWorkers; + private int workerID; + protected Map eofTuples = new HashMap(); + + public CloudSolrStream(String zkHost, String collection, Map params) { + this.zkHost = zkHost; + this.collection = collection; + this.params = params; + this.tuples = new TreeSet(); + String sort = (String)params.get("sort"); + this.comp = parseComp(sort); + } + + //Used by the ParallelStream + protected CloudSolrStream() { + + } + + public void setComp(Comparator comp) { + this.comp = comp; + } + + public void setFieldMappings(Map fieldMappings) { + this.fieldMappings = fieldMappings; + } + + public void setStreamContext(StreamContext context) { + this.numWorkers = context.numWorkers; + this.workerID = context.workerID; + this.cache = context.clientCache; + } + + public void open() throws IOException { + if(this.cache != null) { + this.cloudSolrClient = this.cache.getCloudSolrClient(zkHost); + } else { + this.cloudSolrClient = new CloudSolrClient(zkHost); + this.cloudSolrClient.connect(); + } + constructStreams(); + openStreams(); + } + + + + public List children() { + return solrStreams; + } + + private Comparator parseComp(String sort) { + String[] sorts = sort.split(","); + Comparator[] comps = new Comparator[sorts.length]; + for(int i=0; i 1) { + return new MultiComp(comps); + } else { + return comps[0]; + } + } + + protected void constructStreams() throws IOException { + + try { + + ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader(); + ClusterState clusterState = zkStateReader.getClusterState(); + //System.out.println("Connected to zk an got cluster state."); + + Collection slices = clusterState.getActiveSlices(this.collection); + long time = System.currentTimeMillis(); + params.put("distrib","false"); // We are the aggregator. + + for(Slice slice : slices) { + Collection replicas = slice.getReplicas(); + List shuffler = new ArrayList(); + for(Replica replica : replicas) { + shuffler.add(replica); + } + + Collections.shuffle(shuffler, new Random(time)); + Replica rep = shuffler.get(0); + ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep); + String url = zkProps.getCoreUrl(); + SolrStream solrStream = new SolrStream(url, params); + StreamContext context = new StreamContext(); + context.numWorkers = this.numWorkers; + context.workerID = this.workerID; + solrStream.setStreamContext(context); + solrStream.setFieldMappings(this.fieldMappings); + solrStreams.add(solrStream); + } + } catch (Exception e) { + throw new IOException(e); + } + } + + private void openStreams() throws IOException { + ExecutorService service = Executors.newCachedThreadPool(new SolrjNamedThreadFactory("CloudSolrStream")); + List> futures = new ArrayList(); + for(TupleStream solrStream : solrStreams) { + StreamOpener so = new StreamOpener((SolrStream)solrStream, comp); + Future future = service.submit(so); + futures.add(future); + } + + try { + for(Future f : futures) { + TupleWrapper w = f.get(); + if(w != null) { + tuples.add(w); + } + } + } catch (Exception e) { + throw new IOException(e); + } + + service.shutdown(); + } + + public void close() throws IOException { + for(TupleStream solrStream : solrStreams) { + solrStream.close(); + } + + if(cache == null) { + cloudSolrClient.close(); + } + } + + public Tuple read() throws IOException { + return _read(); + } + + protected Tuple _read() throws IOException { + TupleWrapper tw = tuples.pollFirst(); + if(tw != null) { + Tuple t = tw.getTuple(); + if(tw.next()) { + tuples.add(tw); + } + return t; + } else { + Map m = new HashMap(); + m.put("EOF", true); + + return new Tuple(m); + } + } + + protected class TupleWrapper implements Comparable { + private Tuple tuple; + private SolrStream stream; + private Comparator comp; + + public TupleWrapper(SolrStream stream, Comparator comp) { + this.stream = stream; + this.comp = comp; + } + + public int compareTo(TupleWrapper w) { + if(this == w) { + return 0; + } + + int i = comp.compare(tuple, w.tuple); + if(i == 0) { + return 1; + } else { + return i; + } + } + + public boolean equals(Object o) { + return this == o; + } + + public Tuple getTuple() { + return tuple; + } + + public boolean next() throws IOException { + this.tuple = stream.read(); + + if(tuple.EOF) { + eofTuples.put(stream.getBaseUrl(), tuple); + } + + return !tuple.EOF; + } + } + + protected class StreamOpener implements Callable { + + private SolrStream stream; + private Comparator comp; + + public StreamOpener(SolrStream stream, Comparator comp) { + this.stream = stream; + this.comp = comp; + } + + public TupleWrapper call() throws Exception { + stream.open(); + TupleWrapper wrapper = new TupleWrapper(stream, comp); + if(wrapper.next()) { + return wrapper; + } else { + return null; + } + } + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/CountMetric.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/CountMetric.java new file mode 100644 index 00000000000..5ab4aaa2128 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/CountMetric.java @@ -0,0 +1,58 @@ +/* + * 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.Serializable; +import java.util.Map; +import java.util.HashMap; + +public class CountMetric implements Metric, Serializable { + + private static final long serialVersionUID = 1; + + public static final String COUNT = "count"; + private long count; + + public String getName() { + return "count"; + } + + public void update(Tuple tuple) { + ++count; + } + + public double getValue() { + return count; + } + + public Metric newInstance() { + return new CountMetric(); + } + + public Map metricValues() { + Map m = new HashMap(); + double d = (double)count; + m.put(COUNT, d); + return m; + } + + public void update(Map metricValues) { + double dcount = metricValues.get(COUNT); + count+=(long)dcount; + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/DescBucketComp.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/DescBucketComp.java new file mode 100644 index 00000000000..cf5dbfcccc9 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/DescBucketComp.java @@ -0,0 +1,42 @@ +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.Serializable; +import java.util.Comparator; + +public class DescBucketComp implements Comparator, Serializable { + + private int ord; + + public DescBucketComp(int ord) { + this.ord = ord; + } + + public int compare(BucketMetrics b1, BucketMetrics b2) { + double d1 = b1.getMetrics()[ord].getValue(); + double d2 = b2.getMetrics()[ord].getValue(); + if(d1 > d2) { + return -1; + } else if(d1 < d2) { + return 1; + } else { + return 0; + } + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/DescFieldComp.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/DescFieldComp.java new file mode 100644 index 00000000000..4c69181e745 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/DescFieldComp.java @@ -0,0 +1,43 @@ +/* + * 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.Serializable; +import java.util.Comparator; + +public class DescFieldComp implements Comparator, 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 i = o1.compareTo(o2); + if(i == 0) { + return i; + } else { + return -i; + } + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/DescMetricComp.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/DescMetricComp.java new file mode 100644 index 00000000000..2b514247553 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/DescMetricComp.java @@ -0,0 +1,39 @@ +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.Serializable; +import java.util.Comparator; +import java.util.List; + +public class DescMetricComp implements Comparator, Serializable { + + private static final long serialVersionUID = 1; + + private int ord; + + public DescMetricComp(int ord) { + this.ord = ord; + } + + public int compare(Tuple t1, Tuple t2) { + List values1 = (List)t1.get("metricValues"); + List values2 = (List)t2.get("metricValues"); + return values1.get(ord).compareTo(values2.get(ord))*-1; + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/FilterStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/FilterStream.java new file mode 100644 index 00000000000..ef26a0ccde4 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/FilterStream.java @@ -0,0 +1,101 @@ +/* + * 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; + +public class FilterStream extends TupleStream { + + private static final long serialVersionUID = 1; + + private TupleStream streamA; + private TupleStream streamB; + private Comparator comp; + private Tuple a = null; + private Tuple b = null; + + /* + * Intersects streamA by streamB based on a Comparator. + * Both streams must be sorted by the fields being compared. + * StreamB must be unique for the fields being compared. + **/ + + public FilterStream(TupleStream streamA, TupleStream streamB, Comparator comp) { + this.streamA = streamA; + this.streamB = streamB; + this.comp = comp; + } + + public void setStreamContext(StreamContext context) { + this.streamA.setStreamContext(context); + this.streamB.setStreamContext(context); + } + + public List children() { + List 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 { + a = streamA.read(); + + if(b == null) { + b = streamB.read(); + } + + while(true) { + if(a.EOF) { + return a; + } + + if(b.EOF) { + return b; + } + + int i = comp.compare(a, b); + if(i == 0) { + return a; + } else if(i < 0) { + // a < b so advance a + a = streamA.read(); + } else { + // a > b so advance b + b = streamB.read(); + } + } + } + + public int getCost() { + return 0; + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/GroupByStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/GroupByStream.java new file mode 100644 index 00000000000..1b84dff4990 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/GroupByStream.java @@ -0,0 +1,142 @@ +/* + * 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.Comparator; +import java.util.PriorityQueue; + +/** + * Iterates over a TupleStream Groups The TopN Tuples of a group. + **/ + +public class GroupByStream extends TupleStream { + + private static final long serialVersionUID = 1; + + private TupleStream tupleStream; + private Comparator interGroupComp; + private Comparator intraGroupComp; + private Comparator reverseComp; + private Tuple currentTuple; + private int size; + + public GroupByStream(TupleStream tupleStream, + Comparator interGroupComp, + Comparator intraGroupComp, + int size) { + this.tupleStream = tupleStream; + this.interGroupComp = interGroupComp; + this.intraGroupComp = intraGroupComp; + this.reverseComp = new ReverseComp(intraGroupComp); + this.size = size; + } + + public void setStreamContext(StreamContext context) { + this.tupleStream.setStreamContext(context); + } + + public List children() { + List l = new ArrayList(); + l.add(tupleStream); + return l; + } + + public void open() throws IOException { + tupleStream.open(); + currentTuple = tupleStream.read(); //Read the first Tuple so currentTuple is never null; + } + + public void close() throws IOException { + tupleStream.close(); + } + + public Tuple read() throws IOException { + + if(currentTuple.EOF) { + return currentTuple; + } + + PriorityQueue group = new PriorityQueue<>(size, reverseComp); + group.add(currentTuple); + while(true) { + Tuple t = tupleStream.read(); + + if(t.EOF) { + currentTuple = t; + break; + } + + if(interGroupComp.compare(currentTuple, t) == 0) { + if(group.size() >= size) { + Tuple peek = group.peek(); + if(intraGroupComp.compare(t, peek) < 0) { + group.poll(); + group.add(t); + } + } else { + group.add(t); + } + } else { + currentTuple = t; + break; + } + } + + //We have a finished group so add the Tuples to an array. + Tuple[] members = new Tuple[group.size()]; + for(int i=group.size()-1; i>=0; i--) { + Tuple t = group.poll(); + members[i] = t; + } + + //First Tuple is the group head. + Tuple groupHead = members[0]; + if(members.length > 1) { + List groupList = new ArrayList(); + for(int i=1; i, Serializable { + + private Comparator comp; + + public ReverseComp(Comparator comp) { + this.comp = comp; + } + + public int compare(Tuple t1, Tuple t2) { + return comp.compare(t1, t2)*(-1); + } + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/HashJoinStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/HashJoinStream.java new file mode 100644 index 00000000000..f89b164ea18 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/HashJoinStream.java @@ -0,0 +1,121 @@ +/* + * 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.List; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedList; + +/** + * + * + **/ + +public class HashJoinStream extends TupleStream { + + private static final long serialVersionUID = 1; + + private PushBackStream streamA; + private TupleStream streamB; + private String[] keys; + private HashMap> hashMap = new HashMap(); + + public HashJoinStream(TupleStream streamA, TupleStream streamB, String[] keys) { + this.streamA = new PushBackStream(streamA); + this.streamB = streamB; + this.keys = keys; + } + + public void setStreamContext(StreamContext context) { + this.streamA.setStreamContext(context); + this.streamB.setStreamContext(context); + } + + public List children() { + List l = new ArrayList(); + l.add(streamA); + l.add(streamB); + return l; + } + + public void open() throws IOException { + streamB.open(); + while(true) { + Tuple t = streamB.read(); + if(t.EOF) { + break; + } + + HashKey hashKey = new HashKey(t, keys); + if(hashMap.containsKey(hashKey)) { + List tuples = hashMap.get(hashKey); + tuples.add(t); + } else { + List tuples = new ArrayList(); + tuples.add(t); + hashMap.put(hashKey, tuples); + } + } + + streamB.close(); + streamA.open(); + } + + public void close() throws IOException { + streamA.close(); + } + + private LinkedList joinTuples = new LinkedList(); + + public Tuple read() throws IOException { + while(true) { + Tuple tuple = streamA.read(); + + if(tuple.EOF) { + return tuple; + } + + if(joinTuples.size() > 0) { + Tuple t = tuple.clone(); + Tuple j = joinTuples.removeFirst(); + t.fields.putAll(j.fields); + if(joinTuples.size() > 0) { + streamA.pushBack(tuple); + } + + return t; + } else { + HashKey hashKey = new HashKey(tuple, keys); + + if(hashMap.containsKey(hashKey)) { + List joinWith = hashMap.get(hashKey); + for(Tuple jt : joinWith) { + joinTuples.add(jt); + } + streamA.pushBack(tuple); + } + } + } + } + + public int getCost() { + return 0; + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/HashKey.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/HashKey.java new file mode 100644 index 00000000000..e475edbda9b --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/HashKey.java @@ -0,0 +1,79 @@ +/* + * 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.Serializable; + +public class HashKey implements Serializable { + + private static final long serialVersionUID = 1; + + private Object[] parts; + + + public HashKey(String value) { + parts = (Object[])value.split("::"); + } + + public HashKey(Tuple t, String[] keys) { + this.parts = new Object[keys.length]; + for(int i=0; i 0) { + buf.append("::"); + } + buf.append(parts[i].toString()); + } + + return buf.toString(); + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/JSONTupleStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/JSONTupleStream.java new file mode 100644 index 00000000000..0ffd8f2454a --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/JSONTupleStream.java @@ -0,0 +1,165 @@ +package org.apache.solr.client.solrj.io; + +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.Reader; +import java.util.List; +import java.util.Map; + +import org.apache.solr.client.solrj.SolrClient; +import org.apache.solr.client.solrj.SolrRequest; +import org.apache.solr.client.solrj.SolrServerException; +import org.apache.solr.client.solrj.impl.InputStreamResponseParser; +import org.apache.solr.client.solrj.request.QueryRequest; +import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.common.params.SolrParams; +import org.apache.solr.common.util.NamedList; +import org.noggit.JSONParser; +import org.noggit.ObjectBuilder; + + +/* + * 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. + */ + +/* + Queries a Solr instance, and maps SolrDocs to Tuples. + Initial version works with the json format and only SolrDocs are handled. +*/ + +public class JSONTupleStream { + private List path; // future... for more general stream handling + private Reader reader; + private JSONParser parser; + private boolean atDocs; + + public JSONTupleStream(Reader reader) { + this.reader = reader; + this.parser = new JSONParser(reader); + } + + // temporary... + public static JSONTupleStream create(SolrClient server, SolrParams requestParams) throws IOException, SolrServerException { + String p = requestParams.get("qt"); + if(p != null) { + ModifiableSolrParams modifiableSolrParams = (ModifiableSolrParams) requestParams; + modifiableSolrParams.remove("qt"); + } + + QueryRequest query = new QueryRequest( requestParams ); + query.setPath(p); + query.setResponseParser(new InputStreamResponseParser("json")); + query.setMethod(SolrRequest.METHOD.POST); + NamedList genericResponse = server.request(query); + InputStream stream = (InputStream)genericResponse.get("stream"); + InputStreamReader reader = new InputStreamReader(stream, "UTF-8"); + return new JSONTupleStream(reader); + } + + + /** returns the next Tuple or null */ + public Map next() throws IOException { + if (!atDocs) { + boolean found = advanceToDocs(); + atDocs = true; + if (!found) return null; + } + // advance past ARRAY_START (in the case that we just advanced to docs, or OBJECT_END left over from the last call. + int event = parser.nextEvent(); + if (event == JSONParser.ARRAY_END) return null; + + Object o = ObjectBuilder.getVal(parser); + // right now, getVal will leave the last event read as OBJECT_END + + return (Map)o; + } + + public void close() throws IOException { + reader.close(); + } + + + private void expect(int parserEventType) throws IOException { + int event = parser.nextEvent(); + if (event != parserEventType) { + throw new IOException("JSONTupleStream: expected " + JSONParser.getEventString(parserEventType) + " but got " + JSONParser.getEventString(event) ); + } + } + + private void expect(String mapKey) { + + + } + + private boolean advanceToMapKey(String key, boolean deepSearch) throws IOException { + for (;;) { + int event = parser.nextEvent(); + switch (event) { + case JSONParser.STRING: + if (key != null) { + String val = parser.getString(); + if (key.equals(val)) { + return true; + } + } + break; + case JSONParser.OBJECT_END: + return false; + case JSONParser.OBJECT_START: + if (deepSearch) { + boolean found = advanceToMapKey(key, true); + if (found) { + return true; + } + } else { + advanceToMapKey(null, false); + } + break; + case JSONParser.ARRAY_START: + skipArray(key, deepSearch); + break; + } + } + } + + private void skipArray(String key, boolean deepSearch) throws IOException { + for (;;) { + int event = parser.nextEvent(); + switch (event) { + case JSONParser.OBJECT_START: + advanceToMapKey(key, deepSearch); + break; + case JSONParser.ARRAY_START: + skipArray(key, deepSearch); + break; + case JSONParser.ARRAY_END: + return; + } + } + } + + + private boolean advanceToDocs() throws IOException { + expect(JSONParser.OBJECT_START); + boolean found = advanceToMapKey("docs", true); + expect(JSONParser.ARRAY_START); + return found; + } + + + +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/MaxMetric.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/MaxMetric.java new file mode 100644 index 00000000000..800b14a9fc1 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/MaxMetric.java @@ -0,0 +1,92 @@ +/* + * 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.Serializable; +import java.util.Map; +import java.util.HashMap; + +public class MaxMetric implements Metric, Serializable { + + public static final String MAX = "max"; + private long longMax = -Long.MAX_VALUE; + private double doubleMax = Double.MAX_VALUE; + private boolean isDouble; + private String column; + + public MaxMetric(String column, boolean isDouble) { + this.column = column; + this.isDouble = isDouble; + } + + public String getName() { + return "mix:"+column; + } + + public double getValue() { + if(isDouble) { + return doubleMax; + } else { + return longMax; + } + } + + public void update(Tuple tuple) { + if(isDouble) { + double d = (double)tuple.get(column); + if(d > doubleMax) { + doubleMax = d; + } + } else { + long l = (long)tuple.get(column); + if(l > longMax) { + longMax = l; + } + } + } + + public Metric newInstance() { + return new MaxMetric(column, isDouble); + } + + public Map metricValues() { + Map m = new HashMap(); + if(isDouble) { + m.put(MAX,doubleMax); + } else { + doubleMax = (double)longMax; + m.put(MAX,doubleMax); + } + return m; + } + + public void update(Map metricValues) { + if(isDouble) { + double dmax = metricValues.get(MAX); + if(dmax > doubleMax) { + doubleMax = dmax; + } + } else { + double dmax = metricValues.get(MAX); + long lmax = (long) dmax; + if(lmax > longMax) { + longMax = lmax; + } + } + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/MeanMetric.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/MeanMetric.java new file mode 100644 index 00000000000..b3372ac103a --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/MeanMetric.java @@ -0,0 +1,104 @@ +/* + * 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.Serializable; +import java.util.Map; +import java.util.HashMap; + +public class MeanMetric implements Metric, Serializable { + + private static final long serialVersionUID = 1; + + public static final String SUM = "sum"; + public static final String COUNT = "count"; + public static final String MEAN = "mean"; + + private String column; + private boolean isDouble; + private double doubleSum; + private long longSum; + private long count; + + public MeanMetric(String column, boolean isDouble) { + this.column = column; + this.isDouble = isDouble; + } + + public String getName() { + return "mean:"+column; + } + + public void update(Tuple tuple) { + ++count; + if(isDouble) { + Double d = (Double)tuple.get(column); + doubleSum += d.doubleValue(); + } else { + Long l = (Long)tuple.get(column); + longSum += l.doubleValue(); + } + } + + public Metric newInstance() { + return new MeanMetric(column, isDouble); + } + + public double getValue() { + double dcount = (double)count; + if(isDouble) { + double ave = doubleSum/dcount; + return ave; + + } else { + double ave = longSum/dcount; + return ave; + } + } + + public Map metricValues() { + Map m = new HashMap(); + double dcount = (double)count; + m.put(COUNT, dcount); + if(isDouble) { + double ave = doubleSum/dcount; + m.put(MEAN,ave); + m.put(SUM,doubleSum); + + } else { + double ave = longSum/dcount; + doubleSum = (double)longSum; + m.put(MEAN,ave); + m.put(SUM,doubleSum); + } + + return m; + } + + public void update(Map metricValues) { + double dcount = metricValues.get(COUNT); + count += (long)dcount; + if(isDouble) { + double dsum = metricValues.get(SUM); + doubleSum+=dsum; + } else { + double dsum = metricValues.get(SUM); + longSum+=(long)dsum; + } + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/MergeJoinStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/MergeJoinStream.java new file mode 100644 index 00000000000..5fb07f86c54 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/MergeJoinStream.java @@ -0,0 +1,162 @@ +/* + * 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.List; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.Comparator; + + +/** + * Merge Joins streamA with streamB based on the Comparator. + * Supports: + * one-to-one, one-to-many, many-to-one and many-to-many joins + * + **/ + +public class MergeJoinStream extends TupleStream { + + private static final long serialVersionUID = 1; + + private PushBackStream streamA; + private PushBackStream streamB; + private Comparator comp; + + public MergeJoinStream(TupleStream streamA, TupleStream streamB, Comparator 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 children() { + List 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(); + } + + private LinkedList joinTuples = new LinkedList(); + private List listA = new ArrayList(); + private List listB = new ArrayList(); + + public Tuple read() throws IOException { + + if(joinTuples.size() > 0) { + return joinTuples.removeFirst(); + } + + OUTER: + while(true) { + if(listA.size() == 0) { + //Stream A needs to be advanced. + Tuple a = streamA.read(); + if(a.EOF) { + return a; + } + + listA.add(a); + INNERA: + while(true) { + Tuple a1 = streamA.read(); + if(a1.EOF) { + streamA.pushBack(a1); + break INNERA; + } + + if(comp.compare(a,a1) == 0) { + listA.add(a1); + } else { + streamA.pushBack(a1); + break INNERA; + } + } + } + + if(listB.size() == 0) { + //StreamB needs to be advanced. + Tuple b = streamB.read(); + if(b.EOF) { + return b; + } + + listB.add(b); + INNERA: + while(true) { + Tuple b1 = streamB.read(); + + if(b1.EOF) { + streamB.pushBack(b1); + break INNERA; + } + + if(comp.compare(b,b1) == 0) { + listB.add(b1); + } else { + streamB.pushBack(b1); + break INNERA; + } + } + } + + int c = comp.compare(listA.get(0),listB.get(0)); + if(c == 0) { + //The Tuple lists match. So build all the Tuple combinations. + for(Tuple aa : listA) { + for(Tuple bb : listB) { + Tuple clone = aa.clone(); + clone.fields.putAll(bb.fields); + joinTuples.add(clone); + } + } + + //This will advance both streams. + listA.clear(); + listB.clear(); + + return joinTuples.removeFirst(); + } else if(c < 0) { + //This will advance streamA + listA.clear(); + } else { + //This will advance streamB + listB.clear(); + } + } + } + + public int getCost() { + return 0; + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/MergeStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/MergeStream.java new file mode 100644 index 00000000000..604c63c36d0 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/MergeStream.java @@ -0,0 +1,99 @@ +/* + * 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 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 comp; + + public MergeStream(TupleStream streamA, TupleStream streamB, Comparator 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 children() { + List 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; + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Metric.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Metric.java new file mode 100644 index 00000000000..cce08090a13 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Metric.java @@ -0,0 +1,30 @@ +/* + * 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.Serializable; +import java.util.Map; + +public interface Metric extends Serializable { + public String getName(); + public double getValue(); + public void update(Tuple tuple); + public Metric newInstance(); + public Map metricValues(); + public void update(Map metricValues); +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/MetricStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/MetricStream.java new file mode 100644 index 00000000000..e98c7d3d6d9 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/MetricStream.java @@ -0,0 +1,280 @@ +/* + * 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.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.ArrayList; +import java.util.PriorityQueue; + +/* +String[] buckets = {"a","b"}; +Metric +BucketStream bucketStream = new BucketStream(stream,buckets,metrics,"my-metrics","name"); + +bucketStream.get( +*/ + +public class MetricStream extends TupleStream { + + private static final long serialVersionUID = 1; + + private TupleStream tupleStream; + private Bucket[] buckets; + private Metric[] metrics; + private String outKey; + private Map bucketMap; + private BucketMetrics[] bucketMetrics; + private static final HashKey metricsKey = new HashKey("metrics"); + private int topN; + private Comparator comp; + private Comparator rcomp; + + public MetricStream(TupleStream tupleStream, + Bucket[] buckets, + Metric[] metrics, + String outKey, + Comparator comp, + int topN) { + this.tupleStream = tupleStream; + this.buckets = buckets; + this.metrics = metrics; + this.outKey = outKey; + this.topN = topN; + this.rcomp = new ReverseOrdComp(comp); + this.comp = comp; + this.bucketMap = new HashMap(); + } + + public MetricStream(TupleStream tupleStream, + Metric[] metrics, + String outKey) { + this.tupleStream = tupleStream; + this.metrics = metrics; + this.outKey = outKey; + this.bucketMap = new HashMap(); + } + + public String getOutKey() { + return this.outKey; + } + + public BucketMetrics[] getBucketMetrics(){ + return bucketMetrics; + } + + public void setBucketMetrics(BucketMetrics[] bucketMetrics) { + this.bucketMetrics = bucketMetrics; + } + + BucketMetrics[] merge(List all) { + Map bucketAccumulator = new HashMap(); + + for(Map top : all) { + List ks = (List)top.get("buckets"); + List>> ms = (List>>)top.get("metrics"); + for(int i=0; i> bucketMs = ms.get(i); + + HashKey hashKey = new HashKey(key); + if(bucketAccumulator.containsKey(hashKey)) { + Metric[] mergeMetrics = bucketAccumulator.get(hashKey); + for(int m=0; m> it = bucketAccumulator.entrySet().iterator(); + + PriorityQueue priorityQueue = new PriorityQueue(topN, rcomp); + + while(it.hasNext()) { + Map.Entry entry = it.next(); + BucketMetrics bms = new BucketMetrics(entry.getKey(), entry.getValue()); + if(priorityQueue.size() < topN) { + priorityQueue.add(bms); + } else { + BucketMetrics peek = priorityQueue.peek(); + if(comp.compare(bms, peek) < 0) { + priorityQueue.poll(); + priorityQueue.add(bms); + } + } + } + + int s = priorityQueue.size(); + BucketMetrics[] bucketMetrics = new BucketMetrics[s]; + + for(int i=bucketMetrics.length-1; i>=0; i--) { + BucketMetrics b = priorityQueue.poll(); + bucketMetrics[i]= b; + } + return bucketMetrics; + } + + private class ReverseOrdComp implements Comparator, Serializable { + private Comparator comp; + + public ReverseOrdComp(Comparator comp) { + this.comp = comp; + } + + public int compare(BucketMetrics e1, BucketMetrics e2) { + return comp.compare(e1,e2)*-1; + } + } + + public void setStreamContext(StreamContext context) { + this.tupleStream.setStreamContext(context); + } + + public List children() { + List 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) { + Iterator> it = bucketMap.entrySet().iterator(); + + if(comp == null) { + //Handle No bucket constructor + Map.Entry noBucket = it.next(); + BucketMetrics bms = new BucketMetrics(noBucket.getKey(), noBucket.getValue()); + this.bucketMetrics = new BucketMetrics[1]; + this.bucketMetrics[0] = bms; + List> outMetrics = new ArrayList(); + List outKeys = new ArrayList(); + for(Metric metric : bms.getMetrics()) { + Map outMetricValues = metric.metricValues(); + String outKey = metric.getName(); + outMetrics.add(outMetricValues); + outKeys.add(outKey); + } + Map outMap = new HashMap(); + outMap.put("buckets",outKeys); + outMap.put("metrics",outMetrics); + tuple.set(this.outKey, outMap); + return tuple; + } + + PriorityQueue priorityQueue = new PriorityQueue(topN, rcomp); + + while(it.hasNext()) { + Map.Entry entry = it.next(); + BucketMetrics bms = new BucketMetrics(entry.getKey(), entry.getValue()); + if(priorityQueue.size() < topN) { + priorityQueue.add(bms); + } else { + BucketMetrics peek = priorityQueue.peek(); + + if(comp.compare(bms, peek) < 0) { + priorityQueue.poll(); + priorityQueue.add(bms); + } + + } + } + + int s = priorityQueue.size(); + this.bucketMetrics = new BucketMetrics[s]; + + for(int i=bucketMetrics.length-1; i>=0; i--) { + BucketMetrics b = priorityQueue.poll(); + this.bucketMetrics[i]= b; + } + + List>> outMetrics = new ArrayList(); + List outBuckets = new ArrayList(); + + for(BucketMetrics bms : this.bucketMetrics) { + List outBucketMetrics = new ArrayList(); + for(Metric metric : bms.getMetrics()) { + Map outMetricValues = metric.metricValues(); + outBucketMetrics.add(outMetricValues); + } + outBuckets.add(bms.getKey().toString()); + outMetrics.add(outBucketMetrics); + } + + Map outMap = new HashMap(); + outMap.put("buckets",outBuckets); + outMap.put("metrics",outMetrics); + tuple.set(this.outKey, outMap); + return tuple; + } + + HashKey hashKey = null; + if(buckets != null) { + String[] bucketValues = new String[buckets.length]; + for(int i=0; i metricValues() { + Map m = new HashMap(); + if(isDouble) { + m.put(MIN,doubleMin); + } else { + doubleMin = (double)longMin; + m.put(MIN,doubleMin); + } + return m; + } + + public void update(Map metricValues) { + if(isDouble) { + double dmin = metricValues.get(MIN); + if(dmin < doubleMin) { + doubleMin = dmin; + } + } else { + double dmin = metricValues.get(MIN); + long lmin = (long) dmin; + if(lmin < longMin) { + longMin = lmin; + } + } + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/MultiComp.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/MultiComp.java new file mode 100644 index 00000000000..45808da9077 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/MultiComp.java @@ -0,0 +1,43 @@ +/* + * 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.Serializable; +import java.util.Comparator; + +public class MultiComp implements Comparator, Serializable { + + private static final long serialVersionUID = 1; + + private Comparator[] comps; + + public MultiComp(Comparator... comps) { + this.comps = comps; + } + + public int compare(Tuple t1, Tuple t2) { + for(Comparator comp : comps) { + int i = comp.compare(t1, t2); + if(i != 0) { + return i; + } + } + + return 0; + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ParallelStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ParallelStream.java new file mode 100644 index 00000000000..02b5523e4ee --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ParallelStream.java @@ -0,0 +1,160 @@ +/* + * 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 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 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 children() { + List l = new ArrayList(); + l.add(tupleStream); + return l; + } + + public void merge(List metricStreams) { + for(MetricStream metricStream : metricStreams) { + String outKey = metricStream.getOutKey(); + Iterator it = eofTuples.values().iterator(); + List values = new ArrayList(); + + while(it.hasNext()) { + Tuple t = it.next(); + Map top = (Map)t.get(outKey); + values.add(top); + } + + BucketMetrics[] bucketMetrics = metricStream.merge(values); + metricStream.setBucketMetrics(bucketMetrics); + } + } + + public Tuple read() throws IOException { + Tuple tuple = _read(); + + if(tuple.EOF) { + List metricStreams = new ArrayList(); + getMetricStreams(this, metricStreams); + this.merge(metricStreams); + Map m = new HashMap(); + m.put("EOF", true); + return new Tuple(m); + } + + return tuple; + } + + private void getMetricStreams(TupleStream tupleStream, + List metricStreams) { + if(tupleStream instanceof MetricStream) { + metricStreams.add((MetricStream)tupleStream); + } + + List children = tupleStream.children(); + for(TupleStream ts : children) { + getMetricStreams(ts, metricStreams); + } + } + + protected void constructStreams() throws IOException { + + try { + ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader(); + ClusterState clusterState = zkStateReader.getClusterState(); + Collection slices = clusterState.getActiveSlices(this.collection); + long time = System.currentTimeMillis(); + int workerNum = 0; + for(Slice slice : slices) { + HashMap params = new HashMap(); + + params.put("distrib","false"); // We are the aggregator. + params.put("numWorkers", workers); + params.put("workerID", workerNum); + params.put("stream", this.encoded); + params.put("qt","/stream"); + + Collection replicas = slice.getReplicas(); + List shuffler = new ArrayList(); + for(Replica replica : replicas) { + shuffler.add(replica); + } + + Collections.shuffle(shuffler, new Random(time)); + Replica rep = shuffler.get(0); + ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep); + String url = zkProps.getCoreUrl(); + SolrStream solrStream = new SolrStream(url, params); + solrStreams.add(solrStream); + ++workerNum; + } + } catch (Exception e) { + throw new IOException(e); + } + } +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/PushBackStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/PushBackStream.java new file mode 100644 index 00000000000..cb15eac01c5 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/PushBackStream.java @@ -0,0 +1,77 @@ +/* + * 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; + +public class PushBackStream extends TupleStream { + + private static final long serialVersionUID = 1; + + private TupleStream stream; + private Tuple tuple; + + /* + * A TupleStream that allows a single Tuple to be pushed back into Stream after it's been read. + * + **/ + + public PushBackStream(TupleStream stream) { + this.stream = stream; + + } + + public void setStreamContext(StreamContext context) { + this.stream.setStreamContext(context); + } + + public List children() { + List l = new ArrayList(); + l.add(stream); + return l; + } + + public void open() throws IOException { + stream.open(); + } + + public void close() throws IOException { + stream.close(); + } + + public void pushBack(Tuple tuple) { + this.tuple = tuple; + } + + public Tuple read() throws IOException { + if(tuple != null) { + Tuple t = tuple; + tuple = null; + return t; + } else { + return stream.read(); + } + } + + public int getCost() { + return 0; + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/RankStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/RankStream.java new file mode 100644 index 00000000000..fc778a5382b --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/RankStream.java @@ -0,0 +1,116 @@ +/* + * 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 top; + private Comparator comp; + private boolean finished = false; + private LinkedList topList; + private int size; + + public RankStream(TupleStream tupleStream, int size, Comparator 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 children() { + List 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= 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, Serializable { + + private Comparator comp; + + public ReverseComp(Comparator comp) { + this.comp = comp; + } + + public int compare(Tuple t1, Tuple t2) { + return comp.compare(t1, t2)*(-1); + } + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/RollupStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/RollupStream.java new file mode 100644 index 00000000000..7afd4aa77c3 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/RollupStream.java @@ -0,0 +1,138 @@ +/* + * 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.HashMap; +import java.util.List; +import java.util.Map; +import java.util.ArrayList; + +public class RollupStream extends TupleStream { + + private static final long serialVersionUID = 1; + + private PushBackStream tupleStream; + private Bucket[] buckets; + private Metric[] metrics; + private HashKey currentKey = new HashKey("-"); + private Metric[] currentMetrics; + private boolean finished = false; + + public RollupStream(TupleStream tupleStream, + Bucket[] buckets, + Metric[] metrics) { + this.tupleStream = new PushBackStream(tupleStream); + this.buckets = buckets; + this.metrics = metrics; + } + + public void setStreamContext(StreamContext context) { + this.tupleStream.setStreamContext(context); + } + + public List children() { + List 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 { + + while(true) { + Tuple tuple = tupleStream.read(); + if(tuple.EOF) { + if(!finished) { + Map map = new HashMap(); + if(currentMetrics != null) { + List metricValues = new ArrayList(); + List metricNames = new ArrayList(); + for (Metric metric : currentMetrics) { + metricNames.add(metric.getName()); + metricValues.add(metric.getValue()); + } + map.put("buckets", currentKey.toString()); + map.put("metricNames", metricNames); + map.put("metricValues", metricValues); + Tuple t = new Tuple(map); + tupleStream.pushBack(tuple); + finished = true; + return t; + } else { + return tuple; + } + } else { + return tuple; + } + } + + String[] bucketValues = new String[buckets.length]; + for(int i=0; i metricValues = new ArrayList(); + List metricNames = new ArrayList(); + for(Metric metric : currentMetrics) { + metricNames.add(metric.getName()); + metricValues.add(metric.getValue()); + } + map.put("buckets", currentKey.toString()); + map.put("metricNames", metricNames); + map.put("metricValues", metricValues); + t = new Tuple(map); + } + + currentMetrics = new Metric[metrics.length]; + currentKey = hashKey; + for(int i=0; i solrClients = new HashMap(); + + public synchronized CloudSolrClient getCloudSolrClient(String zkHost) { + CloudSolrClient client = null; + if (solrClients.containsKey(zkHost)) { + client = (CloudSolrClient) solrClients.get(zkHost); + } else { + client = new CloudSolrClient(zkHost); + client.connect(); + solrClients.put(zkHost, client); + } + + return client; + } + + public synchronized HttpSolrClient getHttpSolrClient(String host) { + HttpSolrClient client = null; + if (solrClients.containsKey(host)) { + client = (HttpSolrClient) solrClients.get(host); + } else { + client = new HttpSolrClient(host); + solrClients.put(host, client); + } + return client; + } + + public void close() { + Iterator it = solrClients.values().iterator(); + while(it.hasNext()) { + try { + it.next().close(); + } catch (IOException e) { + log.error(e.getMessage(), e); + } + } + } +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/SolrStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/SolrStream.java new file mode 100644 index 00000000000..e4a176d372d --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/SolrStream.java @@ -0,0 +1,159 @@ +/* + * 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.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.List; +import java.util.Iterator; + +import org.apache.solr.client.solrj.impl.HttpSolrClient; +import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.common.params.SolrParams; + +/** +* Queries a Solr instance, and maps SolrDocs to a Stream of Tuples. +**/ + +public class SolrStream extends TupleStream { + + private static final long serialVersionUID = 1; + + private String baseUrl; + private Map params; + private int numWorkers; + private int workerID; + private Map fieldMappings; + private transient JSONTupleStream jsonTupleStream; + private transient HttpSolrClient client; + private transient SolrClientCache cache; + + public SolrStream(String baseUrl, Map params) { + this.baseUrl = baseUrl; + this.params = params; + } + + public void setFieldMappings(Map fieldMappings) { + this.fieldMappings = fieldMappings; + } + + public List children() { + return new ArrayList(); + } + + public String getBaseUrl() { + return baseUrl; + } + + public void setStreamContext(StreamContext context) { + this.numWorkers = context.numWorkers; + this.workerID = context.workerID; + this.cache = context.clientCache; + } + + /** + * Opens the stream to a single Solr instance. + **/ + + public void open() throws IOException { + + if(cache == null) { + client = new HttpSolrClient(baseUrl); + } else { + client = cache.getHttpSolrClient(baseUrl); + } + + try { + jsonTupleStream = JSONTupleStream.create(client, loadParams(params)); + } catch (Exception e) { + throw new IOException(e); + } + } + + private SolrParams loadParams(Map params) { + ModifiableSolrParams solrParams = new ModifiableSolrParams(); + if(this.numWorkers > 0) { + String partitionFilter = getPartitionFilter(); + solrParams.add("fq", partitionFilter); + } + + Iterator it = params.entrySet().iterator(); + while(it.hasNext()) { + Map.Entry entry = it.next(); + solrParams.add((String)entry.getKey(), entry.getValue().toString()); + } + + return solrParams; + } + + private String getPartitionFilter() { + StringBuilder buf = new StringBuilder("{!hash workers="); + buf.append(this.numWorkers); + buf.append(" worker="); + buf.append(this.workerID); + buf.append("}"); + return buf.toString(); + } + + /** + * Closes the Stream to a single Solr Instance + * */ + + public void close() throws IOException { + jsonTupleStream.close(); + if(cache == null) { + client.close(); + } + } + + /** + * Reads a Tuple from the stream. The Stream is completed when Tuple.EOF == true. + **/ + + public Tuple read() throws IOException { + Map fields = jsonTupleStream.next(); + if(fields == null) { + //Return the EOF tuple. + Map m = new HashMap(); + m.put("EOF", true); + return new Tuple(m); + } else { + if(fieldMappings != null) { + fields = mapFields(fields, fieldMappings); + } + return new Tuple(fields); + } + } + + private Map mapFields(Map fields, Map mappings) { + + Iterator> it = mappings.entrySet().iterator(); + while(it.hasNext()) { + Map.Entry entry = it.next(); + String mapFrom = entry.getKey(); + String mapTo = entry.getValue(); + Object o = fields.get(mapFrom); + fields.remove(mapFrom); + fields.put(mapTo, o); + } + + return fields; + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/StreamContext.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/StreamContext.java new file mode 100644 index 00000000000..a138d5a2cc8 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/StreamContext.java @@ -0,0 +1,46 @@ +/* + * 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.util.Map; +import java.util.HashMap; +import java.util.concurrent.ConcurrentHashMap; + +public class StreamContext { + + private Map entries = new HashMap(); + public int workerID; + public int numWorkers; + public SolrClientCache clientCache; + + public SolrClientCache getClientCache() { + return this.clientCache; + } + + public void setSolrClientCache(SolrClientCache clientCache) { + this.clientCache = clientCache; + } + + public Object get(Object key) { + return entries.get(key); + } + + public void put(Object key, Object value) { + this.entries.put(key, value); + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/SumMetric.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/SumMetric.java new file mode 100644 index 00000000000..af4e55ff5e3 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/SumMetric.java @@ -0,0 +1,88 @@ +/* + * 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.Serializable; +import java.util.Map; +import java.util.HashMap; + +public class SumMetric implements Metric, Serializable { + + private static final long serialVersionUID = 1; + + public static final String SUM = "sum"; + + private String column; + private boolean isDouble; + private double doubleSum; + private long longSum; + + public SumMetric(String column, boolean isDouble) { + this.column = column; + this.isDouble = isDouble; + } + + public String getName() { + return "sum:"+column; + } + + public void update(Tuple tuple) { + if(isDouble) { + Double d = (Double)tuple.get(column); + doubleSum += d.doubleValue(); + } else { + Long l = (Long)tuple.get(column); + longSum += l.doubleValue(); + } + } + + public Metric newInstance() { + return new SumMetric(column, isDouble); + } + + public Map metricValues() { + Map m = new HashMap(); + if(isDouble) { + m.put(SUM,doubleSum); + + } else { + doubleSum = (double)longSum; + m.put(SUM,doubleSum); + } + + return m; + } + + public double getValue() { + if(isDouble) { + return doubleSum; + } else { + return (double)longSum; + } + } + + public void update(Map metricValues) { + if(isDouble) { + double dsum = metricValues.get(SUM); + doubleSum+=dsum; + } else { + double dsum = metricValues.get(SUM); + longSum+=(long)dsum; + } + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Tuple.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Tuple.java new file mode 100644 index 00000000000..77e9efdc52f --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Tuple.java @@ -0,0 +1,93 @@ +/* + * 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.util.HashMap; +import java.util.Map; +import java.util.List; +import java.util.Iterator; + + +/** + * A simple abstraction of a record containing key/value pairs. + * Convenience methods are provided for returning single and multiValue String, Long and Double values. + * Note that ints and floats are treated as longs and doubles respectively. + * +**/ + +public class Tuple implements Cloneable { + + public boolean EOF; + public Map fields = new HashMap(); + + public Tuple(Map fields) { + if(fields.containsKey("EOF")) { + EOF = true; + } + + this.fields.putAll(fields); + } + + public Object get(Object key) { + return this.fields.get(key); + } + + public void set(Object key, Object value) { + this.fields.put(key, value); + } + + public String getString(Object key) { + return (String)this.fields.get(key); + } + + public Long getLong(Object key) { + return (Long)this.fields.get(key); + } + + public Double getDouble(Object key) { + return (Double)this.fields.get(key); + } + + + + public List getStrings(Object key) { + return (List)this.fields.get(key); + } + + + + public List getLongs(Object key) { + return (List)this.fields.get(key); + } + + + public List getDoubles(Object key) { + return (List)this.fields.get(key); + } + + public Iterator getFields() { + return fields.entrySet().iterator(); + } + + public Tuple clone() { + HashMap m = new HashMap(); + m.putAll(fields); + Tuple clone = new Tuple(m); + return clone; + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/TupleStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/TupleStream.java new file mode 100644 index 00000000000..36888e26660 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/TupleStream.java @@ -0,0 +1,51 @@ +/* + * 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.Map; +import java.util.HashMap; +import org.apache.solr.client.solrj.SolrClient; +import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.impl.HttpSolrClient; + + +public abstract class TupleStream implements Serializable { + + private static final long serialVersionUID = 1; + + public TupleStream() { + + } + + public abstract void setStreamContext(StreamContext context); + + public abstract List children(); + + public abstract void open() throws IOException; + + public abstract void close() throws IOException; + + public abstract Tuple read() throws IOException; + + public int getCost() { + return 0; + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/UniqueStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/UniqueStream.java new file mode 100644 index 00000000000..883b46809a3 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/UniqueStream.java @@ -0,0 +1,86 @@ +/* + * 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; + +public class UniqueStream extends TupleStream { + + private static final long serialVersionUID = 1; + + private TupleStream tupleStream; + private Comparator comp; + private Tuple currentTuple = null; + + public UniqueStream(TupleStream tupleStream, Comparator comp) { + this.tupleStream = tupleStream; + this.comp = comp; + } + + public void setStreamContext(StreamContext context) { + this.tupleStream.setStreamContext(context); + } + + public List children() { + List 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; + } +} \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/package-info.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/package-info.java new file mode 100644 index 00000000000..5be9c12a182 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/package-info.java @@ -0,0 +1,24 @@ +/* + * 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. + */ + + + +/** + * Streaming Aggregation API + **/ +package org.apache.solr.client.solrj.io; + diff --git a/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-streaming.xml b/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-streaming.xml new file mode 100644 index 00000000000..216fa2ce85b --- /dev/null +++ b/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-streaming.xml @@ -0,0 +1,599 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + text + id + + + + + + + + + + + + + + + + + + + + + diff --git a/solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-streaming.xml b/solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-streaming.xml new file mode 100644 index 00000000000..ecd0f48bdfc --- /dev/null +++ b/solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-streaming.xml @@ -0,0 +1,83 @@ + + + + + + ${tests.luceneMatchVersion:LUCENE_CURRENT} + + ${useCompoundFile:false} + + ${solr.data.dir:} + + + + + ${solr.data.dir:} + + + + + + + true + + + + + + + + json + false + + + + + + + + + + + + + + + + *:* + + + all + + server-enabled.txt + + + + + solr + + + + diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/StreamingTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/StreamingTest.java new file mode 100644 index 00000000000..6b09fe9bdf8 --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/StreamingTest.java @@ -0,0 +1,1205 @@ +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.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.HashMap; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.LuceneTestCase.Slow; +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; + +import java.util.List; +import java.util.ArrayList; + +/** + * 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 StreamingTest 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 StreamingTest() { + super(); + sliceCount = 2; + } + + private void testUniqueStream() throws Exception { + + //Test CloudSolrStream and UniqueStream + + 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(); + + + String zkHost = zkServer.getZkAddress(); + + 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")); + List tuples = getTuples(ustream); + assert(tuples.size() == 4); + assertOrder(tuples, 0,1,3,4); + + del("*:*"); + commit(); + + } + + private void testHashJoinStream() 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", "join_i", "1000"); + 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, "6", "a_s", "hello1", "a_i", "1", "a_f", "0", "join_i", "2000"); + indexr(id, "7", "a_s", "hello7", "a_i", "1", "a_f", "0"); + + commit(); + + String zkHost = zkServer.getZkAddress(); + + //Test one-to-one + Map paramsA = mapParams("q","id:(0 1 3 4) ","fl","id,a_s,a_f", "sort", "a_s desc"); + CloudSolrStream streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + Map fieldMappings = new HashMap(); + fieldMappings.put("id","streamB.id"); + + Map paramsB = mapParams("q","id:(2)","fl","id,a_s,a_f,join_i", "sort", "a_s desc"); + CloudSolrStream streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB.setFieldMappings(fieldMappings); + + String[] keys = {"a_f"}; + + HashJoinStream fstream = new HashJoinStream(streamA, streamB, keys); + List tuples = getTuples(fstream); + + assert(tuples.size() == 1); + assertOrder(tuples, 0); + assertLong(tuples.get(0), "join_i", 1000); + + + //Test one-to-many + + paramsA = mapParams("q","id:(0 1 3 4) ","fl","id,a_s,a_f", "sort", "a_s desc"); + streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + fieldMappings = new HashMap(); + fieldMappings.put("id","streamB.id"); + + paramsB = mapParams("q","id:(2 6)","fl","id,a_s,a_f,join_i", "sort", "a_s desc"); + streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB.setFieldMappings(fieldMappings); + + + fstream = new HashJoinStream(streamA, streamB, keys); + tuples = getTuples(fstream); + + assert(tuples.size() == 2); + assertOrder(tuples, 0,0); + assertLong(tuples.get(0), "join_i", 1000); + assertLong(tuples.get(1), "join_i", 2000); + + //Test many-to-one + + paramsA = mapParams("q","id:(0 2 1 3 4) ","fl","id,a_s,a_f", "sort", "a_s desc"); + streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + fieldMappings = new HashMap(); + fieldMappings.put("id","streamB.id"); + + paramsB = mapParams("q","id:(6)","fl","id,a_s,a_f,join_i", "sort", "a_s desc"); + streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB.setFieldMappings(fieldMappings); + + + fstream = new HashJoinStream(streamA, streamB, keys); + tuples = getTuples(fstream); + + assert(tuples.size() == 2); + assertOrder(tuples, 2,0); + assertLong(tuples.get(0), "join_i", 2000); + assertLong(tuples.get(1), "join_i", 2000); + + //Test many-to-many + + paramsA = mapParams("q","id:(0 7 1 3 4) ","fl","id,a_s,a_f", "sort", "a_s desc"); + streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + fieldMappings = new HashMap(); + fieldMappings.put("id","streamB.id"); + + paramsB = mapParams("q","id:(6 2)","fl","id,a_s,a_f,join_i", "sort", "a_s desc"); + streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB.setFieldMappings(fieldMappings); + + + fstream = new HashJoinStream(streamA, streamB, keys); + tuples = getTuples(fstream); + + assert(tuples.size() == 4); + assertOrder(tuples, 7,7,0,0); + assertLong(tuples.get(0), "join_i", 1000); + assertLong(tuples.get(1), "join_i", 2000); + assertLong(tuples.get(2), "join_i", 1000); + assertLong(tuples.get(3), "join_i", 2000); + + del("*:*"); + commit(); + + } + + private void testMergeJoinStream() 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", "join_i", "1000"); + 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, "6", "a_s", "hello1", "a_i", "1", "a_f", "0", "join_i", "2000"); + indexr(id, "7", "a_s", "hello7", "a_i", "1", "a_f", "0"); + + commit(); + + String zkHost = zkServer.getZkAddress(); + + //Test one-to-one + Map paramsA = mapParams("q","id:(0 1 3 4) ","fl","id,a_s,a_f", "sort", "a_f desc"); + CloudSolrStream streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + Map fieldMappings = new HashMap(); + fieldMappings.put("id","streamB.id"); + + Map paramsB = mapParams("q","id:(2)","fl","id,a_s,a_f,join_i", "sort", "a_f desc"); + CloudSolrStream streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB.setFieldMappings(fieldMappings); + + String[] keys = {"a_f"}; + + MergeJoinStream fstream = new MergeJoinStream(streamA, streamB, new DescFieldComp("a_f")); + List tuples = getTuples(fstream); + + assert(tuples.size() == 1); + assertOrder(tuples, 0); + assertLong(tuples.get(0), "join_i", 1000); + + + //Test one-to-many + + paramsA = mapParams("q","id:(0 1 3 4) ","fl","id,a_s,a_f", "sort", "a_f desc"); + streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + fieldMappings = new HashMap(); + fieldMappings.put("id","streamB.id"); + + paramsB = mapParams("q","id:(2 6)","fl","id,a_s,a_f,join_i", "sort", "a_f desc"); + streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB.setFieldMappings(fieldMappings); + + + fstream = new MergeJoinStream(streamA, streamB, new DescFieldComp("a_f")); + tuples = getTuples(fstream); + + assert(tuples.size() == 2); + assertOrder(tuples, 0,0); + assertLong(tuples.get(0), "join_i", 1000); + assertLong(tuples.get(1), "join_i", 2000); + + //Test many-to-one + + paramsA = mapParams("q","id:(0 2 1 3 4) ","fl","id,a_s,a_f", "sort", "a_f desc"); + streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + fieldMappings = new HashMap(); + fieldMappings.put("id","streamB.id"); + + paramsB = mapParams("q","id:(6)","fl","id,a_s,a_f,join_i", "sort", "a_f desc"); + streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB.setFieldMappings(fieldMappings); + + + fstream = new MergeJoinStream(streamA, streamB, new DescFieldComp("a_f")); + tuples = getTuples(fstream); + + assert(tuples.size() == 2); + assertOrder(tuples, 2,0); + assertLong(tuples.get(0), "join_i", 2000); + assertLong(tuples.get(1), "join_i", 2000); + + //Test many-to-many + + paramsA = mapParams("q","id:(0 7 1 3 4) ","fl","id,a_s,a_f", "sort", "a_f desc"); + streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + fieldMappings = new HashMap(); + fieldMappings.put("id","streamB.id"); + + paramsB = mapParams("q","id:(6 2)","fl","id,a_s,a_f,join_i", "sort", "a_f desc"); + streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB.setFieldMappings(fieldMappings); + + + fstream = new MergeJoinStream(streamA, streamB, new DescFieldComp("a_f")); + tuples = getTuples(fstream); + + assert(tuples.size() == 4); + assertOrder(tuples, 7,7,0,0); + assertLong(tuples.get(0), "join_i", 1000); + assertLong(tuples.get(1), "join_i", 2000); + assertLong(tuples.get(2), "join_i", 1000); + assertLong(tuples.get(3), "join_i", 2000); + + del("*:*"); + commit(); + + } + + private void testParallelMergeJoinStream() 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", "join_i", "1000"); + 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, "6", "a_s", "hello1", "a_i", "1", "a_f", "0", "join_i", "2000"); + indexr(id, "7", "a_s", "hello7", "a_i", "1", "a_f", "0"); + + commit(); + + String zkHost = zkServer.getZkAddress(); + + //Test one-to-one + Map paramsA = mapParams("q","id:(0 1 3 4) ","fl","id,a_s,a_f", "sort", "a_f desc", "partitionKeys","a_f"); + CloudSolrStream streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + Map fieldMappings = new HashMap(); + fieldMappings.put("id","streamB.id"); + + Map paramsB = mapParams("q","id:(2)","fl","id,a_s,a_f,join_i", "sort", "a_f desc", "partitionKeys","a_f"); + CloudSolrStream streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB.setFieldMappings(fieldMappings); + + String[] keys = {"a_f"}; + + MergeJoinStream mstream = new MergeJoinStream(streamA, streamB, new DescFieldComp("a_f")); + ParallelStream fstream = new ParallelStream(zkHost,"collection1", mstream, 2, new DescFieldComp("a_f")); + + List tuples = getTuples(fstream); + + assert(tuples.size() == 1); + assertOrder(tuples, 0); + assertLong(tuples.get(0), "join_i", 1000); + + + //Test one-to-many + + paramsA = mapParams("q","id:(0 1 3 4) ","fl","id,a_s,a_f", "sort", "a_f desc", "partitionKeys","a_f"); + streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + fieldMappings = new HashMap(); + fieldMappings.put("id","streamB.id"); + + paramsB = mapParams("q","id:(2 6)","fl","id,a_s,a_f,join_i", "sort", "a_f desc", "partitionKeys","a_f"); + streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB.setFieldMappings(fieldMappings); + + + mstream = new MergeJoinStream(streamA, streamB, new DescFieldComp("a_f")); + fstream = new ParallelStream(zkHost,"collection1", mstream, 2, new DescFieldComp("a_f")); + + tuples = getTuples(fstream); + + assert(tuples.size() == 2); + assertOrder(tuples, 0,0); + assertLong(tuples.get(0), "join_i", 1000); + assertLong(tuples.get(1), "join_i", 2000); + + //Test many-to-one + + paramsA = mapParams("q","id:(0 2 1 3 4) ","fl","id,a_s,a_f", "sort", "a_f desc", "partitionKeys","a_f"); + streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + fieldMappings = new HashMap(); + fieldMappings.put("id","streamB.id"); + + paramsB = mapParams("q","id:(6)","fl","id,a_s,a_f,join_i", "sort", "a_f desc", "partitionKeys","a_f"); + streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB.setFieldMappings(fieldMappings); + + + mstream = new MergeJoinStream(streamA, streamB, new DescFieldComp("a_f")); + fstream = new ParallelStream(zkHost,"collection1", mstream, 2, new DescFieldComp("a_f")); + + tuples = getTuples(fstream); + + assert(tuples.size() == 2); + assertOrder(tuples, 2,0); + assertLong(tuples.get(0), "join_i", 2000); + assertLong(tuples.get(1), "join_i", 2000); + + //Test many-to-many + + paramsA = mapParams("q","id:(0 7 1 3 4) ","fl","id,a_s,a_f", "sort", "a_f desc", "partitionKeys","a_f"); + streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + fieldMappings = new HashMap(); + fieldMappings.put("id","streamB.id"); + + paramsB = mapParams("q","id:(6 2)","fl","id,a_s,a_f,join_i", "sort", "a_f desc", "partitionKeys","a_f"); + streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB.setFieldMappings(fieldMappings); + + + mstream = new MergeJoinStream(streamA, streamB, new DescFieldComp("a_f")); + fstream = new ParallelStream(zkHost,"collection1", mstream, 2, new DescFieldComp("a_f")); + + tuples = getTuples(fstream); + + assert(tuples.size() == 4); + assertOrder(tuples, 7,7,0,0); + assertLong(tuples.get(0), "join_i", 1000); + assertLong(tuples.get(1), "join_i", 2000); + assertLong(tuples.get(2), "join_i", 1000); + assertLong(tuples.get(3), "join_i", 2000); + + 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(); + + String zkHost = zkServer.getZkAddress(); + + 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")); + List tuples = getTuples(rstream); + + + assert(tuples.size() == 3); + assertOrder(tuples, 4,3,2); + + del("*:*"); + commit(); + } + + private void testRollupStream() throws Exception { + indexr(id, "0", "a_s", "hello0", "a_i", "100", "a_f", "0"); + indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "0"); + indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); + indexr(id, "4", "a_s", "hello3", "a_i", "4", "a_f", "4"); + indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); + indexr(id, "6", "a_s", "hello1", "a_i", "1", "a_f", "1"); + indexr(id, "7", "a_s", "hello1", "a_i", "1", "a_f", "1"); + + commit(); + + String zkHost = zkServer.getZkAddress(); + + Bucket[] buckets = {new Bucket("a_s")}; + Metric[] metrics = {new SumMetric("a_i", false), + new MeanMetric("a_i", false), + new CountMetric(), + new MinMetric("a_i", false), + new MaxMetric("a_i", false)}; + + Map params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_s asc"); + CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params); + RollupStream rstream = new RollupStream(stream, buckets, metrics); + rstream.open(); + Tuple tuple = rstream.read(); + String b = (String)tuple.get("buckets"); + List values = (List)tuple.get("metricValues"); + assert(b.equals("hello0")); + assert(values.get(0) == 102.0d); + assert(values.get(1) == 51.0d); + assert(values.get(2) == 2.0d); + assert(values.get(3) == 2.0d); + assert(values.get(4) == 100.0d); + + tuple = rstream.read(); + b = (String)tuple.get("buckets"); + values = (List)tuple.get("metricValues"); + assert(b.equals("hello1")); + assert(values.get(0) == 3.0d); + assert(values.get(1) == 1.0d); + assert(values.get(2) == 3.0d); + assert(values.get(3) == 1.0d); + assert(values.get(4) == 1.0d); + + + tuple = rstream.read(); + b = (String)tuple.get("buckets"); + values = (List)tuple.get("metricValues"); + assert(b.equals("hello3")); + assert(values.get(0) == 7.0d); + assert(values.get(1) == 3.5d); + assert(values.get(2) == 2.0d); + assert(values.get(3) == 3.0d); + assert(values.get(4) == 4.0d); + + tuple = rstream.read(); + assert(tuple.EOF); + + rstream.close(); + del("*:*"); + commit(); + } + + private void testParallelRollupStream() throws Exception { + indexr(id, "0", "a_s", "hello0", "a_i", "100", "a_f", "0"); + indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "0"); + indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); + indexr(id, "4", "a_s", "hello3", "a_i", "4", "a_f", "4"); + indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); + indexr(id, "6", "a_s", "hello1", "a_i", "1", "a_f", "1"); + indexr(id, "7", "a_s", "hello1", "a_i", "1", "a_f", "1"); + + commit(); + + String zkHost = zkServer.getZkAddress(); + + Bucket[] buckets = {new Bucket("a_s")}; + Metric[] metrics = {new SumMetric("a_i", false), + new MeanMetric("a_i", false), + new CountMetric(), + new MinMetric("a_i", false), + new MaxMetric("a_i", false)}; + + Map params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_s asc","partitionKeys","a_s"); + CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params); + RollupStream rostream = new RollupStream(stream, buckets, metrics); + ParallelStream rstream = new ParallelStream(zkHost,"collection1", rostream, 2, new AscFieldComp("buckets")); + + rstream.open(); + Tuple tuple = rstream.read(); + String b = (String)tuple.get("buckets"); + List values = (List)tuple.get("metricValues"); + assert(b.equals("hello0")); + assert(values.get(0) == 102.0d); + assert(values.get(1) == 51.0d); + assert(values.get(2) == 2.0d); + assert(values.get(3) == 2.0d); + assert(values.get(4) == 100.0d); + + tuple = rstream.read(); + b = (String)tuple.get("buckets"); + values = (List)tuple.get("metricValues"); + assert(b.equals("hello1")); + assert(values.get(0) == 3.0d); + assert(values.get(1) == 1.0d); + assert(values.get(2) == 3.0d); + assert(values.get(3) == 1.0d); + assert(values.get(4) == 1.0d); + + + tuple = rstream.read(); + b = (String)tuple.get("buckets"); + values = (List)tuple.get("metricValues"); + assert(b.equals("hello3")); + assert(values.get(0) == 7.0d); + assert(values.get(1) == 3.5d); + assert(values.get(2) == 2.0d); + assert(values.get(3) == 3.0d); + assert(values.get(4) == 4.0d); + + tuple = rstream.read(); + assert(tuple.EOF); + + rstream.close(); + del("*:*"); + commit(); + } + + + + private void testMetricStream() throws Exception { + + indexr(id, "0", "a_s", "hello0", "a_i", "100", "a_f", "0"); + indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "0"); + indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); + indexr(id, "4", "a_s", "hello3", "a_i", "4", "a_f", "4"); + indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); + indexr(id, "6", "a_s", "hello1", "a_i", "1", "a_f", "1"); + indexr(id, "7", "a_s", "hello1", "a_i", "1", "a_f", "1"); + + commit(); + + String zkHost = zkServer.getZkAddress(); + + Bucket[] buckets = {new Bucket("a_s")}; + Metric[] metrics = {new SumMetric("a_i", false), + new MeanMetric("a_i", false), + new CountMetric(), + new MinMetric("a_i", false), + new MaxMetric("a_i", false)}; + + Map params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i asc"); + CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params); + MetricStream mstream = new MetricStream(stream, buckets, metrics, "metric1", new DescBucketComp(0),5); + getTuples(mstream); + + BucketMetrics[] bucketMetrics = mstream.getBucketMetrics(); + assert(bucketMetrics.length == 3); + + //Bucket should be is descending order based on Metric 0, which is the SumMetric. + + assert(bucketMetrics[0].getKey().toString().equals("hello0")); + assert(bucketMetrics[1].getKey().toString().equals("hello3")); + assert(bucketMetrics[2].getKey().toString().equals("hello1")); + + assertMetric(bucketMetrics[0].getMetrics()[0], 102.0d); //Test the first Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[1], 51.0d); //Test the second Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[2], 2.0d); //Test the third Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[3], 2.0d); //Test the fourth Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[4], 100.0d); //Test the fifth Metric of the first BucketMetrics + + + assertMetric(bucketMetrics[1].getMetrics()[0], 7.0d); + assertMetric(bucketMetrics[2].getMetrics()[0], 3.0d); + + + params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i asc"); + stream = new CloudSolrStream(zkHost, "collection1", params); + mstream = new MetricStream(stream, buckets, metrics, "metric1", new AscBucketComp(0),5); + getTuples(mstream); + + bucketMetrics = mstream.getBucketMetrics(); + + assertMetric(bucketMetrics[0].getMetrics()[0], 3.0d); //Test the first Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[1], 1.0d); //Test the second Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[2], 3.0d); //Test the third Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[3], 1.0d); //Test the fourth Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[4], 1.0d); //Test the fifth Metric of the first BucketMetrics + + assertMetric(bucketMetrics[1].getMetrics()[0], 7.0d); + assertMetric(bucketMetrics[2].getMetrics()[0], 102.0d); + + indexr(id, "8", "a_s", "hello4", "a_i", "1000", "a_f", "1"); //Add a fourth record. + commit(); + + //Test desc comp with more buckets then priority queue can hold. + params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i asc"); + stream = new CloudSolrStream(zkHost, "collection1", params); + mstream = new MetricStream(stream, buckets, metrics, "metric1", new DescBucketComp(0),3); + getTuples(mstream); + + bucketMetrics = mstream.getBucketMetrics(); + assert(bucketMetrics.length == 3); + assert(bucketMetrics[0].getKey().toString().equals("hello4")); + assert(bucketMetrics[1].getKey().toString().equals("hello0")); + assert(bucketMetrics[2].getKey().toString().equals("hello3")); + + //Test asc comp with more buckets then priority queue can hold. + params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i asc"); + stream = new CloudSolrStream(zkHost, "collection1", params); + mstream = new MetricStream(stream, buckets, metrics, "metric1", new AscBucketComp(0),3); + getTuples(mstream); + + bucketMetrics = mstream.getBucketMetrics(); + assert(bucketMetrics.length == 3); + assert(bucketMetrics[0].getKey().toString().equals("hello1")); + assert(bucketMetrics[1].getKey().toString().equals("hello3")); + assert(bucketMetrics[2].getKey().toString().equals("hello0")); + + + //Test with no buckets + params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i asc"); + stream = new CloudSolrStream(zkHost, "collection1", params); + mstream = new MetricStream(stream, metrics, "metric1"); + getTuples(mstream); + + bucketMetrics = mstream.getBucketMetrics(); + assert(bucketMetrics.length == 1); + assert(bucketMetrics[0].getKey().toString().equals("metrics")); + assertMetric(bucketMetrics[0].getMetrics()[0], 1112.0d); //Test the first Metric of the first BucketMetrics + + del("*:*"); + commit(); + } + + + private void testParallelMetricStream() throws Exception { + + indexr(id, "0", "a_s", "hello0", "a_i", "100", "a_f", "0"); + indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "0"); + indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); + indexr(id, "4", "a_s", "hello3", "a_i", "4", "a_f", "4"); + indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); + indexr(id, "6", "a_s", "hello1", "a_i", "1", "a_f", "1"); + indexr(id, "7", "a_s", "hello1", "a_i", "1", "a_f", "1"); + + commit(); + + String zkHost = zkServer.getZkAddress(); + + Bucket[] buckets = {new Bucket("a_s")}; + Metric[] metrics = {new SumMetric("a_i", false), + new MeanMetric("a_i", false), + new CountMetric(), + new MinMetric("a_i", false), + new MaxMetric("a_i", false)}; + + Map params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i asc", "partitionKeys", "a_i"); + CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params); + MetricStream mstream = new MetricStream(stream, buckets, metrics, "metric1", new DescBucketComp(0),5); + ParallelStream pstream = new ParallelStream(zkHost,"collection1",mstream,2,new AscFieldComp("a_i")); + getTuples(pstream); + + BucketMetrics[] bucketMetrics = mstream.getBucketMetrics(); + assert(bucketMetrics.length == 3); + + //Bucket should be is descending order based on Metric 0, which is the SumMetric. + + assert(bucketMetrics[0].getKey().toString().equals("hello0")); + assert(bucketMetrics[1].getKey().toString().equals("hello3")); + assert(bucketMetrics[2].getKey().toString().equals("hello1")); + + assertMetric(bucketMetrics[0].getMetrics()[0], 102.0d); //Test the first Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[1], 51.0d); //Test the second Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[2], 2.0d); //Test the third Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[3], 2.0d); //Test the fourth Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[4], 100.0d); //Test the fifth Metric of the first BucketMetrics + + + assertMetric(bucketMetrics[1].getMetrics()[0], 7.0d); + assertMetric(bucketMetrics[2].getMetrics()[0], 3.0d); + + + params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i asc"); + stream = new CloudSolrStream(zkHost, "collection1", params); + mstream = new MetricStream(stream, buckets, metrics, "metric1", new AscBucketComp(0),5); + getTuples(mstream); + + bucketMetrics = mstream.getBucketMetrics(); + + assertMetric(bucketMetrics[0].getMetrics()[0], 3.0d); //Test the first Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[1], 1.0d); //Test the second Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[2], 3.0d); //Test the third Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[3], 1.0d); //Test the fourth Metric of the first BucketMetrics + assertMetric(bucketMetrics[0].getMetrics()[4], 1.0d); //Test the fifth Metric of the first BucketMetrics + + assertMetric(bucketMetrics[1].getMetrics()[0], 7.0d); + assertMetric(bucketMetrics[2].getMetrics()[0], 102.0d); + + indexr(id, "8", "a_s", "hello4", "a_i", "1000", "a_f", "1"); //Add a fourth record. + commit(); + + //Test desc comp with more buckets then priority queue can hold. + params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i asc"); + stream = new CloudSolrStream(zkHost, "collection1", params); + mstream = new MetricStream(stream, buckets, metrics, "metric1", new DescBucketComp(0),3); + getTuples(mstream); + + bucketMetrics = mstream.getBucketMetrics(); + assert(bucketMetrics.length == 3); + assert(bucketMetrics[0].getKey().toString().equals("hello4")); + assert(bucketMetrics[1].getKey().toString().equals("hello0")); + assert(bucketMetrics[2].getKey().toString().equals("hello3")); + + //Test asc comp with more buckets then priority queue can hold. + params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i asc"); + stream = new CloudSolrStream(zkHost, "collection1", params); + mstream = new MetricStream(stream, buckets, metrics, "metric1", new AscBucketComp(0),3); + getTuples(mstream); + + bucketMetrics = mstream.getBucketMetrics(); + assert(bucketMetrics.length == 3); + assert(bucketMetrics[0].getKey().toString().equals("hello1")); + assert(bucketMetrics[1].getKey().toString().equals("hello3")); + assert(bucketMetrics[2].getKey().toString().equals("hello0")); + + + //Test with no buckets + params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i asc"); + stream = new CloudSolrStream(zkHost, "collection1", params); + mstream = new MetricStream(stream, metrics, "metric1"); + getTuples(mstream); + + bucketMetrics = mstream.getBucketMetrics(); + assert(bucketMetrics.length == 1); + assert(bucketMetrics[0].getKey().toString().equals("metrics")); + assertMetric(bucketMetrics[0].getMetrics()[0], 1112.0d); //Test the first Metric of the first BucketMetrics + + del("*:*"); + commit(); + } + + private void testGroupByStream() throws Exception { + + indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); + indexr(id, "2", "a_s", "hello0", "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", "hello0", "a_i", "1", "a_f", "1"); + + commit(); + + //Test CloudSolrStream and SumStream over an int field + String zkHost = zkServer.getZkAddress(); + + Map paramsA = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_s asc"); + CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA); + GroupByStream gstream = new GroupByStream(stream, new AscFieldComp("a_s"), new DescFieldComp("a_i"), 5); + + List tuples = getTuples(gstream); + + assert(tuples.size() == 3); + assertOrder(tuples, 2,3,4); + assertGroupOrder(tuples.get(0), 1, 0); + + del("*:*"); + commit(); + } + + + private void testFilterStream() 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(); + + //Test CloudSolrStream and SumStream over an int field + String zkHost = zkServer.getZkAddress(); + + Map paramsA = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_s asc"); + CloudSolrStream streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + Map paramsB = mapParams("q","id:(0 2)","fl","a_s","sort", "a_s asc"); + CloudSolrStream streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + + + FilterStream fstream = new FilterStream(streamA, streamB, new AscFieldComp("a_s")); + List tuples = getTuples(fstream); + + assert(tuples.size() == 2); + assertOrder(tuples, 0,2); + + del("*:*"); + commit(); + } + + private void testParallelStream() 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(); + + String zkHost = zkServer.getZkAddress(); + + Map paramsA = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_s asc", "partitionKeys","a_s"); + CloudSolrStream streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + Map paramsB = mapParams("q","id:(0 2)","fl","a_s","sort", "a_s asc", "partitionKeys","a_s"); + CloudSolrStream streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + + FilterStream fstream = new FilterStream(streamA, streamB, new AscFieldComp("a_s")); + ParallelStream pstream = new ParallelStream(zkHost,"collection1", fstream, 2, new AscFieldComp("a_s")); + List tuples = getTuples(pstream); + + assert(tuples.size() == 2); + assertOrder(tuples, 0,2); + + del("*:*"); + commit(); + } + + + + private void testParallelHashJoinStream() { + + } + + private void testParallelGroupByStream() 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", "hello0", "a_i", "3", "a_f", "3"); + indexr(id, "4", "a_s", "hello0", "a_i", "4", "a_f", "4"); + indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); + + commit(); + + String zkHost = zkServer.getZkAddress(); + + Map params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_s asc", "partitionKeys","a_s"); + CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params); + GroupByStream gstream = new GroupByStream(stream, new AscFieldComp("a_s"), new AscFieldComp("a_i"),5); + ParallelStream pstream = new ParallelStream(zkHost,"collection1", gstream, 2, new AscFieldComp("a_s")); + List tuples = getTuples(pstream); + + assert(tuples.size() == 3); + assertOrder(tuples, 0,1,2); + assertGroupOrder(tuples.get(0),3,4); + del("*:*"); + commit(); + } + + + private void testTuple() throws Exception { + + indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "5.1", "s_multi", "a", "s_multi", "b", "i_multi", "1", "i_multi", "2", "f_multi", "1.2", "f_multi", "1.3"); + + commit(); + + String zkHost = zkServer.getZkAddress(); + + Map params = mapParams("q","*:*","fl","id,a_s,a_i,a_f,s_multi,i_multi,f_multi","sort", "a_s asc", "partitionKeys","a_s"); + CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params); + List tuples = getTuples(stream); + Tuple tuple = tuples.get(0); + + String s = tuple.getString("a_s"); + assert(s.equals("hello0")) ; + + long l = tuple.getLong("a_i"); + assert(l == 0); + + double d = tuple.getDouble("a_f"); + assert(d == 5.1); + + + List stringList = tuple.getStrings("s_multi"); + assert(stringList.get(0).equals("a")); + assert(stringList.get(1).equals("b")); + + List longList = tuple.getLongs("i_multi"); + assert(longList.get(0).longValue() == 1); + assert(longList.get(1).longValue() == 2); + + List doubleList = tuple.getDoubles("f_multi"); + assert(doubleList.get(0).doubleValue() == 1.2); + assert(doubleList.get(1).doubleValue() == 1.3); + + 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(); + + String zkHost = zkServer.getZkAddress(); + + //Test ascending + Map paramsA = mapParams("q","id:(4 1)","fl","id,a_s,a_i","sort", "a_i asc"); + CloudSolrStream streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + 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")); + List tuples = getTuples(mstream); + + assert(tuples.size() == 5); + assertOrder(tuples, 0,1,2,3,4); + + //Test descending + paramsA = mapParams("q","id:(4 1)","fl","id,a_s,a_i","sort", "a_i desc"); + streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + 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")); + tuples = getTuples(mstream); + + assert(tuples.size() == 5); + assertOrder(tuples, 4,3,2,1,0); + + //Test compound sort + + paramsA = mapParams("q","id:(2 4 1)","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i asc"); + streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + 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"))); + tuples = getTuples(mstream); + + assert(tuples.size() == 5); + assertOrder(tuples, 0,2,1,3,4); + + paramsA = mapParams("q","id:(2 4 1)","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i desc"); + streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + + 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"))); + tuples = getTuples(mstream); + + assert(tuples.size() == 5); + assertOrder(tuples, 2,0,1,3,4); + + del("*:*"); + commit(); + } + + + @Test + public void streamTests() throws Exception { + assertNotNull(cloudClient); + + handle.clear(); + handle.put("timestamp", SKIPVAL); + + waitForThingsToLevelOut(30); + + del("*:*"); + + commit(); + + 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(); + + String zkHost = zkServer.getZkAddress(); + Map params = null; + + //Basic CloudSolrStream Test with Ascending Sort + + params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i desc"); + CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params); + List tuples = getTuples(stream); + + assert(tuples.size() == 5); + assertOrder(tuples, 4, 3, 2, 1, 0); + + //With Descending Sort + params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i asc"); + stream = new CloudSolrStream(zkHost, "collection1", params); + tuples = getTuples(stream); + + assert(tuples.size() == 5); + assertOrder(tuples, 0,1,2,3,4); + + + //Test compound sort + params = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i desc"); + stream = new CloudSolrStream(zkHost, "collection1", params); + tuples = getTuples(stream); + + assert(tuples.size() == 5); + assertOrder(tuples, 2,0,1,3,4); + + + params = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i asc"); + stream = new CloudSolrStream(zkHost, "collection1", params); + tuples = getTuples(stream); + + assert(tuples.size() == 5); + assertOrder(tuples, 0,2,1,3,4); + + del("*:*"); + commit(); + + testTuple(); + testUniqueStream(); + testMetricStream(); + testRollupStream(); + testRankStream(); + testFilterStream(); + testGroupByStream(); + testHashJoinStream(); + testMergeJoinStream(); + testMergeStream(); + testParallelStream(); + testParallelRollupStream(); + testParallelMetricStream(); + testParallelGroupByStream(); + testParallelHashJoinStream(); + testParallelMergeJoinStream(); + } + + protected Map mapParams(String... vals) { + Map params = new HashMap(); + String k = null; + for(String val : vals) { + if(k == null) { + k = val; + } else { + params.put(k, val); + k = null; + } + } + + return params; + } + + protected List getTuples(TupleStream tupleStream) throws IOException { + tupleStream.open(); + List tuples = new ArrayList(); + for(Tuple t = tupleStream.read(); !t.EOF; t = tupleStream.read()) { + tuples.add(t); + } + tupleStream.close(); + return tuples; + } + + protected boolean assertOrder(List tuples, int... ids) throws Exception { + int i = 0; + for(int val : ids) { + Tuple t = tuples.get(i); + Long tip = (Long)t.get("id"); + 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 assertMetric(Metric metric, double value) throws Exception { + Double d = metric.getValue(); + if(d.doubleValue() != value) { + throw new Exception("Unexpected Metric "+d+"!="+value); + } + + return true; + } + + @Override + protected void indexr(Object... fields) throws Exception { + SolrInputDocument doc = getDoc(fields); + indexDoc(doc); + } +}