SOLR-10274: The search Streaming Expression should work in non-SolrCloud mode

This commit is contained in:
Joel Bernstein 2017-04-11 15:17:03 -04:00
parent 57c5837183
commit 06a55b73b9
7 changed files with 3491 additions and 2892 deletions

View File

@ -26,8 +26,6 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Random;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
@ -35,7 +33,6 @@ import java.util.concurrent.Future;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.CloudSolrClient.Builder;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.comp.ComparatorOrder;
import org.apache.solr.client.solrj.io.comp.FieldComparator;
@ -52,9 +49,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
import org.apache.solr.common.cloud.Aliases;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
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.params.MapSolrParams;
import org.apache.solr.common.params.ModifiableSolrParams;
@ -178,9 +173,11 @@ public class CloudSolrStream extends TupleStream implements Expressible {
else if(zkHostExpression.getParameter() instanceof StreamExpressionValue){
zkHost = ((StreamExpressionValue)zkHostExpression.getParameter()).getValue();
}
/*
if(null == zkHost){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - zkHost not found for collection '%s'",expression,collectionName));
}
*/
// We've got all the required items
init(collectionName, zkHost, mParams);
@ -299,14 +296,6 @@ public class CloudSolrStream extends TupleStream implements Expressible {
this.tuples = new TreeSet();
this.solrStreams = new ArrayList();
this.eofTuples = Collections.synchronizedMap(new HashMap());
if (this.streamContext != null && this.streamContext.getSolrClientCache() != null) {
this.cloudSolrClient = this.streamContext.getSolrClientCache().getCloudSolrClient(zkHost);
} else {
this.cloudSolrClient = new Builder()
.withZkHost(zkHost)
.build();
this.cloudSolrClient.connect();
}
constructStreams();
openStreams();
}
@ -400,29 +389,15 @@ public class CloudSolrStream extends TupleStream implements Expressible {
protected void constructStreams() throws IOException {
try {
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
ClusterState clusterState = zkStateReader.getClusterState();
Collection<Slice> slices = CloudSolrStream.getSlices(this.collection, zkStateReader, true);
List<String> shardUrls = getShards(this.zkHost, this.collection, this.streamContext);
ModifiableSolrParams mParams = new ModifiableSolrParams(params);
mParams = adjustParams(mParams);
mParams.set(DISTRIB, "false"); // We are the aggregator.
Set<String> liveNodes = clusterState.getLiveNodes();
for(Slice slice : slices) {
Collection<Replica> replicas = slice.getReplicas();
List<Replica> shuffler = new ArrayList<>();
for(Replica replica : replicas) {
if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName()))
shuffler.add(replica);
}
Collections.shuffle(shuffler, new Random());
Replica rep = shuffler.get(0);
ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep);
String url = zkProps.getCoreUrl();
SolrStream solrStream = new SolrStream(url, mParams);
for(String shardUrl : shardUrls) {
SolrStream solrStream = new SolrStream(shardUrl, mParams);
if(streamContext != null) {
solrStream.setStreamContext(streamContext);
}
@ -468,12 +443,6 @@ public class CloudSolrStream extends TupleStream implements Expressible {
solrStream.close();
}
}
if ((this.streamContext == null || this.streamContext.getSolrClientCache() == null) &&
cloudSolrClient != null) {
cloudSolrClient.close();
}
}
/** Return the stream sort - ie, the order in which records are returned */

View File

@ -263,27 +263,7 @@ public class ParallelStream extends CloudSolrStream implements Expressible {
try {
Object pushStream = ((Expressible) tupleStream).toExpression(streamFactory);
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
Collection<Slice> slices = CloudSolrStream.getSlices(this.collection, zkStateReader, true);
ClusterState clusterState = zkStateReader.getClusterState();
Set<String> liveNodes = clusterState.getLiveNodes();
List<Replica> shuffler = new ArrayList<>();
for(Slice slice : slices) {
Collection<Replica> replicas = slice.getReplicas();
for (Replica replica : replicas) {
if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName()))
shuffler.add(replica);
}
}
if(workers > shuffler.size()) {
throw new IOException("Number of workers exceeds nodes in the worker collection");
}
Collections.shuffle(shuffler, new Random());
List<String> shardUrls = getShards(this.zkHost, this.collection, this.streamContext);
for(int w=0; w<workers; w++) {
ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
@ -293,9 +273,8 @@ public class ParallelStream extends CloudSolrStream implements Expressible {
paramsLoc.set("expr", pushStream.toString());
paramsLoc.set("qt","/stream");
Replica rep = shuffler.get(w);
ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep);
String url = zkProps.getCoreUrl();
String url = shardUrls.get(w);
SolrStream solrStream = new SolrStream(url, paramsLoc);
solrStreams.add(solrStream);
}

View File

@ -50,6 +50,10 @@ public class StreamContext implements Serializable{
this.entries.put(key, value);
}
public boolean containsKey(Object key) {
return entries.containsKey(key);
}
public Map getEntries() {
return this.entries;
}

View File

@ -27,6 +27,7 @@ import java.util.List;
import java.util.Locale;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.client.solrj.io.SolrClientCache;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.comp.ComparatorOrder;
import org.apache.solr.client.solrj.io.comp.FieldComparator;
@ -206,6 +207,10 @@ public class JDBCStreamTest extends SolrCloudTestCase {
statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('AL', 'Algeria')");
}
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
// Load Solr
new UpdateRequest()
.add(id, "0", "code_s", "GB", "name_s", "Great Britian")
@ -218,17 +223,24 @@ public class JDBCStreamTest extends SolrCloudTestCase {
List<Tuple> tuples;
try {
// Simple 1
TupleStream jdbcStream = new JDBCStream("jdbc:hsqldb:mem:.", "select CODE,COUNTRY_NAME from COUNTRIES order by CODE", new FieldComparator("CODE", ComparatorOrder.ASCENDING));
TupleStream selectStream = new SelectStream(jdbcStream, new HashMap<String, String>(){{ put("CODE", "code_s"); put("COUNTRY_NAME", "name_s"); }});
TupleStream selectStream = new SelectStream(jdbcStream, new HashMap<String, String>() {{
put("CODE", "code_s");
put("COUNTRY_NAME", "name_s");
}});
TupleStream searchStream = factory.constructStream("search(" + COLLECTIONORALIAS + ", fl=\"code_s,name_s\",q=\"*:*\",sort=\"code_s asc\")");
TupleStream mergeStream = new MergeStream(new FieldComparator("code_s", ComparatorOrder.ASCENDING), new TupleStream[]{selectStream, searchStream});
mergeStream.setStreamContext(streamContext);
tuples = getTuples(mergeStream);
assertEquals(7, tuples.size());
assertOrderOf(tuples, "code_s", "AL", "CA", "GB", "NL", "NO", "NP", "US");
assertOrderOf(tuples, "name_s", "Algeria", "Canada", "Great Britian", "Netherlands", "Norway", "Nepal", "United States");
} finally {
solrClientCache.close();
}
}
@Test
@ -277,7 +289,11 @@ public class JDBCStreamTest extends SolrCloudTestCase {
String expression;
TupleStream stream;
List<Tuple> tuples;
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
// Basic test
expression =
"innerJoin("
@ -295,7 +311,9 @@ public class JDBCStreamTest extends SolrCloudTestCase {
+ " on=\"personId\""
+ ")";
stream = factory.constructStream(expression);
stream.setStreamContext(streamContext);
tuples = getTuples(stream);
assertEquals(10, tuples.size());
@ -303,6 +321,9 @@ public class JDBCStreamTest extends SolrCloudTestCase {
assertOrderOf(tuples, "rating", 3.5d, 5d, 2.2d, 4.3d, 3.5d, 3d, 3d, 4d, 4.1d, 4.8d);
assertOrderOf(tuples, "personName", "Emma", "Grace", "Hailey", "Isabella", "Lily", "Madison", "Mia", "Natalie", "Olivia", "Samantha");
assertOrderOf(tuples, "country", "Netherlands", "United States", "Netherlands", "Netherlands", "Netherlands", "United States", "United States", "Netherlands", "Netherlands", "United States");
} finally {
solrClientCache.close();
}
}
@Test
@ -351,7 +372,11 @@ public class JDBCStreamTest extends SolrCloudTestCase {
String expression;
TupleStream stream;
List<Tuple> tuples;
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
// Basic test for no alias
expression =
"innerJoin("
@ -370,6 +395,7 @@ public class JDBCStreamTest extends SolrCloudTestCase {
+ ")";
stream = factory.constructStream(expression);
stream.setStreamContext(streamContext);
tuples = getTuples(stream);
assertEquals(10, tuples.size());
@ -396,6 +422,7 @@ public class JDBCStreamTest extends SolrCloudTestCase {
+ ")";
stream = factory.constructStream(expression);
stream.setStreamContext(streamContext);
tuples = getTuples(stream);
assertEquals(10, tuples.size());
@ -403,6 +430,9 @@ public class JDBCStreamTest extends SolrCloudTestCase {
assertOrderOf(tuples, "rating", 3.5d, 5d, 2.2d, 4.3d, 3.5d, 3d, 3d, 4d, 4.1d, 4.8d);
assertOrderOf(tuples, "personName", "Emma", "Grace", "Hailey", "Isabella", "Lily", "Madison", "Mia", "Natalie", "Olivia", "Samantha");
assertOrderOf(tuples, "country", "Netherlands", "United States", "Netherlands", "Netherlands", "Netherlands", "United States", "United States", "Netherlands", "Netherlands", "United States");
} finally {
solrClientCache.close();
}
}
@Test
@ -458,6 +488,11 @@ public class JDBCStreamTest extends SolrCloudTestCase {
TupleStream stream;
List<Tuple> tuples;
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
// Basic test
expression =
"rollup("
@ -483,6 +518,7 @@ public class JDBCStreamTest extends SolrCloudTestCase {
+ ")";
stream = factory.constructStream(expression);
stream.setStreamContext(streamContext);
tuples = getTuples(stream);
assertEquals(2, tuples.size());
@ -500,7 +536,9 @@ public class JDBCStreamTest extends SolrCloudTestCase {
assertTrue(3D == tuple.getDouble("min(rating)"));
assertTrue(3.95D == tuple.getDouble("avg(rating)"));
assertTrue(4D == tuple.getDouble("count(*)"));
} finally {
solrClientCache.close();
}
}
@Test(expected=IOException.class)

View File

@ -24,6 +24,7 @@ import java.util.Map;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.client.solrj.io.SolrClientCache;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.eval.AddEvaluator;
import org.apache.solr.client.solrj.io.eval.GreaterThanEvaluator;
@ -92,6 +93,9 @@ public class SelectWithEvaluatorsTest extends SolrCloudTestCase {
String clause;
TupleStream stream;
List<Tuple> tuples;
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
StreamFactory factory = new StreamFactory()
.withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress())
@ -101,7 +105,7 @@ public class SelectWithEvaluatorsTest extends SolrCloudTestCase {
.withFunctionName("if", IfThenElseEvaluator.class)
.withFunctionName("gt", GreaterThanEvaluator.class)
;
try {
// Basic test
clause = "select("
+ "id,"
@ -109,13 +113,16 @@ public class SelectWithEvaluatorsTest extends SolrCloudTestCase {
+ "search(collection1, q=*:*, fl=\"id,a_s,b_i,c_d,d_b\", sort=\"id asc\")"
+ ")";
stream = factory.constructStream(clause);
stream.setStreamContext(streamContext);
tuples = getTuples(stream);
assertFields(tuples, "id", "result");
assertNotFields(tuples, "a_s", "b_i", "c_d", "d_b");
assertEquals(1, tuples.size());
assertDouble(tuples.get(0), "result", 4.3);
assertEquals(4.3, tuples.get(0).get("result"));
} finally {
solrClientCache.close();
}
}
protected List<Tuple> getTuples(TupleStream tupleStream) throws IOException {

View File

@ -131,13 +131,20 @@ public void testUniqueStream() throws Exception {
.add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
SolrParams sParams = StreamingTest.mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
UniqueStream ustream = new UniqueStream(stream, new FieldEqualitor("a_f"));
ustream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(ustream);
assertEquals(4, tuples.size());
assertOrder(tuples, 0, 1, 3, 4);
} finally {
solrClientCache.close();
}
}
@Test
@ -167,15 +174,22 @@ public void testNonePartitionKeys() throws Exception {
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
SolrParams sParamsA = StreamingTest.mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_s asc,a_f asc", "partitionKeys", "none");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
ParallelStream pstream = parallelStream(stream, new FieldComparator("a_s", ComparatorOrder.ASCENDING));
attachStreamFactory(pstream);
pstream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(pstream);
assert (tuples.size() == (10 * numWorkers)); // Each tuple will be double counted.
} finally {
solrClientCache.close();
}
}
@Test
@ -193,11 +207,18 @@ public void testParallelUniqueStream() throws Exception {
.add(id, "8", "a_s", "hello1", "a_i", "13", "a_f", "4")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
SolrParams sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i asc", "partitionKeys", "a_f");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
UniqueStream ustream = new UniqueStream(stream, new FieldEqualitor("a_f"));
ParallelStream pstream = parallelStream(ustream, new FieldComparator("a_f", ComparatorOrder.ASCENDING));
attachStreamFactory(pstream);
pstream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(pstream);
assertEquals(5, tuples.size());
assertOrder(tuples, 0, 1, 3, 4, 6);
@ -206,6 +227,9 @@ public void testParallelUniqueStream() throws Exception {
Map<String, Tuple> eofTuples = pstream.getEofTuples();
assertEquals(numWorkers, eofTuples.size()); //There should be an EOF tuple for each worker.
}finally {
solrClientCache.close();
}
}
@ -226,12 +250,21 @@ public void testMultipleFqClauses() throws Exception {
streamFactory.withCollectionZkHost(COLLECTIONORALIAS, zkHost);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
ModifiableSolrParams params = new ModifiableSolrParams(mapParams("q", "*:*", "fl", "id,a_i",
"sort", "a_i asc", "fq", "a_ss:hello0", "fq", "a_ss:hello1"));
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, params);
stream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(stream);
assertEquals("Multiple fq clauses should have been honored", 1, tuples.size());
assertEquals("should only have gotten back document 0", "0", tuples.get(0).getString("id"));
} finally {
solrClientCache.close();
}
}
@Test
@ -245,15 +278,20 @@ public void testRankStream() throws Exception {
.add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
SolrParams sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
RankStream rstream = new RankStream(stream, 3, new FieldComparator("a_i", ComparatorOrder.DESCENDING));
rstream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(rstream);
assertEquals(3, tuples.size());
assertOrder(tuples, 4, 3, 2);
} finally {
solrClientCache.close();
}
}
@Test
@ -272,15 +310,23 @@ public void testParallelRankStream() throws Exception {
.add(id, "10", "a_s", "hello1", "a_i", "10", "a_f", "1")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
SolrParams sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc", "partitionKeys", "a_i");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
RankStream rstream = new RankStream(stream, 11, new FieldComparator("a_i", ComparatorOrder.DESCENDING));
ParallelStream pstream = parallelStream(rstream, new FieldComparator("a_i", ComparatorOrder.DESCENDING));
attachStreamFactory(pstream);
pstream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(pstream);
assertEquals(10, tuples.size());
assertOrder(tuples, 10, 9, 8, 7, 6, 5, 4, 3, 2, 0);
} finally {
solrClientCache.close();
}
}
@ -300,15 +346,24 @@ public void testTrace() throws Exception {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
//Test with spaces in the parameter lists.
SolrParams sParamsA = mapParams("q", "*:*", "fl", "id,a_s, a_i,a_f", "sort", "a_s asc,a_f asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
stream.setTrace(true);
stream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(stream);
assertEquals(COLLECTIONORALIAS, tuples.get(0).get("_COLLECTION_"));
assertEquals(COLLECTIONORALIAS, tuples.get(1).get("_COLLECTION_"));
assertEquals(COLLECTIONORALIAS, tuples.get(2).get("_COLLECTION_"));
assertEquals(COLLECTIONORALIAS, tuples.get(3).get("_COLLECTION_"));
} finally {
solrClientCache.close();
}
}
@Test
@ -327,6 +382,11 @@ public void testTrace() throws Exception {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
//Test with spaces in the parameter lists.
SolrParams sParamsA = mapParams("q", "*:*", "fl", "id,a_s, a_i, a_f", "sort", "a_s asc , a_f asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
@ -334,6 +394,7 @@ public void testTrace() throws Exception {
new FieldEqualitor("a_s"),
new GroupOperation(new FieldComparator("a_f", ComparatorOrder.ASCENDING), 5));
rstream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(rstream);
assertEquals(3, tuples.size());
@ -356,7 +417,7 @@ public void testTrace() throws Exception {
rstream = new ReducerStream(stream,
new FieldComparator("a_s", ComparatorOrder.ASCENDING),
new GroupOperation(new FieldComparator("a_f", ComparatorOrder.DESCENDING), 5));
rstream.setStreamContext(streamContext);
tuples = getTuples(rstream);
assertEquals(3, tuples.size());
@ -372,7 +433,9 @@ public void testTrace() throws Exception {
t2 = tuples.get(2);
maps2 = t2.getMaps("group");
assertMaps(maps2, 6, 4);
} finally {
solrClientCache.close();
}
}
@Test
@ -392,17 +455,24 @@ public void testTrace() throws Exception {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
//Test with spaces in the parameter lists.
SolrParams sParamsA = mapParams("q", "blah", "fl", "id,a_s, a_i, a_f", "sort", "a_s asc , a_f asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
ReducerStream rstream = new ReducerStream(stream,
new FieldEqualitor("a_s"),
new GroupOperation(new FieldComparator("a_f", ComparatorOrder.ASCENDING), 5));
rstream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(rstream);
assertEquals(0, tuples.size());
} finally {
solrClientCache.close();
}
}
@Test
@ -421,6 +491,11 @@ public void testTrace() throws Exception {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
SolrParams sParamsA = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_s asc,a_f asc", "partitionKeys", "a_s");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
@ -429,6 +504,7 @@ public void testTrace() throws Exception {
new GroupOperation(new FieldComparator("a_f", ComparatorOrder.DESCENDING), 5));
ParallelStream pstream = parallelStream(rstream, new FieldComparator("a_s", ComparatorOrder.ASCENDING));
attachStreamFactory(pstream);
pstream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(pstream);
assertEquals(3, tuples.size());
@ -455,6 +531,7 @@ public void testTrace() throws Exception {
new GroupOperation(new FieldComparator("a_f", ComparatorOrder.ASCENDING), 3));
pstream = parallelStream(rstream, new FieldComparator("a_s", ComparatorOrder.DESCENDING));
attachStreamFactory(pstream);
pstream.setStreamContext(streamContext);
tuples = getTuples(pstream);
assertEquals(3, tuples.size());
@ -470,7 +547,9 @@ public void testTrace() throws Exception {
t2 = tuples.get(2);
maps2 = t2.getMaps("group");
assertMaps(maps2, 0, 2, 1);
} finally {
solrClientCache.close();
}
}
@Test
@ -490,10 +569,15 @@ public void testTrace() throws Exception {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
//Test an error that comes originates from the /select handler
try {
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,blah", "sort", "blah asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
ExceptionStream estream = new ExceptionStream(stream);
estream.setStreamContext(streamContext);
Tuple t = getTuple(estream);
assertTrue(t.EOF);
assertTrue(t.EXCEPTION);
@ -503,11 +587,15 @@ public void testTrace() throws Exception {
sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,score", "sort", "a_s asc", "qt", "/export");
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
estream = new ExceptionStream(stream);
estream.setStreamContext(streamContext);
t = getTuple(estream);
assertTrue(t.EOF);
assertTrue(t.EXCEPTION);
//The /export handler will pass through a real exception.
assertTrue(t.getException().contains("undefined field:"));
} finally {
solrClientCache.close();
}
}
@Test
@ -577,6 +665,11 @@ public void testTrace() throws Exception {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
SolrParams sParamsA = mapParams("q", "*:*");
Metric[] metrics = {new SumMetric("a_i"),
@ -590,7 +683,7 @@ public void testTrace() throws Exception {
new CountMetric()};
StatsStream statsStream = new StatsStream(zkHost, COLLECTIONORALIAS, sParamsA, metrics);
statsStream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(statsStream);
assertEquals(1, tuples.size());
@ -618,7 +711,9 @@ public void testTrace() throws Exception {
assertEquals(7.0, avgi.doubleValue(), .01);
assertEquals(5.5, avgf.doubleValue(), .001);
assertEquals(10, count.doubleValue(), .01);
} finally {
solrClientCache.close();
}
}
@Test
@ -637,6 +732,11 @@ public void testTrace() throws Exception {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc");
Bucket[] buckets = {new Bucket("a_s")};
@ -820,6 +920,7 @@ public void testTrace() throws Exception {
facetStream = new FacetStream(zkHost, COLLECTIONORALIAS, sParamsA, buckets, metrics, sorts, 100);
facetStream.setStreamContext(streamContext);
tuples = getTuples(facetStream);
@ -901,7 +1002,7 @@ public void testTrace() throws Exception {
sorts[0] = new FieldComparator("a_s", ComparatorOrder.ASCENDING);
facetStream = new FacetStream(zkHost, COLLECTIONORALIAS, sParamsA, buckets, metrics, sorts, 100);
facetStream.setStreamContext(streamContext);
tuples = getTuples(facetStream);
assertEquals(3, tuples.size());
@ -974,7 +1075,9 @@ public void testTrace() throws Exception {
assertEquals(7.5, avgi.doubleValue(), 0.1);
assertEquals(5.5, avgf.doubleValue(), 0.1);
assertEquals(2, count.doubleValue(), 0.1);
} finally {
solrClientCache.close();
}
}
@ -1042,7 +1145,11 @@ public void testTrace() throws Exception {
List<String> selectOrder = ("asc".equals(sortDir)) ? Arrays.asList(ascOrder) : Arrays.asList(descOrder);
List<String> selectOrderBool = ("asc".equals(sortDir)) ? Arrays.asList(ascOrderBool) : Arrays.asList(descOrderBool);
SolrParams exportParams = mapParams("q", "*:*", "qt", "/export", "fl", "id," + field, "sort", field + " " + sortDir + ",id asc");
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try (CloudSolrStream solrStream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, exportParams)) {
solrStream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(solrStream);
assertEquals("There should be exactly 32 responses returned", 32, tuples.size());
// Since the getTuples method doesn't return the EOF tuple, these two entries should be the same size.
@ -1053,6 +1160,8 @@ public void testTrace() throws Exception {
"' RESTORE GETTING selectOrder from select statement after LUCENE-7548",
tuples.get(idx).getString("id"), (field.startsWith("b_") ? selectOrderBool.get(idx) : selectOrder.get(idx)));
}
} finally {
solrClientCache.close();
}
}
@ -1081,7 +1190,12 @@ public void testTrace() throws Exception {
}
SolrParams sParams = mapParams("q", "*:*", "qt", "/export", "fl", fl.toString(), "sort", "id asc");
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try (CloudSolrStream solrStream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams)) {
solrStream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(solrStream);
assertEquals("There should be exactly 32 responses returned", 32, tuples.size());
@ -1097,6 +1211,8 @@ public void testTrace() throws Exception {
}
}
}
} finally {
solrClientCache.close();
}
}
@ -1229,6 +1345,12 @@ public void testTrace() throws Exception {
.add(id, "9", "level1_s", "hello0", "level2_s", "b", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_i,a_f");
Bucket[] buckets = {new Bucket("level1_s"), new Bucket("level2_s")};
@ -1246,7 +1368,7 @@ public void testTrace() throws Exception {
metrics,
sorts,
100);
facetStream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(facetStream);
assertEquals(6, tuples.size());
@ -1326,7 +1448,7 @@ public void testTrace() throws Exception {
metrics,
sorts,
100);
facetStream.setStreamContext(streamContext);
tuples = getTuples(facetStream);
assertEquals(6, tuples.size());
@ -1395,7 +1517,9 @@ public void testTrace() throws Exception {
assertEquals("a", bucket2);
assertEquals(2, sumi.longValue());
assertEquals(2, count.doubleValue(), 0.1);
} finally {
solrClientCache.close();
}
}
@Test
@ -1413,7 +1537,11 @@ public void testTrace() throws Exception {
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
@ -1430,6 +1558,7 @@ public void testTrace() throws Exception {
new CountMetric()};
RollupStream rollupStream = new RollupStream(stream, buckets, metrics);
rollupStream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(rollupStream);
assert (tuples.size() == 3);
@ -1510,6 +1639,7 @@ public void testTrace() throws Exception {
// Test will null metrics
rollupStream = new RollupStream(stream, buckets, metrics);
rollupStream.setStreamContext(streamContext);
tuples = getTuples(rollupStream);
assert (tuples.size() == 3);
@ -1533,7 +1663,6 @@ public void testTrace() throws Exception {
sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc", "qt", "/export");
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
Bucket[] buckets1 = {new Bucket("a_s")};
Metric[] metrics1 = {new SumMetric("a_i"),
@ -1547,6 +1676,7 @@ public void testTrace() throws Exception {
new CountMetric()};
rollupStream = new RollupStream(stream, buckets1, metrics1);
rollupStream.setStreamContext(streamContext);
tuples = getTuples(rollupStream);
//Check that we've got the extra NULL bucket
assertEquals(4, tuples.size());
@ -1572,7 +1702,9 @@ public void testTrace() throws Exception {
assertEquals(14, avgi.doubleValue(), 0.01);
assertEquals(10, avgf.doubleValue(), 0.01);
assertEquals(1, count.doubleValue(), 0.01);
} finally {
solrClientCache.close();
}
}
@Test
@ -1583,6 +1715,7 @@ public void testTrace() throws Exception {
SolrClientCache cache = new SolrClientCache();
context.setSolrClientCache(cache);
try {
SolrParams sParams = mapParams("q", "a_s:hello0", "rows", "500", "fl", "id");
TopicStream topicStream = new TopicStream(zkHost,
@ -1605,6 +1738,7 @@ public void testTrace() throws Exception {
int count = 0;
while (count == 0) {
SolrStream solrStream = new SolrStream(jetty.getBaseUrl().toString() + "/" + COLLECTIONORALIAS, sParams1);
solrStream.setStreamContext(context);
List<Tuple> tuples = getTuples(solrStream);
count = tuples.size();
if (count > 0) {
@ -1642,7 +1776,10 @@ public void testTrace() throws Exception {
assertTrue(tuple.EOF);
daemonStream.close();
} finally {
cache.close();
}
}
@ -1662,6 +1799,11 @@ public void testTrace() throws Exception {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc", "partitionKeys", "a_s");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
@ -1680,6 +1822,7 @@ public void testTrace() throws Exception {
RollupStream rollupStream = new RollupStream(stream, buckets, metrics);
ParallelStream parallelStream = parallelStream(rollupStream, new FieldComparator("a_s", ComparatorOrder.ASCENDING));
attachStreamFactory(parallelStream);
parallelStream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(parallelStream);
assertEquals(3, tuples.size());
@ -1754,7 +1897,9 @@ public void testTrace() throws Exception {
assertEquals(7.5, avgi.doubleValue(), 0.001);
assertEquals(5.5, avgf.doubleValue(), 0.001);
assertEquals(2, count.doubleValue(), 0.001);
} finally {
solrClientCache.close();
}
}
@Test
@ -1773,6 +1918,10 @@ public void testTrace() throws Exception {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
SolrParams sParamsA = mapParams("q", "blah", "fl", "id,a_s,a_i,a_f", "sort", "a_s asc,a_f asc", "partitionKeys", "a_s");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
ReducerStream rstream = new ReducerStream(stream,
@ -1780,8 +1929,12 @@ public void testTrace() throws Exception {
new GroupOperation(new FieldComparator("a_s", ComparatorOrder.ASCENDING), 2));
ParallelStream pstream = parallelStream(rstream, new FieldComparator("a_s", ComparatorOrder.ASCENDING));
attachStreamFactory(pstream);
pstream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(pstream);
assert (tuples.size() == 0);
} finally {
solrClientCache.close();
}
}
@ -1793,14 +1946,20 @@ public void testTrace() throws Exception {
"1", "i_multi", "2", "f_multi", "1.2", "f_multi", "1.3")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
SolrParams sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f,s_multi,i_multi,f_multi", "sort", "a_s asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
stream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(stream);
Tuple tuple = tuples.get(0);
String s = tuple.getString("a_s");
assertEquals("hello0", s);
;
long l = tuple.getLong("a_i");
assertEquals(0, l);
@ -1820,7 +1979,9 @@ public void testTrace() throws Exception {
List<Double> doubleList = tuple.getDoubles("f_multi");
assertEquals(1.2, doubleList.get(0).doubleValue(), 0.001);
assertEquals(1.3, doubleList.get(1).doubleValue(), 0.001);
} finally {
solrClientCache.close();
}
}
@Test
@ -1834,6 +1995,11 @@ public void testTrace() throws Exception {
.add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
//Test ascending
SolrParams sParamsA = mapParams("q", "id:(4 1)", "fl", "id,a_s,a_i", "sort", "a_i asc");
CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
@ -1842,6 +2008,7 @@ public void testTrace() throws Exception {
CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsB);
MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i", ComparatorOrder.ASCENDING));
mstream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(mstream);
assertEquals(5, tuples.size());
@ -1855,6 +2022,7 @@ public void testTrace() throws Exception {
streamB = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsB);
mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i", ComparatorOrder.DESCENDING));
mstream.setStreamContext(streamContext);
tuples = getTuples(mstream);
assertEquals(5, tuples.size());
@ -1869,6 +2037,7 @@ public void testTrace() throws Exception {
streamB = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsB);
mstream = new MergeStream(streamA, streamB, new MultipleFieldComparator(new FieldComparator("a_f", ComparatorOrder.ASCENDING), new FieldComparator("a_i", ComparatorOrder.ASCENDING)));
mstream.setStreamContext(streamContext);
tuples = getTuples(mstream);
assertEquals(5, tuples.size());
@ -1881,11 +2050,14 @@ public void testTrace() throws Exception {
streamB = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsB);
mstream = new MergeStream(streamA, streamB, new MultipleFieldComparator(new FieldComparator("a_f", ComparatorOrder.ASCENDING), new FieldComparator("a_i", ComparatorOrder.DESCENDING)));
mstream.setStreamContext(streamContext);
tuples = getTuples(mstream);
assertEquals(5, tuples.size());
assertOrder(tuples, 2, 0, 1, 3, 4);
} finally {
solrClientCache.close();
}
}
@Test
@ -1904,6 +2076,11 @@ public void testTrace() throws Exception {
.add(id, "9", "a_s", "hello1", "a_i", "100", "a_f", "1")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
//Test ascending
SolrParams sParamsA = mapParams("q", "id:(4 1 8 7 9)", "fl", "id,a_s,a_i", "sort", "a_i asc", "partitionKeys", "a_i");
CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
@ -1914,6 +2091,7 @@ public void testTrace() throws Exception {
MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i", ComparatorOrder.ASCENDING));
ParallelStream pstream = parallelStream(mstream, new FieldComparator("a_i", ComparatorOrder.ASCENDING));
attachStreamFactory(pstream);
pstream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(pstream);
assertEquals(9, tuples.size());
@ -1929,11 +2107,14 @@ public void testTrace() throws Exception {
mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i", ComparatorOrder.DESCENDING));
pstream = parallelStream(mstream, new FieldComparator("a_i", ComparatorOrder.DESCENDING));
attachStreamFactory(pstream);
pstream.setStreamContext(streamContext);
tuples = getTuples(pstream);
assertEquals(8, tuples.size());
assertOrder(tuples, 9, 8, 6, 4, 3, 2, 1, 0);
} finally {
solrClientCache.close();
}
}
@Test
@ -1952,6 +2133,11 @@ public void testTrace() throws Exception {
.add(id, "9", "a_s", "hello1", "a_i", "100", "a_f", "1")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
try {
//Test ascending
SolrParams sParamsA = mapParams("q", "id:(4 1 8 7 9)", "fl", "id,a_s,a_i", "sort", "a_i asc", "partitionKeys", "a_i");
CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
@ -1962,12 +2148,15 @@ public void testTrace() throws Exception {
MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i", ComparatorOrder.ASCENDING));
ParallelStream pstream = parallelStream(mstream, new FieldComparator("a_i", ComparatorOrder.ASCENDING));
attachStreamFactory(pstream);
pstream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(pstream);
assertEquals(9, tuples.size());
Map<String, Tuple> eofTuples = pstream.getEofTuples();
assertEquals(numWorkers, eofTuples.size()); // There should be an EOF Tuple for each worker.
} finally {
solrClientCache.close();
}
}
@Test
@ -1982,10 +2171,15 @@ public void testTrace() throws Exception {
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
//Basic CloudSolrStream Test with Descending Sort
try {
SolrParams sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i desc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
stream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(stream);
assertEquals(5, tuples.size());
@ -1994,6 +2188,7 @@ public void testTrace() throws Exception {
//With Ascending Sort
sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc");
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
stream.setStreamContext(streamContext);
tuples = getTuples(stream);
assertEquals(5, tuples.size());
@ -2003,6 +2198,7 @@ public void testTrace() throws Exception {
//Test compound sort
sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i desc");
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
stream.setStreamContext(streamContext);
tuples = getTuples(stream);
assertEquals(5, tuples.size());
@ -2011,11 +2207,14 @@ public void testTrace() throws Exception {
sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i asc");
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
stream.setStreamContext(streamContext);
tuples = getTuples(stream);
assertEquals(5, tuples.size());
assertOrder(tuples, 0, 2, 1, 3, 4);
} finally {
solrClientCache.close();
}
}
@Test
@ -2037,8 +2236,13 @@ public void testTrace() throws Exception {
//Basic CloudSolrStream Test bools desc
SolrParams sParams = mapParams("q", "*:*", "qt", which, "fl", "id,b_sing", "sort", "b_sing asc,id asc");
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
try {
stream.setStreamContext(streamContext);
List<Tuple> tuples = getTuples(stream);
assertEquals(5, tuples.size());
@ -2047,6 +2251,7 @@ public void testTrace() throws Exception {
//Basic CloudSolrStream Test bools desc
sParams = mapParams("q", "*:*", "qt", which, "fl", "id,b_sing", "sort", "b_sing desc,id desc");
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
stream.setStreamContext(streamContext);
tuples = getTuples(stream);
assertEquals(5, tuples.size());
@ -2055,6 +2260,7 @@ public void testTrace() throws Exception {
//Basic CloudSolrStream Test dates desc
sParams = mapParams("q", "*:*", "qt", which, "fl", "id,dt_sing", "sort", "dt_sing desc,id asc");
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
stream.setStreamContext(streamContext);
tuples = getTuples(stream);
assertEquals(5, tuples.size());
@ -2063,14 +2269,13 @@ public void testTrace() throws Exception {
//Basic CloudSolrStream Test ates desc
sParams = mapParams("q", "*:*", "qt", which, "fl", "id,dt_sing", "sort", "dt_sing asc,id desc");
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
stream.setStreamContext(streamContext);
tuples = getTuples(stream);
assertEquals (5,tuples.size());
assertOrder(tuples, 3, 4, 1, 0, 2);
} finally {
if (stream != null) {
stream.close();
}
solrClientCache.close();
}
}
@ -2099,11 +2304,15 @@ public void testTrace() throws Exception {
// We should be getting the exact same thing back with both the export and select handlers, so test
private void tryWithQt(String which) throws IOException {
StreamContext streamContext = new StreamContext();
SolrClientCache solrClientCache = new SolrClientCache();
streamContext.setSolrClientCache(solrClientCache);
SolrParams sParams = StreamingTest.mapParams("q", "*:*", "qt", which, "fl",
"id,i_sing,i_multi,l_sing,l_multi,f_sing,f_multi,d_sing,d_multi,dt_sing,dt_multi,s_sing,s_multi,b_sing,b_multi",
"sort", "i_sing asc");
try (CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams)) {
stream.setStreamContext(streamContext);
Tuple tuple = getTuple(stream); // All I really care about is that all the fields are returned. There's
assertEquals("Integers should be returned", 11, tuple.getLong("i_sing").longValue());
@ -2141,6 +2350,8 @@ public void testTrace() throws Exception {
assertTrue("Booleans should be returned", tuple.getBool("b_sing"));
assertFalse("MV boolean should be returned for b_multi", tuple.getBools("b_multi").get(0));
assertTrue("MV boolean should be returned for b_multi", tuple.getBools("b_multi").get(1));
} finally {
solrClientCache.close();
}
}