LUCENE-6766: merge master

This commit is contained in:
Mike McCandless 2016-05-10 05:44:36 -04:00
commit e3ecc6a536
38 changed files with 2411 additions and 864 deletions

View File

@ -164,7 +164,7 @@
<property name="javac.target" value="1.8"/>
<property name="javac.args" value="-Xlint -Xlint:-deprecation -Xlint:-serial -Xlint:-options"/>
<property name="javac.profile.args" value="-profile compact2"/>
<property name="javadoc.link" value="http://download.oracle.com/javase/8/docs/api/"/>
<property name="javadoc.link" value="https://docs.oracle.com/javase/8/docs/api/"/>
<property name="javadoc.link.junit" value="http://junit.sourceforge.net/javadoc/"/>
<property name="javadoc.packagelist.dir" location="${common.dir}/tools/javadoc"/>
<available file="${javadoc.packagelist.dir}/java8/package-list" property="javadoc.java8.packagelist.exists"/>

View File

@ -126,6 +126,15 @@ New Features
* SOLR-8972: Add GraphHandler and GraphMLResponseWriter to support graph visualizations (Joel Bernstein)
* SOLR-9026: Extend facet telemetry support to legacy (non-json) facets under "debug/facet-debug" in
the response. (Michael Sun, yonik)
* SOLR-7117: Provide an option to limit the maximum number of cores that can be created on a node by the
Auto Add Replica feature. For this you can set a "maxCoresPerNode" property via the Cluster Property API
(Varun Thacker, Mark Miller)
* SOLR-8208: [subquery] document transformer executes separate requests per result document. (Cao Manh Dat via Mikhail Khludnev)
Bug Fixes
----------------------
@ -264,6 +273,9 @@ Other Changes
* SOLR-8458: Add Streaming Expressions tests for parameter substitution (Joel Bernstein, Cao Manh Dat, Dennis Gove, Kevin Risden)
* SOLR-8467: CloudSolrStream and FacetStream should take a SolrParams object rather than a
Map<String, String> to allow more complex Solr queries to be specified. (Erick Erickson)
================== 6.0.0 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release

View File

