mirror of https://github.com/apache/lucene.git
SOLR-5359: ZooKeeper client is not closed when it fails to connect to an ensemble.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1533786 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
3f289b737b
commit
d4f285c177
|
@ -108,6 +108,9 @@ Bug Fixes
|
|||
|
||||
* SOLR-5367: Unmarshalling delete by id commands with JavaBin can lead to class cast
|
||||
exception. (Mark Miller)
|
||||
|
||||
* SOLR-5359: ZooKeeper client is not closed when it fails to connect to an ensemble.
|
||||
(Mark Miller, Klaus Herrmann)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
|
|
|
@ -34,21 +34,40 @@ public class DefaultConnectionStrategy extends ZkClientConnectionStrategy {
|
|||
|
||||
@Override
|
||||
public void connect(String serverAddress, int timeout, Watcher watcher, ZkUpdate updater) throws IOException, InterruptedException, TimeoutException {
|
||||
updater.update(new SolrZooKeeper(serverAddress, timeout, watcher));
|
||||
SolrZooKeeper zk = new SolrZooKeeper(serverAddress, timeout, watcher);
|
||||
boolean success = false;
|
||||
try {
|
||||
updater.update(zk);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
zk.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reconnect(final String serverAddress, final int zkClientTimeout,
|
||||
final Watcher watcher, final ZkUpdate updater) throws IOException {
|
||||
log.info("Connection expired - starting a new one...");
|
||||
|
||||
SolrZooKeeper zk = new SolrZooKeeper(serverAddress, zkClientTimeout, watcher);
|
||||
boolean success = false;
|
||||
try {
|
||||
updater
|
||||
.update(new SolrZooKeeper(serverAddress, zkClientTimeout, watcher));
|
||||
.update(zk);
|
||||
success = true;
|
||||
log.info("Reconnected to ZooKeeper");
|
||||
} catch (Exception e) {
|
||||
SolrException.log(log, "Reconnect to ZooKeeper failed", e);
|
||||
log.info("Reconnect to ZooKeeper failed");
|
||||
} finally {
|
||||
if (!success) {
|
||||
try {
|
||||
zk.close();
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -120,6 +120,13 @@ public class SolrZkClient {
|
|||
});
|
||||
} catch (Throwable e) {
|
||||
connManager.close();
|
||||
if (keeper != null) {
|
||||
try {
|
||||
keeper.close();
|
||||
} catch (InterruptedException e1) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
|
@ -127,6 +134,11 @@ public class SolrZkClient {
|
|||
connManager.waitForConnected(clientConnectTimeout);
|
||||
} catch (Throwable e) {
|
||||
connManager.close();
|
||||
try {
|
||||
keeper.close();
|
||||
} catch (InterruptedException e1) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
numOpens.incrementAndGet();
|
||||
|
|
|
@ -18,11 +18,10 @@ package org.apache.solr.client.solrj.impl;
|
|||
*/
|
||||
|
||||
import java.io.File;
|
||||
import java.util.HashSet;
|
||||
import java.net.MalformedURLException;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
|
||||
|
@ -41,6 +40,7 @@ import org.junit.AfterClass;
|
|||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
|
||||
/**
|
||||
* This test would be faster if we simulated the zk state instead.
|
||||
*/
|
||||
|
@ -201,5 +201,17 @@ public class CloudSolrServerTest extends AbstractFullDistribZkTestBase {
|
|||
SolrInputDocument doc = getDoc(fields);
|
||||
indexDoc(doc);
|
||||
}
|
||||
|
||||
public void testShutdown() throws MalformedURLException {
|
||||
CloudSolrServer server = new CloudSolrServer("[ff01::114]:33332");
|
||||
server.setZkConnectTimeout(100);
|
||||
try {
|
||||
server.connect();
|
||||
fail("Expected exception");
|
||||
} catch(RuntimeException e) {
|
||||
assertTrue(e.getCause() instanceof TimeoutException);
|
||||
}
|
||||
server.shutdown();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue