You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2014/10/13 16:24:46 UTC

svn commit: r1631411 - in /lucene/dev/branches/branch_5x: ./ solr/ solr/solrj/ solr/solrj/src/java/org/apache/solr/client/solrj/impl/ solr/solrj/src/test/org/apache/solr/client/solrj/impl/

Author: shalin
Date: Mon Oct 13 14:24:45 2014
New Revision: 1631411

URL: http://svn.apache.org/r1631411
Log:
SOLR-5852: Add CloudSolrServer helper method to connect to a ZK ensemble

Added:
    lucene/dev/branches/branch_5x/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerMultiConstructorTest.java
      - copied unchanged from r1631409, lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerMultiConstructorTest.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/solr/solrj/   (props changed)
    lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java
    lucene/dev/branches/branch_5x/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerTest.java

Modified: lucene/dev/branches/branch_5x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/CHANGES.txt?rev=1631411&r1=1631410&r2=1631411&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/solr/CHANGES.txt Mon Oct 13 14:24:45 2014
@@ -273,6 +273,9 @@ Other Changes
 * SOLR-6529: Stop command in the start scripts should only stop the instance that it had started.
   (Varun Thacker, Timothy Potter)
 
+* SOLR-5852: Add CloudSolrServer helper method to connect to a ZK ensemble. (Varun Thacker, Furkan KAMACI,
+  Shawn Heisey, Mark Miller, Erick Erickson via shalin)
+
 ==================  4.10.1 ==================
 
 Bug Fixes

Modified: lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java?rev=1631411&r1=1631410&r2=1631411&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java (original)
+++ lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java Mon Oct 13 14:24:45 2014
@@ -84,11 +84,12 @@ import org.slf4j.LoggerFactory;
  * 'id' - if this is not the case, you must set the right name
  * with {@link #setIdField(String)}.
  */
+@SuppressWarnings("serial")
 public class CloudSolrServer extends SolrServer {
-  private static final Logger log = LoggerFactory.getLogger(CloudSolrServer.class);
+  protected static final Logger log = LoggerFactory.getLogger(CloudSolrServer.class);
 
   private volatile ZkStateReader zkStateReader;
-  private String zkHost; // the zk server address
+  private String zkHost; // the zk server connect string
   private int zkConnectTimeout = 10000;
   private int zkClientTimeout = 10000;
   private volatile String defaultCollection;
@@ -187,6 +188,57 @@ public class CloudSolrServer extends Sol
   }
   
   /**
+   * Create a new client object using multiple string values in a Collection
+   * instead of a standard zkHost connection string. Note that this method will
+   * not be used if there is only one String argument - that will use
+   * {@link #CloudSolrServer(String)} instead.
+   * 
+   * @param zkHosts
+   *          A Java Collection (List, Set, etc) of HOST:PORT strings, one for
+   *          each host in the zookeeper ensemble. Note that with certain
+   *          Collection types like HashSet, the order of hosts in the final
+   *          connect string may not be in the same order you added them.
+   * @param chroot
+   *          A chroot value for zookeeper, starting with a forward slash. If no
+   *          chroot is required, use null.
+   * @throws IllegalArgumentException
+   *           if the chroot value does not start with a forward slash.
+   * @see #CloudSolrServer(String)
+   */
+  public CloudSolrServer(Collection<String> zkHosts, String chroot) {
+    
+    StringBuilder zkBuilder = new StringBuilder();
+    int lastIndexValue = zkHosts.size() - 1;
+    int i = 0;
+    for (String zkHost : zkHosts) {
+      zkBuilder.append(zkHost);
+      if (i < lastIndexValue) {
+        zkBuilder.append(",");
+      }
+      i++;
+    }
+    if (chroot != null) {
+      if (chroot.startsWith("/")) {
+        zkBuilder.append(chroot);
+      } else {
+        throw new IllegalArgumentException(
+            "The chroot must start with a forward slash.");
+      }
+    }
+    
+    /* Log the constructed connection string and then initialize. */
+    log.info("Final constructed zkHost string: " + zkBuilder.toString());
+    
+    this.zkHost = zkBuilder.toString();
+    this.myClient = HttpClientUtil.createClient(null);
+    this.lbServer = new LBHttpSolrServer(myClient);
+    this.lbServer.setRequestWriter(new BinaryRequestWriter());
+    this.lbServer.setParser(new BinaryResponseParser());
+    this.updatesToLeaders = true;
+    shutdownLBHttpSolrServer = true;
+  }
+  
+  /**
    * @param zkHost
    *          A zookeeper client endpoint.
    * @param updatesToLeaders
@@ -265,6 +317,13 @@ public class CloudSolrServer extends Sol
     lbServer.setRequestWriter(requestWriter);
   }
 
+  /**
+   * @return the zkHost value used to connect to zookeeper.
+   */
+  public String getZkHost() {
+    return zkHost;
+  }
+
   public ZkStateReader getZkStateReader() {
     return zkStateReader;
   }
@@ -349,7 +408,7 @@ public class CloudSolrServer extends Sol
     this.parallelUpdates = parallelUpdates;
   }
 
