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 2011/11/22 05:44:13 UTC

svn commit: r1204808 - in /lucene/dev/branches/solrcloud/solr: core/src/java/org/apache/solr/cloud/ core/src/java/org/apache/solr/handler/ core/src/java/org/apache/solr/update/ core/src/java/org/apache/solr/update/processor/ core/src/test/org/apache/so...

Author: markrmiller
Date: Tue Nov 22 04:44:11 2011
New Revision: 1204808

URL: http://svn.apache.org/viewvc?rev=1204808&view=rev
Log:
add new shard prop called state that can be active or recovering - also start doing replication synchronously - also cleanup system.outs

Modified:
    lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
    lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/HashPartitioner.java
    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/ReplicationHandler.java
    lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/SnapPuller.java
    lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
    lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
    lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/FullDistributedZkTest.java
    lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java
    lucene/dev/branches/solrcloud/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java?rev=1204808&r1=1204807&r2=1204808&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java Tue Nov 22 04:44:11 2011
@@ -24,7 +24,7 @@ public class CloudDescriptor {
   private String shardId;
   private String collectionName;
   private SolrParams params;
-  private String roles;
+  private String roles = "";
   
   public void setShardId(String shardId) {
     this.shardId = shardId;

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/HashPartitioner.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/HashPartitioner.java?rev=1204808&r1=1204807&r2=1204808&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/HashPartitioner.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/HashPartitioner.java Tue Nov 22 04:44:11 2011
@@ -38,12 +38,6 @@ public class HashPartitioner {
     long range = Integer.MAX_VALUE + (Math.abs((long)Integer.MIN_VALUE));
     long srange = range / partitions;
     
-    System.out.println("min:" + Integer.MIN_VALUE);
-    System.out.println("max:" + Integer.MAX_VALUE);
-    
-    System.out.println("range:" + range);
-    System.out.println("srange:" + srange);
-    
     List<Range> ranges = new ArrayList<Range>(partitions);
     
     long end = 0;
@@ -51,7 +45,6 @@ public class HashPartitioner {
 
     while (end < Integer.MAX_VALUE) {
       end = start + srange;
-      System.out.println("from:" + start + ":" + end);
       start = end + 1L;
       ranges.add(new Range(start, end));
     }

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=1204808&r1=1204807&r2=1204808&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 Tue Nov 22 04:44:11 2011
@@ -26,12 +26,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.ReentrantLock;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
-import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.CloudState;
 import org.apache.solr.common.cloud.OnReconnect;
@@ -40,9 +39,12 @@ import org.apache.solr.common.cloud.Solr
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.cloud.ZooKeeperException;
-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.core.CoreDescriptor;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.ReplicationHandler;
+import org.apache.solr.handler.SnapPuller;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.Code;
@@ -410,6 +412,8 @@ public final class ZkController {
    * @throws Exception 
    */
   public String register(String coreName, final CoreDescriptor desc, final CloudDescriptor cloudDesc) throws Exception {
+    // nocommit: TODO: on core reload we don't want to do recovery or anything...
+    
     String shardUrl = localHostName + ":" + localHostPort + "/" + localHostContext
         + "/" + coreName;
     
@@ -422,10 +426,10 @@ public final class ZkController {
     CloudState state = CloudState.load(data);
     String shardZkNodeName = getNodeName() + "_" + coreName;
     
-    boolean recover = getIsRecover(cloudDesc, state, shardZkNodeName);
+    boolean doRecovery = checkRecovery(cloudDesc, state, shardZkNodeName);
     
     String shardId = cloudDesc.getShardId();
-    if (shardId == null && !recover) {
+    if (shardId == null && !doRecovery) {
       shardId = assignShard.assignShard(collection, numShards);
       cloudDesc.setShardId(shardId);
     }
@@ -437,7 +441,7 @@ public final class ZkController {
     
     leaderElector.setupForSlice(shardId, collection);
     
-    ZkNodeProps props = addToZk(collection, desc, cloudDesc, shardUrl, shardZkNodeName);
+    ZkNodeProps props = addToZk(collection, desc, cloudDesc, shardUrl, shardZkNodeName, "recovering");
     
     // leader election
     doLeaderElectionProcess(shardId, collection, shardZkNodeName, props);
@@ -451,22 +455,23 @@ public final class ZkController {
       iamleader = true;
     } else {
       // we are not the leader, so catch up with recovery
-      recover = true;
+      doRecovery = true;
     }
     
-    if (recover) {
+    if (doRecovery) {
       if (desc.getCoreContainer() != null) {
         doRecovery(collection, desc, cloudDesc, iamleader);
       } else {
         log.warn("For some odd reason a SolrCore is trying to recover but does not have access to a CoreContainer - skipping recovery.");
       }
     }
+    addToZk(collection, desc, cloudDesc, shardUrl, shardZkNodeName, "active");
 
     return shardId;
   }
 
 
-  private boolean getIsRecover(final CloudDescriptor cloudDesc,
+  private boolean checkRecovery(final CloudDescriptor cloudDesc,
       CloudState state, String shardZkNodeName) {
     boolean recover = false;
     Map<String,Slice> slices = state.getSlices(cloudDesc.getCollectionName());
@@ -475,13 +480,10 @@ public final class ZkController {
       Map<String,String> nodes = new HashMap<String,String>();
 
       for (Slice s : slices.values()) {
-        System.out.println("add slice: "+ s.getName());
         for (String node : s.getShards().keySet()) {
-          System.out.println("add node: "+ node);
           nodes.put(node, s.getName());
         }
       }
-      System.out.println("print recovery:" + nodes + " name: " + shardZkNodeName);
       if (nodes.containsKey(shardZkNodeName)) {
         // TODO: we where already registered - go into recovery mode
         cloudDesc.setShardId(nodes.get(shardZkNodeName));
@@ -493,7 +495,7 @@ public final class ZkController {
 
 
   ZkNodeProps addToZk(String collection, final CoreDescriptor desc, final CloudDescriptor cloudDesc, String shardUrl,
-      final String shardZkNodeName)
+      final String shardZkNodeName, String state)
       throws Exception {
     ZkNodeProps props = new ZkNodeProps();
     props.put(ZkStateReader.URL_PROP, shardUrl);
@@ -501,6 +503,8 @@ public final class ZkController {
     props.put(ZkStateReader.NODE_NAME, getNodeName());
     
     props.put("roles", cloudDesc.getRoles());
+    
+    props.put("state", state);
 
     Map<String, ZkNodeProps> shardProps = new HashMap<String, ZkNodeProps>();
     shardProps.put(shardZkNodeName, props);
@@ -511,12 +515,12 @@ public final class ZkController {
 		if (stat == null) {
 			log.info(ZkStateReader.CLUSTER_STATE + " does not exist, attempting to create");
 			try {
-				CloudState state = new CloudState();
+				CloudState clusterState = new CloudState();
 
-				state.addSlice(cloudDesc.getCollectionName(), slice);
+				clusterState.addSlice(cloudDesc.getCollectionName(), slice);
 
 				zkClient.create(ZkStateReader.CLUSTER_STATE,
-						CloudState.store(state), Ids.OPEN_ACL_UNSAFE,
+						CloudState.store(clusterState), Ids.OPEN_ACL_UNSAFE,
 						CreateMode.PERSISTENT);
 				persisted = true;
 				log.info(ZkStateReader.CLUSTER_STATE);
@@ -539,16 +543,16 @@ public final class ZkController {
 						null, stat);
 				log.info("Attempting to update " + ZkStateReader.CLUSTER_STATE + " version "
 						+ stat.getVersion());
-				CloudState state = CloudState.load(data);
-				// our second state read - should only need one (see register)
+				CloudState clusterState = CloudState.load(data);
+				// our second state read - should only need one? (see register)
         
 				// we need a new copy to modify
-        state = new CloudState(state.getLiveNodes(), state.getCollectionStates());
-				state.addSlice(cloudDesc.getCollectionName(), slice);
+				clusterState = new CloudState(clusterState.getLiveNodes(), clusterState.getCollectionStates());
+				clusterState.addSlice(cloudDesc.getCollectionName(), slice);
 
 				try {
 					zkClient.setData(ZkStateReader.CLUSTER_STATE,
-							CloudState.store(state), stat.getVersion());
+							CloudState.store(clusterState), stat.getVersion());
 					updated = true;
 				} catch (KeeperException e) {
 					if (e.code() != Code.BADVERSION) {
@@ -582,23 +586,45 @@ public final class ZkController {
       // if we are the leader, either we are trying to recover faster
       // then our ephemeral timed out or we are the only node
       
-      ModifiableSolrParams params = new ModifiableSolrParams();
-      params.set("command", "fetchindex");
-      params.set("force", true); // force replication regardless of
-                                 // versions
-      params.set("masterUrl", leaderUrl + "replication");
-      QueryRequest req = new QueryRequest(params);
-      req.setPath("/replication");
-      System.out.println("Make replication call to:" + leaderUrl);
-      System.out.println("params:" + params);
+      // TODO: first, issue a hard commit?
+      
       
       // if we want to buffer updates while recovering, this
       // will have to trigger later - http is not yet up
       
-      // we need to use embedded cause http is not up yet anyhow
-      EmbeddedSolrServer server = new EmbeddedSolrServer(
-          desc.getCoreContainer(), desc.getName());
-      server.request(req);
+      // use rep handler and SnapPuller directly, so we can do this sync rather than async
+      SolrCore core = desc.getCoreContainer().getCore(desc.getName());
+      try {
+        ReplicationHandler replicationHandler = (ReplicationHandler) core
+            .getRequestHandler("/replication");
+        
+        if (replicationHandler == null) {
+          log.error("Skipping recovery, no /replication handler found");
+          return;
+        }
+        
+        ReentrantLock snapPullLock = replicationHandler.snapPullLock;
+        if (!snapPullLock.tryLock()) return;
+        SnapPuller tempSnapPuller;
+        try {
+          
+          NamedList<Object> nl = new NamedList<Object>();
+          nl.add(ReplicationHandler.MASTER_URL, leaderUrl + "replication");
+          nl.remove(SnapPuller.POLL_INTERVAL);
+          tempSnapPuller = new SnapPuller(nl, replicationHandler, core);
+          
+          tempSnapPuller.fetchLatestIndex(core, true);
+        } catch (Exception e) {
+          log.error("SnapPull failed ", e);
+        } finally {
+          
+          snapPullLock.unlock();
+        }
+      } finally {
+        core.close();
+      }
+
+      // TODO: once done replicating, mark as active
     }
   }
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java?rev=1204808&r1=1204807&r2=1204808&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java Tue Nov 22 04:44:11 2011
@@ -88,7 +88,8 @@ public class ReplicationHandler extends 
 
   private SnapPuller snapPuller;
 
-  private ReentrantLock snapPullLock = new ReentrantLock();
+  // nocommit: made this public
+  public ReentrantLock snapPullLock = new ReentrantLock();
 
   private String includeConfFiles;
 
@@ -282,22 +283,27 @@ public class ReplicationHandler extends 
 
   private volatile SnapPuller tempSnapPuller;
 
-  void doFetch(SolrParams solrParams) {
+  public void doFetch(SolrParams solrParams) {
     String masterUrl = solrParams == null ? null : solrParams.get(MASTER_URL);
     if (!snapPullLock.tryLock())
       return;
     try {
-      tempSnapPuller = snapPuller;
+      
       if (masterUrl != null) {
         NamedList<Object> nl = solrParams.toNamedList();
         nl.remove(SnapPuller.POLL_INTERVAL);
         tempSnapPuller = new SnapPuller(nl, this, core);
+      } else {
+        tempSnapPuller = snapPuller;
       }
+      
       tempSnapPuller.fetchLatestIndex(core, solrParams == null ? false : solrParams.getBool(FORCE, false));
     } catch (Exception e) {
       LOG.error("SnapPull failed ", e);
     } finally {
-      tempSnapPuller = snapPuller;
+      if (snapPuller != null) {
+        tempSnapPuller = snapPuller;
+      }
       snapPullLock.unlock();
     }
   }
@@ -444,7 +450,7 @@ public class ReplicationHandler extends 
   }
 
   boolean isPollingDisabled() {
-    return snapPuller.isPollingDisabled();
+    return snapPuller == null ? false : snapPuller.isPollingDisabled();
   }
 
   int getTimesReplicatedSinceStartup() {
@@ -456,7 +462,7 @@ public class ReplicationHandler extends 
   }
 
   long getIndexSize() {
-    return FileUtils.sizeOfDirectory(new File(core.getIndexDir()));
+    return FileUtils.sizeOfDirectory(new File(core.getNewIndexDir()));
   }
 
   /**

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/SnapPuller.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/SnapPuller.java?rev=1204808&r1=1204807&r2=1204808&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/SnapPuller.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/SnapPuller.java Tue Nov 22 04:44:11 2011
@@ -247,7 +247,7 @@ public class SnapPuller {
   @SuppressWarnings("unchecked")
   boolean successfulInstall = false;
 
-  boolean fetchLatestIndex(SolrCore core, boolean force) throws IOException {
+  public boolean fetchLatestIndex(SolrCore core, boolean force) throws IOException {
     replicationStartTime = System.currentTimeMillis();
     try {
       //get the current 'replicateable' index version in the master

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java?rev=1204808&r1=1204807&r2=1204808&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java Tue Nov 22 04:44:11 2011
@@ -316,13 +316,12 @@ public class SolrCmdDistributor {
             } else {
               url = shard;
             }
-            System.out.println("URL:" + url);
+
             SolrServer server = new CommonsHttpSolrServer(url, client);
             clonedRequest.ursp = server.request(clonedRequest.ureq);
             
             // currently no way to get the request body.
           } catch (Exception e) {
-            e.printStackTrace(System.out);
             clonedRequest.exception = e;
             if (e instanceof SolrException) {
               clonedRequest.rspCode = ((SolrException) e).code();
@@ -330,7 +329,6 @@ public class SolrCmdDistributor {
               clonedRequest.rspCode = -1;
             }
           }
-          System.out.println("RSPFirst:" + clonedRequest.rspCode);
           return clonedRequest;
         }
       };
@@ -352,7 +350,6 @@ public class SolrCmdDistributor {
         
         try {
           Request sreq = future.get();
-          System.out.println("RSP:" + sreq.rspCode);
           if (sreq.rspCode != 0) {
             // error during request
             failed++;
@@ -388,8 +385,6 @@ public class SolrCmdDistributor {
       }
     }
     
-    System.out.println("check failed rate:" + failed + " " + expectedResponses
-        / 2);
     if (failed <= (expectedResponses / 2)) {
       // don't fail if half or more where fine
       rsp.setException(null);

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1204808&r1=1204807&r2=1204808&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Tue Nov 22 04:44:11 2011
@@ -107,7 +107,6 @@ public class DistributedUpdateProcessor 
   }
 
   private void setupRequest(int hash) {
-    System.out.println("hash:" + hash);
     CoreDescriptor coreDesc = req.getCore().getCoreDescriptor();
     
     CloudState cloudState = req.getCore().getCoreDescriptor().getCoreContainer().getZkController().getCloudState();
@@ -152,7 +151,6 @@ public class DistributedUpdateProcessor 
             if (shardZkNodeName.equals(leader)) {
               isLeader = true;
             }
-            System.out.println(" go local");
           } else if (shardZkNodeName.equals(leader)) {
             isLeader = true;
             // that means I want to forward onto my replicas...
@@ -162,13 +160,11 @@ public class DistributedUpdateProcessor 
             
             // mark that this req has been to the leader
             params.set(SEEN_LEADER, true);
-            System.out.println("mark leader seen");
           } else {
             // I need to forward onto the leader...
             shardStr = leaderUrl;
             forwardToLeader  = true;
           }
-          System.out.println("set params on req:" + params);
           req.setParams(params);
         }
       } catch (KeeperException e) {
@@ -245,10 +241,6 @@ public class DistributedUpdateProcessor 
       return;
     }
 
-    System.out.println("LeaderParam:"
-        + req.getParams().get(SEEN_LEADER));
-    System.out.println("leader? " + isLeader);
-
     // at this point, there is an update we need to try and apply.
     // we may or may not be the leader.
 
@@ -280,10 +272,8 @@ public class DistributedUpdateProcessor 
           cmd.setVersion(version);
           cmd.getSolrInputDocument().setField(VersionInfo.VERSION_FIELD, version);
           bucket.updateHighest(version);
-          System.out.println("add version field to doc:" + version);
         } else {
           // The leader forwarded us this update.
-          System.out.println("got version from leader:" + versionOnUpdate);
           cmd.setVersion(versionOnUpdate);
 
           // if we aren't the leader, then we need to check that updates were not re-ordered

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/FullDistributedZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/FullDistributedZkTest.java?rev=1204808&r1=1204807&r2=1204808&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/FullDistributedZkTest.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/FullDistributedZkTest.java Tue Nov 22 04:44:11 2011
@@ -236,7 +236,6 @@ public class FullDistributedZkTest exten
     boolean pick = random.nextBoolean();
     
     int which = (doc.getField(id).toString().hashCode() & 0x7fffffff) % sliceCount;
-    System.out.println("add doc to shard:" + which);
     
     if (pick) {
       which = which + ((shardCount / sliceCount) * random.nextInt(sliceCount-1));
@@ -247,7 +246,6 @@ public class FullDistributedZkTest exten
     UpdateRequest ureq = new UpdateRequest();
     ureq.add(doc);
     ureq.setParam("update.chain", "distrib-update-chain");
-    System.out.println("set update.chain on req");
     ureq.process(client);
   }
   
@@ -529,10 +527,6 @@ public class FullDistributedZkTest exten
     System.out.println("shard2_1 port:" + ((CommonsHttpSolrServer)s2c.get(0)).getBaseURL());
     System.out.println("shard2_2 port:" + ((CommonsHttpSolrServer)s2c.get(1)).getBaseURL());
     
-    // wait a bit for replication
-    // TODO: poll or something..
-    Thread.sleep(5000);
-    
 
     // if we properly recovered, we should now have the couple missing docs that
     // came in while shard was down
@@ -557,10 +551,6 @@ public class FullDistributedZkTest exten
       System.out.println("total:" + client.query(new SolrQuery("*:*")).getResults().getNumFound());
     }
     
-    // wait a bit for replication
-    // TODO: poll or something..
-    Thread.sleep(5000);
-    
     // assert the new server has the same number of docs as another server in
     // that shard
     assertEquals(shardToClient.get("shard1").get(0).query(new SolrQuery("*:*"))

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java?rev=1204808&r1=1204807&r2=1204808&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java (original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java Tue Nov 22 04:44:11 2011
@@ -36,6 +36,7 @@ import javax.xml.parsers.DocumentBuilder
 import javax.xml.parsers.ParserConfigurationException;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.XMLErrorLogger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -175,14 +176,17 @@ public class CloudState {
 					}
 				}
 			} catch (SAXException e) {
-				// TODO Auto-generated catch block
-				e.printStackTrace();
+        log.error("", e);
+        throw new ZooKeeperException(
+            SolrException.ErrorCode.SERVER_ERROR, "", e);
 			} catch (IOException e) {
-				// TODO Auto-generated catch block
-				e.printStackTrace();
+        log.error("", e);
+        throw new ZooKeeperException(
+            SolrException.ErrorCode.SERVER_ERROR, "", e);
 			} catch (ParserConfigurationException e) {
-				// TODO Auto-generated catch block
-				e.printStackTrace();
+        log.error("", e);
+        throw new ZooKeeperException(
+            SolrException.ErrorCode.SERVER_ERROR, "", e);
 			} finally {
 				// some XML parsers are broken and don't close the byte stream (but
 				// they should according to spec)

Modified: lucene/dev/branches/solrcloud/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java?rev=1204808&r1=1204807&r2=1204808&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java (original)
+++ lucene/dev/branches/solrcloud/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java Tue Nov 22 04:44:11 2011
@@ -330,7 +330,6 @@ public abstract class BaseDistributedSea
     for (int i = 0; i < q.length; i += 2) {
       params.add(q[i].toString(), q[i + 1].toString());
     }
-    System.out.println("Q:" + params);
     final QueryResponse controlRsp = controlClient.query(params);
 
     setDistributedParams(params);