SOLR-13014: URI Too Long with large streaming expressions in SolrJ

This commit is contained in:
Jan Høydahl 2018-12-10 16:05:29 +01:00
parent 0650f995f9
commit b6cbb8494e
8 changed files with 81 additions and 7 deletions

View File

@ -147,6 +147,8 @@ Bug Fixes
* SOLR-12933: Fix SolrCloud distributed commit. (Mark Miller)
* SOLR-13014: URI Too Long with large streaming expressions in SolrJ (janhoy)
Improvements
----------------------

View File

@ -28,6 +28,7 @@ import java.util.Map.Entry;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.CloudSolrClient.Builder;
import org.apache.solr.client.solrj.io.SolrClientCache;
@ -496,7 +497,7 @@ public class FacetStream extends TupleStream implements Expressible {
paramsLoc.set("json.facet", json);
paramsLoc.set("rows", "0");
QueryRequest request = new QueryRequest(paramsLoc);
QueryRequest request = new QueryRequest(paramsLoc, SolrRequest.METHOD.POST);
try {
NamedList response = cloudSolrClient.request(request, collection);
getTuples(response, buckets, metrics);

View File

@ -29,6 +29,7 @@ import java.util.Optional;
import java.util.Random;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.io.SolrClientCache;
import org.apache.solr.client.solrj.io.Tuple;
@ -191,7 +192,7 @@ public class RandomStream extends TupleStream implements Expressible {
String sortField = "random_"+seed;
params.add(SORT, sortField+" asc");
QueryRequest request = new QueryRequest(params);
QueryRequest request = new QueryRequest(params, SolrRequest.METHOD.POST);
try {
QueryResponse response = request.process(cloudSolrClient, collection);
SolrDocumentList docs = response.getResults();

View File

@ -28,6 +28,7 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.io.SolrClientCache;
import org.apache.solr.client.solrj.io.Tuple;
@ -185,7 +186,7 @@ public class SearchStream extends TupleStream implements Expressible {
}
QueryRequest request = new QueryRequest(params);
QueryRequest request = new QueryRequest(params, SolrRequest.METHOD.POST);
try {
QueryResponse response = request.process(cloudSolrClient, collection);
SolrDocumentList docs = response.getResults();

View File

@ -30,6 +30,7 @@ import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.io.SolrClientCache;
import org.apache.solr.client.solrj.io.Tuple;
@ -398,7 +399,7 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
params.add("field", field);
params.add("numTerms", String.valueOf(numTerms*5));
QueryRequest request= new QueryRequest(params);
QueryRequest request= new QueryRequest(params, SolrRequest.METHOD.POST);
QueryResponse response = request.process(solrClient);
NamedList res = response.getResponse();
return res;

View File

@ -25,6 +25,7 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.io.SolrClientCache;
@ -208,7 +209,7 @@ public class StatsStream extends TupleStream implements Expressible {
Map<String, List<String>> shardsMap = (Map<String, List<String>>)streamContext.get("shards");
if(shardsMap == null) {
QueryRequest request = new QueryRequest(paramsLoc);
QueryRequest request = new QueryRequest(paramsLoc, SolrRequest.METHOD.POST);
CloudSolrClient cloudSolrClient = cache.getCloudSolrClient(zkHost);
try {
NamedList response = cloudSolrClient.request(request, collection);
@ -226,7 +227,7 @@ public class StatsStream extends TupleStream implements Expressible {
paramsLoc.add("distrib", "true");
}
QueryRequest request = new QueryRequest(paramsLoc);
QueryRequest request = new QueryRequest(paramsLoc, SolrRequest.METHOD.POST);
try {
NamedList response = client.request(request);
this.tuple = getTuple(response);

View File

@ -30,6 +30,7 @@ import java.util.Optional;
import java.util.Map.Entry;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.CloudSolrClient.Builder;
import org.apache.solr.client.solrj.io.SolrClientCache;
@ -288,7 +289,7 @@ public class TimeSeriesStream extends TupleStream implements Expressible {
paramsLoc.set("json.facet", json);
paramsLoc.set("rows", "0");
QueryRequest request = new QueryRequest(paramsLoc);
QueryRequest request = new QueryRequest(paramsLoc, SolrRequest.METHOD.POST);
try {
NamedList response = cloudSolrClient.request(request, collection);
getTuples(response, field, metrics);

View File

@ -23,10 +23,13 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.io.ClassificationEvaluation;
import org.apache.solr.client.solrj.io.SolrClientCache;
@ -2669,6 +2672,69 @@ public class StreamExpressionTest extends SolrCloudTestCase {
}
}
@Test
public void tooLargeForGetRequest() throws IOException, SolrServerException {
// Test expressions which are larger than GET can handle
UpdateRequest updateRequest = new UpdateRequest();
for (int i = 0; i < 10; i++) {
updateRequest.add(id, "a"+i, "test_t", "a b c d m l");
}
for(int i=1; i<=50; i++) {
updateRequest.add(id, "id_"+(i),"test_dt", getDateString("2016", "5", "1"), "price_f", "400.00");
}
updateRequest.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
SolrClientCache cache = new SolrClientCache();
StreamContext streamContext = new StreamContext();
streamContext.setSolrClientCache(cache);
String longQuery = "\"id:(" + IntStream.range(0, 4000).mapToObj(i -> "a").collect(Collectors.joining(" ", "", "")) + ")\"";
try {
assertSuccess("significantTerms("+COLLECTIONORALIAS+", q="+longQuery+", field=\"test_t\", limit=3, minTermLength=1, maxDocFreq=\".5\")", streamContext);
String expr = "timeseries("+COLLECTIONORALIAS+", q="+longQuery+", start=\"2013-01-01T01:00:00.000Z\", " +
"end=\"2016-12-01T01:00:00.000Z\", " +
"gap=\"+1YEAR\", " +
"field=\"test_dt\", " +
"format=\"yyyy\", " +
"count(*), sum(price_f), max(price_f), min(price_f))";
assertSuccess(expr, streamContext);
expr = "facet("
+ "collection1, "
+ "q="+longQuery+", "
+ "fl=\"a_s,a_i,a_f\", "
+ "sort=\"a_s asc\", "
+ "buckets=\"a_s\", "
+ "bucketSorts=\"sum(a_i) asc\", "
+ "bucketSizeLimit=100, "
+ "sum(a_i), sum(a_f), "
+ "min(a_i), min(a_f), "
+ "max(a_i), max(a_f), "
+ "avg(a_i), avg(a_f), "
+ "count(*)"
+ ")";
assertSuccess(expr, streamContext);
expr = "stats(" + COLLECTIONORALIAS + ", q="+longQuery+", sum(a_i), sum(a_f), min(a_i), min(a_f), max(a_i), max(a_f), avg(a_i), avg(a_f), count(*))";
assertSuccess(expr, streamContext);
expr = "search(" + COLLECTIONORALIAS + ", q="+longQuery+", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")";
assertSuccess(expr, streamContext);
expr = "random(" + COLLECTIONORALIAS + ", q="+longQuery+", rows=\"1000\", fl=\"id, a_i\")";
assertSuccess(expr, streamContext);
} finally {
cache.close();
}
}
private void assertSuccess(String expr, StreamContext streamContext) throws IOException {
ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
paramsLoc.set("expr", expr);
paramsLoc.set("qt", "/stream");
String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString()+"/"+COLLECTIONORALIAS;
TupleStream solrStream = new SolrStream(url, paramsLoc);
solrStream.setStreamContext(streamContext);
getTuples(solrStream);
}
protected List<Tuple> getTuples(TupleStream tupleStream) throws IOException {
List<Tuple> tuples = new ArrayList<Tuple>();