SOLR-5678: SolrZkClient should throw a SolrException when connect times out rather than a RuntimeException.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1565056 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Mark Robert Miller 2014-02-06 03:01:36 +00:00
parent a2bf844c90
commit 8b4ea8885e
4 changed files with 14 additions and 3 deletions

View File

@ -273,6 +273,9 @@ Bug Fixes
* SOLR-5593: Replicas should accept the last updates from a leader that has just
lost it's connection to ZooKeeper. (Christine Poerschke via Mark Miller)
* SOLR-5678: SolrZkClient should throw a SolrException when connect times out
rather than a RuntimeException. (Karl Wright, Anshum Gupta
Optimizations
----------------------

View File

@ -29,6 +29,7 @@ import org.apache.commons.io.IOUtils;
import org.apache.commons.io.filefilter.RegexFileFilter;
import org.apache.commons.io.filefilter.TrueFileFilter;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
@ -305,6 +306,12 @@ public class ZkCLITest extends SolrTestCaseJ4 {
}
}
@Test(expected = SolrException.class)
public void testInvalidZKAddress() throws SolrException{
SolrZkClient zkClient = new SolrZkClient("----------:33332", 100);
zkClient.close();
}
@Override
public void tearDown() throws Exception {
if (VERBOSE) {

View File

@ -229,7 +229,7 @@ public class ConnectionManager implements Watcher {
wait(500);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
break;
}
left = expire - System.currentTimeMillis();
}

View File

@ -24,6 +24,7 @@ import java.io.StringWriter;
import java.io.UnsupportedEncodingException;
import java.net.URLDecoder;
import java.util.List;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicLong;
import javax.xml.transform.OutputKeys;
@ -132,7 +133,7 @@ public class SolrZkClient {
Thread.currentThread().interrupt();
}
}
throw new RuntimeException(e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
try {
@ -144,7 +145,7 @@ public class SolrZkClient {
} catch (InterruptedException e1) {
Thread.currentThread().interrupt();
}
throw new RuntimeException(e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
numOpens.incrementAndGet();
}