-  private NamedList directUpdate(AbstractUpdateRequest request, ClusterState clusterState) throws SolrServerException {
+  private NamedList<Object> directUpdate(AbstractUpdateRequest request, ClusterState clusterState) throws SolrServerException {
     UpdateRequest updateRequest = (UpdateRequest) request;
     ModifiableSolrParams params = (ModifiableSolrParams) request.getParams();
     ModifiableSolrParams routableParams = new ModifiableSolrParams();
@@ -396,8 +455,8 @@ public class CloudSolrServer extends Sol
       return null;
     }
 
-    NamedList<Throwable> exceptions = new NamedList<Throwable>();
-    NamedList<NamedList> shardResponses = new NamedList<NamedList>();
+    NamedList<Throwable> exceptions = new NamedList<>();
+    NamedList<NamedList> shardResponses = new NamedList<>();
 
     Map<String, LBHttpSolrServer.Req> routes = updateRequest.getRoutes(router, col, urlMap, routableParams, this.idField);
     if (routes == null) {
@@ -440,7 +499,7 @@ public class CloudSolrServer extends Sol
         String url = entry.getKey();
         LBHttpSolrServer.Req lbRequest = entry.getValue();
         try {
-          NamedList rsp = lbServer.request(lbRequest).getResponse();
+          NamedList<Object> rsp = lbServer.request(lbRequest).getResponse();
           shardResponses.add(url, rsp);
         } catch (Exception e) {
           throw new SolrServerException(e);
@@ -752,7 +811,7 @@ public class CloudSolrServer extends Sol
     
     if (request instanceof IsUpdateRequest) {
       if (request instanceof UpdateRequest) {
-        NamedList response = directUpdate((AbstractUpdateRequest) request,
+        NamedList<Object> response = directUpdate((AbstractUpdateRequest) request,
             clusterState);
         if (response != null) {
           return response;

Modified: lucene/dev/branches/branch_5x/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerTest.java?rev=1631411&r1=1631410&r2=1631411&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerTest.java (original)
+++ lucene/dev/branches/branch_5x/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerTest.java Mon Oct 13 14:24:45 2014
@@ -158,7 +158,7 @@ public class CloudSolrServerTest extends
     request.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, false);
     
     // Test single threaded routed updates for UpdateRequest
-    NamedList response = cloudClient.request(request);
+    NamedList<Object> response = cloudClient.request(request);
     CloudSolrServer.RouteResponse rr = (CloudSolrServer.RouteResponse) response;
     Map<String,LBHttpSolrServer.Req> routes = rr.getRoutes();
     Iterator<Map.Entry<String,LBHttpSolrServer.Req>> it = routes.entrySet()
@@ -330,11 +330,8 @@ public class CloudSolrServerTest extends
     // use generic request to avoid extra processing of queries
     QueryRequest req = new QueryRequest(params);
     NamedList<Object> resp = server.request(req);
-    NamedList mbeans = (NamedList) resp.get("solr-mbeans");
-    NamedList queryHandler = (NamedList) mbeans.get("QUERYHANDLER");
-    NamedList select = (NamedList) queryHandler.get("org.apache.solr.handler.StandardRequestHandler");
-    NamedList stats = (NamedList) select.get("stats");
-    return (Long) stats.get("requests");
+    return (Long) resp.findRecursive("solr-mbeans", "QUERYHANDLER",
+        "org.apache.solr.handler.StandardRequestHandler", "stats", "requests");
   }
   
   @Override