SOLR-13306 Add a request parameter to execute a streaming expression locally

This commit is contained in:
Gus Heck 2019-05-08 12:13:07 -04:00
parent f4399a495e
commit 76b854cb4f
11 changed files with 246 additions and 74 deletions

View File

@ -73,7 +73,9 @@ Jetty 9.4.14.v20181114
New Features
----------------------
* SOLR-13320 : add an update param failOnVersionConflicts=false to updates not fail when there is a version conflict (noble)
* SOLR-13320: add an update param failOnVersionConflicts=false to updates not fail when there is a version conflict (noble)
* SOLR-13306: Add a request parameter to execute a streaming expression locally
================== 8.1.0 ==================
@ -252,7 +254,7 @@ Bug Fixes
* SOLR-5970: Return correct status upon collection creation failure (Abraham Elmahrek, Ishan Chattopadhyaya,
Jason Gerlowski, Kesharee Nandan Vishwakarma)
* SOLR-12291: prematurely reporting not yet finished async Collections API call as completed
* SOLR-12291: prematurely reporting not yet finished async Collections API call as completed
when collection's replicas are collocated at least at one node (Varun Thacker, Mikhail Khludnev)
* SOLR-13410: Designated overseer wasn't able to rejoin election queue upon restart (Ishan Chattopadhyaya,

View File

@ -87,10 +87,10 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
/*
* The stream factory will always contain the zkUrl for the given collection Adds default streams with their
* corresponding function names. These defaults can be overridden or added to in the solrConfig in the stream
* RequestHandler def. Example config override
* <lst name="streamFunctions">
* <str name="group">org.apache.solr.client.solrj.io.stream.ReducerStream</str>
* <str name="count">org.apache.solr.client.solrj.io.stream.RecordCountStream</str>
* RequestHandler def. Example config override
* <lst name="streamFunctions">
* <str name="group">org.apache.solr.client.solrj.io.stream.ReducerStream</str>
* <str name="count">org.apache.solr.client.solrj.io.stream.RecordCountStream</str>
* </lst>
*/
@ -162,6 +162,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
int worker = params.getInt("workerID", 0);
int numWorkers = params.getInt("numWorkers", 1);
boolean local = params.getBool("streamLocalOnly", false);
StreamContext context = new StreamContext();
context.put("shards", getCollectionShards(params));
context.workerID = worker;
@ -171,6 +172,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
context.setObjectCache(objectCache);
context.put("core", this.coreName);
context.put("solr-core", req.getCore());
context.setLocal(local);
tupleStream.setStreamContext(context);
// if asking for explanation then go get it

View File