@ -229,7 +229,8 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
private boolean addReplica(final String collection, DownReplica badReplica) {
// first find best home - first strategy, sort by number of cores
// hosted where maxCoresPerNode is not violated
final String createUrl = getBestCreateUrl(zkStateReader, badReplica);
final Integer maxCoreCount = (Integer) zkStateReader.getClusterProps().get(ZkStateReader.MAX_CORES_PER_NODE);
final String createUrl = getBestCreateUrl(zkStateReader, badReplica, maxCoreCount);
if (createUrl == null) {
log.warn("Could not find a node to create new replica on.");
return false;
@ -301,15 +302,16 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
* @return the best node to replace the badReplica on or null if there is no
* such node
*/
static String getBestCreateUrl(ZkStateReader zkStateReader, DownReplica badReplica) {
static String getBestCreateUrl(ZkStateReader zkStateReader, DownReplica badReplica, Integer maxCoreCount) {
assert badReplica != null;
assert badReplica.collection != null;
assert badReplica.slice != null;
log.debug("getBestCreateUrl for " + badReplica.replica);
Map<String,Counts> counts = new HashMap<String, Counts>();
Set<String> unsuitableHosts = new HashSet<String>();
Map<String,Counts> counts = new HashMap<>();
Set<String> unsuitableHosts = new HashSet<>();
Set<String> liveNodes = new HashSet<>(zkStateReader.getClusterState().getLiveNodes());
Map<String, Integer> coresPerNode = new HashMap<>();
ClusterState clusterState = zkStateReader.getClusterState();
if (clusterState != null) {
@ -329,8 +331,13 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
for (Replica replica : replicas) {
liveNodes.remove(replica.getNodeName());
String baseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
if (baseUrl.equals(
badReplica.replica.getStr(ZkStateReader.BASE_URL_PROP))) {
if (coresPerNode.containsKey(baseUrl)) {
Integer nodeCount = coresPerNode.get(baseUrl);
coresPerNode.put(baseUrl, nodeCount++);
} else {
coresPerNode.put(baseUrl, 1);
}
if (baseUrl.equals(badReplica.replica.getStr(ZkStateReader.BASE_URL_PROP))) {
continue;
}
// on a live node?
@ -351,16 +358,15 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
if (badReplica.collection.getName().equals(collection) && badReplica.slice.getName().equals(slice.getName())) {
cnt.ourReplicas++;
}
// TODO: this is collection wide and we want to take into
// account cluster wide - use new cluster sys prop
Integer maxShardsPerNode = badReplica.collection.getMaxShardsPerNode();
if (maxShardsPerNode == null) {
log.warn("maxShardsPerNode is not defined for collection, name=" + badReplica.collection.getName());
maxShardsPerNode = Integer.MAX_VALUE;
}
log.debug("collection={} node={} max shards per node={} potential hosts={}", collection, baseUrl, maxShardsPerNode, cnt);
log.debug("collection={} node={} maxShardsPerNode={} maxCoresPerNode={} potential hosts={}",
collection, baseUrl, maxShardsPerNode, maxCoreCount, cnt);
Collection<Replica> badSliceReplicas = null;
DocCollection c = clusterState.getCollection(badReplica.collection.getName());
if (c != null) {
@ -370,7 +376,8 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
}
}
boolean alreadyExistsOnNode = replicaAlreadyExistsOnNode(zkStateReader.getClusterState(), badSliceReplicas, badReplica, baseUrl);
if (unsuitableHosts.contains(baseUrl) || alreadyExistsOnNode || cnt.collectionShardsOnNode >= maxShardsPerNode) {
if (unsuitableHosts.contains(baseUrl) || alreadyExistsOnNode || cnt.collectionShardsOnNode >= maxShardsPerNode
|| (maxCoreCount != null && coresPerNode.get(baseUrl) >= maxCoreCount) ) {
counts.remove(baseUrl);
unsuitableHosts.add(baseUrl);
log.debug("not a candidate node, collection={} node={} max shards per node={} good replicas={}", collection, baseUrl, maxShardsPerNode, cnt);

View File

@ -144,9 +144,8 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe
}
private SolrParams adjustParams(SolrParams params) {
ModifiableSolrParams adjustedParams = new ModifiableSolrParams();
adjustedParams.add(params);
adjustedParams.add(CommonParams.OMIT_HEADER, "true");
ModifiableSolrParams adjustedParams = new ModifiableSolrParams(params);
adjustedParams.set(CommonParams.OMIT_HEADER, "true");
return adjustedParams;
}
@ -230,18 +229,18 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe
String zkHost = tableSpec.zkHost;
String collection = tableSpec.collection;
Map<String, String> params = new HashMap();
ModifiableSolrParams params = new ModifiableSolrParams();
params.put(CommonParams.FL, fl);
params.put(CommonParams.Q, sqlVisitor.query);
params.set(CommonParams.FL, fl);
params.set(CommonParams.Q, sqlVisitor.query);
//Always use the /export handler for Group By Queries because it requires exporting full result sets.
params.put(CommonParams.QT, "/export");
params.set(CommonParams.QT, "/export");
if(numWorkers > 1) {
params.put("partitionKeys", getPartitionKeys(buckets));
params.set("partitionKeys", getPartitionKeys(buckets));
}
params.put("sort", sort);
params.set("sort", sort);
TupleStream tupleStream = null;
@ -370,18 +369,18 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe
String zkHost = tableSpec.zkHost;
String collection = tableSpec.collection;
Map<String, String> params = new HashMap();
ModifiableSolrParams params = new ModifiableSolrParams();
params.put(CommonParams.FL, fl);
params.put(CommonParams.Q, sqlVisitor.query);
params.set(CommonParams.FL, fl);
params.set(CommonParams.Q, sqlVisitor.query);
//Always use the /export handler for Distinct Queries because it requires exporting full result sets.
params.put(CommonParams.QT, "/export");
params.set(CommonParams.QT, "/export");
if(numWorkers > 1) {
params.put("partitionKeys", getPartitionKeys(buckets));
params.set("partitionKeys", getPartitionKeys(buckets));
}
params.put("sort", sort);
params.set("sort", sort);
TupleStream tupleStream = null;
@ -463,9 +462,9 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe
String zkHost = tableSpec.zkHost;
String collection = tableSpec.collection;
Map<String, String> params = new HashMap();
ModifiableSolrParams params = new ModifiableSolrParams();
params.put(CommonParams.Q, sqlVisitor.query);
params.set(CommonParams.Q, sqlVisitor.query);
int limit = sqlVisitor.limit > 0 ? sqlVisitor.limit : 100;
@ -512,9 +511,9 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe
String zkHost = tableSpec.zkHost;
String collection = tableSpec.collection;
Map<String, String> params = new HashMap();
ModifiableSolrParams params = new ModifiableSolrParams();
params.put(CommonParams.Q, sqlVisitor.query);
params.set(CommonParams.Q, sqlVisitor.query);
int limit = sqlVisitor.limit > 0 ? sqlVisitor.limit : 100;
@ -628,22 +627,22 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe
}
}
Map<String, String> params = new HashMap();
params.put("fl", fl.toString());
params.put("q", sqlVisitor.query);
ModifiableSolrParams params = new ModifiableSolrParams();
params.set("fl", fl.toString());
params.set("q", sqlVisitor.query);
if(siBuf.length() > 0) {
params.put("sort", siBuf.toString());
params.set("sort", siBuf.toString());
}
TupleStream tupleStream;
if(sqlVisitor.limit > -1) {
params.put("rows", Integer.toString(sqlVisitor.limit));
params.set("rows", Integer.toString(sqlVisitor.limit));
tupleStream = new LimitStream(new CloudSolrStream(zkHost, collection, params), sqlVisitor.limit);
} else {
//Only use the export handler when no limit is specified.
params.put(CommonParams.QT, "/export");
params.set(CommonParams.QT, "/export");
tupleStream = new CloudSolrStream(zkHost, collection, params);
}
@ -681,9 +680,9 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe
String zkHost = tableSpec.zkHost;
String collection = tableSpec.collection;
Map<String, String> params = new HashMap();
ModifiableSolrParams params = new ModifiableSolrParams();
params.put(CommonParams.Q, sqlVisitor.query);
params.set(CommonParams.Q, sqlVisitor.query);
TupleStream tupleStream = new StatsStream(zkHost,
collection,

View File

@ -114,6 +114,11 @@ public class DebugComponent extends SearchComponent
if (fdebug != null) {
info.add("facet-trace", fdebug.getFacetDebugInfo());
}
fdebug = (FacetDebugInfo)(rb.req.getContext().get("FacetDebugInfo-nonJson"));
if (fdebug != null) {
info.add("facet-debug", fdebug.getFacetDebugInfo());
}
if (rb.req.getJSON() != null) {
info.add(JSON, rb.req.getJSON());

View File

@ -22,7 +22,6 @@ import java.net.URL;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
@ -50,6 +49,8 @@ import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.schema.FieldType;
import org.apache.solr.search.QueryParsing;
import org.apache.solr.search.SyntaxError;
import org.apache.solr.search.facet.FacetDebugInfo;
import org.apache.solr.util.RTimer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -252,7 +253,16 @@ public class FacetComponent extends SearchComponent {
SolrParams params = rb.req.getParams();
SimpleFacets f = new SimpleFacets(rb.req, rb.getResults().docSet, params, rb);
NamedList<Object> counts = FacetComponent.getFacetCounts(f);
RTimer timer = null;
FacetDebugInfo fdebug = null;
if (rb.isDebug()) {
fdebug = new FacetDebugInfo();
rb.req.getContext().put("FacetDebugInfo-nonJson", fdebug);
timer = new RTimer();
}
NamedList<Object> counts = FacetComponent.getFacetCounts(f, fdebug);
String[] pivots = params.getParams(FacetParams.FACET_PIVOT);
if (!ArrayUtils.isEmpty(pivots)) {
PivotFacetProcessor pivotProcessor
@ -264,10 +274,19 @@ public class FacetComponent extends SearchComponent {
}
}
if (fdebug != null) {
long timeElapsed = (long) timer.getTime();
fdebug.setElapse(timeElapsed);
}
rb.rsp.add("facet_counts", counts);
}
}
public static NamedList<Object> getFacetCounts(SimpleFacets simpleFacets) {
return getFacetCounts(simpleFacets, null);
}
/**
* Looks at various Params to determining if any simple Facet Constraint count
* computations are desired.
@ -279,7 +298,7 @@ public class FacetComponent extends SearchComponent {
* @see FacetParams#FACET
* @return a NamedList of Facet Count info or null
*/
public static NamedList<Object> getFacetCounts(SimpleFacets simpleFacets) {
public static NamedList<Object> getFacetCounts(SimpleFacets simpleFacets, FacetDebugInfo fdebug) {
// if someone called this method, benefit of the doubt: assume true
if (!simpleFacets.getGlobalParams().getBool(FacetParams.FACET, true))
return null;
@ -288,7 +307,19 @@ public class FacetComponent extends SearchComponent {
NamedList<Object> counts = new SimpleOrderedMap<>();
try {
counts.add(FACET_QUERY_KEY, simpleFacets.getFacetQueryCounts());
counts.add(FACET_FIELD_KEY, simpleFacets.getFacetFieldCounts());
if (fdebug != null) {
FacetDebugInfo fd = new FacetDebugInfo();
fd.putInfoItem("action", "field facet");
fd.setProcessor(simpleFacets.getClass().getSimpleName());
fdebug.addChild(fd);
simpleFacets.setFacetDebugInfo(fd);
final RTimer timer = new RTimer();
counts.add(FACET_FIELD_KEY, simpleFacets.getFacetFieldCounts());
long timeElapsed = (long) timer.getTime();
fd.setElapse(timeElapsed);
} else {
counts.add(FACET_FIELD_KEY, simpleFacets.getFacetFieldCounts());
}
counts.add(FACET_RANGES_KEY, rangeFacetProcessor.getFacetRangeCounts());
counts.add(FACET_INTERVALS_KEY, simpleFacets.getFacetIntervalCounts());
counts.add(SpatialHeatmapFacets.RESPONSE_KEY, simpleFacets.getHeatmapCounts());

View File

@ -74,16 +74,17 @@ public class MoreLikeThisComponent extends SearchComponent {
public void process(ResponseBuilder rb) throws IOException {
SolrParams params = rb.req.getParams();
ReturnFields returnFields = new SolrReturnFields( rb.req );
int flags = 0;
if (returnFields.wantsScore()) {
flags |= SolrIndexSearcher.GET_SCORES;
}
rb.setFieldFlags(flags);
if (params.getBool(MoreLikeThisParams.MLT, false)) {
ReturnFields returnFields = new SolrReturnFields( rb.req );
int flags = 0;
if (returnFields.wantsScore()) {
flags |= SolrIndexSearcher.GET_SCORES;
}
rb.setFieldFlags(flags);
log.debug("Starting MoreLikeThis.Process. isShard: "
+ params.getBool(ShardParams.IS_SHARD));
SolrIndexSearcher searcher = rb.req.getSearcher();

View File

@ -16,6 +16,9 @@
*/
package org.apache.solr.request;
import java.io.IOException;
import java.util.List;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues;
@ -37,11 +40,9 @@ import org.apache.solr.schema.SchemaField;
import org.apache.solr.search.DocSet;
import org.apache.solr.search.Filter;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.search.facet.FacetDebugInfo;
import org.apache.solr.util.LongPriorityQueue;
import java.io.IOException;
import java.util.List;
/**
* Computes term facets for docvalues field (single or multivalued).
* <p>
@ -57,7 +58,7 @@ import java.util.List;
public class DocValuesFacets {
private DocValuesFacets() {}
public static NamedList<Integer> getCounts(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix, String contains, boolean ignoreCase) throws IOException {
public static NamedList<Integer> getCounts(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix, String contains, boolean ignoreCase, FacetDebugInfo fdebug) throws IOException {
SchemaField schemaField = searcher.getSchema().getField(fieldName);
FieldType ft = schemaField.getType();
NamedList<Integer> res = new NamedList<>();
@ -118,6 +119,9 @@ public class DocValuesFacets {
// count collection array only needs to be as big as the number of terms we are
// going to collect counts for.
final int[] counts = new int[nTerms];
if (fdebug != null) {
fdebug.putInfoItem("numBuckets", nTerms);
}
Filter filter = docs.getTopFilter();
List<LeafReaderContext> leaves = searcher.getTopReaderContext().leaves();

View File

@ -16,6 +16,24 @@
*/
package org.apache.solr.request;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.IdentityHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.Future;
import java.util.concurrent.FutureTask;
import java.util.concurrent.RunnableFuture;
import java.util.concurrent.Semaphore;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.TimeUnit;
import org.apache.commons.lang.StringUtils;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.LeafReader;
@ -49,7 +67,6 @@ import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.handler.component.FacetComponent;
import org.apache.solr.handler.component.ResponseBuilder;
import org.apache.solr.handler.component.SpatialHeatmapFacets;
import org.apache.solr.request.IntervalFacets.FacetInterval;
@ -69,28 +86,12 @@ import org.apache.solr.search.QueryParsing;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.search.SortedIntDocSet;
import org.apache.solr.search.SyntaxError;
import org.apache.solr.search.facet.FacetDebugInfo;
import org.apache.solr.search.facet.FacetProcessor;
import org.apache.solr.search.grouping.GroupingSpecification;
import org.apache.solr.util.BoundedTreeSet;
import org.apache.solr.util.DefaultSolrThreadFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.IdentityHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.Future;
import java.util.concurrent.FutureTask;
import java.util.concurrent.RunnableFuture;
import java.util.concurrent.Semaphore;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.TimeUnit;
import org.apache.solr.util.RTimer;
/**
* A class that generates simple Facet information for a request.
@ -109,6 +110,9 @@ public class SimpleFacets {
protected final SolrQueryRequest req;
protected final ResponseBuilder rb;
protected FacetDebugInfo fdebugParent;
protected FacetDebugInfo fdebug;
// per-facet values
protected final static class ParsedParams {
final public SolrParams localParams; // localParams on this particular facet command
@ -160,6 +164,10 @@ public class SimpleFacets {
this.rb = rb;
}
public void setFacetDebugInfo(FacetDebugInfo fdebugParent) {
this.fdebugParent = fdebugParent;
}
/**
* Returns <code>true</code> if a String contains the given substring. Otherwise
* <code>false</code>.
@ -450,6 +458,14 @@ public class SimpleFacets {
method = FacetMethod.FC;
}
RTimer timer = null;
if (fdebug != null) {
fdebug.putInfoItem("method", method.name());
fdebug.putInfoItem("inputDocSetSize", docs.size());
fdebug.putInfoItem("field", field);
timer = new RTimer();
}
if (params.getFieldBool(field, GroupParams.GROUP_FACET, false)) {
counts = getGroupedCounts(searcher, docs, field, multiToken, offset,limit, mincount, missing, sort, prefix, contains, ignoreCase);
} else {
@ -535,13 +551,18 @@ public class SimpleFacets {
}
break;
case FC:
counts = DocValuesFacets.getCounts(searcher, docs, field, offset,limit, mincount, missing, sort, prefix, contains, ignoreCase);
counts = DocValuesFacets.getCounts(searcher, docs, field, offset,limit, mincount, missing, sort, prefix, contains, ignoreCase, fdebug);
break;
default:
throw new AssertionError();
}
}
if (fdebug != null) {
long timeElapsed = (long) timer.getTime();
fdebug.setElapse(timeElapsed);
}
return counts;
}
@ -654,9 +675,17 @@ public class SimpleFacets {
final Semaphore semaphore = new Semaphore((maxThreads <= 0) ? Integer.MAX_VALUE : maxThreads);
List<Future<NamedList>> futures = new ArrayList<>(facetFs.length);
if (fdebugParent != null) {
fdebugParent.putInfoItem("maxThreads", maxThreads);
}
try {
//Loop over fields; submit to executor, keeping the future
for (String f : facetFs) {
if (fdebugParent != null) {
fdebug = new FacetDebugInfo();
fdebugParent.addChild(fdebug);
}
final ParsedParams parsed = parseParams(FacetParams.FACET_FIELD, f);
final SolrParams localParams = parsed.localParams;
final String termList = localParams == null ? null : localParams.get(CommonParams.TERMS);

View File

@ -0,0 +1,360 @@
/*
* 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.response.transform;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.Callable;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.Query;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrRequestInfo;
import org.apache.solr.response.ResultContext;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.search.DocList;
import org.apache.solr.search.DocSlice;
import org.apache.solr.search.JoinQParserPlugin;
import org.apache.solr.search.ReturnFields;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.search.SolrReturnFields;
import org.apache.solr.search.TermsQParserPlugin;
/**
*
* This transformer executes subquery per every result document. It must be be given uniq name.
* There might be a few of them, eg <code>fl=*,foo:[subquery],bar:[subquery]</code>.
* Every [subquery] occurrence adds a field into a result document with the given name,
* the value of this field is a document list, which is a result of executing subquery using
* document fields as an input.
*
* <h3>Subquery Parameters Shift</h3>
* if subquery is declared as <code>fl=*,foo:[subquery]</code>, subquery parameters
* are prefixed with the given name and period. eg <br>
* <code>q=*:*&amp;fl=*,foo:[subquery]&amp;foo.q=to be continued&amp;foo.rows=10&amp;foo.sort=id desc</code>
*
* <h3>Document field as an input param for subquery</h3>
*
* It's necessary to pass some document field value as a parametr for subquery. It's supported via
* implicit <code>row.<i>fieldname</i></code> parameter, and can be (but might not only) referred via
* Local Parameters syntax.<br>
* <code>q=namne:john&amp;fl=name,id,depts:[subquery]&amp;depts.q={!terms f=id v=$row.dept_id}&amp;depts.rows=10</code>
* Here departmens are retrieved per every employee in search result. We can say that it's like SQL
* <code> join ON emp.dept_id=dept.id </code><br>
* Note, when document field has multiple values they are concatenated with comma by default, it can be changed by
* <code>foo:[subquery separator=' ']</code> local parameter, this mimics {@link TermsQParserPlugin} to work smoothly with.
*
* <h3>Cores and Collections in Cloud</h3>
* use <code>foo:[subquery fromIndex=departments]</code> invoke subquery on another core on this node, it's like
* {@link JoinQParserPlugin} for non SolrCloud mode. <b>But for SolrCloud</b> just (and only) <b>explicitly specify</b>
* its' native parameters like <code>collection, shards</code> for subquery, eg<br>
* <code>q=*:*&amp;fl=*,foo:[subquery]&amp;foo.q=cloud&amp;foo.collection=departments</code>
*
*/
public class SubQueryAugmenterFactory extends TransformerFactory{
@Override
public DocTransformer create(String field, SolrParams params, SolrQueryRequest req) {
if (field.contains("[") || field.contains("]")) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"please give an exlicit name for [subquery] column ie fl=relation:[subquery ..]");
}
checkThereIsNoDupe(field, req.getContext());
String fromIndex = params.get("fromIndex");
final SolrClient solrClient;
solrClient = new EmbeddedSolrServer(req.getCore());
SolrParams subParams = retainAndShiftPrefix(req.getParams(), field+".");
return new SubQueryAugmenter(solrClient, fromIndex, field,
field,
subParams,
params.get(TermsQParserPlugin.SEPARATOR, ","));
}
@SuppressWarnings("unchecked")
private void checkThereIsNoDupe(String field, Map<Object,Object> context) {
// find a map
final Map conflictMap;
final String conflictMapKey = getClass().getSimpleName();
if (context.containsKey(conflictMapKey)) {
conflictMap = (Map) context.get(conflictMapKey);
} else {
conflictMap = new HashMap<>();
context.put(conflictMapKey, conflictMap);
}
// check entry absence
if (conflictMap.containsKey(field)) {
throw new SolrException(ErrorCode.BAD_REQUEST,
"[subquery] name "+field+" is duplicated");
} else {
conflictMap.put(field, true);
}
}
private SolrParams retainAndShiftPrefix(SolrParams params, String subPrefix) {
ModifiableSolrParams out = new ModifiableSolrParams();
Iterator<String> baseKeyIt = params.getParameterNamesIterator();
while (baseKeyIt.hasNext()) {
String key = baseKeyIt.next();
if (key.startsWith(subPrefix)) {
out.set(key.substring(subPrefix.length()), params.getParams(key));
}
}
return out;
}
}
class SubQueryAugmenter extends DocTransformer {
private static final class Result extends ResultContext {
private final SolrDocumentList docList;
final SolrReturnFields justWantAllFields = new SolrReturnFields();
private Result(SolrDocumentList docList) {
this.docList = docList;
}
@Override
public ReturnFields getReturnFields() {
return justWantAllFields;
}
@Override
public Iterator<SolrDocument> getProcessedDocuments(){
return docList.iterator();
}
@Override
public boolean wantsScores() {
return justWantAllFields.wantsScore();
}
@Override
public DocList getDocList() {
return new DocSlice((int)docList.getStart(),
docList.size(), new int[0], new float[docList.size()],
(int) docList.getNumFound(),
docList.getMaxScore() == null ? Float.NaN : docList.getMaxScore());
}
@Override
public SolrIndexSearcher getSearcher() {
return null;
}
@Override
public SolrQueryRequest getRequest() {
return null;
}
@Override
public Query getQuery() {
return null;
}
}
/** project document values to prefixed parameters
* multivalues are joined with a separator, it always return single value */
static final class DocRowParams extends SolrParams {
final private SolrDocument doc;
final private String prefixDotRowDot;
final private String separator;
public DocRowParams(SolrDocument doc, String prefix, String separator ) {
this.doc = doc;
this.prefixDotRowDot = "row.";//prefix+ ".row.";
this.separator = separator;
}
@Override
public String[] getParams(String param) {
final Collection<Object> vals = mapToDocField(param);
if (vals != null) {
StringBuilder rez = new StringBuilder();
int i = 0;
for (Iterator iterator = vals.iterator(); iterator.hasNext();) {
Object object = (Object) iterator.next();
rez.append(convertFieldValue(object));
if (iterator.hasNext()) {
rez.append(separator);
}
}
return new String[]{rez.toString()};
}
return null;
}
@Override
public String get(String param) {
final String[] aVal = this.getParams(param);
if (aVal != null) {
assert aVal.length == 1 : "that's how getParams is written" ;
return aVal[0];
}
return null;
}
/** @return null if prefix doesn't match, field is absent or empty */
protected Collection<Object> mapToDocField(String param) {
if (param.startsWith(prefixDotRowDot)) {
final String docFieldName = param.substring(prefixDotRowDot.length());
final Collection<Object> vals = doc.getFieldValues(docFieldName);
if (vals == null || vals.isEmpty()) {
return null;
} else {
return vals;
}
}
return null;
}
protected String convertFieldValue(Object val) {
if (val instanceof IndexableField) {
IndexableField f = (IndexableField)val;
return f.stringValue();
}
return val.toString();
}
@Override
public Iterator<String> getParameterNamesIterator() {
final Iterator<String> fieldNames = doc.getFieldNames().iterator();
return new Iterator<String>() {
@Override
public boolean hasNext() {
return fieldNames.hasNext();
}
@Override
public String next() {
final String fieldName = fieldNames.next();
return prefixDotRowDot + fieldName;
}
};
}
}
final private String name;
final private SolrParams baseSubParams;
final private String prefix;
final private String separator;
final private SolrClient server;
final private String coreName;
public SubQueryAugmenter(SolrClient server, String coreName,
String name,String prefix, SolrParams baseSubParams, String separator) {
this.name = name;
this.prefix = prefix;
this.baseSubParams = baseSubParams;
this.separator = separator;
this.server = server;
this.coreName = coreName;
}
@Override
public String getName() {
return name;
}
@Override
public void transform(SolrDocument doc, int docid, float score) {
final SolrParams docWithDeprefixed = SolrParams.wrapDefaults(
new DocRowParams(doc, prefix, separator), baseSubParams);
try {
Callable<QueryResponse> subQuery = new Callable<QueryResponse>() {
@Override
public QueryResponse call() throws Exception {
try {
return new QueryResponse(
server.request(
new QueryRequest(docWithDeprefixed), coreName)
, server);
} finally {
}
}
};
QueryResponse response =
SolrRequestInfoSuspender.doInSuspension(subQuery);
final SolrDocumentList docList = (SolrDocumentList) response.getResults();
doc.setField(getName(), new Result(docList));
} catch (Exception e) {
String docString = doc.toString();
throw new SolrException(ErrorCode.BAD_REQUEST, "while invoking " +
name + ":[subquery"+ (coreName!=null ? "fromIndex="+coreName : "") +"] on doc=" +
docString.substring(0, Math.min(100, docString.length())), e.getCause());
} finally {}
}
// look ma!! no hands..
final static class SolrRequestInfoSuspender extends SolrRequestInfo {
private SolrRequestInfoSuspender(SolrQueryRequest req, SolrQueryResponse rsp) {
super(req, rsp);
}
/** Suspends current SolrRequestInfo invoke the given action, and resumes then */
static <T> T doInSuspension(Callable<T> action) throws Exception {
final SolrRequestInfo info = threadLocal.get();
try {
threadLocal.remove();
return action.call();
} finally {
setRequestInfo(info);
}
}
}
}

View File

@ -47,6 +47,7 @@ public abstract class TransformerFactory implements NamedListInitializedPlugin
defaultFactories.put( "docid", new DocIdAugmenterFactory() );
defaultFactories.put( "shard", new ShardAugmenterFactory() );
defaultFactories.put( "child", new ChildDocTransformerFactory() );
defaultFactories.put( "subquery", new SubQueryAugmenterFactory() );
defaultFactories.put( "json", new RawValueTransformerFactory("json") );
defaultFactories.put( "xml", new RawValueTransformerFactory("xml") );
defaultFactories.put( "geo", new GeoTransformerFactory() );

View File

@ -1493,10 +1493,10 @@ public class IndexSchema {
return Stream.of(Handler.values())
.filter(it -> name == null || it.nameLower.equals(name))
.map(it -> new Pair<>(it.realName, it.fun.apply(this)))
.filter(it->it.getValue() != null)
.filter(it->it.second() != null)
.collect(Collectors.toMap(
Pair::getKey,
Pair::getValue,
Pair::first,
Pair::second,
(v1, v2) -> v2,
LinkedHashMap::new));
}

View File

@ -25,7 +25,7 @@ import org.apache.solr.common.util.SimpleOrderedMap;
public class FacetDebugInfo {
String processor;
long elapse;
long elapse = -1;
String filter;
Map<String, Object> info; // additional information
final List<FacetDebugInfo> children;
@ -69,8 +69,8 @@ public class FacetDebugInfo {
SimpleOrderedMap<Object> info = new SimpleOrderedMap<>();
if (filter != null) info.add("filter", filter);
info.add("processor", processor);
info.add("elapse", elapse);
if (processor != null) info.add("processor", processor);
if (elapse != -1) info.add("elapse", elapse);
if (reqDescription != null) {
info.addAll(reqDescription);
}

View File

@ -101,8 +101,8 @@ public class TestInjection {
public static boolean injectRandomDelayInCoreCreation() {
if (randomDelayInCoreCreation != null) {
Pair<Boolean,Integer> pair = parseValue(randomDelayInCoreCreation);
boolean enabled = pair.getKey();
int chanceIn100 = pair.getValue();
boolean enabled = pair.first();
int chanceIn100 = pair.second();
if (enabled && RANDOM.nextInt(100) >= (100 - chanceIn100)) {
int delay = RANDOM.nextInt(randomDelayMaxInCoreCreationInSec);
log.info("Inject random core creation delay of {}s", delay);
@ -119,8 +119,8 @@ public class TestInjection {
public static boolean injectNonGracefullClose(CoreContainer cc) {
if (cc.isShutDown() && nonGracefullClose != null) {
Pair<Boolean,Integer> pair = parseValue(nonGracefullClose);
boolean enabled = pair.getKey();
int chanceIn100 = pair.getValue();
boolean enabled = pair.first();
int chanceIn100 = pair.second();
if (enabled && RANDOM.nextInt(100) >= (100 - chanceIn100)) {
if (RANDOM.nextBoolean()) {
throw new TestShutdownFailError("Test exception for non graceful close");
@ -157,8 +157,8 @@ public class TestInjection {
public static boolean injectFailReplicaRequests() {
if (failReplicaRequests != null) {
Pair<Boolean,Integer> pair = parseValue(failReplicaRequests);
boolean enabled = pair.getKey();
int chanceIn100 = pair.getValue();
boolean enabled = pair.first();
int chanceIn100 = pair.second();
if (enabled && RANDOM.nextInt(100) >= (100 - chanceIn100)) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Random test update fail");
}
@ -170,8 +170,8 @@ public class TestInjection {
public static boolean injectFailUpdateRequests() {
if (failUpdateRequests != null) {
Pair<Boolean,Integer> pair = parseValue(failUpdateRequests);
boolean enabled = pair.getKey();
int chanceIn100 = pair.getValue();
boolean enabled = pair.first();
int chanceIn100 = pair.second();
if (enabled && RANDOM.nextInt(100) >= (100 - chanceIn100)) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Random test update fail");
}
@ -183,8 +183,8 @@ public class TestInjection {
public static boolean injectNonExistentCoreExceptionAfterUnload(String cname) {
if (nonExistentCoreExceptionAfterUnload != null) {
Pair<Boolean,Integer> pair = parseValue(nonExistentCoreExceptionAfterUnload);
boolean enabled = pair.getKey();
int chanceIn100 = pair.getValue();
boolean enabled = pair.first();
int chanceIn100 = pair.second();
if (enabled && RANDOM.nextInt(100) >= (100 - chanceIn100)) {
throw new NonExistentCoreException("Core not found to unload: " + cname);
}
@ -196,8 +196,8 @@ public class TestInjection {
public static boolean injectUpdateLogReplayRandomPause() {
if (updateLogReplayRandomPause != null) {
Pair<Boolean,Integer> pair = parseValue(updateLogReplayRandomPause);
boolean enabled = pair.getKey();
int chanceIn100 = pair.getValue();
boolean enabled = pair.first();
int chanceIn100 = pair.second();
if (enabled && RANDOM.nextInt(100) >= (100 - chanceIn100)) {
long rndTime = RANDOM.nextInt(1000);
log.info("inject random log replay delay of {}ms", rndTime);
@ -215,8 +215,8 @@ public class TestInjection {
public static boolean injectUpdateRandomPause() {
if (updateRandomPause != null) {
Pair<Boolean,Integer> pair = parseValue(updateRandomPause);
boolean enabled = pair.getKey();
int chanceIn100 = pair.getValue();
boolean enabled = pair.first();
int chanceIn100 = pair.second();
if (enabled && RANDOM.nextInt(100) >= (100 - chanceIn100)) {
long rndTime = RANDOM.nextInt(1000);
log.info("inject random update delay of {}ms", rndTime);

View File

@ -87,6 +87,10 @@
<dynamicField name="*_dt_dv" type="date" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_dts_dv" type="date" indexed="true" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_t" type="text" indexed="true" stored="true"/>
<field name="_version_" type="long" indexed="true" stored="true" multiValued="false"/>
</fields>
<defaultSearchField>id</defaultSearchField>

View File

@ -68,7 +68,7 @@ public class SharedFSAutoReplicaFailoverUtilsTest extends SolrTestCaseJ4 {
@Before
public void setUp() throws Exception {
super.setUp();
results = new ArrayList<Result>();
results = new ArrayList<>();
}
@After
@ -82,27 +82,27 @@ public class SharedFSAutoReplicaFailoverUtilsTest extends SolrTestCaseJ4 {
@Test
public void testGetBestCreateUrlBasics() {
Result result = buildClusterState("csr1R*r2", NODE1);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertNull("Should be no live node to failover to", createUrl);
result = buildClusterState("csr1R*r2", NODE1, NODE2);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertNull("Only failover candidate node already has a replica", createUrl);
result = buildClusterState("csr1R*r2sr3", NODE1, NODE2, NODE3);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals("Node3 does not have a replica from the bad slice and should be the best choice", NODE3_URL, createUrl);
result = buildClusterState("csr1R*r2-4sr3r4r5", NODE1, NODE2, NODE3);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
assertTrue(createUrl.equals(NODE2_URL) || createUrl.equals(NODE3_URL));
result = buildClusterState("csr1R*r2Fsr3r4r5", NODE1, NODE2, NODE3);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertTrue(createUrl.equals(NODE3_URL));
result = buildClusterState("csr1*r2r3sr3r3sr4", NODE1, NODE2, NODE3, NODE4);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE4_URL, createUrl);
result = buildClusterState("csr1*r2sr3r3sr4sr4", NODE1, NODE2, NODE3, NODE4);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertTrue(createUrl.equals(NODE3_URL) || createUrl.equals(NODE4_URL));
}
@ -121,27 +121,27 @@ public class SharedFSAutoReplicaFailoverUtilsTest extends SolrTestCaseJ4 {
public void testGetBestCreateUrlMultipleCollections() throws Exception {
Result result = buildClusterState("csr*r2csr2", NODE1);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
assertEquals(null, createUrl);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertNull(createUrl);
result = buildClusterState("csr*r2csr2", NODE1);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
assertEquals(null, createUrl);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertNull(createUrl);
result = buildClusterState("csr*r2csr2", NODE1, NODE2);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
assertEquals(null, createUrl);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertNull(createUrl);
}
@Test
public void testGetBestCreateUrlMultipleCollections2() {
Result result = buildClusterState("csr*r2sr3cr2", NODE1);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
assertEquals(null, createUrl);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertNull(createUrl);
result = buildClusterState("csr*r2sr3cr2", NODE1, NODE2, NODE3);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE3_URL, createUrl);
}
@ -149,49 +149,74 @@ public class SharedFSAutoReplicaFailoverUtilsTest extends SolrTestCaseJ4 {
@Test
public void testGetBestCreateUrlMultipleCollections3() {
Result result = buildClusterState("csr5r1sr4r2sr3r6csr2*r6sr5r3sr4r3", NODE1, NODE4, NODE5, NODE6);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE1_URL, createUrl);
}
@Test
public void testGetBestCreateUrlMultipleCollections4() {
Result result = buildClusterState("csr1r4sr3r5sr2r6csr5r6sr4r6sr5*r4", NODE6);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE6_URL, createUrl);
}
@Test
public void testFailOverToEmptySolrInstance() {
Result result = buildClusterState("csr1*r1sr1csr1", NODE2);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE2_URL, createUrl);
}
@Test
public void testFavorForeignSlices() {
Result result = buildClusterState("csr*sr2csr3r3", NODE2, NODE3);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE3_URL, createUrl);
result = buildClusterState("csr*sr2csr3r3r3r3r3r3r3", NODE2, NODE3);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE2_URL, createUrl);
}
@Test
public void testCollectionMaxNodesPerShard() {
Result result = buildClusterState("csr*sr2", 1, 1, NODE2);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
assertEquals(null, createUrl);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertNull(createUrl);
result = buildClusterState("csr*sr2", 1, 2, NODE2);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE2_URL, createUrl);
result = buildClusterState("csr*csr2r2", 1, 1, NODE2);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE2_URL, createUrl);
}
@Test
public void testMaxCoresPerNode() {
Result result = buildClusterState("csr*sr2", 1, 1, NODE2);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 1);
assertNull(createUrl);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 2);
assertNull(createUrl);
result = buildClusterState("csr*sr2", 1, 2, NODE2);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 2);
assertEquals(NODE2_URL, createUrl);
result = buildClusterState("csr*sr2sr3sr4", 1, 1, NODE2, NODE3, NODE4);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 1);
assertNull(createUrl);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 2);
assertNull(createUrl);
result = buildClusterState("csr*sr2sr3sr4", 1, 2, NODE2, NODE3, NODE4);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 2);
assertTrue(createUrl.equals(NODE3_URL) || createUrl.equals(NODE4_URL));
}
private Result buildClusterState(String string, String ... liveNodes) {
return buildClusterState(string, 1, liveNodes);
@ -351,7 +376,7 @@ public class SharedFSAutoReplicaFailoverUtilsTest extends SolrTestCaseJ4 {
// trunk briefly had clusterstate taking a zkreader :( this was required to work around that - leaving
// until that issue is resolved.
MockZkStateReader reader = new MockZkStateReader(null, collectionStates.keySet());
ClusterState clusterState = new ClusterState(1, new HashSet<String>(Arrays.asList(liveNodes)), collectionStates);
ClusterState clusterState = new ClusterState(1, new HashSet<>(Arrays.asList(liveNodes)), collectionStates);
reader = new MockZkStateReader(clusterState, collectionStates.keySet());
String json;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.response;
package org.apache.solr.response.transform;
import org.apache.lucene.util.TestUtil;
import org.apache.solr.SolrTestCaseJ4;
@ -47,6 +47,8 @@ public class TestChildDocTransformer extends SolrTestCaseJ4 {
createIndex(titleVals);
testParentFilterJSON();
testParentFilterXML();
testSubQueryParentFilterJSON();
testSubQueryParentFilterXML();
}
@Test
@ -54,6 +56,9 @@ public class TestChildDocTransformer extends SolrTestCaseJ4 {
createSimpleIndex();
testChildDoctransformerJSON();
testChildDoctransformerXML();
testSubQueryXML();
testSubQueryJSON();
}
private void testChildDoctransformerXML() {
@ -88,6 +93,96 @@ public class TestChildDocTransformer extends SolrTestCaseJ4 {
assertQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
"fl", "subject,[child parentFilter=\"subject:parentDocument\" childFilter=\"title:bar\" limit=2]"), test3);
}
private void testSubQueryXML() {
String test1[];
{
final String subqueryPath = "/result[@name='children'][@numFound='6']";
test1 = new String[] {
"//*[@numFound='1']",
"/response/result/doc[1]" + subqueryPath + "/doc[1]/int[@name='id']='2'" ,
"/response/result/doc[1]" + subqueryPath + "/doc[2]/int[@name='id']='3'" ,
"/response/result/doc[1]" + subqueryPath + "/doc[3]/int[@name='id']='4'" ,
"/response/result/doc[1]" + subqueryPath + "/doc[4]/int[@name='id']='5'" ,
"/response/result/doc[1]" + subqueryPath + "/doc[5]/int[@name='id']='6'" ,
"/response/result/doc[1]" + subqueryPath + "/doc[6]/int[@name='id']='7'"};
}
assertQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
"fl", "*,children:[subquery]",
"children.q","{!child of=subject:parentDocument}{!terms f=id v=$row.id}",
"children.rows","10"),
test1);
String test2[] = new String[] {
"//*[@numFound='1']",
"/response/result/doc[1]/result[@name='children'][@numFound='3']/doc[1]/int[@name='id']='2'" ,
"/response/result/doc[1]/result[@name='children'][@numFound='3']/doc[2]/int[@name='id']='4'" ,
"/response/result/doc[1]/result[@name='children'][@numFound='3']/doc[3]/int[@name='id']='6'" };
assertQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
"fl", "*,children:[subquery]",
"children.q","{!child of=subject:parentDocument}{!terms f=id v=$row.id}",
"children.rows","10",
"children.fq","title:foo"
), test2);
String test3[] = new String[] {
"//*[@numFound='1']",
"/response/result/doc[1]/result[@name='children'][@numFound='3']/doc[1]/int[@name='id']='3'" ,
"/response/result/doc[1]/result[@name='children'][@numFound='3']/doc[2]/int[@name='id']='5'" };
assertQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
"fl", "*,children:[subquery]",
"children.q","{!child of=subject:parentDocument}{!terms f=id v=$row.id}",
"children.rows","2",
"children.fq","title:bar",
"children.sort","_docid_ asc"
), test3);
}
private void testSubQueryJSON() throws Exception {
String[] test1 = new String[] {
"/response/docs/[0]/children/docs/[0]/id==2",
"/response/docs/[0]/children/docs/[1]/id==3",
"/response/docs/[0]/children/docs/[2]/id==4",
"/response/docs/[0]/children/docs/[3]/id==5",
"/response/docs/[0]/children/docs/[4]/id==6",
"/response/docs/[0]/children/docs/[5]/id==7"
};
String[] test2 = new String[] {
"/response/docs/[0]/children/docs/[0]/id==2",
"/response/docs/[0]/children/docs/[1]/id==4",
"/response/docs/[0]/children/docs/[2]/id==6"
};
String[] test3 = new String[] {
"/response/docs/[0]/children/docs/[0]/id==3",
"/response/docs/[0]/children/docs/[1]/id==5"
};
assertJQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
"fl", "*,children:[subquery]",
"children.q","{!child of=subject:parentDocument}{!terms f=id v=$row.id}",
"children.rows","10"), test1);
assertJQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
"fl", "*,children:[subquery]",
"children.q","{!child of=subject:parentDocument}{!terms f=id v=$row.id}",
"children.rows","10",
"children.fq","title:foo"), test2);
assertJQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
"fl", "*,children:[subquery]",
"children.q","{!child of=subject:parentDocument}{!terms f=id v=$row.id}",
"children.rows","2",
"children.fq","title:bar",
"children.sort","_docid_ asc"), test3);
}
private void testChildDoctransformerJSON() throws Exception {
String[] test1 = new String[] {
@ -214,6 +309,38 @@ public class TestChildDocTransformer extends SolrTestCaseJ4 {
tests);
}
private void testSubQueryParentFilterJSON() throws Exception {
String[] tests = new String[] {
"/response/docs/[0]/id==1",
"/response/docs/[0]/children/docs/[0]/id==2",
"/response/docs/[0]/children/docs/[0]/cat/[0]/=='childDocument'",
"/response/docs/[0]/children/docs/[0]/title/[0]/=='" + titleVals[0] + "'",
"/response/docs/[1]/id==4",
"/response/docs/[1]/children/docs/[0]/id==5",
"/response/docs/[1]/children/docs/[0]/cat/[0]/=='childDocument'",
"/response/docs/[1]/children/docs/[0]/title/[0]/=='" + titleVals[1] + "'"
};
assertJQ(req(
"q", "*:*", "fq", "subject:\"parentDocument\" ",
"fl", "*,children:[subquery]",
"sort", "id asc",
"children.q","{!child of=subject:parentDocument}{!terms f=id v=$row.id}",
"children.fq","cat:childDocument",
"children.sort","_docid_ asc"),
tests);
assertJQ(req(
"q", "*:*", "fq", "subject:\"parentDocument\" ",
"fl", "id,children:[subquery]",
"sort", "id asc",
"children.q","{!child of=subject:parentDocument}{!terms f=id v=$row.id}",
"children.fq","cat:childDocument",
"children.sort","_docid_ asc"),
tests);
}
private void testParentFilterXML() {
@ -241,4 +368,36 @@ public class TestChildDocTransformer extends SolrTestCaseJ4 {
tests);
}
private void testSubQueryParentFilterXML() {
String tests[] = new String[] {
"//*[@numFound='2']",
"/response/result/doc[1]/int[@name='id']='1'" ,
"/response/result/doc[1]/result[@name='children'][@numFound=1]/doc[1]/int[@name='id']='2'" ,
"/response/result/doc[1]/result[@name='children'][@numFound=1]/doc[1]/arr[@name='cat']/str[1]='childDocument'" ,
"/response/result/doc[1]/result[@name='children'][@numFound=1]/doc[1]/arr[@name='title']/str[1]='" + titleVals[0] + "'" ,
"/response/result/doc[2]/int[@name='id']='4'" ,
"/response/result/doc[2]/result[@name='children'][@numFound=1]/doc[1]/int[@name='id']='5'",
"/response/result/doc[2]/result[@name='children'][@numFound=1]/doc[1]/arr[@name='cat']/str[1]='childDocument'",
"/response/result/doc[2]/result[@name='children'][@numFound=1]/doc[1]/arr[@name='title']/str[1]='" + titleVals[1] + "'"};
assertQ(req(
"q", "*:*", "fq", "subject:\"parentDocument\" ",
"fl", "*,children:[subquery]",
"sort", "id asc",
"children.q","{!child of=subject:parentDocument}{!terms f=id v=$row.id}",
"children.fq","cat:childDocument",
"children.sort","_docid_ asc"
),
tests);
assertQ(req("q", "*:*", "fq", "subject:\"parentDocument\" ",
"fl", "id,children:[subquery]",
"sort", "id asc",
"children.q","{!child of=subject:parentDocument}{!terms f=id v=$row.id}",
"children.fq","cat:childDocument",
"children.sort","_docid_ asc"),
tests);
}
}

View File

@ -0,0 +1,571 @@
/*
* 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.response.transform;
import java.io.ByteArrayInputStream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.StringTokenizer;
import org.apache.commons.io.output.ByteArrayOutputStream;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.JavaBinCodec;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.core.SolrCore;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrRequestInfo;
import org.apache.solr.response.BinaryQueryResponseWriter;
import org.apache.solr.response.SolrQueryResponse;
import org.junit.BeforeClass;
import org.junit.Test;
public class TestSubQueryTransformer extends SolrTestCaseJ4 {
private static int peopleMultiplier;
private static int deptMultiplier;
@BeforeClass
public static void beforeTests() throws Exception {
System.setProperty("enable.update.log", "false");
initCore("solrconfig-basic.xml", "schema-docValuesJoin.xml");
peopleMultiplier = atLeast(1);
deptMultiplier = atLeast(1);
int id=0;
for (int p=0; p < peopleMultiplier; p++){
assertU(add(doc("id", ""+id++,"name_s", "john", "title_s", "Director",
"dept_ss_dv","Engineering",
"dept_i", "0",
"dept_is", "0")));
assertU(add(doc("id", ""+id++,"name_s", "mark", "title_s", "VP",
"dept_ss_dv","Marketing",
"dept_i", "1",
"dept_is", "1")));
assertU(add(doc("id", ""+id++,"name_s", "nancy", "title_s", "MTS",
"dept_ss_dv","Sales",
"dept_i", "2",
"dept_is", "2")));
assertU(add(doc("id", ""+id++,"name_s", "dave", "title_s", "MTS",
"dept_ss_dv","Support", "dept_ss_dv","Engineering",
"dept_i", "3",
"dept_is", "3", "dept_is", "0")));
assertU(add(doc("id", ""+id++,"name_s", "tina", "title_s", "VP",
"dept_ss_dv","Engineering",
"dept_i", "0",
"dept_is", "0")));
if (rarely()) {
assertU(commit("softCommit", "true"));
}
}
for (int d=0; d < deptMultiplier; d++){
assertU(add(doc("id",""+id, "id_i",""+id++,
"dept_id_s", "Engineering", "text_t","These guys develop stuff", "salary_i_dv", "1000",
"dept_id_i", "0")));
assertU(add(doc("id",""+id++,"id_i",""+id++,
"dept_id_s", "Marketing", "text_t","These guys make you look good","salary_i_dv", "1500",
"dept_id_i", "1")));
assertU(add(doc("id",""+id, "id_i",""+id++,
"dept_id_s", "Sales", "text_t","These guys sell stuff","salary_i_dv", "1600",
"dept_id_i", "2")));
assertU(add(doc("id",""+id,"id_i",""+id++,
"dept_id_s", "Support", "text_t","These guys help customers","salary_i_dv", "800",
"dept_id_i", "3")));
if (rarely()) {
assertU(commit("softCommit", "true"));
}
}
assertU(commit());
}
@Test
public void testJohnOrNancySingleField() throws Exception {
//System.out.println("p "+peopleMultiplier+" d "+deptMultiplier);
assertQ("subq1.fl is limited to single field",
req("q","name_s:(john nancy)", "indent","true",
"fl","name_s_dv,depts:[subquery]",
"rows","" + (2 * peopleMultiplier),
"depts.q","{!term f=dept_id_s v=$row.dept_ss_dv}",
"depts.fl","text_t",
"depts.indent","true",
"depts.rows",""+deptMultiplier),
"count(//result/doc/str[@name='name_s_dv'][.='john']/../result[@name='depts'][@numFound='" +
deptMultiplier+ "']/doc/str[@name='text_t'][.='These guys develop stuff'])="+
(peopleMultiplier * deptMultiplier),
"count(//result/doc/str[@name='name_s_dv'][.='nancy']/../result[@name='depts'][@numFound='" +
deptMultiplier+ "']/doc/str[@name='text_t'][.='These guys sell stuff'])="+
(peopleMultiplier * deptMultiplier),
"count((//result/doc/str[@name='name_s_dv'][.='john']/..)[1]/result[@name='depts']/doc[1]/*)=1",
"count((//result/doc/str[@name='name_s_dv'][.='john']/..)[1]/result[@name='depts']/doc["+ deptMultiplier+ "]/*)=1",
"count((//result/doc/str[@name='name_s_dv'][.='john']/..)["+ peopleMultiplier +"]/result[@name='depts'][@numFound='" +
deptMultiplier+ "']/doc[1]/*)=1",
"count((//result/doc/str[@name='name_s_dv'][.='john']/..)["+ peopleMultiplier +"]/result[@name='depts'][@numFound='" +
deptMultiplier+ "']/doc["+ deptMultiplier+ "]/*)=1"
);
}
final String[] johnAndNancyParams = new String[]{"q","name_s:(john nancy)", "indent","true",
"fl","name_s_dv,depts:[subquery]",
"fl","depts_i:[subquery]",
"rows","" + (2 * peopleMultiplier),
"depts.q","{!term f=dept_id_s v=$row.dept_ss_dv}",
"depts.fl","text_t",
"depts.indent","true",
"depts.rows",""+deptMultiplier,
"depts_i.q","{!term f=dept_id_i v=$row.dept_i_dv}",
"depts_i.fl","text_t", // multi val subquery param check
"depts_i.fl","dept_id_s_dv",
"depts_i.indent","true",
"depts_i.rows",""+deptMultiplier};
@Test
public void testTwoSubQueriesAndByNumberWithTwoFields() throws Exception {
final SolrQueryRequest johnOrNancyTwoFL = req(johnAndNancyParams);
assertQ("call subquery twice a row, once by number, with two fls via multival params",
johnOrNancyTwoFL,
"count(//result/doc/str[@name='name_s_dv'][.='john']/../result[@name='depts']/doc/str[@name='text_t'][.='These guys develop stuff'])="+
(peopleMultiplier * deptMultiplier),
"count(//result/doc/str[@name='name_s_dv'][.='john']/../result[@name='depts_i']/doc/str[@name='dept_id_s_dv'][.='Engineering'])="+
(peopleMultiplier * deptMultiplier),
"count(//result/doc/str[@name='name_s_dv'][.='nancy']/../result[@name='depts_i']/doc/str[@name='text_t'][.='These guys sell stuff'])="+
(peopleMultiplier * deptMultiplier),
"count(//result/doc/str[@name='name_s_dv'][.='nancy']/../result[@name='depts_i']/doc/str[@name='dept_id_s_dv'][.='Sales'])="+
(peopleMultiplier * deptMultiplier),
"count((//result/doc/str[@name='name_s_dv'][.='john']/..)["+ peopleMultiplier +"]/result[@name='depts_i']/doc["+ deptMultiplier+ "]/str[@name='dept_id_s_dv'][.='Engineering'])=1",
"count((//result/doc/str[@name='name_s_dv'][.='john']/..)["+ peopleMultiplier +"]/result[@name='depts_i']/doc["+ deptMultiplier+ "]/str[@name='text_t'][.='These guys develop stuff'])=1"
);
}
@Test
public void testRowsStartForSubqueryAndScores() throws Exception {
String johnDeptsIds = h.query(req(new String[]{"q","{!join from=dept_ss_dv to=dept_id_s}name_s:john",
"wt","csv",
"csv.header","false",
"fl","id",
"rows",""+deptMultiplier,
"sort", "id_i desc"
}));
ArrayList<Object> deptIds = Collections.list(
new StringTokenizer( johnDeptsIds));
final int a = random().nextInt(deptMultiplier+1);
final int b = random().nextInt(deptMultiplier+1);
final int start = Math.min(a, b) ;
final int toIndex = Math.max(a, b) ;
List<Object> expectIds = deptIds.subList(start , toIndex);
ArrayList<String> assertions = new ArrayList<>();
// count((//result/doc/str[@name='name_s_dv'][.='john']/../result[@name='depts'])[1]/doc/str[@name='id'])
// random().nextInt(peopleMultiplier);
assertions.add("count((//result/doc/str[@name='name_s_dv'][.='john']/.."
+ "/result[@name='depts'][@numFound='"+deptMultiplier+"'][@start='"+start+"'])["+
(random().nextInt(peopleMultiplier)+1)
+"]/doc/str[@name='id'])=" +(toIndex-start));
// System.out.println(expectIds);
for (int i=0; i< expectIds.size(); i++) {
// (//result/doc/str[@name='name_s_dv'][.='john']/../result[@name='depts'])[1]/doc[1]/str[@name='id']='15'
String ithDoc = "(//result/doc/str[@name='name_s_dv'][.='john']/.."
+ "/result[@name='depts'][@numFound='"+deptMultiplier+"'][@start='"+start+"'])["+
(random().nextInt(peopleMultiplier)+1) +
"]/doc[" +(i+1)+ "]";
assertions.add(ithDoc+"/str[@name='id'][.='"+expectIds.get(i)+"']");
// let's test scores right there
assertions.add(ithDoc+"/float[@name='score'][.='"+expectIds.get(i)+".0']");
}
String[] john = new String[]{"q","name_s:john", "indent","true",
"fl","name_s_dv,depts:[subquery]",
"rows","" + (2 * peopleMultiplier),
"depts.q","+{!term f=dept_id_s v=$row.dept_ss_dv}^=0 _val_:id_i",
"depts.fl","id",
"depts.fl","score",
"depts.indent","true",
"depts.rows",""+(toIndex-start),
"depts.start",""+start};
assertQ(req(john), assertions.toArray(new String[]{}));
}
@Test
public void testThreeLevel() throws Exception {
List<String> asserts = new ArrayList<>();
// dave works in both dept, get his coworkers from both
for (String dept : new String[] {"Engineering", "Support"}) { //dept_id_s_dv">Engineering
ArrayList<Object> deptWorkers = Collections.list(
new StringTokenizer( h.query(req(
"q","dept_ss_dv:"+dept ,//dept_id_i_dv
"wt","csv",
"csv.header","false",
"fl","name_s_dv",
"rows",""+peopleMultiplier*3, // dave has three coworkers in two depts
"sort", "id desc"
))));
// System.out.println(deptWorkers);
// looping dave clones
for (int p : new int []{1, peopleMultiplier}) {
// looping dept clones
for (int d : new int []{1, deptMultiplier}) {
// looping coworkers
int wPos = 1;
for (Object mate : deptWorkers) {
// (/response/result/doc/str[@name='name_s_dv'][.='dave']/..)[1]
// /result[@name='subq1']/doc/str[@name='dept_id_s_dv'][.='Engineering']/..
// /result[@name='neighbours']/doc/str[@name='name_s_dv'][.='tina']
asserts.add("((/response/result/doc/str[@name='name_s_dv'][.='dave']/..)["+p+"]"+
"/result[@name='subq1']/doc/str[@name='dept_id_s_dv'][.='"+dept+"']/..)["+ d +"]"+
"/result[@name='neighbours']/doc[" + wPos + "]/str[@name='name_s_dv'][.='"+ mate+"']");
wPos ++;
}
}
}
}
//System.out.println(asserts);
assertQ("dave works at both dept with other folks",
// System.out.println(h.query(
req(new String[]{"q","name_s:dave", "indent","true",
"fl","name_s_dv,subq1:[subquery]",
"rows","" + peopleMultiplier,
"subq1.q","{!terms f=dept_id_s v=$row.dept_ss_dv}",
"subq1.fl","text_t,dept_id_s_dv,neighbours:[subquery]",
"subq1.indent","true",
"subq1.rows",""+(deptMultiplier*2),
"subq1.neighbours.q",//flipping via numbers
random().nextBoolean() ?
"{!terms f=dept_ss_dv v=$row.dept_id_s_dv}"
: "{!terms f=dept_is v=$row.dept_id_i_dv}",
"subq1.neighbours.fl", "name_s_dv" ,
"subq1.neighbours.rows", ""+peopleMultiplier*3},
"subq1.neighbours.sort", "id desc")//,
,asserts.toArray(new String[]{})
// )
);
}
@Test
public void testNoExplicitName() throws Exception {
String[] john = new String[]{"q","name_s:john", "indent","true",
"fl","name_s_dv,"
+ "[subquery]",
"rows","" + (2 * peopleMultiplier),
"depts.q","+{!term f=dept_id_s v=$row.dept_ss_dv}^=0 _val_:id_i",
"depts.fl","id",
"depts.fl","score",
"depts.indent","true",
"depts.rows",""+deptMultiplier,
"depts.start","0"};
assertQEx("no prefix, no subquery", req(john), ErrorCode.BAD_REQUEST);
assertQEx("no prefix, no subsubquery",
req("q","name_s:john", "indent","true",
"fl","name_s_dv,"
+ "depts:[subquery]",
"rows","" + (2 * peopleMultiplier),
"depts.q","+{!term f=dept_id_s v=$row.dept_ss_dv}^=0 _val_:id_i",
"depts.fl","id",
"depts.fl","score",
"depts.fl","[subquery]",// <- here is a trouble
"depts.indent","true",
"depts.rows",""+deptMultiplier,
"depts.start","0"), ErrorCode.BAD_REQUEST);
}
@Test
public void testDupePrefix() throws Exception {
assertQEx("subquery name clash", req(new String[]{"q","name_s:(john nancy)", "indent","true",
"fl","name_s_dv,depts:[subquery]",
"fl","depts:[subquery]",
"rows","" + (2 * peopleMultiplier),
"depts.q","{!term f=dept_id_s v=$row.dept_ss_dv}",
"depts.fl","text_t",
"depts.indent","true",
"depts.rows",""+deptMultiplier,
"depts_i.q","{!term f=dept_id_i v=$depts_i.row.dept_i_dv}",
"depts_i.fl","text_t", // multi val subquery param check
"depts_i.fl","dept_id_s_dv",
"depts_i.indent","true",
"depts_i.rows",""+deptMultiplier}
), ErrorCode.BAD_REQUEST);
}
@Test
public void testJustJohnJson() throws Exception {
final SolrQueryRequest johnTwoFL = req(johnAndNancyParams);
ModifiableSolrParams params = new ModifiableSolrParams(johnTwoFL.getParams());
params.set("q","name_s:john");
johnTwoFL.setParams(params);
assertJQ(johnTwoFL,
"/response/docs/[0]/depts/docs/[0]=={text_t:\"These guys develop stuff\"}",
"/response/docs/[" + (peopleMultiplier-1) + "]/depts/docs/[" + (deptMultiplier-1) + "]=={text_t:\"These guys develop stuff\"}",
"/response/docs/[0]/depts_i/docs/[0]=={dept_id_s_dv:\"Engineering\", text_t:\"These guys develop stuff\"}",// seem like key order doesn't matter , well
"/response/docs/[" + (peopleMultiplier-1) + "]/depts_i/docs/[" + (deptMultiplier-1) + "]=="
+ "{text_t:\"These guys develop stuff\", dept_id_s_dv:\"Engineering\"}");
}
@SuppressWarnings("unchecked")
@Test
public void testJustJohnJavabin() throws Exception {
final SolrQueryRequest johnTwoFL = req(johnAndNancyParams);
ModifiableSolrParams params = new ModifiableSolrParams(johnTwoFL.getParams());
params.set("q","name_s:john");
params.set("wt","javabin");
johnTwoFL.setParams(params);
final NamedList<Object> unmarshalled;
{
SolrCore core = johnTwoFL.getCore();
SolrQueryResponse rsp = new SolrQueryResponse();
SolrRequestInfo.setRequestInfo(new SolrRequestInfo(johnTwoFL, rsp));
SolrQueryResponse response = h.queryAndResponse(
johnTwoFL.getParams().get(CommonParams.QT), johnTwoFL);
BinaryQueryResponseWriter responseWriter = (BinaryQueryResponseWriter) core.getQueryResponseWriter(johnTwoFL);
ByteArrayOutputStream bytes = new ByteArrayOutputStream();
responseWriter.write(bytes,johnTwoFL,response);
unmarshalled = (NamedList<Object>) new JavaBinCodec().unmarshal(
new ByteArrayInputStream(bytes.toByteArray()));
johnTwoFL.close();
SolrRequestInfo.clearRequestInfo();
}
SolrDocumentList resultDocs = (SolrDocumentList)(unmarshalled.get("response"));
{
Map<String,String> engText = new HashMap<>();
engText.put("text_t", "These guys develop stuff");
Map<String,String> engId = new HashMap<>();
engId.put("text_t", "These guys develop stuff");
engId.put("dept_id_s_dv", "Engineering");
for (int docNum : new int []{0, peopleMultiplier-1}) {
SolrDocument employeeDoc = resultDocs.get(docNum);
assertEquals("john", employeeDoc.getFieldValue("name_s_dv"));
for (String subResult : new String []{"depts", "depts_i"}) {
SolrDocumentList subDoc = (SolrDocumentList)employeeDoc.getFieldValue(subResult);
for (int deptNum : new int []{0, deptMultiplier-1}) {
SolrDocument deptDoc = subDoc.get(deptNum);
Object expectedDept = (subResult.equals("depts") ? engText : engId);
assertTrue( "" + expectedDept + " equals to " + deptDoc,
expectedDept.equals(deptDoc));
}
}
}
}
}
@Test
public void testExceptionPropagation() throws Exception {
final SolrQueryRequest r = req("q","name_s:dave", "indent","true",
"fl","depts:[subquery]",
"rows","" + ( peopleMultiplier),
"depts.q","{!lucene}(",
"depts.fl","text_t",
"depts.indent","true",
"depts.rows",""+(deptMultiplier*2),
"depts.logParamsList","q,fl,rows,subq1.row.dept_ss_dv");
// System.out.println(h.query(r));
assertQEx("wrong subquery",
r,
ErrorCode.BAD_REQUEST);
assertQEx( "", req("q","name_s:dave", "indent","true",
"fl","depts:[subquery]",
"rows","1",
"depts.q","{!lucene}",
"depts.fl","text_t",
"depts.indent","true",
"depts.rows","NAN",
"depts.logParamsList","q,fl,rows,subq1.row.dept_ss_dv"),
ErrorCode.BAD_REQUEST);
}
@Test
public void testMultiValue() throws Exception {
String [] happyPathAsserts = new String[]{
"count(//result/doc/str[@name='name_s_dv'][.='dave']/../result[@name='subq1']/doc/str[@name='text_t'][.='These guys develop stuff'])="+
(peopleMultiplier * deptMultiplier),
"count(//result/doc/str[@name='name_s_dv'][.='dave']/../result[@name='subq1']/doc/str[@name='text_t'][.='These guys help customers'])="+
(peopleMultiplier * deptMultiplier),
"//result[@numFound="+peopleMultiplier+"]"};
Random random1 = random();
assertQ("dave works at both, whether we set a default separator or both",
req(new String[]{"q","name_s:dave", "indent","true",
"fl",(random().nextBoolean() ? "name_s_dv" : "*")+ //"dept_ss_dv,
",subq1:[subquery "
+((random1.nextBoolean() ? "" : "separator=,"))+"]",
"rows","" + peopleMultiplier,
"subq1.q","{!terms f=dept_id_s v=$row.dept_ss_dv "+((random1.nextBoolean() ? "" : "separator=,"))+"}",
"subq1.fl","text_t",
"subq1.indent","true",
"subq1.rows",""+(deptMultiplier*2),
"subq1.logParamsList","q,fl,rows,row.dept_ss_dv"}),
happyPathAsserts
);
assertQ("even via numbers",
req("q","name_s:dave", "indent","true",
"fl","dept_is_dv,name_s_dv,subq1:[subquery]",
"rows","" + ( peopleMultiplier),
"subq1.q","{!terms f=dept_id_i v=$row.dept_is_dv}",
"subq1.fl","text_t",
"subq1.indent","true",
"subq1.rows",""+(deptMultiplier*2)),
happyPathAsserts
);
assertQ("even if we set a separator both",
req("q","name_s:dave", "indent","true",
"fl","dept_ss_dv,name_s_dv,name_s_dv,subq1:[subquery separator=\" \"]",
"rows","" + ( peopleMultiplier),
"subq1.q","{!terms f=dept_id_s v=$row.dept_ss_dv separator=\" \"}",
"subq1.fl","text_t",
"subq1.indent","true",
"subq1.rows",""+(deptMultiplier*2)),
happyPathAsserts
);
String [] noMatchAtSubQ = new String[] {
"count(//result/doc/str[@name='name_s_dv'][.='dave']/../result[@name='subq1'][@numFound=0])="+
(peopleMultiplier),
"//result[@numFound="+peopleMultiplier+"]" };
assertQ("different separators, no match",
req("q","name_s:dave", "indent","true",
"fl","dept_ss_dv,name_s_dv,subq1:[subquery]",
"rows","" + ( peopleMultiplier),
"subq1.q","{!terms f=dept_id_s v=$row.dept_ss_dv separator=\" \"}",
"subq1.fl","text_t",
"subq1.indent","true",
"subq1.rows",""+(deptMultiplier*2)),
noMatchAtSubQ
);
assertQ("and no matter where",
req("q","name_s:dave", "indent","true",
"fl","dept_ss_dv,name_s_dv,subq1:[subquery separator=\" \"]",
"rows","" + ( peopleMultiplier),
"subq1.q","{!terms f=dept_id_s v=$row.dept_ss_dv}",
"subq1.fl","text_t",
"subq1.indent","true",
"subq1.rows",""+(deptMultiplier*2)),
noMatchAtSubQ
);
assertQ("setting a wrong parser gets you nowhere",
req("q","name_s:dave", "indent","true",
"fl","dept_ss_dv,name_s_dv,subq1:[subquery]",
"rows","" + ( peopleMultiplier),
"subq1.q","{!term f=dept_id_s v=$row.dept_ss_dv}",
"subq1.fl","text_t",
"subq1.indent","true",
"subq1.rows",""+(deptMultiplier*2)),
noMatchAtSubQ
);
assertQ("but it luckily works with default query parser, but it's not really reliable",
req("q","name_s:dave", "indent","true",
"fl","dept_ss_dv,name_s_dv,subq1:[subquery separator=\" \"]",
"rows","" + ( peopleMultiplier),
"subq1.q","{!lucene df=dept_id_s v=$row.dept_ss_dv}",
"subq1.fl","text_t",
"subq1.indent","true",
"subq1.rows",""+(deptMultiplier*2)),
happyPathAsserts
);
assertQ("even lucene qp can't help at any separator but space",
req("q","name_s:dave", "indent","true",
"fl","dept_ss_dv,name_s_dv,"
+ "subq1:[subquery "+(random().nextBoolean() ? "" : "separator=" +((random().nextBoolean() ? "" : ",")))+"]",
"rows","" + ( peopleMultiplier),
"subq1.q","{!lucene df=dept_id_s v=$row.dept_ss_dv}",
"subq1.fl","text_t",
"subq1.indent","true",
"subq1.rows",""+(deptMultiplier*2)),
noMatchAtSubQ
);
}
static String[] daveMultiValueSearchParams(Random random, int peopleMult, int deptMult) {
return new String[]{"q","name_s:dave", "indent","true",
"fl",(random().nextBoolean() ? "name_s_dv" : "*")+ //"dept_ss_dv,
",subq1:[subquery "
+((random.nextBoolean() ? "" : "separator=,"))+"]",
"rows","" + peopleMult,
"subq1.q","{!terms f=dept_id_s v=$row.dept_ss_dv "+((random.nextBoolean() ? "" : "separator=,"))+"}",
"subq1.fl","text_t",
"subq1.indent","true",
"subq1.rows",""+(deptMult*2),
"subq1.logParamsList","q,fl,rows,row.dept_ss_dv"};
}
}

View File

@ -0,0 +1,131 @@
/*
* 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.response.transform;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrException;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.SolrCore;
import org.apache.solr.request.SolrRequestHandler;
import org.apache.solr.servlet.DirectSolrConnection;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import com.google.common.collect.ImmutableMap;
public class TestSubQueryTransformerCrossCore extends SolrTestCaseJ4 {
private static SolrCore fromCore;
@BeforeClass
public static void beforeTests() throws Exception {
System.setProperty("enable.update.log", "false"); // schema12 doesn't support _version_
initCore("solrconfig-basic.xml","schema-docValuesJoin.xml");
final CoreContainer coreContainer = h.getCoreContainer();
fromCore = coreContainer.create("fromCore", //FileSystems.getDefault().getPath( TEST_HOME()), ImmutableMap.of("config","solrconfig-basic.xml","schema","schema-docValuesJoin.xml"
ImmutableMap.of("configSet", "minimal")
);
assertU(add(doc("id", "1","name_s", "john", "title_s", "Director", "dept_ss_dv","Engineering",
"text_t","These guys develop stuff")));
assertU(add(doc("id", "2","name_s", "mark", "title_s", "VP", "dept_ss_dv","Marketing",
"text_t","These guys make you look good")));
assertU(add(doc("id", "3","name_s", "nancy", "title_s", "MTS", "dept_ss_dv","Sales",
"text_t","These guys sell stuff")));
assertU(add(doc("id", "4","name_s", "dave", "title_s", "MTS", "dept_ss_dv","Support", "dept_ss_dv","Engineering"
, "text_t","These guys help customers")));
assertU(add(doc("id", "5","name_s", "tina", "title_s", "VP", "dept_ss_dv","Engineering",
"text_t","These guys develop stuff")));
assertU(commit());
update(fromCore, add(doc("id","10", "dept_id_s", "Engineering", "text_t","These guys develop stuff", "salary_i_dv", "1000")));
update(fromCore, add(doc("id","11", "dept_id_s", "Marketing", "text_t","These guys make you look good","salary_i_dv", "1500")));
update(fromCore, add(doc("id","12", "dept_id_s", "Sales", "text_t","These guys sell stuff","salary_i_dv", "1600")));
update(fromCore, add(doc("id","13", "dept_id_s", "Support", "text_t","These guys help customers","salary_i_dv", "800")));
update(fromCore, commit());
}
public static String update(SolrCore core, String xml) throws Exception {
DirectSolrConnection connection = new DirectSolrConnection(core);
SolrRequestHandler handler = core.getRequestHandler("/update");
return connection.request(handler, null, xml);
}
@Test
public void testSameCoreSingleField() throws Exception {
assertQ("subq1.fl is limited to single field",
req("q","name_s:john",
"fl","*,depts:[subquery fromIndex=fromCore]",
"depts.q","{!term f=dept_id_s v=$row.dept_ss_dv}",
"depts.fl","text_t"),
"//result/doc/str[@name='name_s_dv'][.='john']/../result[@name='depts']/doc/str[@name='text_t'][.='These guys develop stuff']",
"count(//result/doc/str[@name='name_s_dv'][.='john']/../result[@name='depts']/doc/*)=1");// only text_t
}
@Test
public void testAbsentCore() throws Exception {
assertQEx("from index not exist",
req("q","name_s:dave",
"fl","*,depts:[subquery fromIndex=fromCore2]",
"depts.q","{!term f=dept_id_s v=$row.dept_ss_dv}",
"depts.fl","text_t"),
SolrException.ErrorCode.BAD_REQUEST
);
}
@Test
public void testCrossCoreSubQueryTransformer() throws Exception {
assertQ("make sure request is parsed in this core",
req("q","name_s:john",
"fl","*,depts:[subquery]",
// text is tokenized and can be found, despite there is no substitution magic
"depts.q","{!field f=text_t}These guys"),
"//result/doc/str[@name='name_s_dv'][.='john']/../result[@name='depts']/doc"
);
assertQ("make sure request is parsed in that core",
req("q","name_s:john",
"fl","*,depts:[subquery fromIndex=fromCore]",
// text is NOT tokenized and can NOT be found
"depts.q","{!field f=text_t}These guys"),
"count(//result/doc/str[@name='name_s_dv'][.='john']/../result[@name='depts']/doc)=0"
);
assertQ("make sure request is parsed in that core",
req("q","-name_s:dave", "indent", "true",
"fl","*,depts:[subquery fromIndex=fromCore]",
// stored text (text_t is string in minimal configset) can be found as
"depts.q","{!field f=text_t v=$row.text_t}",
"depts.fl", "dept_id_s" ),
"//result/doc/str[@name='name_s_dv'][.='john']/.."
+ "/result[@name='depts']/doc/str[@name='dept_id_s'][.='Engineering']",
"//result/doc/str[@name='name_s_dv'][.='tina']/.."
+ "/result[@name='depts']/doc/str[@name='dept_id_s'][.='Engineering']",
"//result/doc/str[@name='name_s_dv'][.='mark']/.."
+ "/result[@name='depts']/doc/str[@name='dept_id_s'][.='Marketing']"
);
}
@AfterClass
public static void nukeAll() {
fromCore = null;
}
}

View File

@ -0,0 +1,186 @@
/*
* 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.response.transform;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Random;
import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.util.ContentStreamBase;
import org.junit.Test;
@SuppressSSL
public class TestSubQueryTransformerDistrib extends AbstractFullDistribZkTestBase {
@Override
protected String getCloudSchemaFile() {
return "schema-docValuesJoin.xml";
}
@Override
protected String getCloudSolrConfig() {
return "solrconfig-basic.xml";
}
@SuppressWarnings("serial")
@Test
public void test() throws SolrServerException, IOException {
int peopleMultiplier = atLeast(1);
int deptMultiplier = atLeast(1);
String people = "people";
int numPeopleShards;
createCollection(people, atLeast(1), numPeopleShards = atLeast(2), numPeopleShards);
String depts = "departments";
int numDeptsShards;
createCollection(depts, atLeast(1), numDeptsShards = atLeast(2), numDeptsShards);
createIndex(people, peopleMultiplier, depts, deptMultiplier);
Random random1 = random();
{
final QueryRequest qr = new QueryRequest(params(
new String[]{"q","name_s:dave", "indent","true",
"fl","*,depts:[subquery "+((random1.nextBoolean() ? "" : "separator=,"))+"]",
"rows","" + peopleMultiplier,
"depts.q","{!terms f=dept_id_s v=$row.dept_ss_dv "+((random1.nextBoolean() ? "" : "separator=,"))+"}",
"depts.fl","text_t",
"depts.indent","true",
"depts.collection","departments",
"depts.rows",""+(deptMultiplier*2),
"depts.logParamsList","q,fl,rows,row.dept_ss_dv"}));
final QueryResponse rsp = new QueryResponse();
rsp.setResponse(cloudClient.request(qr, people));
final SolrDocumentList hits = rsp.getResults();
assertEquals(peopleMultiplier, hits.getNumFound());
Map<String,String> engText = new HashMap<String,String>() {
{ put("text_t", "These guys develop stuff");
}
};
Map<String,String> suppText = new HashMap<String,String>() {
{ put("text_t", "These guys help customers");
}
};
int engineer = 0;
int support = 0;
for (int res : new int [] {0, (peopleMultiplier-1) /2, peopleMultiplier-1}) {
SolrDocument doc = hits.get(res);
assertEquals("dave", doc.getFieldValue("name_s_dv"));
SolrDocumentList relDepts = (SolrDocumentList) doc.getFieldValue("depts");
assertEquals("dave works in both depts "+rsp,
deptMultiplier * 2, relDepts.getNumFound());
for (int deptN = 0 ; deptN < relDepts.getNumFound(); deptN++ ) {
SolrDocument deptDoc = relDepts.get(deptN);
assertTrue(deptDoc + "should be either "+engText +" or "+suppText,
(engText.equals(deptDoc) && ++engineer>0) ||
(suppText.equals(deptDoc) && ++support>0));
}
}
assertEquals(hits.toString(), engineer, support);
}
}
private void createIndex(String people, int peopleMultiplier, String depts, int deptMultiplier)
throws SolrServerException, IOException {
int id=0;
List<String> peopleDocs = new ArrayList<>();
for (int p=0; p < peopleMultiplier; p++){
peopleDocs.add(add(doc("id", ""+id++,"name_s", "john", "title_s", "Director",
"dept_ss_dv","Engineering",
"dept_i", "0",
"dept_is", "0")));
peopleDocs.add(add(doc("id", ""+id++,"name_s", "mark", "title_s", "VP",
"dept_ss_dv","Marketing",
"dept_i", "1",
"dept_is", "1")));
peopleDocs.add(add(doc("id", ""+id++,"name_s", "nancy", "title_s", "MTS",
"dept_ss_dv","Sales",
"dept_i", "2",
"dept_is", "2")));
peopleDocs.add(add(doc("id", ""+id++,"name_s", "dave", "title_s", "MTS",
"dept_ss_dv","Support", "dept_ss_dv","Engineering",
"dept_i", "3",
"dept_is", "3", "dept_is", "0")));
peopleDocs.add(add(doc("id", ""+id++,"name_s", "tina", "title_s", "VP",
"dept_ss_dv","Engineering",
"dept_i", "0",
"dept_is", "0")));
}
addDocs(people, peopleDocs);
List<String> deptsDocs = new ArrayList<>();
for (int d=0; d < deptMultiplier; d++) {
deptsDocs.add(add(doc("id",""+id++, "dept_id_s", "Engineering", "text_t","These guys develop stuff", "salary_i_dv", "1000",
"dept_id_i", "0")));
deptsDocs.add(add(doc("id",""+id++, "dept_id_s", "Marketing", "text_t","These guys make you look good","salary_i_dv", "1500",
"dept_id_i", "1")));
deptsDocs.add(add(doc("id",""+id++, "dept_id_s", "Sales", "text_t","These guys sell stuff","salary_i_dv", "1600",
"dept_id_i", "2")));
deptsDocs.add(add(doc("id",""+id++, "dept_id_s", "Support", "text_t","These guys help customers","salary_i_dv", "800",
"dept_id_i", "3")));
}
addDocs(depts, deptsDocs);
}
private void addDocs(String collection, List<String> docs) throws SolrServerException, IOException {
StringBuilder upd = new StringBuilder("<update>");
for (Iterator<String> iterator = docs.iterator(); iterator.hasNext();) {
String add = iterator.next();
upd.append(add);
if (rarely()) {
upd.append(commit("softCommit", "true"));
}
if (!rarely() || !iterator.hasNext()) {
if (!iterator.hasNext()) {
upd.append(commit("softCommit", "false"));
}
upd.append("</update>");
ContentStreamUpdateRequest req = new ContentStreamUpdateRequest("/update");
req.addContentStream(new ContentStreamBase.StringStream(upd.toString(),"text/xml"));
cloudClient.request(req, collection);
upd.setLength("<update>".length());
}
}
}
}

View File

@ -45,7 +45,10 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParamete
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
public class GatherNodesStream extends TupleStream implements Expressible {
@ -404,7 +407,7 @@ public class GatherNodesStream extends TupleStream implements Expressible {
public List<Tuple> call() {
Map joinParams = new HashMap();
Set<String> flSet = new HashSet();
flSet.add(gather);
flSet.add(traverseTo);
@ -435,11 +438,11 @@ public class GatherNodesStream extends TupleStream implements Expressible {
buf.append(",");
}
}
joinParams.putAll(queryParams);
joinParams.put("fl", buf.toString());
joinParams.put("qt", "/export");
joinParams.put("sort", gather + " asc,"+traverseTo +" asc");
ModifiableSolrParams joinSParams = new ModifiableSolrParams(SolrParams.toMultiMap(new NamedList(queryParams)));
joinSParams.set("fl", buf.toString());
joinSParams.set("qt", "/export");
joinSParams.set("sort", gather + " asc,"+traverseTo +" asc");
StringBuffer nodeQuery = new StringBuffer();
@ -454,14 +457,14 @@ public class GatherNodesStream extends TupleStream implements Expressible {
if(maxDocFreq > -1) {
String docFreqParam = " maxDocFreq="+maxDocFreq;
joinParams.put("q", "{!graphTerms f=" + traverseTo + docFreqParam + "}" + nodeQuery.toString());
joinSParams.set("q", "{!graphTerms f=" + traverseTo + docFreqParam + "}" + nodeQuery.toString());
} else {
joinParams.put("q", "{!terms f=" + traverseTo+"}" + nodeQuery.toString());
joinSParams.set("q", "{!terms f=" + traverseTo+"}" + nodeQuery.toString());
}
TupleStream stream = null;
try {
stream = new UniqueStream(new CloudSolrStream(zkHost, collection, joinParams), new MultipleFieldEqualitor(new FieldEqualitor(gather), new FieldEqualitor(traverseTo)));
stream = new UniqueStream(new CloudSolrStream(zkHost, collection, joinSParams), new MultipleFieldEqualitor(new FieldEqualitor(gather), new FieldEqualitor(traverseTo)));
stream.setStreamContext(streamContext);
stream.open();
BATCH:

View File

@ -46,6 +46,9 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
import org.apache.solr.common.params.MapSolrParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
@ -65,8 +68,9 @@ public class ShortestPathStream extends TupleStream implements Expressible {
private boolean found;
private StreamContext streamContext;
private int threads;
private Map<String,String> queryParams;
private SolrParams queryParams;
@Deprecated
public ShortestPathStream(String zkHost,
String collection,
String fromNode,
@ -78,6 +82,29 @@ public class ShortestPathStream extends TupleStream implements Expressible {
int threads,
int maxDepth) {
init(zkHost,
collection,
fromNode,
toNode,
fromField,
toField,
new MapSolrParams(queryParams),
joinBatchSize,
threads,
maxDepth);
}
public ShortestPathStream(String zkHost,
String collection,
String fromNode,
String toNode,
String fromField,
String toField,
SolrParams queryParams,
int joinBatchSize,
int threads,
int maxDepth) {
init(zkHost,
collection,
fromNode,
@ -162,7 +189,7 @@ public class ShortestPathStream extends TupleStream implements Expressible {
maxDepth = Integer.parseInt(((StreamExpressionValue) depthExpression.getParameter()).getValue());
}
Map<String,String> params = new HashMap<String,String>();
ModifiableSolrParams params = new ModifiableSolrParams();
for(StreamExpressionNamedParameter namedParam : namedParams){
if(!namedParam.getName().equals("zkHost") &&
!namedParam.getName().equals("to") &&
@ -172,7 +199,7 @@ public class ShortestPathStream extends TupleStream implements Expressible {
!namedParam.getName().equals("threads") &&
!namedParam.getName().equals("partitionSize"))
{
params.put(namedParam.getName(), namedParam.getParameter().toString().trim());
params.set(namedParam.getName(), namedParam.getParameter().toString().trim());
}
}
@ -201,7 +228,7 @@ public class ShortestPathStream extends TupleStream implements Expressible {
String toNode,
String fromField,
String toField,
Map queryParams,
SolrParams queryParams,
int joinBatchSize,
int threads,
int maxDepth) {
@ -225,10 +252,10 @@ public class ShortestPathStream extends TupleStream implements Expressible {
// collection
expression.addParameter(collection);
Set<Map.Entry<String,String>> entries = queryParams.entrySet();
// parameters
for(Map.Entry param : entries){
String value = param.getValue().toString();
ModifiableSolrParams mParams = new ModifiableSolrParams(queryParams);
for(Map.Entry<String, String[]> param : mParams.getMap().entrySet()){
String value = String.join(",", param.getValue());
// SOLR-8409: This is a special case where the params contain a " character
// Do note that in any other BASE streams with parameters where a " might come into play
@ -262,8 +289,9 @@ public class ShortestPathStream extends TupleStream implements Expressible {
StreamExplanation child = new StreamExplanation(getStreamNodeId() + "-datastore");
child.setFunctionName("solr (graph)");
child.setImplementingClass("Solr/Lucene");
child.setExpressionType(ExpressionType.DATASTORE);
child.setExpression(queryParams.entrySet().stream().map(e -> String.format(Locale.ROOT, "%s=%s", e.getKey(), e.getValue())).collect(Collectors.joining(",")));
child.setExpressionType(ExpressionType.DATASTORE);
ModifiableSolrParams mParams = new ModifiableSolrParams(queryParams);
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;
@ -417,13 +445,12 @@ public class ShortestPathStream extends TupleStream implements Expressible {
public List<Edge> call() {
Map joinParams = new HashMap();
ModifiableSolrParams joinParams = new ModifiableSolrParams(queryParams);
String fl = fromField + "," + toField;
joinParams.putAll(queryParams);
joinParams.put("fl", fl);
joinParams.put("qt", "/export");
joinParams.put("sort", toField + " asc,"+fromField +" asc");
joinParams.set("fl", fl);
joinParams.set("qt", "/export");
joinParams.set("sort", toField + " asc,"+fromField +" asc");
StringBuffer nodeQuery = new StringBuffer();
@ -433,7 +460,7 @@ public class ShortestPathStream extends TupleStream implements Expressible {
String q = fromField + ":(" + nodeQuery.toString().trim() + ")";
joinParams.put("q", q);
joinParams.set("q", q);
TupleStream stream = null;
try {
stream = new UniqueStream(new CloudSolrStream(zkHost, collection, joinParams), new MultipleFieldEqualitor(new FieldEqualitor(toField), new FieldEqualitor(fromField)));

View File

@ -26,8 +26,6 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.HashMap;
import java.util.Random;
import org.apache.solr.client.solrj.io.stream.SolrStream;
@ -37,6 +35,7 @@ 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.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
class StatementImpl implements Statement {
@ -96,11 +95,11 @@ class StatementImpl implements Statement {
Collections.shuffle(shuffler, new Random());
Map<String, String> params = new HashMap<>();
params.put(CommonParams.QT, "/sql");
params.put("stmt", sql);
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CommonParams.QT, "/sql");
params.set("stmt", sql);
for(String propertyName : this.connection.getProperties().stringPropertyNames()) {
params.put(propertyName, this.connection.getProperties().getProperty(propertyName));
params.set(propertyName, this.connection.getProperties().getProperty(propertyName));
}
Replica rep = shuffler.get(0);

View File

@ -36,7 +36,6 @@ 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.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;
@ -56,6 +55,9 @@ 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;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
@ -72,16 +74,11 @@ public class CloudSolrStream extends TupleStream implements Expressible {
protected String zkHost;
protected String collection;
protected Map<String,String> params;
protected SolrParams params;
private Map<String, String> fieldMappings;
protected StreamComparator comp;
private int zkConnectTimeout = 10000;
private int zkClientTimeout = 10000;
private int numWorkers;
private int workerID;
private boolean trace;
protected transient Map<String, Tuple> eofTuples;
protected transient SolrClientCache cache;
protected transient CloudSolrClient cloudSolrClient;
protected transient List<TupleStream> solrStreams;
protected transient TreeSet<TupleWrapper> tuples;
@ -91,7 +88,34 @@ public class CloudSolrStream extends TupleStream implements Expressible {
protected CloudSolrStream(){
}
/**
* @param zkHost Zookeeper ensemble connection string
* @param collectionName Name of the collection to operate on
* @param params Map&lt;String, String&gt; of parameter/value pairs
* @throws IOException Something went wrong
* <p>
* This form does not allow specifying multiple clauses, say "fq" clauses, use the form that
* takes a SolrParams. Transition code can call the preferred method that takes SolrParams
* by calling CloudSolrStream(zkHost, collectionName,
* new ModifiableSolrParams(SolrParams.toMultiMap(new NamedList(Map&lt;String, String&gt;)));
* @deprecated Use the constructor that has a SolrParams obj rather than a Map
*/
@Deprecated
public CloudSolrStream(String zkHost, String collectionName, Map params) throws IOException {
init(collectionName, zkHost, new MapSolrParams(params));
}
/**
* @param zkHost Zookeeper ensemble connection string
* @param collectionName Name of the collection to operate on
* @param params Map&lt;String, String[]&gt; of parameter/value pairs
* @throws IOException Something went wrong
*/
public CloudSolrStream(String zkHost, String collectionName, SolrParams params) throws IOException {
init(collectionName, zkHost, params);
}
@ -117,16 +141,16 @@ public class CloudSolrStream extends TupleStream implements Expressible {
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - at least one named parameter expected. eg. 'q=*:*'",expression));
}
Map<String,String> params = new HashMap<String,String>();
ModifiableSolrParams mParams = new ModifiableSolrParams();
for(StreamExpressionNamedParameter namedParam : namedParams){
if(!namedParam.getName().equals("zkHost") && !namedParam.getName().equals("aliases")){
params.put(namedParam.getName(), namedParam.getParameter().toString().trim());
mParams.add(namedParam.getName(), namedParam.getParameter().toString().trim());
}
}
// Aliases, optional, if provided then need to split
if(null != aliasExpression && aliasExpression.getParameter() instanceof StreamExpressionValue){
fieldMappings = new HashMap<String,String>();
fieldMappings = new HashMap<>();
for(String mapping : ((StreamExpressionValue)aliasExpression.getParameter()).getValue().split(",")){
String[] parts = mapping.trim().split("=");
if(2 == parts.length){
@ -154,7 +178,7 @@ public class CloudSolrStream extends TupleStream implements Expressible {
}
// We've got all the required items
init(collectionName, zkHost, params);
init(collectionName, zkHost, mParams);
}
@Override
@ -168,14 +192,16 @@ public class CloudSolrStream extends TupleStream implements Expressible {
expression.addParameter(collection);
// parameters
for(Entry<String,String> param : params.entrySet()){
String value = param.getValue();
ModifiableSolrParams mParams = new ModifiableSolrParams(SolrParams.toMultiMap(params.toNamedList()));
for (Entry<String, String[]> param : mParams.getMap().entrySet()) {
String value = String.join(",", param.getValue());
// SOLR-8409: This is a special case where the params contain a " character
// Do note that in any other BASE streams with parameters where a " might come into play
// that this same replacement needs to take place.
value = value.replace("\"", "\\\"");
expression.addParameter(new StreamExpressionNamedParameter(param.getKey(), value));
}
@ -213,29 +239,34 @@ public class CloudSolrStream extends TupleStream implements Expressible {
child.setFunctionName(String.format(Locale.ROOT, "solr (%s)", collection));
child.setImplementingClass("Solr/Lucene");
child.setExpressionType(ExpressionType.DATASTORE);
if(null != params){
child.setExpression(params.entrySet().stream().map(e -> String.format(Locale.ROOT, "%s=%s", e.getKey(), e.getValue())).collect(Collectors.joining(",")));
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;
}
private void init(String collectionName, String zkHost, Map params) throws IOException {
private void init(String collectionName, String zkHost, SolrParams params) throws IOException {
this.zkHost = zkHost;
this.collection = collectionName;
this.params = params;
this.params = new ModifiableSolrParams(params);
// If the comparator is null then it was not explicitly set so we will create one using the sort parameter
// of the query. While doing this we will also take into account any aliases such that if we are sorting on
// fieldA but fieldA is aliased to alias.fieldA then the comparater will be against alias.fieldA.
if(!params.containsKey("fl")){
String fls = String.join(",", params.getParams("fl"));
if (fls == null) {
throw new IOException("fl param expected for a stream");
}
if(!params.containsKey("sort")){
String sorts = String.join(",", params.getParams("sort"));
if (sorts == null) {
throw new IOException("sort param expected for a stream");
}
this.comp = parseComp((String)params.get("sort"), (String)params.get("fl"));
this.comp = parseComp(sorts, fls);
}
public void setFieldMappings(Map<String, String> fieldMappings) {
@ -247,9 +278,6 @@ public class CloudSolrStream extends TupleStream implements Expressible {
}
public void setStreamContext(StreamContext context) {
this.numWorkers = context.numWorkers;
this.workerID = context.workerID;
this.cache = context.getSolrClientCache();
this.streamContext = context;
}
@ -261,8 +289,8 @@ public class CloudSolrStream extends TupleStream implements Expressible {
this.tuples = new TreeSet();
this.solrStreams = new ArrayList();
this.eofTuples = Collections.synchronizedMap(new HashMap());
if(this.cache != null) {
this.cloudSolrClient = this.cache.getCloudSolrClient(zkHost);
if (this.streamContext != null && this.streamContext.getSolrClientCache() != null) {
this.cloudSolrClient = this.streamContext.getSolrClientCache().getCloudSolrClient(zkHost);
} else {
this.cloudSolrClient = new Builder()
.withZkHost(zkHost)
@ -345,7 +373,8 @@ public class CloudSolrStream extends TupleStream implements Expressible {
}
}
params.put("distrib","false"); // We are the aggregator.
ModifiableSolrParams mParams = new ModifiableSolrParams(params);
mParams.set("distrib", "false"); // We are the aggregator.
for(Slice slice : slices) {
Collection<Replica> replicas = slice.getReplicas();
@ -359,7 +388,7 @@ public class CloudSolrStream extends TupleStream implements Expressible {
Replica rep = shuffler.get(0);
ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep);
String url = zkProps.getCoreUrl();
SolrStream solrStream = new SolrStream(url, params);
SolrStream solrStream = new SolrStream(url, mParams);
if(streamContext != null) {
solrStream.setStreamContext(streamContext);
}
@ -406,7 +435,9 @@ public class CloudSolrStream extends TupleStream implements Expressible {
}
}
if(cache == null && cloudSolrClient != null) {
if ((this.streamContext == null || this.streamContext.getSolrClientCache() == null) &&
cloudSolrClient != null) {
cloudSolrClient.close();
}
}

View File

@ -46,7 +46,9 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
import org.apache.solr.client.solrj.io.stream.metrics.Bucket;
import org.apache.solr.client.solrj.io.stream.metrics.Metric;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.common.params.MapSolrParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
/**
@ -65,11 +67,16 @@ public class FacetStream extends TupleStream implements Expressible {
private List<Tuple> tuples = new ArrayList<Tuple>();
private int index;
private String zkHost;
private Map<String, String> props;
private SolrParams params;
private String collection;
protected transient SolrClientCache cache;
protected transient CloudSolrClient cloudSolrClient;
/*
*
* @deprecated. Use the form that takes a SolrParams rather than Map&ltString, String&gt;
*/
@Deprecated
public FacetStream(String zkHost,
String collection,
Map<String, String> props,
@ -77,7 +84,17 @@ public class FacetStream extends TupleStream implements Expressible {
Metric[] metrics,
FieldComparator[] bucketSorts,
int bucketSizeLimit) throws IOException {
init(collection, props, buckets, bucketSorts, metrics, bucketSizeLimit, zkHost);
init(collection, new MapSolrParams(props), buckets, bucketSorts, metrics, bucketSizeLimit, zkHost);
}
public FacetStream(String zkHost,
String collection,
SolrParams params,
Bucket[] buckets,
Metric[] metrics,
FieldComparator[] bucketSorts,
int bucketSizeLimit) throws IOException {
init(collection, params, buckets, bucketSorts, metrics, bucketSizeLimit, zkHost);
}
public FacetStream(StreamExpression expression, StreamFactory factory) throws IOException{
@ -106,10 +123,10 @@ public class FacetStream extends TupleStream implements Expressible {
}
// pull out known named params
Map<String,String> params = new HashMap<String,String>();
ModifiableSolrParams params = new ModifiableSolrParams();
for(StreamExpressionNamedParameter namedParam : namedParams){
if(!namedParam.getName().equals("zkHost") && !namedParam.getName().equals("buckets") && !namedParam.getName().equals("bucketSorts") && !namedParam.getName().equals("limit")){
params.put(namedParam.getName(), namedParam.getParameter().toString().trim());
params.add(namedParam.getName(), namedParam.getParameter().toString().trim());
}
}
@ -204,10 +221,10 @@ public class FacetStream extends TupleStream implements Expressible {
return comps;
}
private void init(String collection, Map<String, String> props, Bucket[] buckets, FieldComparator[] bucketSorts, Metric[] metrics, int bucketSizeLimit, String zkHost) throws IOException {
private void init(String collection, SolrParams params, Bucket[] buckets, FieldComparator[] bucketSorts, Metric[] metrics, int bucketSizeLimit, String zkHost) throws IOException {
this.zkHost = zkHost;
this.props = props;
this.params = params;
this.buckets = buckets;
this.metrics = metrics;
this.bucketSizeLimit = bucketSizeLimit;
@ -233,8 +250,11 @@ public class FacetStream extends TupleStream implements Expressible {
expression.addParameter(collection);
// parameters
for(Entry<String,String> param : props.entrySet()){
expression.addParameter(new StreamExpressionNamedParameter(param.getKey(), param.getValue()));
ModifiableSolrParams tmpParams = new ModifiableSolrParams(params);
for (Entry<String, String[]> param : tmpParams.getMap().entrySet()) {
expression.addParameter(new StreamExpressionNamedParameter(param.getKey(),
String.join(",", param.getValue())));
}
// buckets
@ -288,8 +308,10 @@ public class FacetStream extends TupleStream implements Expressible {
// parallel stream.
child.setImplementingClass("Solr/Lucene");
child.setExpressionType(ExpressionType.DATASTORE);
child.setExpression(props.entrySet().stream().map(e -> String.format(Locale.ROOT, "%s=%s", e.getKey(), e.getValue())).collect(Collectors.joining(",")));
child.setExpressionType(ExpressionType.DATASTORE);
ModifiableSolrParams tmpParams = new ModifiableSolrParams(SolrParams.toMultiMap(params.toNamedList()));
child.setExpression(tmpParams.getMap().entrySet().stream().map(e -> String.format(Locale.ROOT, "%s=%s", e.getKey(), e.getValue())).collect(Collectors.joining(",")));
explanation.addChild(child);
@ -301,8 +323,7 @@ public class FacetStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
return l;
return new ArrayList();
}
public void open() throws IOException {
@ -317,11 +338,11 @@ public class FacetStream extends TupleStream implements Expressible {
FieldComparator[] adjustedSorts = adjustSorts(buckets, bucketSorts);
String json = getJsonFacetString(buckets, metrics, adjustedSorts, bucketSizeLimit);
ModifiableSolrParams params = getParams(this.props);
params.add("json.facet", json);
params.add("rows", "0");
ModifiableSolrParams paramsLoc = new ModifiableSolrParams(params);
paramsLoc.set("json.facet", json);
paramsLoc.set("rows", "0");
QueryRequest request = new QueryRequest(params);
QueryRequest request = new QueryRequest(paramsLoc);
try {
NamedList response = cloudSolrClient.request(request, collection);
getTuples(response, buckets, metrics);
@ -350,15 +371,6 @@ public class FacetStream extends TupleStream implements Expressible {
}
}
private ModifiableSolrParams getParams(Map<String, String> props) {
ModifiableSolrParams params = new ModifiableSolrParams();
for(String key : props.keySet()) {
String value = props.get(key);
params.add(key, value);
}
return params;
}
private String getJsonFacetString(Bucket[] _buckets, Metric[] _metrics, FieldComparator[] _sorts, int _limit) {
StringBuilder buf = new StringBuilder();
appendJson(buf, _buckets, _metrics, _sorts, _limit, 0);

View File

@ -16,21 +16,15 @@
*/
package org.apache.solr.client.solrj.io.stream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.ObjectOutputStream;
import java.net.URLEncoder;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Set;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Map.Entry;
import java.util.stream.Collectors;
import java.util.Random;
import org.apache.solr.client.solrj.io.Tuple;
@ -49,7 +43,7 @@ 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;
import org.apache.solr.common.params.ModifiableSolrParams;
/**
* The ParallelStream decorates a TupleStream implementation and pushes it to N workers for parallel execution.
@ -287,16 +281,17 @@ public class ParallelStream extends CloudSolrStream implements Expressible {
Collections.shuffle(shuffler, new Random());
for(int w=0; w<workers; w++) {
HashMap params = new HashMap();
params.put("distrib","false"); // We are the aggregator.
params.put("numWorkers", workers);
params.put("workerID", w);
params.put("expr", pushStream);
params.put("qt","/stream");
ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
paramsLoc.set("distrib","false"); // We are the aggregator.
paramsLoc.set("numWorkers", workers);
paramsLoc.set("workerID", w);
paramsLoc.set("expr", pushStream.toString());
paramsLoc.set("qt","/stream");
Replica rep = shuffler.get(w);
ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep);
String url = zkProps.getCoreUrl();
SolrStream solrStream = new SolrStream(url, params);
SolrStream solrStream = new SolrStream(url, paramsLoc);
solrStreams.add(solrStream);
}

View File

@ -32,6 +32,7 @@ import org.apache.solr.client.solrj.io.stream.expr.Explanation;
import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
import org.apache.solr.common.params.MapSolrParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.slf4j.Logger;
@ -48,7 +49,7 @@ public class SolrStream extends TupleStream {
private static final long serialVersionUID = 1;
private String baseUrl;
private Map params;
private SolrParams params;
private int numWorkers;
private int workerID;
private boolean trace;
@ -59,7 +60,25 @@ public class SolrStream extends TupleStream {
private String slice;
private long checkpoint = -1;
/**
* @param baseUrl Base URL of the stream.
* @param params Map&lt;String, String&gt; of parameters
* @deprecated, use the form that thakes SolrParams. Existing code can use
* new ModifiableSolrParams(SolrParams.toMultiMap(new NamedList(params)))
* for existing calls that use Map&lt;String, String&gt;
*/
@Deprecated
public SolrStream(String baseUrl, Map params) {
this.baseUrl = baseUrl;
this.params = new ModifiableSolrParams(new MapSolrParams(params));
}
/**
* @param baseUrl Base URL of the stream.
* @param params Map&lt;String, String&gt; of parameters
*/
public SolrStream(String baseUrl, SolrParams params) {
this.baseUrl = baseUrl;
this.params = params;
}
@ -118,9 +137,9 @@ public class SolrStream extends TupleStream {
this.checkpoint = checkpoint;
}
private SolrParams loadParams(Map params) throws IOException {
ModifiableSolrParams solrParams = new ModifiableSolrParams();
if(params.containsKey("partitionKeys")) {
private SolrParams loadParams(SolrParams paramsIn) throws IOException {
ModifiableSolrParams solrParams = new ModifiableSolrParams(paramsIn);
if (params.get("partitionKeys") != null) {
if(!params.get("partitionKeys").equals("none")) {
String partitionFilter = getPartitionFilter();
solrParams.add("fq", partitionFilter);
@ -135,12 +154,6 @@ public class SolrStream extends TupleStream {
solrParams.add("fq", "{!frange cost=100 incl=false l="+checkpoint+"}_version_");
}
Iterator<Map.Entry> it = params.entrySet().iterator();
while(it.hasNext()) {
Map.Entry entry = it.next();
solrParams.add((String)entry.getKey(), entry.getValue().toString());
}
return solrParams;
}

View File

@ -42,7 +42,9 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
import org.apache.solr.client.solrj.io.stream.metrics.Metric;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.params.MapSolrParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
public class StatsStream extends TupleStream implements Expressible {
@ -52,7 +54,7 @@ public class StatsStream extends TupleStream implements Expressible {
private Metric[] metrics;
private String zkHost;
private Tuple tuple;
private Map<String, String> props;
private SolrParams params;
private String collection;
private boolean done;
private long count;
@ -60,20 +62,29 @@ public class StatsStream extends TupleStream implements Expressible {
protected transient SolrClientCache cache;
protected transient CloudSolrClient cloudSolrClient;
// Use StatsStream(String, String, SolrParams, Metric[]
@Deprecated
public StatsStream(String zkHost,
String collection,
Map<String, String> props,
Metric[] metrics) {
init(zkHost, collection, props, metrics);
init(zkHost, collection, new MapSolrParams(props), metrics);
}
private void init(String zkHost, String collection, Map<String, String> props, Metric[] metrics) {
public StatsStream(String zkHost,
String collection,
SolrParams params,
Metric[] metrics) {
init(zkHost, collection, params, metrics);
}
private void init(String zkHost, String collection, SolrParams params, Metric[] metrics) {
this.zkHost = zkHost;
this.props = props;
this.params = params;
this.metrics = metrics;
this.collection = collection;
}
public StatsStream(StreamExpression expression, StreamFactory factory) throws IOException{
// grab all parameters out
String collectionName = factory.getValueOperand(expression, 0);
@ -95,11 +106,11 @@ public class StatsStream extends TupleStream implements Expressible {
if(0 == namedParams.size()){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - at least one named parameter expected. eg. 'q=*:*'",expression));
}
Map<String,String> params = new HashMap<String,String>();
ModifiableSolrParams params = new ModifiableSolrParams();
for(StreamExpressionNamedParameter namedParam : namedParams){
if(!namedParam.getName().equals("zkHost")){
params.put(namedParam.getName(), namedParam.getParameter().toString().trim());
params.set(namedParam.getName(), namedParam.getParameter().toString().trim());
}
}
@ -139,8 +150,9 @@ public class StatsStream extends TupleStream implements Expressible {
expression.addParameter(collection);
// parameters
for(Entry<String,String> param : props.entrySet()){
expression.addParameter(new StreamExpressionNamedParameter(param.getKey(), param.getValue()));
ModifiableSolrParams mParams = new ModifiableSolrParams(params);
for (Entry<String, String[]> param : mParams.getMap().entrySet()) {
expression.addParameter(new StreamExpressionNamedParameter(param.getKey(), String.join(",", param.getValue())));
}
// zkHost
@ -170,8 +182,9 @@ public class StatsStream extends TupleStream implements Expressible {
// parallel stream.
child.setImplementingClass("Solr/Lucene");
child.setExpressionType(ExpressionType.DATASTORE);
child.setExpression(props.entrySet().stream().map(e -> String.format(Locale.ROOT, "%s=%s", e.getKey(), e.getValue())).collect(Collectors.joining(",")));
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;
@ -195,12 +208,12 @@ public class StatsStream extends TupleStream implements Expressible {
.build();
}
ModifiableSolrParams params = getParams(this.props);
addStats(params, metrics);
params.add("stats", "true");
params.add("rows", "0");
ModifiableSolrParams paramsLoc = new ModifiableSolrParams(this.params);
addStats(paramsLoc, metrics);
paramsLoc.set("stats", "true");
paramsLoc.set("rows", "0");
QueryRequest request = new QueryRequest(params);
QueryRequest request = new QueryRequest(paramsLoc);
try {
NamedList response = cloudSolrClient.request(request, collection);
this.tuple = getTuple(response);
@ -275,15 +288,6 @@ public class StatsStream extends TupleStream implements Expressible {
}
}
private ModifiableSolrParams getParams(Map<String, String> props) {
ModifiableSolrParams params = new ModifiableSolrParams();
for(String key : props.keySet()) {
String value = props.get(key);
params.add(key, value);
}
return params;
}
private Tuple getTuple(NamedList response) {
Map map = new HashMap();

View File

@ -56,6 +56,9 @@ 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;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.slf4j.Logger;
@ -75,6 +78,8 @@ public class TopicStream extends CloudSolrStream implements Expressible {
private Map<String, Long> checkpoints = new HashMap<String, Long>();
private String checkpointCollection;
// Use TopicStream that takes a SolrParams
@Deprecated
public TopicStream(String zkHost,
String checkpointCollection,
String collection,
@ -86,25 +91,42 @@ public class TopicStream extends CloudSolrStream implements Expressible {
collection,
id,
checkpointEvery,
params);
new MapSolrParams(params));
}
public TopicStream(String zkHost,
String checkpointCollection,
String collection,
String id,
long checkpointEvery,
SolrParams params) {
init(zkHost,
checkpointCollection,
collection,
id,
checkpointEvery,
params);
}
private void init(String zkHost,
String checkpointCollection,
String collection,
String id,
long checkpointEvery,
Map<String, String> params) {
SolrParams params) {
this.zkHost = zkHost;
this.params = params;
ModifiableSolrParams mParams = new ModifiableSolrParams(params);
if(mParams.getParams("rows") == null) {
mParams.set("rows", "500");
}
this.params = mParams;
this.collection = collection;
this.checkpointCollection = checkpointCollection;
this.checkpointEvery = checkpointEvery;
this.id = id;
this.comp = new FieldComparator("_version_", ComparatorOrder.ASCENDING);
if(!params.containsKey("rows")) {
params.put("rows", "500");
}
}
public TopicStream(StreamExpression expression, StreamFactory factory) throws IOException{
@ -147,12 +169,12 @@ public class TopicStream extends CloudSolrStream implements Expressible {
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - at least one named parameter expected. eg. 'q=*:*'",expression));
}
Map<String,String> params = new HashMap<String,String>();
ModifiableSolrParams params = new ModifiableSolrParams();
for(StreamExpressionNamedParameter namedParam : namedParams){
if(!namedParam.getName().equals("zkHost") &&
!namedParam.getName().equals("id") &&
!namedParam.getName().equals("checkpointEvery")) {
params.put(namedParam.getName(), namedParam.getParameter().toString().trim());
params.set(namedParam.getName(), namedParam.getParameter().toString().trim());
}
}
@ -189,8 +211,9 @@ public class TopicStream extends CloudSolrStream implements Expressible {
// collection
expression.addParameter(collection);
for(Entry<String,String> param : params.entrySet()) {
String value = param.getValue();
ModifiableSolrParams mParams = new ModifiableSolrParams(params);
for(Entry<String, String[]> param : mParams.getMap().entrySet()) {
String value = String.join(",", param.getValue());
// SOLR-8409: This is a special case where the params contain a " character
// Do note that in any other BASE streams with parameters where a " might come into play
@ -226,8 +249,10 @@ public class TopicStream extends CloudSolrStream implements Expressible {
// parallel stream.
child.setImplementingClass("Solr/Lucene");
child.setExpressionType(ExpressionType.DATASTORE);
child.setExpression(params.entrySet().stream().map(e -> String.format(Locale.ROOT, "%s=%s", e.getKey(), e.getValue())).collect(Collectors.joining(",")));
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);
}
@ -254,8 +279,8 @@ public class TopicStream extends CloudSolrStream implements Expressible {
this.solrStreams = new ArrayList();
this.eofTuples = Collections.synchronizedMap(new HashMap());
if(cache != null) {
cloudSolrClient = cache.getCloudSolrClient(zkHost);
if(streamContext.getSolrClientCache() != null) {
cloudSolrClient = streamContext.getSolrClientCache().getCloudSolrClient(zkHost);
} else {
cloudSolrClient = new Builder()
.withZkHost(zkHost)
@ -313,7 +338,7 @@ public class TopicStream extends CloudSolrStream implements Expressible {
}
}
if (cache == null) {
if (streamContext.getSolrClientCache() == null) {
cloudSolrClient.close();
}
}
@ -369,11 +394,11 @@ public class TopicStream extends CloudSolrStream implements Expressible {
private long getCheckpoint(Slice slice, Set<String> liveNodes) throws IOException {
Collection<Replica> replicas = slice.getReplicas();
long checkpoint = -1;
Map params = new HashMap();
params.put("q","*:*");
params.put("sort", "_version_ desc");
params.put("distrib", "false");
params.put("rows", 1);
ModifiableSolrParams params = new ModifiableSolrParams();
params.set("q","*:*");
params.set("sort", "_version_ desc");
params.set("distrib", "false");
params.set("rows", 1);
for(Replica replica : replicas) {
if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())) {
String coreUrl = replica.getCoreUrl();
@ -432,7 +457,7 @@ public class TopicStream extends CloudSolrStream implements Expressible {
if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())){
HttpSolrClient httpClient = cache.getHttpSolrClient(replica.getCoreUrl());
HttpSolrClient httpClient = streamContext.getSolrClientCache().getHttpSolrClient(replica.getCoreUrl());
try {
SolrDocument doc = httpClient.getById(id);
@ -477,20 +502,19 @@ public class TopicStream extends CloudSolrStream implements Expressible {
throw new Exception("Collection not found:" + this.collection);
}
}
params.put("distrib", "false"); // We are the aggregator.
String fl = params.get("fl");
params.put("sort", "_version_ asc");
ModifiableSolrParams mParams = new ModifiableSolrParams(params);
mParams.set("distrib", "false"); // We are the aggregator.
String fl = mParams.get("fl");
mParams.set("sort", "_version_ asc");
if(!fl.contains("_version_")) {
fl += ",_version_";
}
params.put("fl", fl);
mParams.set("fl", fl);
Random random = new Random();
for(Slice slice : slices) {
Map localParams = new HashMap();
localParams.putAll(params);
ModifiableSolrParams localParams = new ModifiableSolrParams(mParams);
long checkpoint = checkpoints.get(slice.getName());
Collection<Replica> replicas = slice.getReplicas();

View File

@ -88,6 +88,7 @@ public class ZkStateReader implements Closeable {
public static final String REPLICATION_FACTOR = "replicationFactor";
public static final String MAX_SHARDS_PER_NODE = "maxShardsPerNode";
public static final String AUTO_ADD_REPLICAS = "autoAddReplicas";
public static final String MAX_CORES_PER_NODE = "maxCoresPerNode";
public static final String ROLES = "/roles.json";
@ -137,7 +138,8 @@ public class ZkStateReader implements Closeable {
LEGACY_CLOUD,
URL_SCHEME,
AUTO_ADD_REPLICAS,
BACKUP_LOCATION)));
BACKUP_LOCATION,
MAX_CORES_PER_NODE)));
/**
* Returns config set name for collection.
@ -385,8 +387,8 @@ public class ZkStateReader implements Closeable {
if (securityNodeListener != null) {
addSecuritynodeWatcher(pair -> {
ConfigData cd = new ConfigData();
cd.data = pair.getKey() == null || pair.getKey().length == 0 ? EMPTY_MAP : Utils.getDeepCopy((Map) fromJSON(pair.getKey()), 4, false);
cd.version = pair.getValue() == null ? -1 : pair.getValue().getVersion();
cd.data = pair.first() == null || pair.first().length == 0 ? EMPTY_MAP : Utils.getDeepCopy((Map) fromJSON(pair.first()), 4, false);
cd.version = pair.second() == null ? -1 : pair.second().getVersion();
securityData = cd;
securityNodeListener.run();
});

View File

@ -17,30 +17,43 @@
package org.apache.solr.common.util;
import java.io.Serializable;
import java.util.Objects;
public class Pair<K, V> implements Serializable {
private K key;
import static org.apache.solr.common.util.Utils.makeMap;
import static org.apache.solr.common.util.Utils.toJSONString;
public K getKey() {
return key;
public class Pair<T1, T2> implements Serializable {
private final T1 first;
private final T2 second;
public T1 first() {
return first;
}
private V value;
public K _1() {
return key;
public T2 second() {
return second;
}
public V _2() {
return value;
public Pair(T1 key, T2 value) {
this.first = key;
this.second = value;
}
public V getValue() {
return value;
@Override
public boolean equals(Object that) {
return that instanceof Pair &&
Objects.equals(this.first, ((Pair) that).first) &&
Objects.equals(this.second, ((Pair) that).second);
}
public Pair(K key, V value) {
this.key = key;
this.value = value;
@Override
public String toString() {
return toJSONString(makeMap("first", first, "second", second));
}
@Override
public int hashCode() {
return Objects.hash(first, second);
}
}

View File

@ -397,7 +397,6 @@ public class GraphExpressionTest extends SolrCloudTestCase {
.commit(cluster.getSolrClient(), COLLECTION);
List<Tuple> tuples = null;
Set<String> paths = null;
GatherNodesStream stream = null;
StreamContext context = new StreamContext();
SolrClientCache cache = new SolrClientCache();

View File

@ -29,11 +29,13 @@ 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.stream.StreamContext;
import org.apache.solr.client.solrj.io.stream.StreamingTest;
import org.apache.solr.client.solrj.io.stream.TupleStream;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.cloud.AbstractDistribZkTestBase;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.params.SolrParams;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@ -100,8 +102,7 @@ public class GraphTest extends SolrCloudTestCase {
SolrClientCache cache = new SolrClientCache();
context.setSolrClientCache(cache);
Map params = new HashMap();
params.put("fq", "predicate_s:knows");
SolrParams sParams = StreamingTest.mapParams("fq", "predicate_s:knows");
stream = new ShortestPathStream(zkHost,
"collection1",
@ -109,7 +110,7 @@ public class GraphTest extends SolrCloudTestCase {
"steve",
"from_s",
"to_s",
params,
sParams,
20,
3,
6);
@ -131,7 +132,7 @@ public class GraphTest extends SolrCloudTestCase {
//Test with batch size of 1
params.put("fq", "predicate_s:knows");
sParams = StreamingTest.mapParams("fq", "predicate_s:knows");
stream = new ShortestPathStream(zkHost,
"collection1",
@ -139,7 +140,7 @@ public class GraphTest extends SolrCloudTestCase {
"steve",
"from_s",
"to_s",
params,
sParams,
1,
3,
6);
@ -159,7 +160,7 @@ public class GraphTest extends SolrCloudTestCase {
//Test with bad predicate
params.put("fq", "predicate_s:crap");
sParams = StreamingTest.mapParams("fq", "predicate_s:crap");
stream = new ShortestPathStream(zkHost,
"collection1",
@ -167,7 +168,7 @@ public class GraphTest extends SolrCloudTestCase {
"steve",
"from_s",
"to_s",
params,
sParams,
1,
3,
6);
@ -180,7 +181,7 @@ public class GraphTest extends SolrCloudTestCase {
//Test with depth 2
params.put("fq", "predicate_s:knows");
sParams = StreamingTest.mapParams("fq", "predicate_s:knows");
stream = new ShortestPathStream(zkHost,
"collection1",
@ -188,7 +189,7 @@ public class GraphTest extends SolrCloudTestCase {
"steve",
"from_s",
"to_s",
params,
sParams,
1,
3,
2);
@ -202,7 +203,7 @@ public class GraphTest extends SolrCloudTestCase {
//Take out alex
params.put("fq", "predicate_s:knows NOT to_s:alex");
sParams = StreamingTest.mapParams("fq", "predicate_s:knows NOT to_s:alex");
stream = new ShortestPathStream(zkHost,
"collection1",
@ -210,7 +211,7 @@ public class GraphTest extends SolrCloudTestCase {
"steve",
"from_s",
"to_s",
params,
sParams,
10,
3,
6);

View File

@ -19,7 +19,6 @@ package org.apache.solr.client.solrj.io.stream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
@ -47,6 +46,7 @@ import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.cloud.AbstractDistribZkTestBase;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@ -175,6 +175,26 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assert(tuples.size() == 3);
assertOrder(tuples, 0, 3, 4);
assertLong(tuples.get(1), "a_i", 3);
// Test a couple of multile field lists.
expression = StreamExpressionParser.parse("search(collection1, fq=\"a_s:hello0\", fq=\"a_s:hello1\", q=\"id:(*)\", " +
"zkHost=" + cluster.getZkServer().getZkAddress()+ ", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")");
stream = new CloudSolrStream(expression, factory);
tuples = getTuples(stream);
assertEquals("fq clauses should have prevented any docs from coming back", tuples.size(), 0);
expression = StreamExpressionParser.parse("search(collection1, fq=\"a_s:(hello0 OR hello1)\", q=\"id:(*)\", " +
"zkHost=" + cluster.getZkServer().getZkAddress() + ", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")");
stream = new CloudSolrStream(expression, factory);
tuples = getTuples(stream);
assertEquals("Combining an f1 clause should show us 2 docs", tuples.size(), 2);
}
@Test
@ -193,33 +213,33 @@ public class StreamExpressionTest extends SolrCloudTestCase {
TupleStream stream;
// Basic test
Map<String,String> params = new HashMap<>();
params.put("expr","merge("
ModifiableSolrParams sParams = new ModifiableSolrParams();
sParams.set("expr", "merge("
+ "${q1},"
+ "${q2},"
+ "on=${mySort})");
params.put(CommonParams.QT, "/stream");
params.put("q1", "search(" + COLLECTION + ", q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=${mySort})");
params.put("q2", "search(" + COLLECTION + ", q=\"id:(1)\", fl=\"id,a_s,a_i,a_f\", sort=${mySort})");
params.put("mySort", "a_f asc");
stream = new SolrStream(url, params);
sParams.set(CommonParams.QT, "/stream");
sParams.set("q1", "search(" + COLLECTION + ", q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=${mySort})");
sParams.set("q2", "search(" + COLLECTION + ", q=\"id:(1)\", fl=\"id,a_s,a_i,a_f\", sort=${mySort})");
sParams.set("mySort", "a_f asc");
stream = new SolrStream(url, sParams);
tuples = getTuples(stream);
assertEquals(4, tuples.size());
assertOrder(tuples, 0,1,3,4);
// Basic test desc
params.put("mySort", "a_f desc");
stream = new SolrStream(url, params);
sParams.set("mySort", "a_f desc");
stream = new SolrStream(url, sParams);
tuples = getTuples(stream);
assertEquals(4, tuples.size());
assertOrder(tuples, 4,3,1,0);
// Basic w/ multi comp
params.put("q2", "search(" + COLLECTION + ", q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=${mySort})");
params.put("mySort", "\"a_f asc, a_s asc\"");
stream = new SolrStream(url, params);
sParams.set("q2", "search(" + COLLECTION + ", q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=${mySort})");
sParams.set("mySort", "\"a_f asc, a_s asc\"");
stream = new SolrStream(url, sParams);
tuples = getTuples(stream);
assertEquals(5, tuples.size());
@ -2677,16 +2697,14 @@ public class StreamExpressionTest extends SolrCloudTestCase {
//Lets sleep long enough for daemon updates to run.
//Lets stop the daemons
Map params = new HashMap();
params.put(CommonParams.QT,"/stream");
params.put("action","list");
ModifiableSolrParams sParams = new ModifiableSolrParams(StreamingTest.mapParams(CommonParams.QT, "/stream", "action", "list"));
int workersComplete = 0;
for(JettySolrRunner jetty : cluster.getJettySolrRunners()) {
int iterations = 0;
INNER:
while(iterations == 0) {
SolrStream solrStream = new SolrStream(jetty.getBaseUrl().toString() + "/collection1", params);
SolrStream solrStream = new SolrStream(jetty.getBaseUrl().toString() + "/collection1", sParams);
solrStream.open();
Tuple tupleResponse = solrStream.read();
if (tupleResponse.EOF) {
@ -2714,27 +2732,27 @@ public class StreamExpressionTest extends SolrCloudTestCase {
cluster.getSolrClient().commit("parallelDestinationCollection1");
//Lets stop the daemons
params = new HashMap();
params.put(CommonParams.QT,"/stream");
params.put("action", "stop");
params.put("id", "test");
sParams = new ModifiableSolrParams();
sParams.set(CommonParams.QT, "/stream");
sParams.set("action", "stop");
sParams.set("id", "test");
for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
SolrStream solrStream = new SolrStream(jetty.getBaseUrl() + "/collection1", params);
SolrStream solrStream = new SolrStream(jetty.getBaseUrl() + "/collection1", sParams);
solrStream.open();
Tuple tupleResponse = solrStream.read();
solrStream.close();
}
params = new HashMap();
params.put(CommonParams.QT,"/stream");
params.put("action","list");
sParams = new ModifiableSolrParams();
sParams.set(CommonParams.QT, "/stream");
sParams.set("action", "list");
workersComplete = 0;
for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
long stopTime = 0;
INNER:
while(stopTime == 0) {
SolrStream solrStream = new SolrStream(jetty.getBaseUrl() + "/collection1", params);
SolrStream solrStream = new SolrStream(jetty.getBaseUrl() + "/collection1", sParams);
solrStream.open();
Tuple tupleResponse = solrStream.read();
if (tupleResponse.EOF) {

View File

@ -18,7 +18,6 @@ package org.apache.solr.client.solrj.io.stream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -43,6 +42,8 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.cloud.AbstractDistribZkTestBase;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
@ -107,8 +108,8 @@ public class StreamingTest extends SolrCloudTestCase {
.add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
.commit(cluster.getSolrClient(), COLLECTION);
Map params = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, params);
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, COLLECTION, sParams);
UniqueStream ustream = new UniqueStream(stream, new FieldEqualitor("a_f"));
List<Tuple> tuples = getTuples(ustream);
assertEquals(4, tuples.size());
@ -119,13 +120,13 @@ public class StreamingTest extends SolrCloudTestCase {
@Test
public void testSpacesInParams() throws Exception {
Map params = mapParams("q", "*:*", "fl", "id , a_s , a_i , a_f", "sort", "a_f asc , a_i asc");
SolrParams sParams = StreamingTest.mapParams("q", "*:*", "fl", "id , a_s , a_i , a_f", "sort", "a_f asc , a_i asc");
//CloudSolrStream compares the values of the sort with the fl field.
//The constructor will throw an exception if the sort fields do not the
//a value in the field list.
CloudSolrStream stream = new CloudSolrStream("", "collection1", params);
CloudSolrStream stream = new CloudSolrStream("", "collection1", sParams);
}
@Test
@ -144,8 +145,8 @@ public class StreamingTest extends SolrCloudTestCase {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTION);
Map paramsA = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_s asc,a_f asc", "partitionKeys", "none");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
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, COLLECTION, sParamsA);
ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, stream, 2, new FieldComparator("a_s",ComparatorOrder.ASCENDING));
attachStreamFactory(pstream);
@ -170,8 +171,8 @@ public class StreamingTest extends SolrCloudTestCase {
.add(id, "8", "a_s", "hello1", "a_i", "13", "a_f", "4")
.commit(cluster.getSolrClient(), COLLECTION);
Map params = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i asc", "partitionKeys", "a_f");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, params);
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, COLLECTION, sParams);
UniqueStream ustream = new UniqueStream(stream, new FieldEqualitor("a_f"));
ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, ustream, 2, new FieldComparator("a_f",ComparatorOrder.ASCENDING));
attachStreamFactory(pstream);
@ -186,6 +187,31 @@ public class StreamingTest extends SolrCloudTestCase {
}
@Test
public void testMultipleFqClauses() throws Exception {
new UpdateRequest()
.add(id, "0", "a_ss", "hello0", "a_ss", "hello1", "a_i", "0", "a_f", "0")
.add(id, "2", "a_ss", "hello2", "a_i", "2", "a_f", "0")
.add(id, "3", "a_ss", "hello3", "a_i", "3", "a_f", "3")
.add(id, "4", "a_ss", "hello4", "a_i", "4", "a_f", "4")
.add(id, "1", "a_ss", "hello1", "a_i", "1", "a_f", "1")
.add(id, "5", "a_ss", "hello1", "a_i", "10", "a_f", "1")
.add(id, "6", "a_ss", "hello1", "a_i", "11", "a_f", "5")
.add(id, "7", "a_ss", "hello1", "a_i", "12", "a_f", "5")
.add(id, "8", "a_ss", "hello1", "a_i", "13", "a_f", "4")
.commit(cluster.getSolrClient(), COLLECTION);
streamFactory.withCollectionZkHost(COLLECTION, zkHost);
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, COLLECTION, params);
List<Tuple> tuples = getTuples(stream);
assertEquals("Multiple fq clauses should have been honored", tuples.size(), 1);
assertEquals("should only have gotten back document 0", tuples.get(0).getString("id"), "0");
}
@Test
public void testRankStream() throws Exception {
@ -198,8 +224,8 @@ public class StreamingTest extends SolrCloudTestCase {
.commit(cluster.getSolrClient(), COLLECTION);
Map params = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, params);
SolrParams sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
RankStream rstream = new RankStream(stream, 3, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
List<Tuple> tuples = getTuples(rstream);
@ -224,8 +250,8 @@ public class StreamingTest extends SolrCloudTestCase {
.add(id, "10", "a_s", "hello1", "a_i", "10", "a_f", "1")
.commit(cluster.getSolrClient(), COLLECTION);
Map params = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc", "partitionKeys", "a_i");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, params);
SolrParams sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc", "partitionKeys", "a_i");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
RankStream rstream = new RankStream(stream, 11, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, rstream, 2, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
attachStreamFactory(pstream);
@ -253,8 +279,8 @@ public class StreamingTest extends SolrCloudTestCase {
.commit(cluster.getSolrClient(), COLLECTION);
//Test with spaces in the parameter lists.
Map paramsA = mapParams("q","*:*","fl","id,a_s, a_i, a_f","sort", "a_s asc , a_f asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
SolrParams sParamsA = mapParams("q", "*:*", "fl", "id,a_s, a_i,a_f", "sort", "a_s asc,a_f asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
stream.setTrace(true);
List<Tuple> tuples = getTuples(stream);
assert(tuples.get(0).get("_COLLECTION_").equals(COLLECTION));
@ -280,8 +306,8 @@ public class StreamingTest extends SolrCloudTestCase {
.commit(cluster.getSolrClient(), COLLECTION);
//Test with spaces in the parameter lists.
Map paramsA = mapParams("q","*:*","fl","id,a_s, a_i, a_f","sort", "a_s asc , a_f asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
SolrParams sParamsA = mapParams("q", "*:*", "fl", "id,a_s, a_i, a_f", "sort", "a_s asc , a_f asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
ReducerStream rstream = new ReducerStream(stream,
new FieldEqualitor("a_s"),
new GroupOperation(new FieldComparator("a_f", ComparatorOrder.ASCENDING), 5));
@ -303,8 +329,8 @@ public class StreamingTest extends SolrCloudTestCase {
assertMaps(maps2, 4, 6);
//Test with spaces in the parameter lists using a comparator
paramsA = mapParams("q","*:*","fl","id,a_s, a_i, a_f","sort", "a_s asc , a_f asc");
stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
sParamsA = mapParams("q", "*:*", "fl", "id,a_s, a_i, a_f", "sort", "a_s asc , a_f asc");
stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
rstream = new ReducerStream(stream,
new FieldComparator("a_s", ComparatorOrder.ASCENDING),
new GroupOperation(new FieldComparator("a_f", ComparatorOrder.DESCENDING), 5));
@ -345,8 +371,8 @@ public class StreamingTest extends SolrCloudTestCase {
.commit(cluster.getSolrClient(), COLLECTION);
//Test with spaces in the parameter lists.
Map paramsA = mapParams("q", "blah", "fl", "id,a_s, a_i, a_f", "sort", "a_s asc , a_f asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
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, COLLECTION, sParamsA);
ReducerStream rstream = new ReducerStream(stream,
new FieldEqualitor("a_s"),
new GroupOperation(new FieldComparator("a_f", ComparatorOrder.ASCENDING), 5));
@ -373,8 +399,8 @@ public class StreamingTest extends SolrCloudTestCase {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTION);
Map paramsA = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_s asc,a_f asc", "partitionKeys", "a_s");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
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, COLLECTION, sParamsA);
ReducerStream rstream = new ReducerStream(stream,
new FieldEqualitor("a_s"),
@ -401,8 +427,8 @@ public class StreamingTest extends SolrCloudTestCase {
//Test Descending with Ascending subsort
paramsA = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_s desc,a_f asc", "partitionKeys", "a_s");
stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
sParamsA = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_s desc,a_f asc", "partitionKeys", "a_s");
stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
rstream = new ReducerStream(stream,
new FieldEqualitor("a_s"),
@ -447,8 +473,8 @@ public class StreamingTest extends SolrCloudTestCase {
.commit(cluster.getSolrClient(), COLLECTION);
//Test an error that comes originates from the /select handler
Map paramsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,blah", "sort", "blah asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,blah", "sort", "blah asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
ExceptionStream estream = new ExceptionStream(stream);
Tuple t = getTuple(estream);
assert(t.EOF);
@ -456,8 +482,8 @@ public class StreamingTest extends SolrCloudTestCase {
assert(t.getException().contains("sort param field can't be found: blah"));
//Test an error that comes originates from the /export handler
paramsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,score", "sort", "a_s asc", "qt","/export");
stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,score", "sort", "a_s asc", "qt", "/export");
stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
estream = new ExceptionStream(stream);
t = getTuple(estream);
assert(t.EOF);
@ -483,8 +509,8 @@ public class StreamingTest extends SolrCloudTestCase {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTION);
Map paramsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,blah", "sort", "blah asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,blah", "sort", "blah asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, stream, 2, new FieldComparator("blah", ComparatorOrder.ASCENDING));
ExceptionStream estream = new ExceptionStream(pstream);
Tuple t = getTuple(estream);
@ -495,8 +521,8 @@ public class StreamingTest extends SolrCloudTestCase {
//Test an error that originates from the /select handler
paramsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,blah", "sort", "blah asc", "partitionKeys","a_s");
stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,blah", "sort", "blah asc", "partitionKeys", "a_s");
stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
pstream = new ParallelStream(zkHost, COLLECTION, stream, 2, new FieldComparator("blah", ComparatorOrder.ASCENDING));
estream = new ExceptionStream(pstream);
t = getTuple(estream);
@ -506,8 +532,8 @@ public class StreamingTest extends SolrCloudTestCase {
//Test an error that originates from the /export handler
paramsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,score", "sort", "a_s asc", "qt","/export", "partitionKeys","a_s");
stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,score", "sort", "a_s asc", "qt", "/export", "partitionKeys", "a_s");
stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
pstream = new ParallelStream(zkHost, COLLECTION, stream, 2, new FieldComparator("a_s", ComparatorOrder.ASCENDING));
estream = new ExceptionStream(pstream);
t = getTuple(estream);
@ -533,7 +559,7 @@ public class StreamingTest extends SolrCloudTestCase {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTION);
Map paramsA = mapParams("q", "*:*");
SolrParams sParamsA = mapParams("q", "*:*");
Metric[] metrics = {new SumMetric("a_i"),
new SumMetric("a_f"),
@ -545,7 +571,7 @@ public class StreamingTest extends SolrCloudTestCase {
new MeanMetric("a_f"),
new CountMetric()};
StatsStream statsStream = new StatsStream(zkHost, COLLECTION, paramsA, metrics);
StatsStream statsStream = new StatsStream(zkHost, COLLECTION, sParamsA, metrics);
List<Tuple> tuples = getTuples(statsStream);
@ -593,7 +619,7 @@ public class StreamingTest extends SolrCloudTestCase {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTION);
Map paramsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc");
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc");
Bucket[] buckets = {new Bucket("a_s")};
@ -610,7 +636,7 @@ public class StreamingTest extends SolrCloudTestCase {
FieldComparator[] sorts = {new FieldComparator("sum(a_i)",
ComparatorOrder.ASCENDING)};
FacetStream facetStream = new FacetStream(zkHost, COLLECTION, paramsA, buckets, metrics, sorts, 100);
FacetStream facetStream = new FacetStream(zkHost, COLLECTION, sParamsA, buckets, metrics, sorts, 100);
List<Tuple> tuples = getTuples(facetStream);
@ -692,7 +718,7 @@ public class StreamingTest extends SolrCloudTestCase {
sorts[0] = new FieldComparator("sum(a_i)", ComparatorOrder.DESCENDING);
facetStream = new FacetStream(zkHost, COLLECTION, paramsA, buckets, metrics, sorts, 100);
facetStream = new FacetStream(zkHost, COLLECTION, sParamsA, buckets, metrics, sorts, 100);
tuples = getTuples(facetStream);
@ -775,7 +801,7 @@ public class StreamingTest extends SolrCloudTestCase {
sorts[0] = new FieldComparator("a_s", ComparatorOrder.DESCENDING);
facetStream = new FacetStream(zkHost, COLLECTION, paramsA, buckets, metrics, sorts, 100);
facetStream = new FacetStream(zkHost, COLLECTION, sParamsA, buckets, metrics, sorts, 100);
tuples = getTuples(facetStream);
@ -856,7 +882,7 @@ public class StreamingTest extends SolrCloudTestCase {
sorts[0] = new FieldComparator("a_s", ComparatorOrder.ASCENDING);
facetStream = new FacetStream(zkHost, COLLECTION, paramsA, buckets, metrics, sorts, 100);
facetStream = new FacetStream(zkHost, COLLECTION, sParamsA, buckets, metrics, sorts, 100);
tuples = getTuples(facetStream);
@ -949,7 +975,7 @@ public class StreamingTest extends SolrCloudTestCase {
.add(id, "9", "level1_s", "hello0", "level2_s", "b", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTION);
Map paramsA = mapParams("q","*:*","fl","a_i,a_f");
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_i,a_f");
Bucket[] buckets = {new Bucket("level1_s"), new Bucket("level2_s")};
@ -961,7 +987,7 @@ public class StreamingTest extends SolrCloudTestCase {
FacetStream facetStream = new FacetStream(
zkHost,
COLLECTION,
paramsA,
sParamsA,
buckets,
metrics,
sorts,
@ -1041,7 +1067,7 @@ public class StreamingTest extends SolrCloudTestCase {
facetStream = new FacetStream(
zkHost,
COLLECTION,
paramsA,
sParamsA,
buckets,
metrics,
sorts,
@ -1134,8 +1160,8 @@ public class StreamingTest extends SolrCloudTestCase {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTION);
Map paramsA = mapParams("q","*:*","fl","a_s,a_i,a_f","sort", "a_s asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
Bucket[] buckets = {new Bucket("a_s")};
@ -1234,8 +1260,8 @@ public class StreamingTest extends SolrCloudTestCase {
.add(id, "12", "a_s", null, "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTION);
paramsA = mapParams("q","*:*","fl","a_s,a_i,a_f","sort", "a_s asc", "qt", "/export");
stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc", "qt", "/export");
stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
Bucket[] buckets1 = {new Bucket("a_s")};
@ -1285,12 +1311,9 @@ public class StreamingTest extends SolrCloudTestCase {
SolrClientCache cache = new SolrClientCache();
context.setSolrClientCache(cache);
Map params = new HashMap();
params.put("q","a_s:hello0");
params.put("rows", "500");
params.put("fl", "id");
SolrParams sParams = mapParams("q", "a_s:hello0", "rows", "500", "fl", "id");
TopicStream topicStream = new TopicStream(zkHost, COLLECTION, COLLECTION, "50000000", 1000000, params);
TopicStream topicStream = new TopicStream(zkHost, COLLECTION, COLLECTION, "50000000", 1000000, sParams);
DaemonStream daemonStream = new DaemonStream(topicStream, "daemon1", 1000, 500);
daemonStream.setStreamContext(context);
@ -1300,13 +1323,11 @@ public class StreamingTest extends SolrCloudTestCase {
// Wait for the checkpoint
JettySolrRunner jetty = cluster.getJettySolrRunners().get(0);
Map params1 = new HashMap();
params1.put("qt","/get");
params1.put("ids","50000000");
params1.put("fl","id");
SolrParams sParams1 = mapParams("qt", "/get", "ids", "50000000", "fl", "id");
int count = 0;
while(count == 0) {
SolrStream solrStream = new SolrStream(jetty.getBaseUrl().toString() + "/" + COLLECTION, params1);
SolrStream solrStream = new SolrStream(jetty.getBaseUrl().toString() + "/" + COLLECTION, sParams1);
List<Tuple> tuples = getTuples(solrStream);
count = tuples.size();
if(count > 0) {
@ -1364,8 +1385,8 @@ public class StreamingTest extends SolrCloudTestCase {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTION);
Map paramsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc", "partitionKeys", "a_s");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc", "partitionKeys", "a_s");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
Bucket[] buckets = {new Bucket("a_s")};
@ -1475,8 +1496,8 @@ public class StreamingTest extends SolrCloudTestCase {
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
.commit(cluster.getSolrClient(), COLLECTION);
Map paramsA = mapParams("q", "blah", "fl", "id,a_s,a_i,a_f","sort", "a_s asc,a_f asc", "partitionKeys", "a_s");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA);
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, COLLECTION, sParamsA);
ReducerStream rstream = new ReducerStream(stream,
new FieldEqualitor("a_s"),
new GroupOperation(new FieldComparator("a_s", ComparatorOrder.ASCENDING), 2));
@ -1497,8 +1518,8 @@ public class StreamingTest extends SolrCloudTestCase {
"1", "i_multi", "2", "f_multi", "1.2", "f_multi", "1.3")
.commit(cluster.getSolrClient(), COLLECTION);
Map params = mapParams("q","*:*","fl","id,a_s,a_i,a_f,s_multi,i_multi,f_multi","sort", "a_s asc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, params);
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, COLLECTION, sParams);
List<Tuple> tuples = getTuples(stream);
Tuple tuple = tuples.get(0);
@ -1538,11 +1559,11 @@ public class StreamingTest extends SolrCloudTestCase {
.commit(cluster.getSolrClient(), COLLECTION);
//Test ascending
Map paramsA = mapParams("q","id:(4 1)","fl","id,a_s,a_i","sort", "a_i asc");
CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTION, paramsA);
SolrParams sParamsA = mapParams("q", "id:(4 1)", "fl", "id,a_s,a_i", "sort", "a_i asc");
CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
Map paramsB = mapParams("q","id:(0 2 3)","fl","id,a_s,a_i","sort", "a_i asc");
CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTION, paramsB);
SolrParams sParamsB = mapParams("q", "id:(0 2 3)", "fl", "id,a_s,a_i", "sort", "a_i asc");
CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTION, sParamsB);
MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.ASCENDING));
List<Tuple> tuples = getTuples(mstream);
@ -1551,11 +1572,11 @@ public class StreamingTest extends SolrCloudTestCase {
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, COLLECTION, paramsA);
sParamsA = mapParams("q", "id:(4 1)", "fl", "id,a_s,a_i", "sort", "a_i desc");
streamA = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
paramsB = mapParams("q","id:(0 2 3)","fl","id,a_s,a_i","sort", "a_i desc");
streamB = new CloudSolrStream(zkHost, COLLECTION, paramsB);
sParamsB = mapParams("q", "id:(0 2 3)", "fl", "id,a_s,a_i", "sort", "a_i desc");
streamB = new CloudSolrStream(zkHost, COLLECTION, sParamsB);
mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
tuples = getTuples(mstream);
@ -1565,11 +1586,11 @@ public class StreamingTest extends SolrCloudTestCase {
//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, COLLECTION, paramsA);
sParamsA = 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, COLLECTION, sParamsA);
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, COLLECTION, paramsB);
sParamsB = mapParams("q", "id:(0 3)", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i asc");
streamB = new CloudSolrStream(zkHost, COLLECTION, sParamsB);
mstream = new MergeStream(streamA, streamB, new MultipleFieldComparator(new FieldComparator("a_f",ComparatorOrder.ASCENDING),new FieldComparator("a_i",ComparatorOrder.ASCENDING)));
tuples = getTuples(mstream);
@ -1577,11 +1598,11 @@ public class StreamingTest extends SolrCloudTestCase {
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, COLLECTION, paramsA);
sParamsA = 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, COLLECTION, sParamsA);
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, COLLECTION, paramsB);
sParamsB = mapParams("q", "id:(0 3)", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i desc");
streamB = new CloudSolrStream(zkHost, COLLECTION, sParamsB);
mstream = new MergeStream(streamA, streamB, new MultipleFieldComparator(new FieldComparator("a_f",ComparatorOrder.ASCENDING),new FieldComparator("a_i",ComparatorOrder.DESCENDING)));
tuples = getTuples(mstream);
@ -1608,11 +1629,11 @@ public class StreamingTest extends SolrCloudTestCase {
.commit(cluster.getSolrClient(), COLLECTION);
//Test ascending
Map paramsA = mapParams("q","id:(4 1 8 7 9)","fl","id,a_s,a_i","sort", "a_i asc", "partitionKeys", "a_i");
CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTION, paramsA);
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, COLLECTION, sParamsA);
Map paramsB = mapParams("q","id:(0 2 3 6)","fl","id,a_s,a_i","sort", "a_i asc", "partitionKeys", "a_i");
CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTION, paramsB);
SolrParams sParamsB = mapParams("q", "id:(0 2 3 6)", "fl", "id,a_s,a_i", "sort", "a_i asc", "partitionKeys", "a_i");
CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTION, sParamsB);
MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.ASCENDING));
ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, mstream, 2, new FieldComparator("a_i",ComparatorOrder.ASCENDING));
@ -1623,11 +1644,11 @@ public class StreamingTest extends SolrCloudTestCase {
assertOrder(tuples, 0,1,2,3,4,7,6,8,9);
//Test descending
paramsA = mapParams("q", "id:(4 1 8 9)", "fl", "id,a_s,a_i", "sort", "a_i desc", "partitionKeys", "a_i");
streamA = new CloudSolrStream(zkHost, COLLECTION, paramsA);
sParamsA = mapParams("q", "id:(4 1 8 9)", "fl", "id,a_s,a_i", "sort", "a_i desc", "partitionKeys", "a_i");
streamA = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
paramsB = mapParams("q","id:(0 2 3 6)","fl","id,a_s,a_i","sort", "a_i desc", "partitionKeys", "a_i");
streamB = new CloudSolrStream(zkHost, COLLECTION, paramsB);
sParamsB = mapParams("q", "id:(0 2 3 6)", "fl", "id,a_s,a_i", "sort", "a_i desc", "partitionKeys", "a_i");
streamB = new CloudSolrStream(zkHost, COLLECTION, sParamsB);
mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
pstream = new ParallelStream(zkHost, COLLECTION, mstream, 2, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
@ -1656,11 +1677,11 @@ public class StreamingTest extends SolrCloudTestCase {
.commit(cluster.getSolrClient(), COLLECTION);
//Test ascending
Map paramsA = mapParams("q","id:(4 1 8 7 9)","fl","id,a_s,a_i","sort", "a_i asc", "partitionKeys", "a_i");
CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTION, paramsA);
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, COLLECTION, sParamsA);
Map paramsB = mapParams("q","id:(0 2 3 6)","fl","id,a_s,a_i","sort", "a_i asc", "partitionKeys", "a_i");
CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTION, paramsB);
SolrParams sParamsB = mapParams("q", "id:(0 2 3 6)", "fl", "id,a_s,a_i", "sort", "a_i asc", "partitionKeys", "a_i");
CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTION, sParamsB);
MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.ASCENDING));
ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, mstream, 2, new FieldComparator("a_i",ComparatorOrder.ASCENDING));
@ -1685,20 +1706,19 @@ public class StreamingTest extends SolrCloudTestCase {
.add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
.commit(cluster.getSolrClient(), COLLECTION);
Map params = null;
//Basic CloudSolrStream Test with Descending Sort
params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i desc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, params);
SolrParams sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i desc");
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
List<Tuple> tuples = getTuples(stream);
assert(tuples.size() == 5);
assertOrder(tuples, 4, 3, 2, 1, 0);
//With Ascending Sort
params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i asc");
stream = new CloudSolrStream(zkHost, COLLECTION, params);
sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc");
stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
tuples = getTuples(stream);
assert(tuples.size() == 5);
@ -1706,16 +1726,16 @@ public class StreamingTest extends SolrCloudTestCase {
//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, COLLECTION, params);
sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i desc");
stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
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, COLLECTION, params);
sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i asc");
stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
tuples = getTuples(stream);
assert (tuples.size() == 5);
@ -1723,21 +1743,6 @@ public class StreamingTest extends SolrCloudTestCase {
}
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<Tuple> getTuples(TupleStream tupleStream) throws IOException {
tupleStream.open();
List<Tuple> tuples = new ArrayList();
@ -1819,4 +1824,15 @@ public class StreamingTest extends SolrCloudTestCase {
streamContext.setStreamFactory(streamFactory);
tupleStream.setStreamContext(streamContext);
}
public static SolrParams mapParams(String... vals) {
ModifiableSolrParams params = new ModifiableSolrParams();
assertEquals("Parameters passed in here must be in pairs!", 0, (vals.length % 2));
for (int idx = 0; idx < vals.length; idx += 2) {
params.add(vals[idx], vals[idx + 1]);
}
return params;
}
}

View File

@ -70,7 +70,7 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
System.setProperty(ZOOKEEPER_FORCE_SYNC, "false");
System.setProperty(MockDirectoryFactory.SOLR_TESTS_ALLOW_READING_FILES_STILL_OPEN_FOR_WRITE, "true");
String schema = getSchemaFile();
String schema = getCloudSchemaFile();
if (schema == null) schema = "schema.xml";
AbstractZkTestCase.buildZooKeeper(zkServer.getZkHost(), zkServer.getZkAddress(), getCloudSolrConfig(), schema);
@ -83,6 +83,10 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
return "solrconfig-tlog.xml";
}
protected String getCloudSchemaFile() {
return getSchemaFile();
}
@Override
protected void createServers(int numShards) throws Exception {
// give everyone there own solrhome

View File

@ -44,8 +44,6 @@ import org.apache.solr.client.solrj.impl.HttpClientUtil;
import org.apache.solr.client.solrj.impl.HttpClientUtil.SchemaRegistryProvider;
import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
import org.apache.lucene.util.Constants;
import org.eclipse.jetty.util.resource.Resource;
import org.eclipse.jetty.util.security.CertificateUtils;
import org.eclipse.jetty.util.ssl.SslContextFactory;
@ -101,7 +99,7 @@ public class SSLTestConfig extends SSLConfig {
assert isSSLMode();
SSLContextBuilder builder = SSLContexts.custom();
builder.setSecureRandom(NullSecureRandom.INSTANCE);
builder.setSecureRandom(NotSecurePsuedoRandom.INSTANCE);
// NOTE: KeyStore & TrustStore are swapped because they are from configured from server perspective...
// we are a client - our keystore contains the keys the server trusts, and vice versa
@ -130,7 +128,7 @@ public class SSLTestConfig extends SSLConfig {
assert isSSLMode();
SSLContextBuilder builder = SSLContexts.custom();
builder.setSecureRandom(NullSecureRandom.INSTANCE);
builder.setSecureRandom(NotSecurePsuedoRandom.INSTANCE);
builder.loadKeyMaterial(buildKeyStore(getKeyStore(), getKeyStorePassword()), getKeyStorePassword().toCharArray());
@ -263,93 +261,53 @@ public class SSLTestConfig extends SSLConfig {
}
/**
* A mocked up instance of SecureRandom that always does the minimal amount of work to generate
* "random" numbers. This is to prevent blocking issues that arise in platform default
* A mocked up instance of SecureRandom that just uses {@link Random} under the covers.
* This is to prevent blocking issues that arise in platform default
* SecureRandom instances due to too many instances / not enough random entropy.
* Tests do not need secure SSL.
*/
private static class NullSecureRandom extends SecureRandom {
/**
* The one and only instance that should be used, specific impl may vary based on platform
* @see Constants#SUN_OS
* @see <a href="https://issues.apache.org/jira/browse/SOLR-9068">SOLR-9068</a>
*/
public static final SecureRandom INSTANCE = Constants.SUN_OS
? new NullSecureRandom(NullSecureRandomSpi.PSUEDO_RAND_INSTANCE)
: new NullSecureRandom(NullSecureRandomSpi.NULL_INSTANCE);
/** A source of psuedo random data if needed */
private static class NotSecurePsuedoRandom extends SecureRandom {
public static final SecureRandom INSTANCE = new NotSecurePsuedoRandom();
private static final Random RAND = new Random(42);
/** SPI base class for all NullSecureRandom instances */
private static class NullSecureRandomSpi extends SecureRandomSpi {
private NullSecureRandomSpi() {
/* NOOP */
}
/**
* Helper method that can be used to fill an array with non-zero data.
* Default impl is No-Op
*/
public byte[] fillData(byte[] data) {
return data; /* NOOP */
}
/**
* Helper method that can be used to fill an array with non-zero data.
* (Attempted workarround of Solaris SSL Padding bug: SOLR-9068)
*/
private static final byte[] fillData(byte[] data) {
RAND.nextBytes(data);
return data;
}
/** SPI Used to init all instances */
private static final SecureRandomSpi NOT_SECURE_SPI = new SecureRandomSpi() {
/** returns a new byte[] filled with static data */
@Override
public byte[] engineGenerateSeed(int numBytes) {
return fillData(new byte[numBytes]);
}
/** fills the byte[] with static data */
@Override
public void engineNextBytes(byte[] bytes) {
fillData(bytes);
}
/** NOOP */
@Override
public void engineSetSeed(byte[] seed) { /* NOOP */ }
/** Instance to use on platforms w/SSLEngines that work fine when SecureRandom returns constant bytes */
public static final NullSecureRandomSpi NULL_INSTANCE = new NullSecureRandomSpi();
/**
* Instance to use on platforms that need at least psuedo-random data for the SSLEngine to not break
* (Attempted workarround of Solaris SSL Padding bug: SOLR-9068)
*/
public static final NullSecureRandomSpi PSUEDO_RAND_INSTANCE = new NullSecureRandomSpi() {
/**
* Fill with Psuedo-Random data.
* (Attempted workarround of Solaris SSL Padding bug: SOLR-9068)
*/
@Override
public byte[] fillData(byte[] data) {
RAND.nextBytes(data);
return data;
}
};
};
private NotSecurePsuedoRandom() {
super(NOT_SECURE_SPI, null) ;
}
private NullSecureRandom(NullSecureRandomSpi spi) {
super(spi, null);
this.spi = spi;
}
private NullSecureRandomSpi spi;
/** fills a new byte[] with data from SPI */
@Override
/** returns a new byte[] filled with static data */
public byte[] generateSeed(int numBytes) {
return spi.fillData(new byte[numBytes]);
return fillData(new byte[numBytes]);
}
/** fills the byte[] with data from SPI */
@Override
/** fills the byte[] with static data */
synchronized public void nextBytes(byte[] bytes) {
spi.fillData(bytes);
fillData(bytes);
}
/** NOOP */
@Override
synchronized public void setSeed(byte[] seed) { /* NOOP */ }
/** NOOP */
@Override
synchronized public void setSeed(long seed) { /* NOOP */ }
}