SOLR-7155: Add optional collection parameter to all SolrClient methods

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1664310 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Alan Woodward 2015-03-05 11:29:24 +00:00
parent 739e9b489d
commit a43843701d
10 changed files with 1042 additions and 201 deletions

View File

@ -133,6 +133,9 @@ New Features
* SOLR-7164: BBoxField defaults sub fields to not-stored (ryan)
* SOLR-7155: All SolrClient methods now take an optional 'collection' argument
(Alan Woodward)
Bug Fixes
----------------------

View File

@ -86,13 +86,16 @@ public class EmbeddedSolrServer extends SolrClient
}
@Override
public NamedList<Object> request(SolrRequest request) throws SolrServerException, IOException
public NamedList<Object> request(SolrRequest request, String coreName) throws SolrServerException, IOException
{
String path = request.getPath();
if( path == null || !path.startsWith( "/" ) ) {
path = "/select";
}
if (coreName == null)
coreName = this.coreName;
// Check for cores action
SolrCore core = coreContainer.getCore( coreName );
if( core == null ) {

View File

@ -72,7 +72,7 @@ public class MockStreamingSolrClients extends StreamingSolrClients {
}
@Override
public NamedList<Object> request(SolrRequest request)
public NamedList<Object> request(SolrRequest request, String collection)
throws SolrServerException, IOException {
if (exp != null) {
if (LuceneTestCase.random().nextBoolean()) {

View File

@ -120,18 +120,36 @@ public abstract class SolrRequest<T extends SolrResponse> implements Serializabl
/**
* Send this request to a {@link SolrClient} and return the response
*
* @param client the SolrClient to communicate with
* @param collection the collection to execute the request against
*
* @return the response
*
* @throws SolrServerException if there is an error on the Solr server
* @throws IOException if there is a communication error
*/
public final T process(SolrClient client) throws SolrServerException, IOException {
public final T process(SolrClient client, String collection) throws SolrServerException, IOException {
long startTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
T res = createResponse(client);
res.setResponse(client.request(this));
res.setResponse(client.request(this, collection));
long endTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
res.setElapsedTime(endTime - startTime);
return res;
}
/**
* Send this request to a {@link SolrClient} and return the response
*
* @param client the SolrClient to communicate with
*
* @return the response
*
* @throws SolrServerException if there is an error on the Solr server
* @throws IOException if there is a communication error
*/
public final T process(SolrClient client) throws SolrServerException, IOException {
return process(client, null);
}
}

View File

@ -501,7 +501,7 @@ public class CloudSolrClient extends SolrClient {
zkStateReader.getConfigManager().downloadConfigDir(configName, downloadPath);
}
private NamedList<Object> directUpdate(AbstractUpdateRequest request, ClusterState clusterState) throws SolrServerException {
private NamedList<Object> directUpdate(AbstractUpdateRequest request, String collection, ClusterState clusterState) throws SolrServerException {
UpdateRequest updateRequest = (UpdateRequest) request;
ModifiableSolrParams params = (ModifiableSolrParams) request.getParams();
ModifiableSolrParams routableParams = new ModifiableSolrParams();
@ -515,7 +515,6 @@ public class CloudSolrClient extends SolrClient {
}
}
String collection = nonRoutableParams.get(UpdateParams.COLLECTION, defaultCollection);
if (collection == null) {
throw new SolrServerException("No collection param specified on request and no default collection has been set.");
}
@ -747,9 +746,11 @@ public class CloudSolrClient extends SolrClient {
}
@Override
public NamedList<Object> request(SolrRequest request) throws SolrServerException, IOException {
public NamedList<Object> request(SolrRequest request, String collection) throws SolrServerException, IOException {
SolrParams reqParams = request.getParams();
String collection = (reqParams != null) ? reqParams.get("collection", getDefaultCollection()) : getDefaultCollection();
if (collection == null)
collection = (reqParams != null) ? reqParams.get("collection", getDefaultCollection()) : getDefaultCollection();
return requestWithRetryOnStaleState(request, 0, collection);
}
@ -807,7 +808,7 @@ public class CloudSolrClient extends SolrClient {
NamedList<Object> resp = null;
try {
resp = sendRequest(request);
resp = sendRequest(request, collection);
//to avoid an O(n) operation we always add STATE_VERSION to the last and try to read it from there
Object o = resp.get(STATE_VERSION, resp.size()-1);
if(o != null && o instanceof Map) {
@ -905,7 +906,7 @@ public class CloudSolrClient extends SolrClient {
return resp;
}
protected NamedList<Object> sendRequest(SolrRequest request)
protected NamedList<Object> sendRequest(SolrRequest request, String collection)
throws SolrServerException, IOException {
connect();
@ -916,8 +917,7 @@ public class CloudSolrClient extends SolrClient {
if (request instanceof IsUpdateRequest) {
if (request instanceof UpdateRequest) {
NamedList<Object> response = directUpdate((AbstractUpdateRequest) request,
clusterState);
NamedList<Object> response = directUpdate((AbstractUpdateRequest) request, collection, clusterState);
if (response != null) {
return response;
}
@ -938,7 +938,6 @@ public class CloudSolrClient extends SolrClient {
theUrlList.add(zkStateReader.getBaseUrlForNodeName(liveNode));
}
} else {
String collection = reqParams.get(UpdateParams.COLLECTION, defaultCollection);
if (collection == null) {
throw new SolrServerException(
@ -983,22 +982,18 @@ public class CloudSolrClient extends SolrClient {
if (!liveNodes.contains(coreNodeProps.getNodeName())
|| !coreNodeProps.getState().equals(ZkStateReader.ACTIVE)) continue;
if (nodes.put(node, nodeProps) == null) {
if (!sendToLeaders || (sendToLeaders && coreNodeProps.isLeader())) {
if (!sendToLeaders || coreNodeProps.isLeader()) {
String url;
if (reqParams.get(UpdateParams.COLLECTION) == null) {
url = ZkCoreNodeProps.getCoreUrl(
nodeProps.getStr(ZkStateReader.BASE_URL_PROP),
defaultCollection);
url = ZkCoreNodeProps.getCoreUrl(nodeProps.getStr(ZkStateReader.BASE_URL_PROP), collection);
} else {
url = coreNodeProps.getCoreUrl();
}
urlList2.add(url);
} else if (sendToLeaders) {
} else {
String url;
if (reqParams.get(UpdateParams.COLLECTION) == null) {
url = ZkCoreNodeProps.getCoreUrl(
nodeProps.getStr(ZkStateReader.BASE_URL_PROP),
defaultCollection);
url = ZkCoreNodeProps.getCoreUrl(nodeProps.getStr(ZkStateReader.BASE_URL_PROP), collection);
} else {
url = coreNodeProps.getCoreUrl();
}
@ -1039,7 +1034,7 @@ public class CloudSolrClient extends SolrClient {
}
}
LBHttpSolrClient.Req req = new LBHttpSolrClient.Req(request, theUrlList);
LBHttpSolrClient.Rsp rsp = lbClient.request(req);
return rsp.getResponse();

View File

@ -291,10 +291,10 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
}
@Override
public NamedList<Object> request(final SolrRequest request)
public NamedList<Object> request(final SolrRequest request, String collection)
throws SolrServerException, IOException {
if (!(request instanceof UpdateRequest)) {
return client.request(request);
return client.request(request, collection);
}
UpdateRequest req = (UpdateRequest) request;
@ -305,13 +305,13 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
&& (req.getDeleteByIdMap() == null || req.getDeleteByIdMap().isEmpty())) {
if (req.getDeleteQuery() == null) {
blockUntilFinished();
return client.request(request);
return client.request(request, collection);
}
}
} else {
if ((req.getDocuments() == null || req.getDocuments().isEmpty())) {
blockUntilFinished();
return client.request(request);
return client.request(request, collection);
}
}
@ -322,7 +322,7 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
if (params.getBool(UpdateParams.WAIT_SEARCHER, false)) {
log.info("blocking for commit/optimize");
blockUntilFinished(); // empty the queue
return client.request(request);
return client.request(request, collection);
}
}

View File

@ -43,15 +43,12 @@ import org.apache.http.message.BasicHeader;
import org.apache.http.message.BasicNameValuePair;
import org.apache.http.util.EntityUtils;
import org.apache.solr.client.solrj.ResponseParser;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.request.RequestWriter;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.response.UpdateResponse;
import org.apache.solr.client.solrj.util.ClientUtils;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
@ -201,7 +198,7 @@ public class HttpSolrClient extends SolrClient {
* org.apache.solr.client.solrj.ResponseParser)
*/
@Override
public NamedList<Object> request(final SolrRequest request)
public NamedList<Object> request(final SolrRequest request, String collection)
throws SolrServerException, IOException {
ResponseParser responseParser = request.getResponseParser();
if (responseParser == null) {
@ -700,54 +697,6 @@ public class HttpSolrClient extends SolrClient {
this.requestWriter = requestWriter;
}
/**
* Adds the documents supplied by the given iterator.
*
* @param docIterator
* the iterator which returns SolrInputDocument instances
*
* @return the response from the SolrServer
*/
public UpdateResponse add(Iterator<SolrInputDocument> docIterator)
throws SolrServerException, IOException {
UpdateRequest req = new UpdateRequest();
req.setDocIterator(docIterator);
return req.process(this);
}
/**
* Adds the beans supplied by the given iterator.
*
* @param beanIterator
* the iterator which returns Beans
*
* @return the response from the SolrServer
*/
public UpdateResponse addBeans(final Iterator<?> beanIterator)
throws SolrServerException, IOException {
UpdateRequest req = new UpdateRequest();
req.setDocIterator(new Iterator<SolrInputDocument>() {
@Override
public boolean hasNext() {
return beanIterator.hasNext();
}
@Override
public SolrInputDocument next() {
Object o = beanIterator.next();
if (o == null) return null;
return getBinder().toSolrInputDocument(o);
}
@Override
public void remove() {
beanIterator.remove();
}
});
return req.process(this);
}
/**
* Close the {@link ClientConnectionManager} from the internal client.
*/

View File

@ -367,7 +367,7 @@ public class LBHttpSolrClient extends SolrClient {
boolean isZombie, String zombieKey) throws SolrServerException, IOException {
Exception ex = null;
try {
rsp.rsp = client.request(req.getRequest());
rsp.rsp = client.request(req.getRequest(), (String) null);
if (isZombie) {
zombieServers.remove(zombieKey);
}
@ -491,7 +491,7 @@ public class LBHttpSolrClient extends SolrClient {
* @throws IOException If there is a low-level I/O error.
*/
@Override
public NamedList<Object> request(final SolrRequest request)
public NamedList<Object> request(final SolrRequest request, String collection)
throws SolrServerException, IOException {
Exception ex = null;
ServerWrapper[] serverList = aliveServerList;
@ -511,7 +511,7 @@ public class LBHttpSolrClient extends SolrClient {
wrapper.lastUsed = System.currentTimeMillis();
try {
return wrapper.client.request(request);
return wrapper.client.request(request, collection);
} catch (SolrException e) {
// Server is alive but the request was malformed or invalid
throw e;
@ -537,7 +537,7 @@ public class LBHttpSolrClient extends SolrClient {
if (wrapper.standard==false || justFailed!=null && justFailed.containsKey(wrapper.getKey())) continue;
try {
NamedList<Object> rsp = wrapper.client.request(request);
NamedList<Object> rsp = wrapper.client.request(request, collection);
// remove from zombie list *before* adding to alive to avoid a race that could lose a server
zombieServers.remove(wrapper.getKey());
addToAlive(wrapper);

View File

@ -17,12 +17,6 @@ package org.apache.solr.client.solrj.impl;
* limitations under the License.
*/
import static org.apache.solr.cloud.OverseerCollectionProcessor.CREATE_NODE_SET;
import static org.apache.solr.cloud.OverseerCollectionProcessor.NUM_SLICES;
import static org.apache.solr.common.cloud.ZkNodeProps.makeMap;
import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
@ -72,6 +66,11 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeoutException;
import static org.apache.solr.cloud.OverseerCollectionProcessor.NUM_SLICES;
import static org.apache.solr.common.cloud.ZkNodeProps.makeMap;
import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
/**
* This test would be faster if we simulated the zk state instead.
@ -122,6 +121,7 @@ public class CloudSolrClientTest extends AbstractFullDistribZkTestBase {
@Test
public void test() throws Exception {
checkCollectionParameters();
allTests();
stateVersionParamTest();
customHttpClientTest();
@ -498,6 +498,44 @@ public class CloudSolrClientTest extends AbstractFullDistribZkTestBase {
indexDoc(doc);
}
private void checkCollectionParameters() throws Exception {
try (CloudSolrClient client = createCloudClient("multicollection1")) {
createCollection("multicollection1", client, 2, 2);
createCollection("multicollection2", client, 2, 2);
waitForRecoveriesToFinish("multicollection1", false);
waitForRecoveriesToFinish("multicollection2", false);
List<SolrInputDocument> docs = new ArrayList<>(3);
for (int i = 0; i < 3; i++) {
SolrInputDocument doc = new SolrInputDocument();
doc.addField(id, Integer.toString(i));
doc.addField("a_t", "hello");
docs.add(doc);
}
client.add(docs); // default - will add them to multicollection1
client.commit();
ModifiableSolrParams queryParams = new ModifiableSolrParams();
queryParams.add("q", "*:*");
assertEquals(3, client.query(queryParams).getResults().size());
assertEquals(0, client.query("multicollection2", queryParams).getResults().size());
SolrQuery query = new SolrQuery("*:*");
query.set("collection", "multicollection2");
assertEquals(0, client.query(query).getResults().size());
client.add("multicollection2", docs);
client.commit("multicollection2");
assertEquals(3, client.query("multicollection2", queryParams).getResults().size());
}
}
private void stateVersionParamTest() throws Exception {
try (CloudSolrClient client = createCloudClient(null)) {