HBASE-14498 Revert for on-going review
This commit is contained in:
parent
c36b9489aa
commit
3e551ea538
|
@ -26,7 +26,6 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -39,7 +38,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
|||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.security.Superusers;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.WatchedEvent;
|
||||
|
@ -78,7 +76,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
private RecoverableZooKeeper recoverableZooKeeper;
|
||||
|
||||
// abortable in case of zk failure
|
||||
protected final Abortable abortable;
|
||||
protected Abortable abortable;
|
||||
// Used if abortable is null
|
||||
private boolean aborted = false;
|
||||
|
||||
|
@ -90,10 +88,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
// negotiation to complete
|
||||
public CountDownLatch saslLatch = new CountDownLatch(1);
|
||||
|
||||
// Connection timeout on disconnect event
|
||||
private long connWaitTimeOut;
|
||||
private AtomicBoolean isConnected = new AtomicBoolean(false);
|
||||
|
||||
// node names
|
||||
|
||||
// base znode for this cluster
|
||||
|
@ -183,9 +177,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
this.identifier = identifier + "0x0";
|
||||
this.abortable = abortable;
|
||||
setNodeNames(conf);
|
||||
// On Disconnected event a thread will wait for sometime (2/3 of zookeeper.session.timeout),
|
||||
// it will abort the process if no SyncConnected event reported by the time.
|
||||
connWaitTimeOut = this.conf.getLong("zookeeper.session.timeout", 90000) * 2 / 3;
|
||||
this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, this, identifier);
|
||||
if (canCreateBaseZNode) {
|
||||
createBaseZNodes();
|
||||
|
@ -620,7 +611,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
private void connectionEvent(WatchedEvent event) {
|
||||
switch(event.getState()) {
|
||||
case SyncConnected:
|
||||
isConnected.set(true);
|
||||
// Now, this callback can be invoked before the this.zookeeper is set.
|
||||
// Wait a little while.
|
||||
long finished = System.currentTimeMillis() +
|
||||
|
@ -650,35 +640,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
|
||||
// Abort the server if Disconnected or Expired
|
||||
case Disconnected:
|
||||
LOG.debug("Received Disconnected from ZooKeeper.");
|
||||
isConnected.set(false);
|
||||
|
||||
Thread t = new Thread() {
|
||||
public void run() {
|
||||
long startTime = EnvironmentEdgeManager.currentTime();
|
||||
while (EnvironmentEdgeManager.currentTime() - startTime < connWaitTimeOut) {
|
||||
if (isConnected.get()) {
|
||||
LOG.debug("Client got reconnected to zookeeper.");
|
||||
return;
|
||||
}
|
||||
try {
|
||||
Thread.sleep(100);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!isConnected.get() && abortable != null) {
|
||||
String msg =
|
||||
prefix("Couldn't connect to ZooKeeper after waiting " + connWaitTimeOut
|
||||
+ " ms, aborting");
|
||||
abortable.abort(msg, new KeeperException.ConnectionLossException());
|
||||
}
|
||||
};
|
||||
};
|
||||
t.setDaemon(true);
|
||||
t.start();
|
||||
LOG.debug(prefix("Received Disconnected from ZooKeeper, ignoring"));
|
||||
break;
|
||||
|
||||
case Expired:
|
||||
|
|
|
@ -22,22 +22,14 @@ import static org.junit.Assert.*;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.zookeeper.WatchedEvent;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category({SmallTests.class})
|
||||
public class TestZooKeeperWatcher {
|
||||
private final static Log LOG = LogFactory.getLog(TestZooKeeperWatcher.class);
|
||||
|
||||
@Test
|
||||
public void testIsClientReadable() throws ZooKeeperConnectionException, IOException {
|
||||
|
@ -65,43 +57,4 @@ public class TestZooKeeperWatcher {
|
|||
watcher.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConnectionEvent() throws ZooKeeperConnectionException, IOException {
|
||||
long zkSessionTimeout = 15000l;
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
conf.set("zookeeper.session.timeout", "15000");
|
||||
|
||||
Abortable abortable = new Abortable() {
|
||||
boolean aborted = false;
|
||||
|
||||
@Override
|
||||
public void abort(String why, Throwable e) {
|
||||
aborted = true;
|
||||
LOG.error(why, e);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAborted() {
|
||||
return aborted;
|
||||
}
|
||||
};
|
||||
ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, "testConnectionEvent", abortable, false);
|
||||
|
||||
WatchedEvent event =
|
||||
new WatchedEvent(Watcher.Event.EventType.None, Watcher.Event.KeeperState.Disconnected, null);
|
||||
|
||||
long startTime = EnvironmentEdgeManager.currentTime();
|
||||
while (!abortable.isAborted()
|
||||
&& (EnvironmentEdgeManager.currentTime() - startTime < zkSessionTimeout)) {
|
||||
watcher.process(event);
|
||||
try {
|
||||
Thread.sleep(1000);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
assertTrue(abortable.isAborted());
|
||||
watcher.close();
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue