mirror of https://github.com/apache/lucene.git
SOLR-8599: Improved the tests for this issue to avoid changing a variable to non-final
This commit is contained in:
parent
d392940092
commit
e3b785a906
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.solr.cloud;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -23,9 +24,14 @@ import java.util.concurrent.TimeUnit;
|
|||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||
import org.apache.solr.SolrTestCaseJ4;
|
||||
import org.apache.solr.common.cloud.ConnectionManager;
|
||||
import org.apache.solr.common.cloud.DefaultConnectionStrategy;
|
||||
import org.apache.solr.common.cloud.OnReconnect;
|
||||
import org.apache.solr.common.cloud.SolrZkClient;
|
||||
import org.apache.solr.common.cloud.ZkClientConnectionStrategy;
|
||||
import org.apache.solr.common.cloud.ZkClientConnectionStrategy.ZkUpdate;
|
||||
import org.apache.solr.util.DefaultSolrThreadFactory;
|
||||
import org.apache.zookeeper.WatchedEvent;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.zookeeper.Watcher.Event.EventType;
|
||||
import org.apache.zookeeper.Watcher.Event.KeeperState;
|
||||
import org.junit.Ignore;
|
||||
|
@ -129,22 +135,19 @@ public class ConnectionManagerTest extends SolrTestCaseJ4 {
|
|||
AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
|
||||
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
|
||||
|
||||
SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
|
||||
MockZkClientConnectionStrategy strat = new MockZkClientConnectionStrategy();
|
||||
SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT, strat , null);
|
||||
ConnectionManager cm = zkClient.getConnectionManager();
|
||||
|
||||
try {
|
||||
assertFalse(cm.isLikelyExpired());
|
||||
assertTrue(cm.isConnected());
|
||||
|
||||
|
||||
cm.setZkServerAddress("http://BADADDRESS");
|
||||
executor.schedule(() -> {
|
||||
cm.setZkServerAddress(server.getZkAddress());
|
||||
}, 5, TimeUnit.SECONDS);
|
||||
|
||||
// reconnect -- should no longer be likely expired
|
||||
cm.process(new WatchedEvent(EventType.None, KeeperState.Expired, ""));
|
||||
assertFalse(cm.isLikelyExpired());
|
||||
assertTrue(cm.isConnected());
|
||||
assertTrue(strat.isExceptionThrow());
|
||||
} finally {
|
||||
cm.close();
|
||||
zkClient.close();
|
||||
|
@ -154,4 +157,25 @@ public class ConnectionManagerTest extends SolrTestCaseJ4 {
|
|||
server.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
private class MockZkClientConnectionStrategy extends DefaultConnectionStrategy {
|
||||
int called = 0;
|
||||
boolean exceptionThrown = false;
|
||||
|
||||
@Override
|
||||
public void reconnect(final String serverAddress, final int zkClientTimeout,
|
||||
final Watcher watcher, final ZkUpdate updater) throws IOException {
|
||||
|
||||
if(called++ < 1) {
|
||||
exceptionThrown = true;
|
||||
throw new IOException("Testing");
|
||||
}
|
||||
|
||||
super.reconnect(serverAddress, zkClientTimeout, watcher, updater);
|
||||
}
|
||||
|
||||
public boolean isExceptionThrow() {
|
||||
return exceptionThrown;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,8 +37,7 @@ public class ConnectionManager implements Watcher {
|
|||
|
||||
private final ZkClientConnectionStrategy connectionStrategy;
|
||||
|
||||
//expert: mutable for testing
|
||||
private String zkServerAddress;
|
||||
private final String zkServerAddress;
|
||||
|
||||
|
||||
|
||||
|
@ -255,9 +254,4 @@ public class ConnectionManager implements Watcher {
|
|||
"", e);
|
||||
}
|
||||
}
|
||||
|
||||
//expert: mutable for testing
|
||||
public void setZkServerAddress(String zkServerAddress) {
|
||||
this.zkServerAddress = zkServerAddress;
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue