You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2012/01/14 22:10:57 UTC
svn commit: r1231585 - in /lucene/dev/branches/solrcloud/solr:
core/src/java/org/apache/solr/cloud/
core/src/java/org/apache/solr/handler/component/
core/src/test/org/apache/solr/cloud/
solrj/src/java/org/apache/solr/client/solrj/impl/ solrj/src/java/o...
Author: markrmiller
Date: Sat Jan 14 21:10:56 2012
New Revision: 1231585
URL: http://svn.apache.org/viewvc?rev=1231585&view=rev
Log:
SOLR-2287: SolrCloud - Allow users to query by multiple, compatible collections
Modified:
lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZkController.java
lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java
lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/util/ClientUtils.java
Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1231585&r1=1231584&r2=1231585&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZkController.java Sat Jan 14 21:10:56 2012
@@ -707,7 +707,7 @@ public final class ZkController {
// if the config name wasn't passed in, use the default
if (!collectionProps.containsKey(CONFIGNAME_PROP))
- collectionProps.put(CONFIGNAME_PROP, defaultConfigName);
+ getConfName(collection, collectionPath, collectionProps);
} else if(System.getProperty("bootstrap_confdir") != null) {
// if we are bootstrapping a collection, default the config for
@@ -726,33 +726,7 @@ public final class ZkController {
collectionProps.put(CONFIGNAME_PROP, defaultConfigName);
} else {
- // check for configName
- log.info("Looking for collection configName");
- int retry = 1;
- for (; retry < 6; retry++) {
- if (zkClient.exists(collectionPath, true)) {
- ZkNodeProps cProps = ZkNodeProps.load(zkClient.getData(collectionPath, null, null, true));
- if (cProps.containsKey(CONFIGNAME_PROP)) {
- break;
- }
- }
- // if there is only one conf, use that
- List<String> configNames = zkClient.getChildren(CONFIGS_ZKNODE, null, true);
- if (configNames.size() == 1) {
- // no config set named, but there is only 1 - use it
- log.info("Only one config set found in zk - using it:" + configNames.get(0));
- collectionProps.put(CONFIGNAME_PROP, configNames.get(0));
- break;
- }
- log.info("Could not find collection configName - pausing for 2 seconds and trying again - try: " + retry);
- Thread.sleep(2000);
- }
- if (retry == 6) {
- log.error("Could not find configName for collection " + collection);
- throw new ZooKeeperException(
- SolrException.ErrorCode.SERVER_ERROR,
- "Could not find configName for collection " + collection);
- }
+ getConfName(collection, collectionPath, collectionProps);
}
ZkNodeProps zkProps = new ZkNodeProps(collectionProps);
@@ -778,6 +752,39 @@ public final class ZkController {
}
}
+
+
+ private void getConfName(String collection, String collectionPath,
+ Map<String,String> collectionProps) throws KeeperException,
+ InterruptedException {
+ // check for configName
+ log.info("Looking for collection configName");
+ int retry = 1;
+ for (; retry < 6; retry++) {
+ if (zkClient.exists(collectionPath, true)) {
+ ZkNodeProps cProps = ZkNodeProps.load(zkClient.getData(collectionPath, null, null, true));
+ if (cProps.containsKey(CONFIGNAME_PROP)) {
+ break;
+ }
+ }
+ // if there is only one conf, use that
+ List<String> configNames = zkClient.getChildren(CONFIGS_ZKNODE, null, true);
+ if (configNames.size() == 1) {
+ // no config set named, but there is only 1 - use it
+ log.info("Only one config set found in zk - using it:" + configNames.get(0));
+ collectionProps.put(CONFIGNAME_PROP, configNames.get(0));
+ break;
+ }
+ log.info("Could not find collection configName - pausing for 2 seconds and trying again - try: " + retry);
+ Thread.sleep(2000);
+ }
+ if (retry == 6) {
+ log.error("Could not find configName for collection " + collection);
+ throw new ZooKeeperException(
+ SolrException.ErrorCode.SERVER_ERROR,
+ "Could not find configName for collection " + collection);
+ }
+ }
public ZkStateReader getZkStateReader() {
return zkStateReader;
Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java?rev=1231585&r1=1231584&r2=1231585&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java Sat Jan 14 21:10:56 2012
@@ -22,6 +22,7 @@ import org.apache.solr.client.solrj.Solr
import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
import org.apache.solr.client.solrj.impl.LBHttpSolrServer;
import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.util.ClientUtils;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException;
@@ -274,11 +275,36 @@ public class HttpShardHandler extends Sh
cloudState = zkController.getCloudState();
- // TODO: check "collection" for which collection(s) to search.. but for now, just default
- // to the collection for this core.
- // This can be more efficient... we only record the name, even though we have the
- // shard info we need in the next step of mapping slice->shards
- slices = cloudState.getSlices(cloudDescriptor.getCollectionName());
+ // This can be more efficient... we only record the name, even though we
+ // have the shard info we need in the next step of mapping slice->shards
+
+ // Stores the comma-separated list of specified collections.
+ // Eg: "collection1,collection2,collection3"
+ String collections = params.get("collection");
+ if (collections != null) {
+ // If there were one or more collections specified in the query, split
+ // each parameter and store as a seperate member of a List.
+ List<String> collectionList = StrUtils.splitSmart(collections, ",",
+ true);
+
+ // First create an empty HashMap to add the slice info to.
+ slices = new HashMap<String,Slice>();
+
+ // In turn, retrieve the slices that cover each collection from the
+ // cloud state and add them to the Map 'slices'.
+ for (int i = 0; i < collectionList.size(); i++) {
+ String collection = collectionList.get(i);
+ ClientUtils.appendMap(collection, slices, cloudState.getSlices(collection));
+ }
+ } else {
+ // If no collections were specified, default to the collection for
+ // this core.
+ slices = cloudState.getSlices(cloudDescriptor.getCollectionName());
+ }
+
+ // Store the logical slices in the ResponseBuilder and create a new
+ // String array to hold the physical shards (which will be mapped
+ // later).
rb.slices = slices.keySet().toArray(new String[slices.size()]);
rb.shards = new String[rb.slices.length];
Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java?rev=1231585&r1=1231584&r2=1231585&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java Sat Jan 14 21:10:56 2012
@@ -31,6 +31,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.UnicodeUtil;
+import org.apache.solr.client.solrj.util.ClientUtils;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrDocument;
@@ -246,11 +247,36 @@ public class QueryComponent extends Sear
cloudState = zkController.getCloudState();
- // TODO: check "collection" for which collection(s) to search.. but for now, just default
- // to the collection for this core.
// This can be more efficient... we only record the name, even though we have the
// shard info we need in the next step of mapping slice->shards
- slices = cloudState.getSlices(cloudDescriptor.getCollectionName());
+
+ // Stores the comma-separated list of specified collections.
+ // Eg: "collection1,collection2,collection3"
+ String collections = params.get("collection");
+ if (collections != null) {
+ // If there were one or more collections specified in the query, split
+ // each parameter and store as a seperate member of a List.
+ List<String> collectionList = StrUtils.splitSmart(collections, ",",
+ true);
+
+ // First create an empty HashMap to add the slice info to.
+ slices = new HashMap<String,Slice>();
+
+ // In turn, retrieve the slices that cover each collection from the
+ // cloud state and add them to the Map 'slices'.
+ for (int i = 0; i < collectionList.size(); i++) {
+ String collection = collectionList.get(i);
+ ClientUtils.appendMap(collection, slices, cloudState.getSlices(collection));
+ }
+ } else {
+ // If no collections were specified, default to the collection for
+ // this core.
+ slices = cloudState.getSlices(cloudDescriptor.getCollectionName());
+ }
+
+ // Store the logical slices in the ResponseBuilder and create a new
+ // String array to hold the physical shards (which will be mapped
+ // later).
rb.slices = slices.keySet().toArray(new String[slices.size()]);
rb.shards = new String[rb.slices.length];
Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java?rev=1231585&r1=1231584&r2=1231585&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java Sat Jan 14 21:10:56 2012
@@ -17,11 +17,21 @@ package org.apache.solr.cloud;
* limitations under the License.
*/
+import java.io.IOException;
import java.net.MalformedURLException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.CloudSolrServer;
+import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
@@ -49,6 +59,8 @@ public class BasicDistributedZkTest exte
String missingField="ignore_exception__missing_but_valid_field_t";
String invalidField="ignore_exception__invalid_field_not_in_schema";
+ private Map<String,List<SolrServer>> otherCollectionClients = new HashMap<String,List<SolrServer>>();
+
public BasicDistributedZkTest() {
fixShardCount = true;
}
@@ -228,13 +240,106 @@ public class BasicDistributedZkTest exte
// TODO: This test currently fails because debug info is obtained only
// on shards with matches.
// query("q","matchesnothing","fl","*,score", "debugQuery", "true");
-
+
+ testMultipleCollections();
// Thread.sleep(10000000000L);
if (DEBUG) {
super.printLayout();
}
}
+ private void testMultipleCollections() throws MalformedURLException,
+ SolrServerException, IOException, Exception {
+ // create another 2 collections and search across them
+ createNewCollection("collection2");
+ indexDoc("collection2", getDoc(id, "10000000"));
+ indexDoc("collection2", getDoc(id, "10000001"));
+ indexDoc("collection2", getDoc(id, "10000003"));
+
+ createNewCollection("collection3");
+ indexDoc("collection3", getDoc(id, "20000000"));
+ indexDoc("collection3", getDoc(id, "20000001"));
+
+ otherCollectionClients.get("collection2").get(0).commit();
+ otherCollectionClients.get("collection3").get(0).commit();
+
+ long collection1Docs = solrj.query(new SolrQuery("*:*")).getResults()
+ .getNumFound();
+ long collection2Docs = otherCollectionClients.get("collection2").get(0)
+ .query(new SolrQuery("*:*")).getResults().getNumFound();
+ long collection3Docs = otherCollectionClients.get("collection3").get(0)
+ .query(new SolrQuery("*:*")).getResults().getNumFound();
+
+ SolrQuery query = new SolrQuery("*:*");
+ query.set("collection", "collection2,collection3");
+ long found = clients.get(0).query(query).getResults().getNumFound();
+ assertEquals(collection2Docs + collection3Docs, found);
+
+ query = new SolrQuery("*:*");
+ query.set("collection", "collection1,collection2,collection3");
+ found = clients.get(0).query(query).getResults().getNumFound();
+ assertEquals(collection1Docs + collection2Docs + collection3Docs, found);
+
+ // try to search multiple with cloud client
+ found = solrj.query(query).getResults().getNumFound();
+ assertEquals(collection1Docs + collection2Docs + collection3Docs, found);
+
+ query.set("collection", "collection2,collection3");
+ found = solrj.query(query).getResults().getNumFound();
+ assertEquals(collection2Docs + collection3Docs, found);
+
+ query.set("collection", "collection3");
+ found = solrj.query(query).getResults().getNumFound();
+ assertEquals(collection3Docs, found);
+
+ query.remove("collection");
+ found = solrj.query(query).getResults().getNumFound();
+ assertEquals(collection1Docs, found);
+ }
+
+ protected SolrInputDocument getDoc(Object... fields) throws Exception {
+ SolrInputDocument doc = new SolrInputDocument();
+ addFields(doc, fields);
+ return doc;
+ }
+
+ protected void indexDoc(String collection, SolrInputDocument doc) throws IOException, SolrServerException {
+ List<SolrServer> clients = otherCollectionClients.get(collection);
+ int which = (doc.getField(id).toString().hashCode() & 0x7fffffff) % clients.size();
+ SolrServer client = clients.get(which);
+ client.add(doc);
+ }
+
+ private void createNewCollection(String collection)
+ throws MalformedURLException, SolrServerException, IOException {
+ List<SolrServer> collectionClients = new ArrayList<SolrServer>();
+ otherCollectionClients.put(collection, collectionClients);
+ for (SolrServer client : clients) {
+ CommonsHttpSolrServer server = new CommonsHttpSolrServer(
+ ((CommonsHttpSolrServer) client).getBaseURL());
+ Create createCmd = new Create();
+ createCmd.setCoreName(collection);
+ createCmd.setDataDir(collection);
+ server.request(createCmd);
+ collectionClients.add(createNewSolrServer(collection,
+ ((CommonsHttpSolrServer) client).getBaseURL()));
+ }
+ }
+
+ protected SolrServer createNewSolrServer(String collection, String baseUrl) {
+ try {
+ // setup the server...
+ CommonsHttpSolrServer s = new CommonsHttpSolrServer(baseUrl + "/" + collection);
+ s.setConnectionTimeout(100); // 1/10th sec
+ s.setDefaultMaxConnectionsPerHost(100);
+ s.setMaxTotalConnections(100);
+ return s;
+ }
+ catch (Exception ex) {
+ throw new RuntimeException(ex);
+ }
+ }
+
volatile CloudSolrServer solrj;
@Override
Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java?rev=1231585&r1=1231584&r2=1231585&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java (original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java Sat Jan 14 21:10:56 2012
@@ -31,6 +31,7 @@ import java.util.concurrent.TimeoutExcep
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.util.ClientUtils;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.CloudState;
import org.apache.solr.common.cloud.Slice;
@@ -41,6 +42,7 @@ import org.apache.solr.common.cloud.ZooK
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.StrUtils;
import org.apache.zookeeper.KeeperException;
public class CloudSolrServer extends SolrServer {
@@ -118,7 +120,6 @@ public class CloudSolrServer extends Sol
}
}
-
@Override
public NamedList<Object> request(SolrRequest request) throws SolrServerException, IOException {
connect();
@@ -132,15 +133,18 @@ public class CloudSolrServer extends Sol
reqParams = new ModifiableSolrParams();
}
String collection = reqParams.get("collection", defaultCollection);
-
- // TODO: allow multiple collections to be specified via comma separated list
-
- Map<String,Slice> slices = cloudState.getSlices(collection);
- if (slices == null) {
- throw new RuntimeException("Could not find collection in zk: " + cloudState);
- }
+ // Extract each comma separated collection name and store in a List.
+ List<String> collectionList = StrUtils.splitSmart(collection, ",", true);
+ // Retrieve slices from the cloud state and, for each collection specified,
+ // add it to the Map of slices.
+ Map<String,Slice> slices = new HashMap<String,Slice>();
+ for (int i = 0; i < collectionList.size(); i++) {
+ String coll= collectionList.get(i);
+ ClientUtils.appendMap(coll, slices, cloudState.getSlices(coll));
+ }
+
Set<String> liveNodes = cloudState.getLiveNodes();
// IDEA: have versions on various things... like a global cloudState version
Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/util/ClientUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/util/ClientUtils.java?rev=1231585&r1=1231584&r2=1231585&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/util/ClientUtils.java (original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/client/solrj/util/ClientUtils.java Sat Jan 14 21:10:56 2012
@@ -27,7 +27,10 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.Date;
import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
import java.util.TimeZone;
+import java.util.Map.Entry;
import java.nio.ByteBuffer;
import org.apache.commons.httpclient.util.DateParseException;
@@ -35,6 +38,7 @@ import org.apache.commons.httpclient.uti
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.SolrInputField;
+import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.*;
@@ -229,4 +233,11 @@ public class ClientUtils
catch (IOException e) {throw new RuntimeException(e);} // can't happen
return sb.toString();
}
+
+ public static void appendMap(String collection, Map<String,Slice> map1, Map<String,Slice> map2) {
+ Set<Entry<String,Slice>> entrySet = map2.entrySet();
+ for (Entry<String,Slice> entry : entrySet) {
+ map1.put(collection + "_" + entry.getKey(), entry.getValue());
+ }
+ }
}