@ -114,6 +114,15 @@ unless the jvm has been started with `-DStreamingExpressionMacros=true` (usually
Because streaming expressions relies on the `/export` handler, many of the field and field type requirements to use `/export` are also requirements for `/stream`, particularly for `sort` and `fl` parameters. Please see the section <<exporting-result-sets.adoc#exporting-result-sets,Exporting Result Sets>> for details.
=== Local Execution
In certain special cases such as joining documents on a value that is 1:1 with the portion of the id used in
composite routing, the relevant data is always co-located on the same node. When this happens, fanning out requests
among many nodes and waiting for a response from all nodes is inefficient. In cases where data co-location holds true
for the entire expression, it may be faster for the client to send the expression to each slice with
`&streamLocalOnly=true` and handle merging of the results (if required) locally. This is an advanced option, relying
on a convenient organization of the index, and should only be considered if normal usage poses a performance issue.
== Types of Streaming Expressions
=== About Stream Sources

View File

@ -85,7 +85,7 @@ public class CloudSolrStream extends TupleStream implements Expressible {
// Used by parallel stream
protected CloudSolrStream(){
}
/**
@ -98,7 +98,7 @@ public class CloudSolrStream extends TupleStream implements Expressible {
init(collectionName, zkHost, params);
}
public CloudSolrStream(StreamExpression expression, StreamFactory factory) throws IOException{
public CloudSolrStream(StreamExpression expression, StreamFactory factory) throws IOException{
// grab all parameters out
String collectionName = factory.getValueOperand(expression, 0);
List<StreamExpressionNamedParameter> namedParams = factory.getNamedOperands(expression);
@ -114,12 +114,12 @@ public class CloudSolrStream extends TupleStream implements Expressible {
if(expression.getParameters().size() != 1 + namedParams.size()){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - unknown operands found",expression));
}
// Named parameters - passed directly to solr as solrparams
if(0 == namedParams.size()){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - at least one named parameter expected. eg. 'q=*:*'",expression));
}
ModifiableSolrParams mParams = new ModifiableSolrParams();
for(StreamExpressionNamedParameter namedParam : namedParams){
if(!namedParam.getName().equals("zkHost") && !namedParam.getName().equals("aliases")){
@ -157,25 +157,25 @@ public class CloudSolrStream extends TupleStream implements Expressible {
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);
}
@Override
public StreamExpression toExpression(StreamFactory factory) throws IOException {
// functionName(collectionName, param1, param2, ..., paramN, sort="comp", [aliases="field=alias,..."])
// function name
StreamExpression expression = new StreamExpression("search");
// collection
if(collection.indexOf(',') > -1) {
expression.addParameter("\""+collection+"\"");
} else {
expression.addParameter(collection);
}
for (Entry<String, String[]> param : params.getMap().entrySet()) {
for (String val : param.getValue()) {
// SOLR-8409: Escaping the " is a special case.
@ -185,10 +185,10 @@ public class CloudSolrStream extends TupleStream implements Expressible {
val.replace("\"", "\\\"")));
}
}
// zkHost
expression.addParameter(new StreamExpressionNamedParameter("zkHost", zkHost));
// aliases
if(null != fieldMappings && 0 != fieldMappings.size()){
StringBuilder sb = new StringBuilder();
@ -198,35 +198,35 @@ public class CloudSolrStream extends TupleStream implements Expressible {
sb.append("=");
sb.append(mapping.getValue());
}
expression.addParameter(new StreamExpressionNamedParameter("aliases", sb.toString()));
}
return expression;
return expression;
}
@Override
public Explanation toExplanation(StreamFactory factory) throws IOException {
StreamExplanation explanation = new StreamExplanation(getStreamNodeId().toString());
explanation.setFunctionName("search");
explanation.setImplementingClass(this.getClass().getName());
explanation.setExpressionType(ExpressionType.STREAM_SOURCE);
explanation.setExpression(toExpression(factory).toString());
// child is a datastore so add it at this point
StreamExplanation child = new StreamExplanation(getStreamNodeId() + "-datastore");
child.setFunctionName(String.format(Locale.ROOT, "solr (%s)", collection));
child.setImplementingClass("Solr/Lucene");
child.setExpressionType(ExpressionType.DATASTORE);
if(null != params){
ModifiableSolrParams mParams = new ModifiableSolrParams(params);
child.setExpression(mParams.getMap().entrySet().stream().map(e -> String.format(Locale.ROOT, "%s=%s", e.getKey(), e.getValue())).collect(Collectors.joining(",")));
}
explanation.addChild(child);
return explanation;
}
@ -254,7 +254,7 @@ public class CloudSolrStream extends TupleStream implements Expressible {
String sorts = String.join(",", params.getParams(SORT));
this.comp = parseComp(sorts, fls);
}
public void setFieldMappings(Map<String, String> fieldMappings) {
this.fieldMappings = fieldMappings;
}
@ -302,23 +302,23 @@ public class CloudSolrStream extends TupleStream implements Expressible {
String s = sorts[i];
String[] spec = s.trim().split("\\s+"); //This should take into account spaces in the sort spec.
if (spec.length != 2) {
throw new IOException("Invalid sort spec:" + s);
}
String fieldName = spec[0].trim();
String order = spec[1].trim();
if(!fieldSet.contains(spec[0])) {
throw new IOException("Fields in the sort spec must be included in the field list:"+spec[0]);
}
// if there's an alias for the field then use the alias
if(null != fieldMappings && fieldMappings.containsKey(fieldName)){
fieldName = fieldMappings.get(fieldName);
}
comps[i] = new FieldComparator(fieldName, order.equalsIgnoreCase("asc") ? ComparatorOrder.ASCENDING : ComparatorOrder.DESCENDING);
}
@ -381,6 +381,9 @@ public class CloudSolrStream extends TupleStream implements Expressible {
SolrStream solrStream = new SolrStream(shardUrl, mParams);
if(streamContext != null) {
solrStream.setStreamContext(streamContext);
if (streamContext.isLocal()) {
solrStream.setDistrib(false);
}
}
solrStream.setFieldMappings(this.fieldMappings);
solrStreams.add(solrStream);
@ -425,7 +428,7 @@ public class CloudSolrStream extends TupleStream implements Expressible {
}
}
}
/** Return the stream sort - ie, the order in which records are returned */
public StreamComparator getStreamSort(){
return comp;

View File

@ -398,6 +398,9 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
params.add("minTermLength", Integer.toString(minTermLength));
params.add("field", field);
params.add("numTerms", String.valueOf(numTerms*5));
if (streamContext.isLocal()) {
params.add("distrib", "false");
}
QueryRequest request= new QueryRequest(params, SolrRequest.METHOD.POST);
QueryResponse response = request.process(solrClient);

View File

@ -65,6 +65,7 @@ public class SolrStream extends TupleStream {
private String slice;
private long checkpoint = -1;
private CloseableHttpResponse closeableHttpResponse;
private boolean distrib = true;
/**
* @param baseUrl Base URL of the stream.
@ -89,6 +90,7 @@ public class SolrStream extends TupleStream {
}
public void setStreamContext(StreamContext context) {
this.distrib = !context.isLocal();
this.numWorkers = context.numWorkers;
this.workerID = context.workerID;
this.cache = context.getSolrClientCache();
@ -106,7 +108,11 @@ public class SolrStream extends TupleStream {
}
try {
tupleStreamParser = constructParser(client, loadParams(params));
SolrParams requestParams = loadParams(params);
if (!distrib) {
((ModifiableSolrParams) requestParams).add("distrib","false");
}
tupleStreamParser = constructParser(client, requestParams);
} catch (Exception e) {
throw new IOException("params " + params, e);
}
@ -128,7 +134,7 @@ public class SolrStream extends TupleStream {
this.checkpoint = checkpoint;
}
private SolrParams loadParams(SolrParams paramsIn) throws IOException {
private ModifiableSolrParams loadParams(SolrParams paramsIn) throws IOException {
ModifiableSolrParams solrParams = new ModifiableSolrParams(paramsIn);
if (params.get("partitionKeys") != null) {
if(!params.get("partitionKeys").equals("none") && numWorkers > 1) {
@ -164,7 +170,7 @@ public class SolrStream extends TupleStream {
.withExpressionType(ExpressionType.STREAM_SOURCE)
.withExpression("non-expressible");
}
/**
* Closes the Stream to a single Solr Instance
* */
@ -219,12 +225,20 @@ public class SolrStream extends TupleStream {
}
}
public void setDistrib(boolean distrib) {
this.distrib = distrib;
}
public boolean getDistrib() {
return distrib;
}
public static class HandledException extends IOException {
public HandledException(String msg) {
super(msg);
}
}
/** There is no known sort applied to a SolrStream */
public StreamComparator getStreamSort(){
return null;

View File

@ -200,6 +200,9 @@ public class SqlStream extends TupleStream implements Expressible {
this.tupleStream = new SolrStream(url, mParams);
if(streamContext != null) {
tupleStream.setStreamContext(streamContext);
if(streamContext.isLocal()) {
mParams.add("distrib", "false");
}
}
} catch (Exception e) {
throw new IOException(e);

View File

@ -84,23 +84,23 @@ public class StatsStream extends TupleStream implements Expressible {
}
}
public StatsStream(StreamExpression expression, StreamFactory factory) throws IOException{
public StatsStream(StreamExpression expression, StreamFactory factory) throws IOException{
// grab all parameters out
String collectionName = factory.getValueOperand(expression, 0);
List<StreamExpressionNamedParameter> namedParams = factory.getNamedOperands(expression);
List<StreamExpression> metricExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Metric.class);
StreamExpressionNamedParameter zkHostExpression = factory.getNamedOperand(expression, "zkHost");
// Validate there are no unknown parameters - zkHost is namedParameter so we don't need to count it twice
if(expression.getParameters().size() != 1 + namedParams.size() + metricExpressions.size()){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - unknown operands found",expression));
}
// Collection Name
if(null == collectionName){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - collectionName expected as first operand",expression));
}
// Named parameters - passed directly to solr as solrparams
if(0 == namedParams.size()){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - at least one named parameter expected. eg. 'q=*:*'",expression));
@ -112,7 +112,7 @@ public class StatsStream extends TupleStream implements Expressible {
params.set(namedParam.getName(), namedParam.getParameter().toString().trim());
}
}
// zkHost, optional - if not provided then will look into factory list to get
String zkHost = null;
if(null == zkHostExpression){
@ -130,65 +130,65 @@ public class StatsStream extends TupleStream implements Expressible {
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - zkHost not found for collection '%s'",expression,collectionName));
}
*/
// metrics, optional - if not provided then why are you using this?
Metric[] metrics = new Metric[metricExpressions.size()];
for(int idx = 0; idx < metricExpressions.size(); ++idx){
metrics[idx] = factory.constructMetric(metricExpressions.get(idx));
}
// We've got all the required items
init(zkHost, collectionName, params, metrics);
}
@Override
public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
// functionName(collectionName, param1, param2, ..., paramN, sort="comp", sum(fieldA), avg(fieldB))
// function name
StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
// collection
expression.addParameter(collection);
// parameters
ModifiableSolrParams mParams = new ModifiableSolrParams(params);
for (Entry<String, String[]> param : mParams.getMap().entrySet()) {
expression.addParameter(new StreamExpressionNamedParameter(param.getKey(), String.join(",", param.getValue())));
}
// zkHost
expression.addParameter(new StreamExpressionNamedParameter("zkHost", zkHost));
// metrics
for(Metric metric : metrics){
expression.addParameter(metric.toExpression(factory));
}
return expression;
return expression;
}
@Override
public Explanation toExplanation(StreamFactory factory) throws IOException {
StreamExplanation explanation = new StreamExplanation(getStreamNodeId().toString());
explanation.setFunctionName(factory.getFunctionName(this.getClass()));
explanation.setImplementingClass(this.getClass().getName());
explanation.setExpressionType(ExpressionType.STREAM_SOURCE);
explanation.setExpression(toExpression(factory).toString());
StreamExplanation child = new StreamExplanation(getStreamNodeId() + "-datastore");
child.setFunctionName(String.format(Locale.ROOT, "solr (worker ? of ?)"));
child.setFunctionName(String.format(Locale.ROOT, "solr (worker ? of ?)"));
// TODO: fix this so we know the # of workers - check with Joel about a Stat's ability to be in a
// parallel stream.
child.setImplementingClass("Solr/Lucene");
child.setExpressionType(ExpressionType.DATASTORE);
ModifiableSolrParams mParams = new ModifiableSolrParams(params);
child.setExpression(mParams.getMap().entrySet().stream().map(e -> String.format(Locale.ROOT, "%s=%s", e.getKey(), e.getValue())).collect(Collectors.joining(",")));
explanation.addChild(child);
return explanation;
}
@ -206,6 +206,9 @@ public class StatsStream extends TupleStream implements Expressible {
addStats(paramsLoc, metrics);
paramsLoc.set("stats", "true");
paramsLoc.set("rows", "0");
if (streamContext.isLocal()) {
paramsLoc.set("distrib", "false");
}
Map<String, List<String>> shardsMap = (Map<String, List<String>>)streamContext.get("shards");
if(shardsMap == null) {

View File

@ -26,15 +26,15 @@ import org.apache.solr.client.solrj.io.SolrClientCache;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
/**
* The StreamContext is passed to TupleStreams using the TupleStream.setStreamContext() method.
* The StreamContext is used to pass shared context to concentrically wrapped TupleStreams.
* The StreamContext is passed to TupleStreams using the TupleStream.setStreamContext() method.
* The StreamContext is used to pass shared context to concentrically wrapped TupleStreams.
*
* Note: The StreamContext contains the SolrClientCache which is used to cache SolrClients for reuse
* across multiple TupleStreams.
* Note: The StreamContext contains the SolrClientCache which is used to cache SolrClients for reuse
* across multiple TupleStreams.
**/
public class StreamContext implements Serializable{
public class StreamContext implements Serializable {
private Map entries = new HashMap();
private Map tupleContext = new HashMap();
@ -45,6 +45,7 @@ public class StreamContext implements Serializable{
private SolrClientCache clientCache;
private ModelCache modelCache;
private StreamFactory streamFactory;
private boolean local;
public ConcurrentMap getObjectCache() {
return this.objectCache;
@ -54,7 +55,7 @@ public class StreamContext implements Serializable{
this.objectCache = objectCache;
}
public Map<String, Object> getLets(){
public Map<String, Object> getLets() {
return lets;
}
@ -101,4 +102,12 @@ public class StreamContext implements Serializable{
public StreamFactory getStreamFactory() {
return this.streamFactory;
}
public void setLocal(boolean local) {
this.local = local;
}
public boolean isLocal() {
return local;
}
}

View File

@ -49,7 +49,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
public abstract class TupleStream implements Closeable, Serializable, MapWriter {
private static final long serialVersionUID = 1;
private UUID streamNodeId = UUID.randomUUID();
public TupleStream() {
@ -66,9 +66,9 @@ public abstract class TupleStream implements Closeable, Serializable, MapWriter
public abstract Tuple read() throws IOException;
public abstract StreamComparator getStreamSort();
public abstract Explanation toExplanation(StreamFactory factory) throws IOException;
public int getCost() {
return 0;
}
@ -150,6 +150,10 @@ public abstract class TupleStream implements Closeable, Serializable, MapWriter
shards.add(url);
}
}
Object core = streamContext.get("core");
if (streamContext != null && streamContext.isLocal() && core != null) {
shards.removeIf(shardUrl -> !shardUrl.contains((CharSequence) core));
}
return shards;
}

View File

@ -49,6 +49,9 @@ import org.apache.solr.client.solrj.io.stream.metrics.SumMetric;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.junit.Assume;
@ -2297,6 +2300,122 @@ public void testParallelRankStream() throws Exception {
}
}
/**
* This test verifies that setting a core into the stream context entries and streamContext.local = true causes the
* streaming expression to only consider data found on the local node.
*/
@Test
public void streamLocalTests() throws Exception {
new UpdateRequest()
.add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0")
.add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0")
.add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3")
.add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4")
.add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamContext streamContext = new StreamContext();
streamContext.setLocal(true);
ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
List<String> strings = zkStateReader.aliasesManager.getAliases().resolveAliases(COLLECTIONORALIAS);
String collName = strings.size() > 0 ? strings.get(0) : COLLECTIONORALIAS;
zkStateReader.forceUpdateCollection(collName);
DocCollection collection = zkStateReader.getClusterState().getCollectionOrNull(collName);
List<Replica> replicas = collection.getReplicas();
streamContext.getEntries().put("core",replicas.get(random().nextInt(replicas.size())).getCoreName());
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);
// note if hashing algo changes this might break
switch (tuples.size()) {
case 5: // 1 shard
assertOrder(tuples, 4, 3, 2, 1, 0);
break;
case 3: // 2 shards case 1 (randomized)
assertOrder(tuples, 4, 1, 0);
break;
case 2: // 2 shards case 2 (randomized)
assertOrder(tuples, 3, 2);
break;
default: // nope, no way, no how, never good.
fail("should have 3, 5 or 2 tuples, has hashing algorithm changed?");
}
//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);
// note if hashing algo changes this might break
switch (tuples.size()) {
case 5: // 1 shard
assertOrder(tuples, 0, 1, 2, 3, 4);
break;
case 3: // 2 shards case 1 (randomized)
assertOrder(tuples, 0, 1, 4);
break;
case 2: // 2 shards case 2 (randomized)
assertOrder(tuples, 2, 3);
break;
default: // nope, no way, no how, never good.
fail("should have 3, 5 or 2 tuples, has hashing algorithm changed?");
}
//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);
// note if hashing algo changes this might break
switch (tuples.size()) {
case 5: // 1 shard
assertOrder(tuples, 2, 0, 1, 3, 4);
break;
case 3: // 2 shards case 1 (randomized)
assertOrder(tuples, 0, 1, 4);
break;
case 2: // 2 shards case 2 (randomized)
assertOrder(tuples, 2, 3);
break;
default: // nope, no way, no how, never good.
fail("should have 3, 5 or 2 tuples, has hashing algorithm changed?");
}
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);
// note if hashing algo changes this might break
switch (tuples.size()) {
case 5: // 1 shard
assertOrder(tuples, 0, 2, 1, 3, 4);
break;
case 3: // 2 shards case 1 (randomized)
assertOrder(tuples, 0, 1, 4);
break;
case 2: // 2 shards case 2 (randomized)
assertOrder(tuples, 2, 3);
break;
default: // nope, no way, no how, never good.
fail("should have 3, 5 or 2 tuples, has hashing algorithm changed?");
}
} finally {
solrClientCache.close();
}
}
@Test
public void testDateBoolSorting() throws Exception {
@ -2381,14 +2500,14 @@ public void testParallelRankStream() throws Exception {
tryWithQt("/export");
tryWithQt("/select");
}
// 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",
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)) {
@ -2426,7 +2545,7 @@ public void testParallelRankStream() throws Exception {
assertTrue("MV dates should be returned as Dates for dt_multi", tuple.getDates("dt_multi").get(0).equals(dt));
dt = new Date(Instant.parse("1981-05-24T04:05:06.990Z").toEpochMilli());
assertTrue("MV dates should be returned as Dates for dt_multi", tuple.getDates("dt_multi").get(1).equals(dt));
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));
@ -2463,8 +2582,9 @@ public void testParallelRankStream() throws Exception {
for(int val : ids) {
Tuple t = tuples.get(i);
String tip = (String)t.get("id");
if(!tip.equals(Integer.toString(val))) {
throw new Exception("Found value:"+tip+" expecting:"+val);
String valStr = Integer.toString(val);
if(!tip.equals(valStr)) {
assertEquals("Found value:"+tip+" expecting:"+valStr, val, tip);
}
++i;
}
@ -2510,7 +2630,7 @@ public void testParallelRankStream() throws Exception {
return true;
}
private void attachStreamFactory(TupleStream tupleStream) {
StreamContext streamContext = new StreamContext();
streamContext.setStreamFactory(streamFactory);
@ -2526,10 +2646,10 @@ public void testParallelRankStream() throws Exception {
if(random().nextBoolean()) params.add("wt","javabin");
return params;
}
private ParallelStream parallelStream(TupleStream stream, FieldComparator comparator) throws IOException {
ParallelStream pstream = new ParallelStream(zkHost, COLLECTIONORALIAS, stream, numWorkers, comparator);
return pstream;
}
}
}