HBASE-19474 Brought down the number of Checkstyle errors in hbase-zookeeper

This commit is contained in:
Jan Hentschel 2017-12-10 15:07:01 +01:00
parent 371c243a9b
commit 231ec7e836
18 changed files with 433 additions and 319 deletions

View File

@ -20,11 +20,12 @@ package org.apache.hadoop.hbase.zookeeper;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
import org.apache.zookeeper.KeeperException;
/** /**
* Tracker on cluster settings up in zookeeper. * Tracker on cluster settings up in zookeeper.
@ -42,8 +43,9 @@ public class ClusterStatusTracker extends ZKNodeTracker {
* *
* <p>After construction, use {@link #start} to kick off tracking. * <p>After construction, use {@link #start} to kick off tracking.
* *
* @param watcher * @param watcher reference to the {@link ZKWatcher} which also contains configuration and
* @param abortable * constants
* @param abortable used to abort if a fatal error occurs
*/ */
public ClusterStatusTracker(ZKWatcher watcher, Abortable abortable) { public ClusterStatusTracker(ZKWatcher watcher, Abortable abortable) {
super(watcher, watcher.znodePaths.clusterStateZNode, abortable); super(watcher, watcher.znodePaths.clusterStateZNode, abortable);
@ -52,7 +54,7 @@ public class ClusterStatusTracker extends ZKNodeTracker {
/** /**
* Checks if cluster is up. * Checks if cluster is up.
* @return true if the cluster up ('shutdown' is its name up in zk) znode * @return true if the cluster up ('shutdown' is its name up in zk) znode
* exists with data, false if not * exists with data, false if not
*/ */
public boolean isClusterUp() { public boolean isClusterUp() {
return super.getData(false) != null; return super.getData(false) != null;
@ -88,7 +90,7 @@ public class ClusterStatusTracker extends ZKNodeTracker {
/** /**
* @return Content of the clusterup znode as a serialized pb with the pb * @return Content of the clusterup znode as a serialized pb with the pb
* magic as prefix. * magic as prefix.
*/ */
static byte [] toByteArray() { static byte [] toByteArray() {
ZooKeeperProtos.ClusterUp.Builder builder = ZooKeeperProtos.ClusterUp.Builder builder =

View File

@ -25,7 +25,7 @@ import org.apache.zookeeper.Watcher;
* An empty ZooKeeper watcher * An empty ZooKeeper watcher
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class EmptyWatcher implements Watcher { public final class EmptyWatcher implements Watcher {
// Used in this package but also by tests so needs to be public // Used in this package but also by tests so needs to be public
public static final EmptyWatcher instance = new EmptyWatcher(); public static final EmptyWatcher instance = new EmptyWatcher();
private EmptyWatcher() {} private EmptyWatcher() {}

View File

@ -35,11 +35,11 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.util.DNS; import org.apache.hadoop.hbase.util.DNS;
import org.apache.hadoop.hbase.util.Strings; import org.apache.hadoop.hbase.util.Strings;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.zookeeper.server.ServerConfig; import org.apache.zookeeper.server.ServerConfig;
import org.apache.zookeeper.server.ZooKeeperServerMain; import org.apache.zookeeper.server.ZooKeeperServerMain;
import org.apache.zookeeper.server.quorum.QuorumPeerConfig; import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
@ -54,7 +54,9 @@ import org.apache.zookeeper.server.quorum.QuorumPeerMain;
*/ */
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS) @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class HQuorumPeer { public final class HQuorumPeer {
private HQuorumPeer() {
}
/** /**
* Parse ZooKeeper configuration from HBase XML config and run a QuorumPeer. * Parse ZooKeeper configuration from HBase XML config and run a QuorumPeer.
@ -80,7 +82,8 @@ public class HQuorumPeer {
} }
} }
private static void runZKServer(QuorumPeerConfig zkConfig) throws UnknownHostException, IOException { private static void runZKServer(QuorumPeerConfig zkConfig)
throws UnknownHostException, IOException {
if (zkConfig.isDistributed()) { if (zkConfig.isDistributed()) {
QuorumPeerMain qp = new QuorumPeerMain(); QuorumPeerMain qp = new QuorumPeerMain();
qp.runFromConfig(zkConfig); qp.runFromConfig(zkConfig);
@ -139,8 +142,8 @@ public class HQuorumPeer {
} }
// Set the max session timeout from the provided client-side timeout // Set the max session timeout from the provided client-side timeout
properties.setProperty("maxSessionTimeout", properties.setProperty("maxSessionTimeout", conf.get(HConstants.ZK_SESSION_TIMEOUT,
conf.get(HConstants.ZK_SESSION_TIMEOUT, Integer.toString(HConstants.DEFAULT_ZK_SESSION_TIMEOUT))); Integer.toString(HConstants.DEFAULT_ZK_SESSION_TIMEOUT)));
if (myId == -1) { if (myId == -1) {
throw new IOException("Could not find my address: " + myAddress + throw new IOException("Could not find my address: " + myAddress +

View File

@ -21,13 +21,14 @@ import java.io.IOException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.zookeeper.KeeperException;
/** /**
* Tracks the load balancer state up in ZK * Tracks the load balancer state up in ZK
@ -57,12 +58,14 @@ public class LoadBalancerTracker extends ZKNodeTracker {
} }
/** /**
* Set the balancer on/off * Set the balancer on/off.
* @param balancerOn *
* @throws KeeperException * @param balancerOn true if the balancher should be on, false otherwise
* @throws KeeperException if a ZooKeeper operation fails
*/ */
public void setBalancerOn(boolean balancerOn) throws KeeperException { public void setBalancerOn(boolean balancerOn) throws KeeperException {
byte [] upData = toByteArray(balancerOn); byte [] upData = toByteArray(balancerOn);
try { try {
ZKUtil.setData(watcher, watcher.znodePaths.balancerZNode, upData); ZKUtil.setData(watcher, watcher.znodePaths.balancerZNode, upData);
} catch(KeeperException.NoNodeException nne) { } catch(KeeperException.NoNodeException nne) {

View File

@ -23,14 +23,15 @@ import java.io.InterruptedIOException;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
/** /**
* Manages the location of the current active Master for the RegionServer. * Manages the location of the current active Master for the RegionServer.
@ -138,12 +139,12 @@ public class MasterAddressTracker extends ZKNodeTracker {
* instance of this tracker in your context. * instance of this tracker in your context.
* @param zkw ZKWatcher to use * @param zkw ZKWatcher to use
* @return ServerName stored in the the master address znode or null if no * @return ServerName stored in the the master address znode or null if no
* znode present. * znode present.
* @throws KeeperException * @throws KeeperException if a ZooKeeper operation fails
* @throws IOException * @throws IOException if the address of the ZooKeeper master cannot be retrieved
*/ */
public static ServerName getMasterAddress(final ZKWatcher zkw) public static ServerName getMasterAddress(final ZKWatcher zkw)
throws KeeperException, IOException { throws KeeperException, IOException {
byte [] data; byte [] data;
try { try {
data = ZKUtil.getData(zkw, zkw.znodePaths.masterAddressZNode); data = ZKUtil.getData(zkw, zkw.znodePaths.masterAddressZNode);
@ -169,13 +170,12 @@ public class MasterAddressTracker extends ZKNodeTracker {
* instance of this tracker in your context. * instance of this tracker in your context.
* @param zkw ZKWatcher to use * @param zkw ZKWatcher to use
* @return master info port in the the master address znode or null if no * @return master info port in the the master address znode or null if no
* znode present. * znode present.
* // TODO can't return null for 'int' return type. non-static verison returns 0 * // TODO can't return null for 'int' return type. non-static verison returns 0
* @throws KeeperException * @throws KeeperException if a ZooKeeper operation fails
* @throws IOException * @throws IOException if the address of the ZooKeeper master cannot be retrieved
*/ */
public static int getMasterInfoPort(final ZKWatcher zkw) throws KeeperException, public static int getMasterInfoPort(final ZKWatcher zkw) throws KeeperException, IOException {
IOException {
byte[] data; byte[] data;
try { try {
data = ZKUtil.getData(zkw, zkw.znodePaths.masterAddressZNode); data = ZKUtil.getData(zkw, zkw.znodePaths.masterAddressZNode);
@ -201,10 +201,10 @@ public class MasterAddressTracker extends ZKNodeTracker {
* path. * path.
* @param zkw The ZKWatcher to use. * @param zkw The ZKWatcher to use.
* @param znode Where to create the znode; could be at the top level or it * @param znode Where to create the znode; could be at the top level or it
* could be under backup masters * could be under backup masters
* @param master ServerName of the current master must not be null. * @param master ServerName of the current master must not be null.
* @return true if node created, false if not; a watch is set in both cases * @return true if node created, false if not; a watch is set in both cases
* @throws KeeperException * @throws KeeperException if a ZooKeeper operation fails
*/ */
public static boolean setMasterAddress(final ZKWatcher zkw, public static boolean setMasterAddress(final ZKWatcher zkw,
final String znode, final ServerName master, int infoPort) final String znode, final ServerName master, int infoPort)
@ -223,7 +223,7 @@ public class MasterAddressTracker extends ZKNodeTracker {
/** /**
* @param sn must not be null * @param sn must not be null
* @return Content of the master znode as a serialized pb with the pb * @return Content of the master znode as a serialized pb with the pb
* magic as prefix. * magic as prefix.
*/ */
static byte[] toByteArray(final ServerName sn, int infoPort) { static byte[] toByteArray(final ServerName sn, int infoPort) {
ZooKeeperProtos.Master.Builder mbuilder = ZooKeeperProtos.Master.newBuilder(); ZooKeeperProtos.Master.Builder mbuilder = ZooKeeperProtos.Master.newBuilder();
@ -240,7 +240,7 @@ public class MasterAddressTracker extends ZKNodeTracker {
/** /**
* @param data zookeeper data. may be null * @param data zookeeper data. may be null
* @return pb object of master, null if no active master * @return pb object of master, null if no active master
* @throws DeserializationException * @throws DeserializationException if the parsing fails
*/ */
public static ZooKeeperProtos.Master parse(byte[] data) throws DeserializationException { public static ZooKeeperProtos.Master parse(byte[] data) throws DeserializationException {
if (data == null) { if (data == null) {

View File

@ -99,9 +99,10 @@ public class MetaTableLocator {
} }
/** /**
* Gets the meta regions and their locations for the given path and replica ID.
* *
* @param zkw * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param replicaId * @param replicaId the ID of the replica
* @return meta table regions and their locations. * @return meta table regions and their locations.
*/ */
public List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZKWatcher zkw, public List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZKWatcher zkw,
@ -114,6 +115,8 @@ public class MetaTableLocator {
} }
/** /**
* Gets the meta regions for the given path with the default replica ID.
*
* @param zkw ZooKeeper watcher to be used * @param zkw ZooKeeper watcher to be used
* @return List of meta regions * @return List of meta regions
*/ */
@ -122,9 +125,10 @@ public class MetaTableLocator {
} }
/** /**
* Gets the meta regions for the given path and replica ID.
* *
* @param zkw * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param replicaId * @param replicaId the ID of the replica
* @return List of meta regions * @return List of meta regions
*/ */
public List<RegionInfo> getMetaRegions(ZKWatcher zkw, int replicaId) { public List<RegionInfo> getMetaRegions(ZKWatcher zkw, int replicaId) {
@ -133,9 +137,11 @@ public class MetaTableLocator {
return getListOfRegionInfos(result); return getListOfRegionInfos(result);
} }
private List<RegionInfo> getListOfRegionInfos( private List<RegionInfo> getListOfRegionInfos(final List<Pair<RegionInfo, ServerName>> pairs) {
final List<Pair<RegionInfo, ServerName>> pairs) { if (pairs == null || pairs.isEmpty()) {
if (pairs == null || pairs.isEmpty()) return Collections.EMPTY_LIST; return Collections.EMPTY_LIST;
}
List<RegionInfo> result = new ArrayList<>(pairs.size()); List<RegionInfo> result = new ArrayList<>(pairs.size());
for (Pair<RegionInfo, ServerName> pair: pairs) { for (Pair<RegionInfo, ServerName> pair: pairs) {
result.add(pair.getFirst()); result.add(pair.getFirst());
@ -159,8 +165,8 @@ public class MetaTableLocator {
/** /**
* Gets the meta region location, if available. Does not block. * Gets the meta region location, if available. Does not block.
* @param zkw * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param replicaId * @param replicaId the ID of the replica
* @return server name * @return server name
*/ */
public ServerName getMetaRegionLocation(final ZKWatcher zkw, int replicaId) { public ServerName getMetaRegionLocation(final ZKWatcher zkw, int replicaId) {
@ -177,12 +183,13 @@ public class MetaTableLocator {
* specified timeout if not immediately available. * specified timeout if not immediately available.
* Given the zookeeper notification could be delayed, we will try to * Given the zookeeper notification could be delayed, we will try to
* get the latest data. * get the latest data.
* @param zkw *
* @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param timeout maximum time to wait, in millis * @param timeout maximum time to wait, in millis
* @return server name for server hosting meta region formatted as per * @return server name for server hosting meta region formatted as per
* {@link ServerName}, or null if none available * {@link ServerName}, or null if none available
* @throws InterruptedException if interrupted while waiting * @throws InterruptedException if interrupted while waiting
* @throws NotAllMetaRegionsOnlineException * @throws NotAllMetaRegionsOnlineException if a meta or root region is not online
*/ */
public ServerName waitMetaRegionLocation(ZKWatcher zkw, long timeout) public ServerName waitMetaRegionLocation(ZKWatcher zkw, long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException { throws InterruptedException, NotAllMetaRegionsOnlineException {
@ -190,17 +197,17 @@ public class MetaTableLocator {
} }
/** /**
* Gets the meta region location, if available, and waits for up to the * Gets the meta region location, if available, and waits for up to the specified timeout if not
* specified timeout if not immediately available. * immediately available. Given the zookeeper notification could be delayed, we will try to
* Given the zookeeper notification could be delayed, we will try to
* get the latest data. * get the latest data.
* @param zkw *
* @param replicaId * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param replicaId the ID of the replica
* @param timeout maximum time to wait, in millis * @param timeout maximum time to wait, in millis
* @return server name for server hosting meta region formatted as per * @return server name for server hosting meta region formatted as per
* {@link ServerName}, or null if none available * {@link ServerName}, or null if none available
* @throws InterruptedException * @throws InterruptedException if waiting for the socket operation fails
* @throws NotAllMetaRegionsOnlineException * @throws NotAllMetaRegionsOnlineException if a meta or root region is not online
*/ */
public ServerName waitMetaRegionLocation(ZKWatcher zkw, int replicaId, long timeout) public ServerName waitMetaRegionLocation(ZKWatcher zkw, int replicaId, long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException { throws InterruptedException, NotAllMetaRegionsOnlineException {
@ -234,7 +241,10 @@ public class MetaTableLocator {
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
while (!stopped) { while (!stopped) {
try { try {
if (waitMetaRegionLocation(zkw, 100) != null) break; if (waitMetaRegionLocation(zkw, 100) != null) {
break;
}
long sleepTime = System.currentTimeMillis() - startTime; long sleepTime = System.currentTimeMillis() - startTime;
// +1 in case sleepTime=0 // +1 in case sleepTime=0
if ((sleepTime + 1) % 10000 == 0) { if ((sleepTime + 1) % 10000 == 0) {
@ -251,33 +261,33 @@ public class MetaTableLocator {
/** /**
* Verify <code>hbase:meta</code> is deployed and accessible. * Verify <code>hbase:meta</code> is deployed and accessible.
* @param hConnection *
* @param zkw * @param hConnection the connection to use
* @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param timeout How long to wait on zk for meta address (passed through to * @param timeout How long to wait on zk for meta address (passed through to
* the internal call to {@link #getMetaServerConnection}. * the internal call to {@link #getMetaServerConnection}.
* @return True if the <code>hbase:meta</code> location is healthy. * @return True if the <code>hbase:meta</code> location is healthy.
* @throws java.io.IOException * @throws IOException if the number of retries for getting the connection is exceeded
* @throws InterruptedException * @throws InterruptedException if waiting for the socket operation fails
*/ */
public boolean verifyMetaRegionLocation(ClusterConnection hConnection, public boolean verifyMetaRegionLocation(ClusterConnection hConnection, ZKWatcher zkw,
ZKWatcher zkw, final long timeout) final long timeout) throws InterruptedException, IOException {
throws InterruptedException, IOException {
return verifyMetaRegionLocation(hConnection, zkw, timeout, RegionInfo.DEFAULT_REPLICA_ID); return verifyMetaRegionLocation(hConnection, zkw, timeout, RegionInfo.DEFAULT_REPLICA_ID);
} }
/** /**
* Verify <code>hbase:meta</code> is deployed and accessible. * Verify <code>hbase:meta</code> is deployed and accessible.
* @param connection *
* @param zkw * @param connection the connection to use
* @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param timeout How long to wait on zk for meta address (passed through to * @param timeout How long to wait on zk for meta address (passed through to
* @param replicaId * @param replicaId the ID of the replica
* @return True if the <code>hbase:meta</code> location is healthy. * @return True if the <code>hbase:meta</code> location is healthy.
* @throws InterruptedException * @throws InterruptedException if waiting for the socket operation fails
* @throws IOException * @throws IOException if the number of retries for getting the connection is exceeded
*/ */
public boolean verifyMetaRegionLocation(ClusterConnection connection, public boolean verifyMetaRegionLocation(ClusterConnection connection, ZKWatcher zkw,
ZKWatcher zkw, final long timeout, int replicaId) final long timeout, int replicaId) throws InterruptedException, IOException {
throws InterruptedException, IOException {
AdminProtos.AdminService.BlockingInterface service = null; AdminProtos.AdminService.BlockingInterface service = null;
try { try {
service = getMetaServerConnection(connection, zkw, timeout, replicaId); service = getMetaServerConnection(connection, zkw, timeout, replicaId);
@ -299,12 +309,10 @@ public class MetaTableLocator {
* Verify we can connect to <code>hostingServer</code> and that its carrying * Verify we can connect to <code>hostingServer</code> and that its carrying
* <code>regionName</code>. * <code>regionName</code>.
* @param hostingServer Interface to the server hosting <code>regionName</code> * @param hostingServer Interface to the server hosting <code>regionName</code>
* @param address The servername that goes with the <code>metaServer</code> * @param address The servername that goes with the <code>metaServer</code> interface.
* Interface. Used logging. * Used logging.
* @param regionName The regionname we are interested in. * @param regionName The regionname we are interested in.
* @return True if we were able to verify the region located at other side of * @return True if we were able to verify the region located at other side of the interface.
* the Interface.
* @throws IOException
*/ */
// TODO: We should be able to get the ServerName from the AdminProtocol // TODO: We should be able to get the ServerName from the AdminProtocol
// rather than have to pass it in. Its made awkward by the fact that the // rather than have to pass it in. Its made awkward by the fact that the
@ -312,8 +320,7 @@ public class MetaTableLocator {
// to be fixed to go to a local method or to a cache before we can do this. // to be fixed to go to a local method or to a cache before we can do this.
private boolean verifyRegionLocation(final ClusterConnection connection, private boolean verifyRegionLocation(final ClusterConnection connection,
AdminService.BlockingInterface hostingServer, final ServerName address, AdminService.BlockingInterface hostingServer, final ServerName address,
final byte [] regionName) final byte [] regionName) {
throws IOException {
if (hostingServer == null) { if (hostingServer == null) {
LOG.info("Passed hostingServer is null"); LOG.info("Passed hostingServer is null");
return false; return false;
@ -347,34 +354,33 @@ public class MetaTableLocator {
} }
/** /**
* Gets a connection to the server hosting meta, as reported by ZooKeeper, * Gets a connection to the server hosting meta, as reported by ZooKeeper, waiting up to the
* waiting up to the specified timeout for availability. * specified timeout for availability.
*
* <p>WARNING: Does not retry. Use an {@link org.apache.hadoop.hbase.client.HTable} instead. * <p>WARNING: Does not retry. Use an {@link org.apache.hadoop.hbase.client.HTable} instead.
* @param connection *
* @param zkw * @param connection the connection to use
* @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param timeout How long to wait on meta location * @param timeout How long to wait on meta location
* @param replicaId * @param replicaId the ID of the replica
* @return connection to server hosting meta * @return connection to server hosting meta
* @throws InterruptedException * @throws InterruptedException if waiting for the socket operation fails
* @throws NotAllMetaRegionsOnlineException if timed out waiting * @throws IOException if the number of retries for getting the connection is exceeded
* @throws IOException
*/ */
private AdminService.BlockingInterface getMetaServerConnection(ClusterConnection connection, private AdminService.BlockingInterface getMetaServerConnection(ClusterConnection connection,
ZKWatcher zkw, long timeout, int replicaId) ZKWatcher zkw, long timeout, int replicaId) throws InterruptedException, IOException {
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
return getCachedConnection(connection, waitMetaRegionLocation(zkw, replicaId, timeout)); return getCachedConnection(connection, waitMetaRegionLocation(zkw, replicaId, timeout));
} }
/** /**
* @param sn ServerName to get a connection against. * @param sn ServerName to get a connection against.
* @return The AdminProtocol we got when we connected to <code>sn</code> * @return The AdminProtocol we got when we connected to <code>sn</code>
* May have come from cache, may not be good, may have been setup by this * May have come from cache, may not be good, may have been setup by this invocation, or
* invocation, or may be null. * may be null.
* @throws IOException * @throws IOException if the number of retries for getting the connection is exceeded
*/ */
private static AdminService.BlockingInterface getCachedConnection(ClusterConnection connection, private static AdminService.BlockingInterface getCachedConnection(ClusterConnection connection,
ServerName sn) ServerName sn) throws IOException {
throws IOException {
if (sn == null) { if (sn == null) {
return null; return null;
} }
@ -430,16 +436,16 @@ public class MetaTableLocator {
} }
/** /**
* Sets the location of <code>hbase:meta</code> in ZooKeeper to the * Sets the location of <code>hbase:meta</code> in ZooKeeper to the specified server address.
* specified server address. * @param zookeeper reference to the {@link ZKWatcher} which also contains configuration and
* @param zookeeper * operation
* @param serverName * @param serverName the name of the server
* @param replicaId * @param replicaId the ID of the replica
* @param state * @param state the state of the region
* @throws KeeperException * @throws KeeperException if a ZooKeeper operation fails
*/ */
public static void setMetaLocation(ZKWatcher zookeeper, public static void setMetaLocation(ZKWatcher zookeeper, ServerName serverName, int replicaId,
ServerName serverName, int replicaId, RegionState.State state) throws KeeperException { RegionState.State state) throws KeeperException {
if (serverName == null) { if (serverName == null) {
LOG.warn("Tried to set null ServerName in hbase:meta; skipping -- ServerName required"); LOG.warn("Tried to set null ServerName in hbase:meta; skipping -- ServerName required");
return; return;
@ -476,13 +482,14 @@ public class MetaTableLocator {
/** /**
* Load the meta region state from the meta server ZNode. * Load the meta region state from the meta server ZNode.
* @param zkw *
* @param replicaId * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param replicaId the ID of the replica
* @return regionstate * @return regionstate
* @throws KeeperException * @throws KeeperException if a ZooKeeper operation fails
*/ */
public static RegionState getMetaRegionState(ZKWatcher zkw, int replicaId) public static RegionState getMetaRegionState(ZKWatcher zkw, int replicaId)
throws KeeperException { throws KeeperException {
RegionState.State state = RegionState.State.OPEN; RegionState.State state = RegionState.State.OPEN;
ServerName serverName = null; ServerName serverName = null;
try { try {
@ -491,8 +498,8 @@ public class MetaTableLocator {
try { try {
int prefixLen = ProtobufUtil.lengthOfPBMagic(); int prefixLen = ProtobufUtil.lengthOfPBMagic();
ZooKeeperProtos.MetaRegionServer rl = ZooKeeperProtos.MetaRegionServer rl =
ZooKeeperProtos.MetaRegionServer.PARSER.parseFrom ZooKeeperProtos.MetaRegionServer.PARSER.parseFrom(data, prefixLen,
(data, prefixLen, data.length - prefixLen); data.length - prefixLen);
if (rl.hasState()) { if (rl.hasState()) {
state = RegionState.State.convert(rl.getState()); state = RegionState.State.convert(rl.getState());
} }
@ -545,24 +552,28 @@ public class MetaTableLocator {
} }
} }
/** /**
* Wait until the primary meta region is available. Get the secondary * Wait until the primary meta region is available. Get the secondary locations as well but don't
* locations as well but don't block for those. * block for those.
* @param zkw *
* @param timeout * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param conf * @param timeout maximum time to wait in millis
* @param conf the {@link Configuration} to use
* @return ServerName or null if we timed out. * @return ServerName or null if we timed out.
* @throws InterruptedException * @throws InterruptedException if waiting for the socket operation fails
*/ */
public List<ServerName> blockUntilAvailable(final ZKWatcher zkw, public List<ServerName> blockUntilAvailable(final ZKWatcher zkw, final long timeout,
final long timeout, Configuration conf) Configuration conf) throws InterruptedException {
throws InterruptedException {
int numReplicasConfigured = 1; int numReplicasConfigured = 1;
List<ServerName> servers = new ArrayList<>(); List<ServerName> servers = new ArrayList<>();
// Make the blocking call first so that we do the wait to know // Make the blocking call first so that we do the wait to know
// the znodes are all in place or timeout. // the znodes are all in place or timeout.
ServerName server = blockUntilAvailable(zkw, timeout); ServerName server = blockUntilAvailable(zkw, timeout);
if (server == null) return null;
if (server == null) {
return null;
}
servers.add(server); servers.add(server);
try { try {
@ -583,27 +594,32 @@ public class MetaTableLocator {
* @param zkw zookeeper connection to use * @param zkw zookeeper connection to use
* @param timeout maximum time to wait, in millis * @param timeout maximum time to wait, in millis
* @return ServerName or null if we timed out. * @return ServerName or null if we timed out.
* @throws InterruptedException * @throws InterruptedException if waiting for the socket operation fails
*/ */
public ServerName blockUntilAvailable(final ZKWatcher zkw, public ServerName blockUntilAvailable(final ZKWatcher zkw, final long timeout)
final long timeout) throws InterruptedException {
throws InterruptedException {
return blockUntilAvailable(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout); return blockUntilAvailable(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
} }
/** /**
* Wait until the meta region is available and is not in transition. * Wait until the meta region is available and is not in transition.
* @param zkw *
* @param replicaId * @param zkw reference to the {@link ZKWatcher} which also contains configuration and constants
* @param timeout * @param replicaId the ID of the replica
* @param timeout maximum time to wait in millis
* @return ServerName or null if we timed out. * @return ServerName or null if we timed out.
* @throws InterruptedException * @throws InterruptedException if waiting for the socket operation fails
*/ */
public ServerName blockUntilAvailable(final ZKWatcher zkw, int replicaId, public ServerName blockUntilAvailable(final ZKWatcher zkw, int replicaId, final long timeout)
final long timeout) throws InterruptedException {
throws InterruptedException { if (timeout < 0) {
if (timeout < 0) throw new IllegalArgumentException(); throw new IllegalArgumentException();
if (zkw == null) throw new IllegalArgumentException(); }
if (zkw == null) {
throw new IllegalArgumentException();
}
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
ServerName sn = null; ServerName sn = null;
while (true) { while (true) {

View File

@ -20,9 +20,9 @@ package org.apache.hadoop.hbase.zookeeper;
import java.io.BufferedReader; import java.io.BufferedReader;
import java.io.File; import java.io.File;
import java.io.InterruptedIOException;
import java.io.IOException; import java.io.IOException;
import java.io.InputStreamReader; import java.io.InputStreamReader;
import java.io.InterruptedIOException;
import java.io.OutputStream; import java.io.OutputStream;
import java.io.Reader; import java.io.Reader;
import java.net.BindException; import java.net.BindException;
@ -34,9 +34,9 @@ import java.util.Random;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.server.NIOServerCnxnFactory; import org.apache.zookeeper.server.NIOServerCnxnFactory;
import org.apache.zookeeper.server.ZooKeeperServer; import org.apache.zookeeper.server.ZooKeeperServer;
import org.apache.zookeeper.server.persistence.FileTxnLog; import org.apache.zookeeper.server.persistence.FileTxnLog;
@ -187,17 +187,18 @@ public class MiniZooKeeperCluster {
} }
/** /**
* @param baseDir * @param baseDir the base directory to use
* @param numZooKeeperServers * @param numZooKeeperServers the number of ZooKeeper servers
* @return ClientPort server bound to, -1 if there was a * @return ClientPort server bound to, -1 if there was a binding problem and we couldn't pick
* binding problem and we couldn't pick another port. * another port.
* @throws IOException * @throws IOException if an operation fails during the startup
* @throws InterruptedException * @throws InterruptedException if the startup fails
*/ */
public int startup(File baseDir, int numZooKeeperServers) throws IOException, public int startup(File baseDir, int numZooKeeperServers) throws IOException,
InterruptedException { InterruptedException {
if (numZooKeeperServers <= 0) if (numZooKeeperServers <= 0) {
return -1; return -1;
}
setupTestEnv(); setupTestEnv();
shutdown(); shutdown();
@ -226,15 +227,18 @@ public class MiniZooKeeperCluster {
ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse); ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse);
// Setting {min,max}SessionTimeout defaults to be the same as in Zookeeper // Setting {min,max}SessionTimeout defaults to be the same as in Zookeeper
server.setMinSessionTimeout(configuration.getInt("hbase.zookeeper.property.minSessionTimeout", -1)); server.setMinSessionTimeout(configuration.getInt(
server.setMaxSessionTimeout(configuration.getInt("hbase.zookeeper.property.maxSessionTimeout", -1)); "hbase.zookeeper.property.minSessionTimeout", -1));
server.setMaxSessionTimeout(configuration.getInt(
"hbase.zookeeper.property.maxSessionTimeout", -1));
NIOServerCnxnFactory standaloneServerFactory; NIOServerCnxnFactory standaloneServerFactory;
while (true) { while (true) {
try { try {
standaloneServerFactory = new NIOServerCnxnFactory(); standaloneServerFactory = new NIOServerCnxnFactory();
standaloneServerFactory.configure( standaloneServerFactory.configure(
new InetSocketAddress(currentClientPort), new InetSocketAddress(currentClientPort),
configuration.getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS, HConstants.DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS)); configuration.getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS,
HConstants.DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS));
} catch (BindException e) { } catch (BindException e) {
LOG.debug("Failed binding ZK Server to client port: " + LOG.debug("Failed binding ZK Server to client port: " +
currentClientPort, e); currentClientPort, e);
@ -290,7 +294,7 @@ public class MiniZooKeeperCluster {
} }
/** /**
* @throws IOException * @throws IOException if waiting for the shutdown of a server fails
*/ */
public void shutdown() throws IOException { public void shutdown() throws IOException {
// shut down all the zk servers // shut down all the zk servers
@ -321,13 +325,12 @@ public class MiniZooKeeperCluster {
} }
} }
/**@return clientPort return clientPort if there is another ZK backup can run /**
* @return clientPort return clientPort if there is another ZK backup can run
* when killing the current active; return -1, if there is no backups. * when killing the current active; return -1, if there is no backups.
* @throws IOException * @throws IOException if waiting for the shutdown of a server fails
* @throws InterruptedException
*/ */
public int killCurrentActiveZooKeeperServer() throws IOException, public int killCurrentActiveZooKeeperServer() throws IOException, InterruptedException {
InterruptedException {
if (!started || activeZKServerIndex < 0) { if (!started || activeZKServerIndex < 0) {
return -1; return -1;
} }
@ -363,14 +366,12 @@ public class MiniZooKeeperCluster {
} }
/** /**
* Kill one back up ZK servers * Kill one back up ZK servers.
* @throws IOException *
* @throws InterruptedException * @throws IOException if waiting for the shutdown of a server fails
*/ */
public void killOneBackupZooKeeperServer() throws IOException, public void killOneBackupZooKeeperServer() throws IOException, InterruptedException {
InterruptedException { if (!started || activeZKServerIndex < 0 || standaloneServerFactoryList.size() <= 1) {
if (!started || activeZKServerIndex < 0 ||
standaloneServerFactoryList.size() <= 1) {
return ; return ;
} }

View File

@ -26,13 +26,13 @@ import java.util.List;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.RetryCounter; import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.RetryCounterFactory; import org.apache.hadoop.hbase.util.RetryCounterFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.htrace.core.TraceScope; import org.apache.htrace.core.TraceScope;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
@ -113,14 +113,19 @@ public class RecoverableZooKeeper {
this.sessionTimeout = sessionTimeout; this.sessionTimeout = sessionTimeout;
this.quorumServers = quorumServers; this.quorumServers = quorumServers;
this.metrics = new ZKMetrics(); this.metrics = new ZKMetrics();
try {checkZk();} catch (Exception x) {/* ignore */}
try {
checkZk();
} catch (Exception x) {
/* ignore */
}
} }
/** /**
* Try to create a ZooKeeper connection. Turns any exception encountered into a * Try to create a ZooKeeper connection. Turns any exception encountered into a
* KeeperException.OperationTimeoutException so it can retried. * KeeperException.OperationTimeoutException so it can retried.
* @return The created ZooKeeper connection object * @return The created ZooKeeper connection object
* @throws KeeperException * @throws KeeperException if a ZooKeeper operation fails
*/ */
protected synchronized ZooKeeper checkZk() throws KeeperException { protected synchronized ZooKeeper checkZk() throws KeeperException {
if (this.zk == null) { if (this.zk == null) {
@ -161,7 +166,8 @@ public class RecoverableZooKeeper {
try { try {
long startTime = EnvironmentEdgeManager.currentTime(); long startTime = EnvironmentEdgeManager.currentTime();
checkZk().delete(path, version); checkZk().delete(path, version);
this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerWriteOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
return; return;
} catch (KeeperException e) { } catch (KeeperException e) {
this.metrics.registerFailedZKCall(); this.metrics.registerFailedZKCall();
@ -205,7 +211,8 @@ public class RecoverableZooKeeper {
try { try {
long startTime = EnvironmentEdgeManager.currentTime(); long startTime = EnvironmentEdgeManager.currentTime();
Stat nodeStat = checkZk().exists(path, watcher); Stat nodeStat = checkZk().exists(path, watcher);
this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerReadOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
return nodeStat; return nodeStat;
} catch (KeeperException e) { } catch (KeeperException e) {
this.metrics.registerFailedZKCall(); this.metrics.registerFailedZKCall();
@ -239,7 +246,8 @@ public class RecoverableZooKeeper {
try { try {
long startTime = EnvironmentEdgeManager.currentTime(); long startTime = EnvironmentEdgeManager.currentTime();
Stat nodeStat = checkZk().exists(path, watch); Stat nodeStat = checkZk().exists(path, watch);
this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerReadOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
return nodeStat; return nodeStat;
} catch (KeeperException e) { } catch (KeeperException e) {
this.metrics.registerFailedZKCall(); this.metrics.registerFailedZKCall();
@ -285,7 +293,8 @@ public class RecoverableZooKeeper {
try { try {
long startTime = EnvironmentEdgeManager.currentTime(); long startTime = EnvironmentEdgeManager.currentTime();
List<String> children = checkZk().getChildren(path, watcher); List<String> children = checkZk().getChildren(path, watcher);
this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerReadOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
return children; return children;
} catch (KeeperException e) { } catch (KeeperException e) {
this.metrics.registerFailedZKCall(); this.metrics.registerFailedZKCall();
@ -320,7 +329,8 @@ public class RecoverableZooKeeper {
try { try {
long startTime = EnvironmentEdgeManager.currentTime(); long startTime = EnvironmentEdgeManager.currentTime();
List<String> children = checkZk().getChildren(path, watch); List<String> children = checkZk().getChildren(path, watch);
this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerReadOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
return children; return children;
} catch (KeeperException e) { } catch (KeeperException e) {
this.metrics.registerFailedZKCall(); this.metrics.registerFailedZKCall();
@ -355,7 +365,8 @@ public class RecoverableZooKeeper {
try { try {
long startTime = EnvironmentEdgeManager.currentTime(); long startTime = EnvironmentEdgeManager.currentTime();
byte[] revData = checkZk().getData(path, watcher, stat); byte[] revData = checkZk().getData(path, watcher, stat);
this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerReadOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
return ZKMetadata.removeMetaData(revData); return ZKMetadata.removeMetaData(revData);
} catch (KeeperException e) { } catch (KeeperException e) {
this.metrics.registerFailedZKCall(); this.metrics.registerFailedZKCall();
@ -390,7 +401,8 @@ public class RecoverableZooKeeper {
try { try {
long startTime = EnvironmentEdgeManager.currentTime(); long startTime = EnvironmentEdgeManager.currentTime();
byte[] revData = checkZk().getData(path, watch, stat); byte[] revData = checkZk().getData(path, watch, stat);
this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerReadOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
return ZKMetadata.removeMetaData(revData); return ZKMetadata.removeMetaData(revData);
} catch (KeeperException e) { } catch (KeeperException e) {
this.metrics.registerFailedZKCall(); this.metrics.registerFailedZKCall();
@ -430,7 +442,8 @@ public class RecoverableZooKeeper {
try { try {
startTime = EnvironmentEdgeManager.currentTime(); startTime = EnvironmentEdgeManager.currentTime();
Stat nodeStat = checkZk().setData(path, newData, version); Stat nodeStat = checkZk().setData(path, newData, version);
this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerWriteOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
return nodeStat; return nodeStat;
} catch (KeeperException e) { } catch (KeeperException e) {
this.metrics.registerFailedZKCall(); this.metrics.registerFailedZKCall();
@ -450,7 +463,8 @@ public class RecoverableZooKeeper {
Stat stat = new Stat(); Stat stat = new Stat();
startTime = EnvironmentEdgeManager.currentTime(); startTime = EnvironmentEdgeManager.currentTime();
byte[] revData = checkZk().getData(path, false, stat); byte[] revData = checkZk().getData(path, false, stat);
this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerReadOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
if(Bytes.compareTo(revData, newData) == 0) { if(Bytes.compareTo(revData, newData) == 0) {
// the bad version is caused by previous successful setData // the bad version is caused by previous successful setData
return stat; return stat;
@ -484,7 +498,8 @@ public class RecoverableZooKeeper {
try { try {
long startTime = EnvironmentEdgeManager.currentTime(); long startTime = EnvironmentEdgeManager.currentTime();
List<ACL> nodeACL = checkZk().getACL(path, stat); List<ACL> nodeACL = checkZk().getACL(path, stat);
this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerReadOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
return nodeACL; return nodeACL;
} catch (KeeperException e) { } catch (KeeperException e) {
this.metrics.registerFailedZKCall(); this.metrics.registerFailedZKCall();
@ -519,7 +534,8 @@ public class RecoverableZooKeeper {
try { try {
long startTime = EnvironmentEdgeManager.currentTime(); long startTime = EnvironmentEdgeManager.currentTime();
Stat nodeStat = checkZk().setACL(path, acls, version); Stat nodeStat = checkZk().setACL(path, acls, version);
this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerWriteOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
return nodeStat; return nodeStat;
} catch (KeeperException e) { } catch (KeeperException e) {
this.metrics.registerFailedZKCall(); this.metrics.registerFailedZKCall();
@ -587,7 +603,8 @@ public class RecoverableZooKeeper {
try { try {
startTime = EnvironmentEdgeManager.currentTime(); startTime = EnvironmentEdgeManager.currentTime();
String nodePath = checkZk().create(path, data, acl, createMode); String nodePath = checkZk().create(path, data, acl, createMode);
this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerWriteOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
return nodePath; return nodePath;
} catch (KeeperException e) { } catch (KeeperException e) {
this.metrics.registerFailedZKCall(); this.metrics.registerFailedZKCall();
@ -599,7 +616,8 @@ public class RecoverableZooKeeper {
// so we read the node and compare. // so we read the node and compare.
startTime = EnvironmentEdgeManager.currentTime(); startTime = EnvironmentEdgeManager.currentTime();
byte[] currentData = checkZk().getData(path, false, null); byte[] currentData = checkZk().getData(path, false, null);
this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerReadOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
if (currentData != null && if (currentData != null &&
Bytes.compareTo(currentData, data) == 0) { Bytes.compareTo(currentData, data) == 0) {
// We successfully created a non-sequential node // We successfully created a non-sequential node
@ -649,7 +667,8 @@ public class RecoverableZooKeeper {
first = false; first = false;
long startTime = EnvironmentEdgeManager.currentTime(); long startTime = EnvironmentEdgeManager.currentTime();
String nodePath = checkZk().create(newPath, data, acl, createMode); String nodePath = checkZk().create(newPath, data, acl, createMode);
this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerWriteOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
return nodePath; return nodePath;
} catch (KeeperException e) { } catch (KeeperException e) {
this.metrics.registerFailedZKCall(); this.metrics.registerFailedZKCall();
@ -674,9 +693,10 @@ public class RecoverableZooKeeper {
* Convert Iterable of {@link org.apache.zookeeper.Op} we got into the ZooKeeper.Op * Convert Iterable of {@link org.apache.zookeeper.Op} we got into the ZooKeeper.Op
* instances to actually pass to multi (need to do this in order to appendMetaData). * instances to actually pass to multi (need to do this in order to appendMetaData).
*/ */
private Iterable<Op> prepareZKMulti(Iterable<Op> ops) private Iterable<Op> prepareZKMulti(Iterable<Op> ops) throws UnsupportedOperationException {
throws UnsupportedOperationException { if(ops == null) {
if(ops == null) return null; return null;
}
List<Op> preparedOps = new LinkedList<>(); List<Op> preparedOps = new LinkedList<>();
for (Op op : ops) { for (Op op : ops) {
@ -689,8 +709,8 @@ public class RecoverableZooKeeper {
preparedOps.add(op); preparedOps.add(op);
} else if (op.getType() == ZooDefs.OpCode.setData) { } else if (op.getType() == ZooDefs.OpCode.setData) {
SetDataRequest setData = (SetDataRequest)op.toRequestRecord(); SetDataRequest setData = (SetDataRequest)op.toRequestRecord();
preparedOps.add(Op.setData(setData.getPath(), ZKMetadata.appendMetaData(id, setData.getData()), preparedOps.add(Op.setData(setData.getPath(),
setData.getVersion())); ZKMetadata.appendMetaData(id, setData.getData()), setData.getVersion()));
} else { } else {
throw new UnsupportedOperationException("Unexpected ZKOp type: " + op.getClass().getName()); throw new UnsupportedOperationException("Unexpected ZKOp type: " + op.getClass().getName());
} }
@ -710,7 +730,8 @@ public class RecoverableZooKeeper {
try { try {
long startTime = EnvironmentEdgeManager.currentTime(); long startTime = EnvironmentEdgeManager.currentTime();
List<OpResult> opResults = checkZk().multi(multiOps); List<OpResult> opResults = checkZk().multi(multiOps);
this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerWriteOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
return opResults; return opResults;
} catch (KeeperException e) { } catch (KeeperException e) {
this.metrics.registerFailedZKCall(); this.metrics.registerFailedZKCall();
@ -741,13 +762,15 @@ public class RecoverableZooKeeper {
String nodePrefix = path.substring(lastSlashIdx+1); String nodePrefix = path.substring(lastSlashIdx+1);
long startTime = EnvironmentEdgeManager.currentTime(); long startTime = EnvironmentEdgeManager.currentTime();
List<String> nodes = checkZk().getChildren(parent, false); List<String> nodes = checkZk().getChildren(parent, false);
this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerReadOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
List<String> matching = filterByPrefix(nodes, nodePrefix); List<String> matching = filterByPrefix(nodes, nodePrefix);
for (String node : matching) { for (String node : matching) {
String nodePath = parent + "/" + node; String nodePath = parent + "/" + node;
startTime = EnvironmentEdgeManager.currentTime(); startTime = EnvironmentEdgeManager.currentTime();
Stat stat = checkZk().exists(nodePath, false); Stat stat = checkZk().exists(nodePath, false);
this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerReadOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
if (stat != null) { if (stat != null) {
return nodePath; return nodePath;
} }
@ -760,7 +783,9 @@ public class RecoverableZooKeeper {
} }
public synchronized void close() throws InterruptedException { public synchronized void close() throws InterruptedException {
if (zk != null) zk.close(); if (zk != null) {
zk.close();
}
} }
public synchronized States getState() { public synchronized States getState() {
@ -778,7 +803,8 @@ public class RecoverableZooKeeper {
public void sync(String path, AsyncCallback.VoidCallback cb, Object ctx) throws KeeperException { public void sync(String path, AsyncCallback.VoidCallback cb, Object ctx) throws KeeperException {
long startTime = EnvironmentEdgeManager.currentTime(); long startTime = EnvironmentEdgeManager.currentTime();
checkZk().sync(path, cb, null); checkZk().sync(path, cb, null);
this.metrics.registerSyncOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); this.metrics.registerSyncOperationLatency(
Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
} }
/** /**

View File

@ -18,16 +18,17 @@
package org.apache.hadoop.hbase.zookeeper; package org.apache.hadoop.hbase.zookeeper;
import java.io.IOException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import java.io.IOException; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos;
/** /**
* Tracks region normalizer state up in ZK * Tracks region normalizer state up in ZK
@ -35,8 +36,7 @@ import java.io.IOException;
public class RegionNormalizerTracker extends ZKNodeTracker { public class RegionNormalizerTracker extends ZKNodeTracker {
private static final Log LOG = LogFactory.getLog(RegionNormalizerTracker.class); private static final Log LOG = LogFactory.getLog(RegionNormalizerTracker.class);
public RegionNormalizerTracker(ZKWatcher watcher, public RegionNormalizerTracker(ZKWatcher watcher, Abortable abortable) {
Abortable abortable) {
super(watcher, watcher.znodePaths.regionNormalizerZNode, abortable); super(watcher, watcher.znodePaths.regionNormalizerZNode, abortable);
} }
@ -59,7 +59,7 @@ public class RegionNormalizerTracker extends ZKNodeTracker {
/** /**
* Set region normalizer on/off * Set region normalizer on/off
* @param normalizerOn whether normalizer should be on or off * @param normalizerOn whether normalizer should be on or off
* @throws KeeperException * @throws KeeperException if a ZooKeeper operation fails
*/ */
public void setNormalizerOn(boolean normalizerOn) throws KeeperException { public void setNormalizerOn(boolean normalizerOn) throws KeeperException {
byte [] upData = toByteArray(normalizerOn); byte [] upData = toByteArray(normalizerOn);

View File

@ -26,9 +26,9 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured; import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.ZooKeeper;

View File

@ -23,8 +23,8 @@ import java.util.UUID;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.ClusterId; import org.apache.hadoop.hbase.ClusterId;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
/** /**
@ -90,7 +90,7 @@ public class ZKClusterId {
* Get the UUID for the provided ZK watcher. Doesn't handle any ZK exceptions * Get the UUID for the provided ZK watcher. Doesn't handle any ZK exceptions
* @param zkw watcher connected to an ensemble * @param zkw watcher connected to an ensemble
* @return the UUID read from zookeeper * @return the UUID read from zookeeper
* @throws KeeperException * @throws KeeperException if a ZooKeeper operation fails
*/ */
public static UUID getUUIDForCluster(ZKWatcher zkw) throws KeeperException { public static UUID getUUIDForCluster(ZKWatcher zkw) throws KeeperException {
String uuid = readClusterIdZNode(zkw); String uuid = readClusterIdZNode(zkw);

View File

@ -23,8 +23,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.Stoppable;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
/** /**

View File

@ -76,22 +76,26 @@ public class ZKMainServer {
} }
/** /**
* @param args * @param args the arguments to check
* @return True if argument strings have a '-server' in them. * @return True if argument strings have a '-server' in them.
*/ */
private static boolean hasServer(final String args[]) { private static boolean hasServer(final String[] args) {
return args.length > 0 && args[0].equals(SERVER_ARG); return args.length > 0 && args[0].equals(SERVER_ARG);
} }
/** /**
* @param args * @param args the arguments to check for command-line arguments
* @return True if command-line arguments were passed. * @return True if command-line arguments were passed.
*/ */
private static boolean hasCommandLineArguments(final String args[]) { private static boolean hasCommandLineArguments(final String[] args) {
if (hasServer(args)) { if (hasServer(args)) {
if (args.length < 2) throw new IllegalStateException("-server param but no value"); if (args.length < 2) {
throw new IllegalStateException("-server param but no value");
}
return args.length > 2; return args.length > 2;
} }
return args.length > 0; return args.length > 0;
} }
@ -99,7 +103,7 @@ public class ZKMainServer {
* Run the tool. * Run the tool.
* @param args Command line arguments. First arg is path to zookeepers file. * @param args Command line arguments. First arg is path to zookeepers file.
*/ */
public static void main(String args[]) throws Exception { public static void main(String[] args) throws Exception {
String [] newArgs = args; String [] newArgs = args;
if (!hasServer(args)) { if (!hasServer(args)) {
// Add the zk ensemble from configuration if none passed on command-line. // Add the zk ensemble from configuration if none passed on command-line.

View File

@ -53,9 +53,10 @@ public abstract class ZKNodeTracker extends ZKListener {
* *
* <p>After construction, use {@link #start} to kick off tracking. * <p>After construction, use {@link #start} to kick off tracking.
* *
* @param watcher * @param watcher reference to the {@link ZKWatcher} which also contains configuration and
* @param node * constants
* @param abortable * @param node path of the node being tracked
* @param abortable used to abort if a fatal error occurs
*/ */
public ZKNodeTracker(ZKWatcher watcher, String node, public ZKNodeTracker(ZKWatcher watcher, String node,
Abortable abortable) { Abortable abortable) {
@ -109,14 +110,17 @@ public abstract class ZKNodeTracker extends ZKListener {
* Gets the data of the node, blocking until the node is available or the * Gets the data of the node, blocking until the node is available or the
* specified timeout has elapsed. * specified timeout has elapsed.
* *
* @param timeout maximum time to wait for the node data to be available, * @param timeout maximum time to wait for the node data to be available, n milliseconds. Pass 0
* n milliseconds. Pass 0 for no timeout. * for no timeout.
* @return data of the node * @return data of the node
* @throws InterruptedException if the waiting thread is interrupted * @throws InterruptedException if the waiting thread is interrupted
*/ */
public synchronized byte [] blockUntilAvailable(long timeout, boolean refresh) public synchronized byte [] blockUntilAvailable(long timeout, boolean refresh)
throws InterruptedException { throws InterruptedException {
if (timeout < 0) throw new IllegalArgumentException(); if (timeout < 0) {
throw new IllegalArgumentException();
}
boolean notimeout = timeout == 0; boolean notimeout = timeout == 0;
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
long remaining = timeout; long remaining = timeout;
@ -137,9 +141,8 @@ public abstract class ZKNodeTracker extends ZKListener {
try { try {
nodeExistsChecked = (ZKUtil.checkExists(watcher, node) != -1); nodeExistsChecked = (ZKUtil.checkExists(watcher, node) != -1);
} catch (KeeperException e) { } catch (KeeperException e) {
LOG.warn( LOG.warn("Got exception while trying to check existence in ZooKeeper" +
"Got exception while trying to check existence in ZooKeeper" + " of the node: " + node + ", retrying if timeout not reached", e);
" of the node: "+node+", retrying if timeout not reached",e );
} }
// It did not exists, and now it does. // It did not exists, and now it does.
@ -188,7 +191,10 @@ public abstract class ZKNodeTracker extends ZKListener {
@Override @Override
public synchronized void nodeCreated(String path) { public synchronized void nodeCreated(String path) {
if (!path.equals(node)) return; if (!path.equals(node)) {
return;
}
try { try {
byte [] data = ZKUtil.getDataAndWatch(watcher, node); byte [] data = ZKUtil.getDataAndWatch(watcher, node);
if (data != null) { if (data != null) {

View File

@ -19,6 +19,9 @@
package org.apache.hadoop.hbase.zookeeper; package org.apache.hadoop.hbase.zookeeper;
import java.util.LinkedList;
import java.util.List;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@ -26,15 +29,15 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import java.util.LinkedList;
import java.util.List;
/** /**
* Tool for reading ZooKeeper servers from HBase XML configuration and producing * Tool for reading ZooKeeper servers from HBase XML configuration and producing
* a line-by-line list for use by bash scripts. * a line-by-line list for use by bash scripts.
*/ */
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS) @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
public class ZKServerTool { public final class ZKServerTool {
private ZKServerTool() {
}
public static ServerName[] readZKNodes(Configuration conf) { public static ServerName[] readZKNodes(Configuration conf) {
List<ServerName> hosts = new LinkedList<>(); List<ServerName> hosts = new LinkedList<>();
String quorum = conf.get(HConstants.ZOOKEEPER_QUORUM, HConstants.LOCALHOST); String quorum = conf.get(HConstants.ZOOKEEPER_QUORUM, HConstants.LOCALHOST);
@ -56,7 +59,7 @@ public class ZKServerTool {
* Run the tool. * Run the tool.
* @param args Command line arguments. * @param args Command line arguments.
*/ */
public static void main(String args[]) { public static void main(String[] args) {
for(ServerName server: readZKNodes(HBaseConfiguration.create())) { for(ServerName server: readZKNodes(HBaseConfiguration.create())) {
// bin/zookeeper.sh relies on the "ZK host" string for grepping which is case sensitive. // bin/zookeeper.sh relies on the "ZK host" string for grepping which is case sensitive.
System.out.println("ZK host: " + server.getHostname()); System.out.println("ZK host: " + server.getHostname());

View File

@ -30,12 +30,16 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
/** /**
* Common methods and attributes used by SplitLogManager and SplitLogWorker running distributed splitting of WAL logs. * Common methods and attributes used by SplitLogManager and SplitLogWorker running distributed
* splitting of WAL logs.
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class ZKSplitLog { public final class ZKSplitLog {
private static final Log LOG = LogFactory.getLog(ZKSplitLog.class); private static final Log LOG = LogFactory.getLog(ZKSplitLog.class);
private ZKSplitLog() {
}
/** /**
* Gets the full path node name for the log file being split. * Gets the full path node name for the log file being split.
* This method will url encode the filename. * This method will url encode the filename.
@ -80,7 +84,9 @@ public class ZKSplitLog {
} }
/** /**
* @param zkw * Checks if the given path represents a rescan node.
*
* @param zkw reference to the {@link ZKWatcher} which also contains configuration and constants
* @param path the absolute path, starts with '/' * @param path the absolute path, starts with '/'
* @return whether the path represents a rescan node * @return whether the path represents a rescan node
*/ */

View File

@ -45,12 +45,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.AuthUtil; import org.apache.hadoop.hbase.AuthUtil;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.security.Superusers;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent; import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
@ -59,6 +55,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.SetData;
import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authentication.util.KerberosUtil; import org.apache.hadoop.security.authentication.util.KerberosUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
@ -77,6 +74,10 @@ import org.apache.zookeeper.proto.DeleteRequest;
import org.apache.zookeeper.proto.SetDataRequest; import org.apache.zookeeper.proto.SetDataRequest;
import org.apache.zookeeper.server.ZooKeeperSaslServer; import org.apache.zookeeper.server.ZooKeeperSaslServer;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
/** /**
* Internal HBase utility class for ZooKeeper. * Internal HBase utility class for ZooKeeper.
* *
@ -87,11 +88,14 @@ import org.apache.zookeeper.server.ZooKeeperSaslServer;
* If ZK is required for the operation, the server will need to be aborted. * If ZK is required for the operation, the server will need to be aborted.
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class ZKUtil { public final class ZKUtil {
private static final Log LOG = LogFactory.getLog(ZKUtil.class); private static final Log LOG = LogFactory.getLog(ZKUtil.class);
private static int zkDumpConnectionTimeOut; private static int zkDumpConnectionTimeOut;
private ZKUtil() {
}
/** /**
* Creates a new connection to ZooKeeper, pulling settings and ensemble config * Creates a new connection to ZooKeeper, pulling settings and ensemble config
* from the specified configuration object using methods from {@link ZKConfig}. * from the specified configuration object using methods from {@link ZKConfig}.
@ -198,13 +202,15 @@ public class ZKUtil {
String userNameKey, String hostname, String userNameKey, String hostname,
String loginContextProperty, String loginContextName) String loginContextProperty, String loginContextName)
throws IOException { throws IOException {
if (!isSecureZooKeeper(conf)) if (!isSecureZooKeeper(conf)) {
return; return;
}
// User has specified a jaas.conf, keep this one as the good one. // User has specified a jaas.conf, keep this one as the good one.
// HBASE_OPTS="-Djava.security.auth.login.config=jaas.conf" // HBASE_OPTS="-Djava.security.auth.login.config=jaas.conf"
if (System.getProperty("java.security.auth.login.config") != null) if (System.getProperty("java.security.auth.login.config") != null) {
return; return;
}
// No keytab specified, no auth // No keytab specified, no auth
String keytabFilename = conf.get(keytabFileKey); String keytabFilename = conf.get(keytabFileKey);
@ -295,7 +301,11 @@ public class ZKUtil {
KEYTAB_KERBEROS_OPTIONS.put("useTicketCache", useTicketCache ? "true" : "false"); KEYTAB_KERBEROS_OPTIONS.put("useTicketCache", useTicketCache ? "true" : "false");
return KEYTAB_KERBEROS_CONF; return KEYTAB_KERBEROS_CONF;
} }
if (baseConfig != null) return baseConfig.getAppConfigurationEntry(appName);
if (baseConfig != null) {
return baseConfig.getAppConfigurationEntry(appName);
}
return(null); return(null);
} }
} }
@ -454,7 +464,7 @@ public class ZKUtil {
* @param zkw zookeeper reference * @param zkw zookeeper reference
* @param znode node to get children of and watch * @param znode node to get children of and watch
* @return list of znode names, null if the node doesn't exist * @return list of znode names, null if the node doesn't exist
* @throws KeeperException * @throws KeeperException if a ZooKeeper operation fails
*/ */
public static List<String> listChildrenAndWatchThem(ZKWatcher zkw, public static List<String> listChildrenAndWatchThem(ZKWatcher zkw,
String znode) throws KeeperException { String znode) throws KeeperException {
@ -544,8 +554,8 @@ public class ZKUtil {
try { try {
return !zkw.getRecoverableZooKeeper().getChildren(znode, null).isEmpty(); return !zkw.getRecoverableZooKeeper().getChildren(znode, null).isEmpty();
} catch(KeeperException.NoNodeException ke) { } catch(KeeperException.NoNodeException ke) {
LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " + LOG.debug(zkw.prefix("Unable to list children of znode " + znode +
"because node does not exist (not an error)")); " because node does not exist (not an error)"));
return false; return false;
} catch (KeeperException e) { } catch (KeeperException e) {
LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e); LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
@ -591,8 +601,8 @@ public class ZKUtil {
/** /**
* Get znode data. Does not set a watcher. * Get znode data. Does not set a watcher.
* @return ZNode data, null if the node does not exist or if there is an *
* error. * @return ZNode data, null if the node does not exist or if there is an error.
*/ */
public static byte [] getData(ZKWatcher zkw, String znode) public static byte [] getData(ZKWatcher zkw, String znode)
throws KeeperException, InterruptedException { throws KeeperException, InterruptedException {
@ -748,17 +758,16 @@ public class ZKUtil {
* Sets no watches under any conditions. * Sets no watches under any conditions.
* *
* @param zkw zk reference * @param zkw zk reference
* @param znode * @param znode the path to the ZNode
* @param data * @param data the data to store in ZooKeeper
* @param expectedVersion * @param expectedVersion the expected version
* @throws KeeperException if unexpected zookeeper exception * @throws KeeperException if unexpected zookeeper exception
* @throws KeeperException.BadVersionException if version mismatch * @throws KeeperException.BadVersionException if version mismatch
* @deprecated Unused * @deprecated Unused
*/ */
@Deprecated @Deprecated
public static void updateExistingNodeData(ZKWatcher zkw, String znode, public static void updateExistingNodeData(ZKWatcher zkw, String znode, byte[] data,
byte [] data, int expectedVersion) int expectedVersion) throws KeeperException {
throws KeeperException {
try { try {
zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion); zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion);
} catch(InterruptedException ie) { } catch(InterruptedException ie) {
@ -808,11 +817,10 @@ public class ZKUtil {
* @param zkw zk reference * @param zkw zk reference
* @param znode path of node * @param znode path of node
* @param data data to set for node * @param data data to set for node
* @throws KeeperException * @throws KeeperException if a ZooKeeper operation fails
*/ */
public static void createSetData(final ZKWatcher zkw, final String znode, public static void createSetData(final ZKWatcher zkw, final String znode, final byte [] data)
final byte [] data) throws KeeperException {
throws KeeperException {
if (checkExists(zkw, znode) == -1) { if (checkExists(zkw, znode) == -1) {
ZKUtil.createWithParents(zkw, znode, data); ZKUtil.createWithParents(zkw, znode, data);
} else { } else {
@ -1025,9 +1033,8 @@ public class ZKUtil {
* @return true name of the newly created znode or null * @return true name of the newly created znode or null
* @throws KeeperException if unexpected zookeeper exception * @throws KeeperException if unexpected zookeeper exception
*/ */
public static String createNodeIfNotExistsNoWatch(ZKWatcher zkw, String znode, public static String createNodeIfNotExistsNoWatch(ZKWatcher zkw, String znode, byte[] data,
byte[] data, CreateMode createMode) throws KeeperException { CreateMode createMode) throws KeeperException {
String createdZNode = null; String createdZNode = null;
try { try {
createdZNode = zkw.getRecoverableZooKeeper().create(znode, data, createdZNode = zkw.getRecoverableZooKeeper().create(znode, data,
@ -1069,7 +1076,8 @@ public class ZKUtil {
throw KeeperException.create(KeeperException.Code.SYSTEMERROR, throw KeeperException.create(KeeperException.Code.SYSTEMERROR,
"ZK.exists returned null (i.e.: znode does not exist) for znode=" + znode); "ZK.exists returned null (i.e.: znode does not exist) for znode=" + znode);
} }
return stat.getVersion();
return stat.getVersion();
} catch (InterruptedException e) { } catch (InterruptedException e) {
zkw.interruptedException(e); zkw.interruptedException(e);
return -1; return -1;
@ -1086,8 +1094,8 @@ public class ZKUtil {
* @param zkw zk reference * @param zkw zk reference
* @param znode path of node to create * @param znode path of node to create
* @param data data of node to create * @param data data of node to create
* @param cb * @param cb the callback to use for the creation
* @param ctx * @param ctx the context to use for the creation
*/ */
public static void asyncCreate(ZKWatcher zkw, public static void asyncCreate(ZKWatcher zkw,
String znode, byte [] data, final AsyncCallback.StringCallback cb, String znode, byte [] data, final AsyncCallback.StringCallback cb,
@ -1232,9 +1240,10 @@ public class ZKUtil {
/** /**
* Deletes the specified node. Fails silent if the node does not exist. * Deletes the specified node. Fails silent if the node does not exist.
* @param zkw *
* @param node * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @throws KeeperException * @param node the node to delete
* @throws KeeperException if a ZooKeeper operation fails
*/ */
public static void deleteNodeFailSilent(ZKWatcher zkw, String node) public static void deleteNodeFailSilent(ZKWatcher zkw, String node)
throws KeeperException { throws KeeperException {
@ -1273,7 +1282,7 @@ public class ZKUtil {
* Sets no watches. Throws all exceptions besides dealing with deletion of * Sets no watches. Throws all exceptions besides dealing with deletion of
* children. * children.
* *
* @throws KeeperException * @throws KeeperException if a ZooKeeper operation fails
*/ */
public static void deleteChildrenRecursively(ZKWatcher zkw, String node) public static void deleteChildrenRecursively(ZKWatcher zkw, String node)
throws KeeperException { throws KeeperException {
@ -1506,7 +1515,7 @@ public class ZKUtil {
* ZKUtilOp representing createAndFailSilent in ZooKeeper * ZKUtilOp representing createAndFailSilent in ZooKeeper
* (attempt to create node, ignore error if already exists) * (attempt to create node, ignore error if already exists)
*/ */
public static class CreateAndFailSilent extends ZKUtilOp { public static final class CreateAndFailSilent extends ZKUtilOp {
private byte [] data; private byte [] data;
private CreateAndFailSilent(String path, byte [] data) { private CreateAndFailSilent(String path, byte [] data) {
@ -1520,8 +1529,12 @@ public class ZKUtil {
@Override @Override
public boolean equals(Object o) { public boolean equals(Object o) {
if (this == o) return true; if (this == o) {
if (!(o instanceof CreateAndFailSilent)) return false; return true;
}
if (!(o instanceof CreateAndFailSilent)) {
return false;
}
CreateAndFailSilent op = (CreateAndFailSilent) o; CreateAndFailSilent op = (CreateAndFailSilent) o;
return getPath().equals(op.getPath()) && Arrays.equals(data, op.data); return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
@ -1538,15 +1551,19 @@ public class ZKUtil {
* ZKUtilOp representing deleteNodeFailSilent in ZooKeeper * ZKUtilOp representing deleteNodeFailSilent in ZooKeeper
* (attempt to delete node, ignore error if node doesn't exist) * (attempt to delete node, ignore error if node doesn't exist)
*/ */
public static class DeleteNodeFailSilent extends ZKUtilOp { public static final class DeleteNodeFailSilent extends ZKUtilOp {
private DeleteNodeFailSilent(String path) { private DeleteNodeFailSilent(String path) {
super(path); super(path);
} }
@Override @Override
public boolean equals(Object o) { public boolean equals(Object o) {
if (this == o) return true; if (this == o) {
if (!(o instanceof DeleteNodeFailSilent)) return false; return true;
}
if (!(o instanceof DeleteNodeFailSilent)) {
return false;
}
return super.equals(o); return super.equals(o);
} }
@ -1560,7 +1577,7 @@ public class ZKUtil {
/** /**
* ZKUtilOp representing setData in ZooKeeper * ZKUtilOp representing setData in ZooKeeper
*/ */
public static class SetData extends ZKUtilOp { public static final class SetData extends ZKUtilOp {
private byte [] data; private byte [] data;
private SetData(String path, byte [] data) { private SetData(String path, byte [] data) {
@ -1574,8 +1591,12 @@ public class ZKUtil {
@Override @Override
public boolean equals(Object o) { public boolean equals(Object o) {
if (this == o) return true; if (this == o) {
if (!(o instanceof SetData)) return false; return true;
}
if (!(o instanceof SetData)) {
return false;
}
SetData op = (SetData) o; SetData op = (SetData) o;
return getPath().equals(op.getPath()) && Arrays.equals(data, op.data); return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
@ -1592,9 +1613,10 @@ public class ZKUtil {
/** /**
* Convert from ZKUtilOp to ZKOp * Convert from ZKUtilOp to ZKOp
*/ */
private static Op toZooKeeperOp(ZKWatcher zkw, ZKUtilOp op) private static Op toZooKeeperOp(ZKWatcher zkw, ZKUtilOp op) throws UnsupportedOperationException {
throws UnsupportedOperationException { if(op == null) {
if(op == null) return null; return null;
}
if (op instanceof CreateAndFailSilent) { if (op instanceof CreateAndFailSilent) {
CreateAndFailSilent cafs = (CreateAndFailSilent)op; CreateAndFailSilent cafs = (CreateAndFailSilent)op;
@ -1629,14 +1651,16 @@ public class ZKUtil {
* In this case, if the previous conditions hold, the commands are run sequentially, which should * In this case, if the previous conditions hold, the commands are run sequentially, which should
* result in the correct final state, but means that the operations will not run atomically. * result in the correct final state, but means that the operations will not run atomically.
* *
* @throws KeeperException * @throws KeeperException if a ZooKeeper operation fails
*/ */
public static void multiOrSequential(ZKWatcher zkw, List<ZKUtilOp> ops, public static void multiOrSequential(ZKWatcher zkw, List<ZKUtilOp> ops,
boolean runSequentialOnMultiFailure) throws KeeperException { boolean runSequentialOnMultiFailure) throws KeeperException {
if (zkw.getConfiguration().get("hbase.zookeeper.useMulti") != null) { if (zkw.getConfiguration().get("hbase.zookeeper.useMulti") != null) {
LOG.warn("hbase.zookeeper.useMulti is deprecated. Default to true always."); LOG.warn("hbase.zookeeper.useMulti is deprecated. Default to true always.");
} }
if (ops == null) return; if (ops == null) {
return;
}
List<Op> zkOps = new LinkedList<>(); List<Op> zkOps = new LinkedList<>();
for (ZKUtilOp op : ops) { for (ZKUtilOp op : ops) {
@ -1646,21 +1670,21 @@ public class ZKUtil {
zkw.getRecoverableZooKeeper().multi(zkOps); zkw.getRecoverableZooKeeper().multi(zkOps);
} catch (KeeperException ke) { } catch (KeeperException ke) {
switch (ke.code()) { switch (ke.code()) {
case NODEEXISTS: case NODEEXISTS:
case NONODE: case NONODE:
case BADVERSION: case BADVERSION:
case NOAUTH: case NOAUTH:
// if we get an exception that could be solved by running sequentially // if we get an exception that could be solved by running sequentially
// (and the client asked us to), then break out and run sequentially // (and the client asked us to), then break out and run sequentially
if (runSequentialOnMultiFailure) { if (runSequentialOnMultiFailure) {
LOG.info("On call to ZK.multi, received exception: " + ke.toString() + "." LOG.info("On call to ZK.multi, received exception: " + ke.toString() + "."
+ " Attempting to run operations sequentially because" + " Attempting to run operations sequentially because"
+ " runSequentialOnMultiFailure is: " + runSequentialOnMultiFailure + "."); + " runSequentialOnMultiFailure is: " + runSequentialOnMultiFailure + ".");
processSequentially(zkw, ops); processSequentially(zkw, ops);
break; break;
} }
default: default:
throw ke; throw ke;
} }
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
zkw.interruptedException(ie); zkw.interruptedException(ie);
@ -1748,14 +1772,19 @@ public class ZKUtil {
/** /**
* Appends replication znodes to the passed StringBuilder. * Appends replication znodes to the passed StringBuilder.
* @param zkw *
* @param sb * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @throws KeeperException * @param sb the {@link StringBuilder} to append to
* @throws KeeperException if a ZooKeeper operation fails
*/ */
private static void getReplicationZnodesDump(ZKWatcher zkw, StringBuilder sb) private static void getReplicationZnodesDump(ZKWatcher zkw, StringBuilder sb)
throws KeeperException { throws KeeperException {
String replicationZnode = zkw.znodePaths.replicationZNode; String replicationZnode = zkw.znodePaths.replicationZNode;
if (ZKUtil.checkExists(zkw, replicationZnode) == -1) return;
if (ZKUtil.checkExists(zkw, replicationZnode) == -1) {
return;
}
// do a ls -r on this znode // do a ls -r on this znode
sb.append("\n").append(replicationZnode).append(": "); sb.append("\n").append(replicationZnode).append(": ");
List<String> children = ZKUtil.listChildrenNoWatch(zkw, replicationZnode); List<String> children = ZKUtil.listChildrenNoWatch(zkw, replicationZnode);
@ -1790,7 +1819,7 @@ public class ZKUtil {
/** /**
* Returns a string with replication znodes and position of the replication log * Returns a string with replication znodes and position of the replication log
* @param zkw * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @return aq string of replication znodes and log positions * @return aq string of replication znodes and log positions
*/ */
public static String getReplicationZnodesDump(ZKWatcher zkw) throws KeeperException { public static String getReplicationZnodesDump(ZKWatcher zkw) throws KeeperException {
@ -1858,13 +1887,16 @@ public class ZKUtil {
} }
} }
private static void appendPeerState(ZKWatcher zkw, String znodeToProcess, private static void appendPeerState(ZKWatcher zkw, String znodeToProcess, StringBuilder sb)
StringBuilder sb) throws KeeperException, InvalidProtocolBufferException { throws KeeperException, InvalidProtocolBufferException {
String peerState = zkw.getConfiguration().get("zookeeper.znode.replication.peers.state", String peerState = zkw.getConfiguration().get("zookeeper.znode.replication.peers.state",
"peer-state"); "peer-state");
int pblen = ProtobufUtil.lengthOfPBMagic(); int pblen = ProtobufUtil.lengthOfPBMagic();
for (String child : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) { for (String child : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
if (!child.equals(peerState)) continue; if (!child.equals(peerState)) {
continue;
}
String peerStateZnode = ZNodePaths.joinZNode(znodeToProcess, child); String peerStateZnode = ZNodePaths.joinZNode(znodeToProcess, child);
sb.append("\n").append(peerStateZnode).append(": "); sb.append("\n").append(peerStateZnode).append(": ");
byte[] peerStateData; byte[] peerStateData;
@ -1929,7 +1961,10 @@ public class ZKUtil {
private static void logRetrievedMsg(final ZKWatcher zkw, private static void logRetrievedMsg(final ZKWatcher zkw,
final String znode, final byte [] data, final boolean watcherSet) { final String znode, final byte [] data, final boolean watcherSet) {
if (!LOG.isTraceEnabled()) return; if (!LOG.isTraceEnabled()) {
return;
}
LOG.trace(zkw.prefix("Retrieved " + ((data == null)? 0: data.length) + LOG.trace(zkw.prefix("Retrieved " + ((data == null)? 0: data.length) +
" byte(s) of data from znode " + znode + " byte(s) of data from znode " + znode +
(watcherSet? " and set watcher; ": "; data=") + (watcherSet? " and set watcher; ": "; data=") +
@ -2008,7 +2043,10 @@ public class ZKUtil {
* @param root name of the root directory in zk to print * @param root name of the root directory in zk to print
*/ */
public static void logZKTree(ZKWatcher zkw, String root) { public static void logZKTree(ZKWatcher zkw, String root) {
if (!LOG.isDebugEnabled()) return; if (!LOG.isDebugEnabled()) {
return;
}
LOG.debug("Current zk system:"); LOG.debug("Current zk system:");
String prefix = "|-"; String prefix = "|-";
LOG.debug(prefix + root); LOG.debug(prefix + root);
@ -2027,7 +2065,11 @@ public class ZKUtil {
protected static void logZKTree(ZKWatcher zkw, String root, String prefix) protected static void logZKTree(ZKWatcher zkw, String root, String prefix)
throws KeeperException { throws KeeperException {
List<String> children = ZKUtil.listChildrenNoWatch(zkw, root); List<String> children = ZKUtil.listChildrenNoWatch(zkw, root);
if (children == null) return;
if (children == null) {
return;
}
for (String child : children) { for (String child : children) {
LOG.debug(prefix + child); LOG.debug(prefix + child);
String node = ZNodePaths.joinZNode(root.equals("/") ? "" : root, child); String node = ZNodePaths.joinZNode(root.equals("/") ? "" : root, child);
@ -2036,7 +2078,7 @@ public class ZKUtil {
} }
/** /**
* @param position * @param position the position to serialize
* @return Serialized protobuf of <code>position</code> with pb magic prefix prepended suitable * @return Serialized protobuf of <code>position</code> with pb magic prefix prepended suitable
* for use as content of an wal position in a replication queue. * for use as content of an wal position in a replication queue.
*/ */
@ -2049,7 +2091,7 @@ public class ZKUtil {
/** /**
* @param bytes - Content of a WAL position znode. * @param bytes - Content of a WAL position znode.
* @return long - The current WAL position. * @return long - The current WAL position.
* @throws DeserializationException * @throws DeserializationException if the WAL position cannot be parsed
*/ */
public static long parseWALPositionFrom(final byte[] bytes) throws DeserializationException { public static long parseWALPositionFrom(final byte[] bytes) throws DeserializationException {
if (bytes == null) { if (bytes == null) {

View File

@ -33,9 +33,9 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.AuthUtil; import org.apache.hadoop.hbase.AuthUtil;
import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.security.Superusers;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher; import org.apache.zookeeper.Watcher;
@ -85,8 +85,6 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
// negotiation to complete // negotiation to complete
public CountDownLatch saslLatch = new CountDownLatch(1); public CountDownLatch saslLatch = new CountDownLatch(1);
private final Configuration conf; private final Configuration conf;
/* A pattern that matches a Kerberos name, borrowed from Hadoop's KerberosName */ /* A pattern that matches a Kerberos name, borrowed from Hadoop's KerberosName */
@ -95,8 +93,8 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
/** /**
* Instantiate a ZooKeeper connection and watcher. * Instantiate a ZooKeeper connection and watcher.
* @param identifier string that is passed to RecoverableZookeeper to be used as * @param identifier string that is passed to RecoverableZookeeper to be used as
* identifier for this instance. Use null for default. * identifier for this instance. Use null for default.
* @throws IOException * @throws IOException if the connection to ZooKeeper fails
* @throws ZooKeeperConnectionException * @throws ZooKeeperConnectionException
*/ */
public ZKWatcher(Configuration conf, String identifier, public ZKWatcher(Configuration conf, String identifier,
@ -106,13 +104,13 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
/** /**
* Instantiate a ZooKeeper connection and watcher. * Instantiate a ZooKeeper connection and watcher.
* @param conf * @param conf the configuration to use
* @param identifier string that is passed to RecoverableZookeeper to be used as identifier for * @param identifier string that is passed to RecoverableZookeeper to be used as identifier for
* this instance. Use null for default. * this instance. Use null for default.
* @param abortable Can be null if there is on error there is no host to abort: e.g. client * @param abortable Can be null if there is on error there is no host to abort: e.g. client
* context. * context.
* @param canCreateBaseZNode * @param canCreateBaseZNode true if a base ZNode can be created
* @throws IOException * @throws IOException if the connection to ZooKeeper fails
* @throws ZooKeeperConnectionException * @throws ZooKeeperConnectionException
*/ */
public ZKWatcher(Configuration conf, String identifier, public ZKWatcher(Configuration conf, String identifier,
@ -211,7 +209,7 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
/** /**
* Set the znode perms recursively. This will do post-order recursion, so that baseZnode ACLs * Set the znode perms recursively. This will do post-order recursion, so that baseZnode ACLs
* will be set last in case the master fails in between. * will be set last in case the master fails in between.
* @param znode * @param znode the ZNode to set the permissions for
*/ */
private void setZnodeAclsRecursive(String znode) throws KeeperException, InterruptedException { private void setZnodeAclsRecursive(String znode) throws KeeperException, InterruptedException {
List<String> children = recoverableZooKeeper.getChildren(znode, false); List<String> children = recoverableZooKeeper.getChildren(znode, false);
@ -228,7 +226,7 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
* Checks whether the ACLs returned from the base znode (/hbase) is set for secure setup. * Checks whether the ACLs returned from the base znode (/hbase) is set for secure setup.
* @param acls acls from zookeeper * @param acls acls from zookeeper
* @return whether ACLs are set for the base znode * @return whether ACLs are set for the base znode
* @throws IOException * @throws IOException if getting the current user fails
*/ */
private boolean isBaseZnodeAclSetup(List<ACL> acls) throws IOException { private boolean isBaseZnodeAclSetup(List<ACL> acls) throws IOException {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
@ -355,7 +353,7 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
* Adds this instance's identifier as a prefix to the passed <code>str</code> * Adds this instance's identifier as a prefix to the passed <code>str</code>
* @param str String to amend. * @param str String to amend.
* @return A new string with this instance's identifier as prefix: e.g. * @return A new string with this instance's identifier as prefix: e.g.
* if passed 'hello world', the returned string could be * if passed 'hello world', the returned string could be
*/ */
public String prefix(final String str) { public String prefix(final String str) {
return this.toString() + " " + str; return this.toString() + " " + str;
@ -364,7 +362,7 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
/** /**
* Get the znodes corresponding to the meta replicas from ZK * Get the znodes corresponding to the meta replicas from ZK
* @return list of znodes * @return list of znodes
* @throws KeeperException * @throws KeeperException if a ZooKeeper operation fails
*/ */
public List<String> getMetaReplicaNodes() throws KeeperException { public List<String> getMetaReplicaNodes() throws KeeperException {
List<String> childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, znodePaths.baseZNode); List<String> childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, znodePaths.baseZNode);
@ -372,7 +370,9 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
if (childrenOfBaseNode != null) { if (childrenOfBaseNode != null) {
String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server"); String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server");
for (String child : childrenOfBaseNode) { for (String child : childrenOfBaseNode) {
if (child.startsWith(pattern)) metaReplicaNodes.add(child); if (child.startsWith(pattern)) {
metaReplicaNodes.add(child);
}
} }
} }
return metaReplicaNodes; return metaReplicaNodes;
@ -380,7 +380,7 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
/** /**
* Register the specified listener to receive ZooKeeper events. * Register the specified listener to receive ZooKeeper events.
* @param listener * @param listener the listener to register
*/ */
public void registerListener(ZKListener listener) { public void registerListener(ZKListener listener) {
listeners.add(listener); listeners.add(listener);
@ -389,7 +389,7 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
/** /**
* Register the specified listener to receive ZooKeeper events and add it as * Register the specified listener to receive ZooKeeper events and add it as
* the first in the list of current listeners. * the first in the list of current listeners.
* @param listener * @param listener the listener to register
*/ */
public void registerListenerFirst(ZKListener listener) { public void registerListenerFirst(ZKListener listener) {
listeners.add(0, listener); listeners.add(0, listener);
@ -512,7 +512,7 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
* call, it's possible for the Abortable to catch it and try to create a new * call, it's possible for the Abortable to catch it and try to create a new
* session with ZooKeeper. This is what the client does in HCM. * session with ZooKeeper. This is what the client does in HCM.
* <p> * <p>
* @param event * @param event the connection-related event
*/ */
private void connectionEvent(WatchedEvent event) { private void connectionEvent(WatchedEvent event) {
switch(event.getState()) { switch(event.getState()) {
@ -571,11 +571,10 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
* <p> * <p>
* TODO: Currently this method rethrows the exception to let the caller handle * TODO: Currently this method rethrows the exception to let the caller handle
* <p> * <p>
* @param ke * @param ke the exception to rethrow
* @throws KeeperException * @throws KeeperException if a ZooKeeper operation fails
*/ */
public void keeperException(KeeperException ke) public void keeperException(KeeperException ke) throws KeeperException {
throws KeeperException {
LOG.error(prefix("Received unexpected KeeperException, re-throwing exception"), ke); LOG.error(prefix("Received unexpected KeeperException, re-throwing exception"), ke);
throw ke; throw ke;
} }
@ -623,8 +622,11 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
@Override @Override
public void abort(String why, Throwable e) { public void abort(String why, Throwable e) {
if (this.abortable != null) this.abortable.abort(why, e); if (this.abortable != null) {
else this.aborted = true; this.abortable.abort(why, e);
} else {
this.aborted = true;
}
} }
@Override @Override