You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2017/11/17 21:20:33 UTC
[01/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper
from hbase-server and hbase-client
Repository: hbase
Updated Branches:
refs/heads/master ca74ec774 -> 330b0d05b
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
new file mode 100644
index 0000000..d0b0081
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
@@ -0,0 +1,634 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CountDownLatch;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.security.Superusers;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooDefs.Perms;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * Acts as the single ZooKeeper Watcher. One instance of this is instantiated
+ * for each Master, RegionServer, and client process.
+ *
+ * <p>This is the only class that implements {@link Watcher}. Other internal
+ * classes which need to be notified of ZooKeeper events must register with
+ * the local instance of this watcher via {@link #registerListener}.
+ *
+ * <p>This class also holds and manages the connection to ZooKeeper. Code to
+ * deal with connection related events and exceptions are handled here.
+ */
+@InterfaceAudience.Private
+public class ZKWatcher implements Watcher, Abortable, Closeable {
+ private static final Log LOG = LogFactory.getLog(ZKWatcher.class);
+
+ // Identifier for this watcher (for logging only). It is made of the prefix
+ // passed on construction and the zookeeper sessionid.
+ private String prefix;
+ private String identifier;
+
+ // zookeeper quorum
+ private String quorum;
+
+ // zookeeper connection
+ private final RecoverableZooKeeper recoverableZooKeeper;
+
+ // abortable in case of zk failure
+ protected Abortable abortable;
+ // Used if abortable is null
+ private boolean aborted = false;
+
+ public final ZNodePaths znodePaths;
+
+ // listeners to be notified
+ private final List<ZKListener> listeners = new CopyOnWriteArrayList<>();
+
+ // Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL
+ // negotiation to complete
+ public CountDownLatch saslLatch = new CountDownLatch(1);
+
+
+
+ private final Configuration conf;
+
+ /* A pattern that matches a Kerberos name, borrowed from Hadoop's KerberosName */
+ private static final Pattern NAME_PATTERN = Pattern.compile("([^/@]*)(/([^/@]*))?@([^/@]*)");
+
+ /**
+ * Instantiate a ZooKeeper connection and watcher.
+ * @param identifier string that is passed to RecoverableZookeeper to be used as
+ * identifier for this instance. Use null for default.
+ * @throws IOException
+ * @throws ZooKeeperConnectionException
+ */
+ public ZKWatcher(Configuration conf, String identifier,
+ Abortable abortable) throws ZooKeeperConnectionException, IOException {
+ this(conf, identifier, abortable, false);
+ }
+
+ /**
+ * Instantiate a ZooKeeper connection and watcher.
+ * @param conf
+ * @param identifier string that is passed to RecoverableZookeeper to be used as identifier for
+ * 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
+ * context.
+ * @param canCreateBaseZNode
+ * @throws IOException
+ * @throws ZooKeeperConnectionException
+ */
+ public ZKWatcher(Configuration conf, String identifier,
+ Abortable abortable, boolean canCreateBaseZNode)
+ throws IOException, ZooKeeperConnectionException {
+ this.conf = conf;
+ this.quorum = ZKConfig.getZKQuorumServersString(conf);
+ this.prefix = identifier;
+ // Identifier will get the sessionid appended later below down when we
+ // handle the syncconnect event.
+ this.identifier = identifier + "0x0";
+ this.abortable = abortable;
+ this.znodePaths = new ZNodePaths(conf);
+ PendingWatcher pendingWatcher = new PendingWatcher();
+ this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, pendingWatcher, identifier);
+ pendingWatcher.prepare(this);
+ if (canCreateBaseZNode) {
+ try {
+ createBaseZNodes();
+ } catch (ZooKeeperConnectionException zce) {
+ try {
+ this.recoverableZooKeeper.close();
+ } catch (InterruptedException ie) {
+ LOG.debug("Encountered InterruptedException when closing " + this.recoverableZooKeeper);
+ Thread.currentThread().interrupt();
+ }
+ throw zce;
+ }
+ }
+ }
+
+ private void createBaseZNodes() throws ZooKeeperConnectionException {
+ try {
+ // Create all the necessary "directories" of znodes
+ ZKUtil.createWithParents(this, znodePaths.baseZNode);
+ ZKUtil.createAndFailSilent(this, znodePaths.rsZNode);
+ ZKUtil.createAndFailSilent(this, znodePaths.drainingZNode);
+ ZKUtil.createAndFailSilent(this, znodePaths.tableZNode);
+ ZKUtil.createAndFailSilent(this, znodePaths.splitLogZNode);
+ ZKUtil.createAndFailSilent(this, znodePaths.backupMasterAddressesZNode);
+ ZKUtil.createAndFailSilent(this, znodePaths.tableLockZNode);
+ ZKUtil.createAndFailSilent(this, znodePaths.masterMaintZNode);
+ } catch (KeeperException e) {
+ throw new ZooKeeperConnectionException(
+ prefix("Unexpected KeeperException creating base node"), e);
+ }
+ }
+
+ /** Returns whether the znode is supposed to be readable by the client
+ * and DOES NOT contain sensitive information (world readable).*/
+ public boolean isClientReadable(String node) {
+ // Developer notice: These znodes are world readable. DO NOT add more znodes here UNLESS
+ // all clients need to access this data to work. Using zk for sharing data to clients (other
+ // than service lookup case is not a recommended design pattern.
+ return
+ node.equals(znodePaths.baseZNode) ||
+ znodePaths.isAnyMetaReplicaZNode(node) ||
+ node.equals(znodePaths.masterAddressZNode) ||
+ node.equals(znodePaths.clusterIdZNode)||
+ node.equals(znodePaths.rsZNode) ||
+ // /hbase/table and /hbase/table/foo is allowed, /hbase/table-lock is not
+ node.equals(znodePaths.tableZNode) ||
+ node.startsWith(znodePaths.tableZNode + "/");
+ }
+
+ /**
+ * On master start, we check the znode ACLs under the root directory and set the ACLs properly
+ * if needed. If the cluster goes from an unsecure setup to a secure setup, this step is needed
+ * so that the existing znodes created with open permissions are now changed with restrictive
+ * perms.
+ */
+ public void checkAndSetZNodeAcls() {
+ if (!ZKUtil.isSecureZooKeeper(getConfiguration())) {
+ LOG.info("not a secure deployment, proceeding");
+ return;
+ }
+
+ // Check the base znodes permission first. Only do the recursion if base znode's perms are not
+ // correct.
+ try {
+ List<ACL> actualAcls = recoverableZooKeeper.getAcl(znodePaths.baseZNode, new Stat());
+
+ if (!isBaseZnodeAclSetup(actualAcls)) {
+ LOG.info("setting znode ACLs");
+ setZnodeAclsRecursive(znodePaths.baseZNode);
+ }
+ } catch(KeeperException.NoNodeException nne) {
+ return;
+ } catch(InterruptedException ie) {
+ interruptedExceptionNoThrow(ie, false);
+ } catch (IOException|KeeperException e) {
+ LOG.warn("Received exception while checking and setting zookeeper ACLs", e);
+ }
+ }
+
+ /**
+ * 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.
+ * @param znode
+ */
+ private void setZnodeAclsRecursive(String znode) throws KeeperException, InterruptedException {
+ List<String> children = recoverableZooKeeper.getChildren(znode, false);
+
+ for (String child : children) {
+ setZnodeAclsRecursive(ZNodePaths.joinZNode(znode, child));
+ }
+ List<ACL> acls = ZKUtil.createACL(this, znode, true);
+ LOG.info("Setting ACLs for znode:" + znode + " , acl:" + acls);
+ recoverableZooKeeper.setAcl(znode, acls, -1);
+ }
+
+ /**
+ * Checks whether the ACLs returned from the base znode (/hbase) is set for secure setup.
+ * @param acls acls from zookeeper
+ * @return whether ACLs are set for the base znode
+ * @throws IOException
+ */
+ private boolean isBaseZnodeAclSetup(List<ACL> acls) throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Checking znode ACLs");
+ }
+ String[] superUsers = conf.getStrings(Superusers.SUPERUSER_CONF_KEY);
+ // Check whether ACL set for all superusers
+ if (superUsers != null && !checkACLForSuperUsers(superUsers, acls)) {
+ return false;
+ }
+
+ // this assumes that current authenticated user is the same as zookeeper client user
+ // configured via JAAS
+ String hbaseUser = UserGroupInformation.getCurrentUser().getShortUserName();
+
+ if (acls.isEmpty()) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("ACL is empty");
+ }
+ return false;
+ }
+
+ for (ACL acl : acls) {
+ int perms = acl.getPerms();
+ Id id = acl.getId();
+ // We should only set at most 3 possible ACLs for 3 Ids. One for everyone, one for superuser
+ // and one for the hbase user
+ if (Ids.ANYONE_ID_UNSAFE.equals(id)) {
+ if (perms != Perms.READ) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
+ id, perms, Perms.READ));
+ }
+ return false;
+ }
+ } else if (superUsers != null && isSuperUserId(superUsers, id)) {
+ if (perms != Perms.ALL) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
+ id, perms, Perms.ALL));
+ }
+ return false;
+ }
+ } else if ("sasl".equals(id.getScheme())) {
+ String name = id.getId();
+ // If ZooKeeper recorded the Kerberos full name in the ACL, use only the shortname
+ Matcher match = NAME_PATTERN.matcher(name);
+ if (match.matches()) {
+ name = match.group(1);
+ }
+ if (name.equals(hbaseUser)) {
+ if (perms != Perms.ALL) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
+ id, perms, Perms.ALL));
+ }
+ return false;
+ }
+ } else {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Unexpected shortname in SASL ACL: " + id);
+ }
+ return false;
+ }
+ } else {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("unexpected ACL id '" + id + "'");
+ }
+ return false;
+ }
+ }
+ return true;
+ }
+
+ /*
+ * Validate whether ACL set for all superusers.
+ */
+ private boolean checkACLForSuperUsers(String[] superUsers, List<ACL> acls) {
+ for (String user : superUsers) {
+ boolean hasAccess = false;
+ // TODO: Validate super group members also when ZK supports setting node ACL for groups.
+ if (!AuthUtil.isGroupPrincipal(user)) {
+ for (ACL acl : acls) {
+ if (user.equals(acl.getId().getId())) {
+ if (acl.getPerms() == Perms.ALL) {
+ hasAccess = true;
+ } else {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(String.format(
+ "superuser '%s' does not have correct permissions: have 0x%x, want 0x%x",
+ acl.getId().getId(), acl.getPerms(), Perms.ALL));
+ }
+ }
+ break;
+ }
+ }
+ if (!hasAccess) {
+ return false;
+ }
+ }
+ }
+ return true;
+ }
+
+ /*
+ * Validate whether ACL ID is superuser.
+ */
+ public static boolean isSuperUserId(String[] superUsers, Id id) {
+ for (String user : superUsers) {
+ // TODO: Validate super group members also when ZK supports setting node ACL for groups.
+ if (!AuthUtil.isGroupPrincipal(user) && new Id("sasl", user).equals(id)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return this.identifier + ", quorum=" + quorum + ", baseZNode=" + znodePaths.baseZNode;
+ }
+
+ /**
+ * Adds this instance's identifier as a prefix to the passed <code>str</code>
+ * @param str String to amend.
+ * @return A new string with this instance's identifier as prefix: e.g.
+ * if passed 'hello world', the returned string could be
+ */
+ public String prefix(final String str) {
+ return this.toString() + " " + str;
+ }
+
+ /**
+ * Get the znodes corresponding to the meta replicas from ZK
+ * @return list of znodes
+ * @throws KeeperException
+ */
+ public List<String> getMetaReplicaNodes() throws KeeperException {
+ List<String> childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, znodePaths.baseZNode);
+ List<String> metaReplicaNodes = new ArrayList<>(2);
+ if (childrenOfBaseNode != null) {
+ String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server");
+ for (String child : childrenOfBaseNode) {
+ if (child.startsWith(pattern)) metaReplicaNodes.add(child);
+ }
+ }
+ return metaReplicaNodes;
+ }
+
+ /**
+ * Register the specified listener to receive ZooKeeper events.
+ * @param listener
+ */
+ public void registerListener(ZKListener listener) {
+ listeners.add(listener);
+ }
+
+ /**
+ * Register the specified listener to receive ZooKeeper events and add it as
+ * the first in the list of current listeners.
+ * @param listener
+ */
+ public void registerListenerFirst(ZKListener listener) {
+ listeners.add(0, listener);
+ }
+
+ public void unregisterListener(ZKListener listener) {
+ listeners.remove(listener);
+ }
+
+ /**
+ * Clean all existing listeners
+ */
+ public void unregisterAllListeners() {
+ listeners.clear();
+ }
+
+ /**
+ * Get a copy of current registered listeners
+ */
+ public List<ZKListener> getListeners() {
+ return new ArrayList<>(listeners);
+ }
+
+ /**
+ * @return The number of currently registered listeners
+ */
+ public int getNumberOfListeners() {
+ return listeners.size();
+ }
+
+ /**
+ * Get the connection to ZooKeeper.
+ * @return connection reference to zookeeper
+ */
+ public RecoverableZooKeeper getRecoverableZooKeeper() {
+ return recoverableZooKeeper;
+ }
+
+ public void reconnectAfterExpiration() throws IOException, KeeperException, InterruptedException {
+ recoverableZooKeeper.reconnectAfterExpiration();
+ }
+
+ /**
+ * Get the quorum address of this instance.
+ * @return quorum string of this zookeeper connection instance
+ */
+ public String getQuorum() {
+ return quorum;
+ }
+
+ /**
+ * Get the znodePaths.
+ * <p>
+ * Mainly used for mocking as mockito can not mock a field access.
+ */
+ public ZNodePaths getZNodePaths() {
+ return znodePaths;
+ }
+
+ /**
+ * Method called from ZooKeeper for events and connection status.
+ * <p>
+ * Valid events are passed along to listeners. Connection status changes
+ * are dealt with locally.
+ */
+ @Override
+ public void process(WatchedEvent event) {
+ LOG.debug(prefix("Received ZooKeeper Event, " +
+ "type=" + event.getType() + ", " +
+ "state=" + event.getState() + ", " +
+ "path=" + event.getPath()));
+
+ switch(event.getType()) {
+
+ // If event type is NONE, this is a connection status change
+ case None: {
+ connectionEvent(event);
+ break;
+ }
+
+ // Otherwise pass along to the listeners
+
+ case NodeCreated: {
+ for(ZKListener listener : listeners) {
+ listener.nodeCreated(event.getPath());
+ }
+ break;
+ }
+
+ case NodeDeleted: {
+ for(ZKListener listener : listeners) {
+ listener.nodeDeleted(event.getPath());
+ }
+ break;
+ }
+
+ case NodeDataChanged: {
+ for(ZKListener listener : listeners) {
+ listener.nodeDataChanged(event.getPath());
+ }
+ break;
+ }
+
+ case NodeChildrenChanged: {
+ for(ZKListener listener : listeners) {
+ listener.nodeChildrenChanged(event.getPath());
+ }
+ break;
+ }
+ }
+ }
+
+ // Connection management
+
+ /**
+ * Called when there is a connection-related event via the Watcher callback.
+ * <p>
+ * If Disconnected or Expired, this should shutdown the cluster. But, since
+ * we send a KeeperException.SessionExpiredException along with the abort
+ * 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.
+ * <p>
+ * @param event
+ */
+ private void connectionEvent(WatchedEvent event) {
+ switch(event.getState()) {
+ case SyncConnected:
+ this.identifier = this.prefix + "-0x" +
+ Long.toHexString(this.recoverableZooKeeper.getSessionId());
+ // Update our identifier. Otherwise ignore.
+ LOG.debug(this.identifier + " connected");
+ break;
+
+ // Abort the server if Disconnected or Expired
+ case Disconnected:
+ LOG.debug(prefix("Received Disconnected from ZooKeeper, ignoring"));
+ break;
+
+ case Expired:
+ String msg = prefix(this.identifier + " received expired from " +
+ "ZooKeeper, aborting");
+ // TODO: One thought is to add call to ZKListener so say,
+ // ZKNodeTracker can zero out its data values.
+ if (this.abortable != null) {
+ this.abortable.abort(msg, new KeeperException.SessionExpiredException());
+ }
+ break;
+
+ case ConnectedReadOnly:
+ case SaslAuthenticated:
+ case AuthFailed:
+ break;
+
+ default:
+ throw new IllegalStateException("Received event is not valid: " + event.getState());
+ }
+ }
+
+ /**
+ * Forces a synchronization of this ZooKeeper client connection.
+ * <p>
+ * Executing this method before running other methods will ensure that the
+ * subsequent operations are up-to-date and consistent as of the time that
+ * the sync is complete.
+ * <p>
+ * This is used for compareAndSwap type operations where we need to read the
+ * data of an existing node and delete or transition that node, utilizing the
+ * previously read version and data. We want to ensure that the version read
+ * is up-to-date from when we begin the operation.
+ */
+ public void sync(String path) throws KeeperException {
+ this.recoverableZooKeeper.sync(path, null, null);
+ }
+
+ /**
+ * Handles KeeperExceptions in client calls.
+ * <p>
+ * This may be temporary but for now this gives one place to deal with these.
+ * <p>
+ * TODO: Currently this method rethrows the exception to let the caller handle
+ * <p>
+ * @param ke
+ * @throws KeeperException
+ */
+ public void keeperException(KeeperException ke)
+ throws KeeperException {
+ LOG.error(prefix("Received unexpected KeeperException, re-throwing exception"), ke);
+ throw ke;
+ }
+
+ /**
+ * Handles InterruptedExceptions in client calls.
+ * @param ie the InterruptedException instance thrown
+ * @throws KeeperException the exception to throw, transformed from the InterruptedException
+ */
+ public void interruptedException(InterruptedException ie) throws KeeperException {
+ interruptedExceptionNoThrow(ie, true);
+ // Throw a system error exception to let upper level handle it
+ throw new KeeperException.SystemErrorException();
+ }
+
+ /**
+ * Log the InterruptedException and interrupt current thread
+ * @param ie The IterruptedException to log
+ * @param throwLater Whether we will throw the exception latter
+ */
+ public void interruptedExceptionNoThrow(InterruptedException ie, boolean throwLater) {
+ LOG.debug(prefix("Received InterruptedException, will interrupt current thread"
+ + (throwLater ? " and rethrow a SystemErrorException" : "")),
+ ie);
+ // At least preserve interrupt.
+ Thread.currentThread().interrupt();
+ }
+
+ /**
+ * Close the connection to ZooKeeper.
+ *
+ */
+ @Override
+ public void close() {
+ try {
+ recoverableZooKeeper.close();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+
+ public Configuration getConfiguration() {
+ return conf;
+ }
+
+ @Override
+ public void abort(String why, Throwable e) {
+ if (this.abortable != null) this.abortable.abort(why, e);
+ else this.aborted = true;
+ }
+
+ @Override
+ public boolean isAborted() {
+ return this.abortable == null? this.aborted: this.abortable.isAborted();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java
new file mode 100644
index 0000000..e67c9fd
--- /dev/null
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestInstancePending {
+ @Test(timeout = 1000)
+ public void test() throws Exception {
+ final InstancePending<String> pending = new InstancePending<>();
+ final AtomicReference<String> getResultRef = new AtomicReference<>();
+
+ new Thread() {
+ @Override
+ public void run() {
+ getResultRef.set(pending.get());
+ }
+ }.start();
+
+ Thread.sleep(100);
+ Assert.assertNull(getResultRef.get());
+
+ pending.prepare("abc");
+ Thread.sleep(100);
+ Assert.assertEquals("abc", getResultRef.get());
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMetrics.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMetrics.java
new file mode 100644
index 0000000..2811cc5
--- /dev/null
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMetrics.java
@@ -0,0 +1,80 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestZKMetrics {
+
+ @Test
+ public void testRegisterExceptions() {
+ MetricsZooKeeperSource zkSource = mock(MetricsZooKeeperSourceImpl.class);
+ ZKMetrics metricsZK = new ZKMetrics(zkSource);
+ metricsZK.registerAuthFailedException();
+ metricsZK.registerConnectionLossException();
+ metricsZK.registerConnectionLossException();
+ metricsZK.registerDataInconsistencyException();
+ metricsZK.registerInvalidACLException();
+ metricsZK.registerNoAuthException();
+ metricsZK.registerOperationTimeoutException();
+ metricsZK.registerOperationTimeoutException();
+ metricsZK.registerRuntimeInconsistencyException();
+ metricsZK.registerSessionExpiredException();
+ metricsZK.registerSystemErrorException();
+ metricsZK.registerSystemErrorException();
+ metricsZK.registerFailedZKCall();
+
+ verify(zkSource, times(1)).incrementAuthFailedCount();
+ // ConnectionLoss Exception was registered twice.
+ verify(zkSource, times(2)).incrementConnectionLossCount();
+ verify(zkSource, times(1)).incrementDataInconsistencyCount();
+ verify(zkSource, times(1)).incrementInvalidACLCount();
+ verify(zkSource, times(1)).incrementNoAuthCount();
+ // OperationTimeout Exception was registered twice.
+ verify(zkSource, times(2)).incrementOperationTimeoutCount();
+ verify(zkSource, times(1)).incrementRuntimeInconsistencyCount();
+ verify(zkSource, times(1)).incrementSessionExpiredCount();
+ // SystemError Exception was registered twice.
+ verify(zkSource, times(2)).incrementSystemErrorCount();
+ verify(zkSource, times(1)).incrementTotalFailedZKCalls();
+ }
+
+ @Test
+ public void testLatencyHistogramUpdates() {
+ MetricsZooKeeperSource zkSource = mock(MetricsZooKeeperSourceImpl.class);
+ ZKMetrics metricsZK = new ZKMetrics(zkSource);
+ long latency = 100;
+
+ metricsZK.registerReadOperationLatency(latency);
+ metricsZK.registerReadOperationLatency(latency);
+ metricsZK.registerWriteOperationLatency(latency);
+ metricsZK.registerSyncOperationLatency(latency);
+ // Read Operation Latency update was registered twice.
+ verify(zkSource, times(2)).recordReadOperationLatency(latency);
+ verify(zkSource, times(1)).recordWriteOperationLatency(latency);
+ verify(zkSource, times(1)).recordSyncOperationLatency(latency);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
new file mode 100644
index 0000000..7006040
--- /dev/null
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.security.Superusers;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooDefs.Perms;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+/**
+ *
+ */
+@Category({SmallTests.class})
+public class TestZKUtil {
+
+ @Test
+ public void testUnsecure() throws ZooKeeperConnectionException, IOException {
+ Configuration conf = HBaseConfiguration.create();
+ conf.set(Superusers.SUPERUSER_CONF_KEY, "user1");
+ String node = "/hbase/testUnsecure";
+ ZKWatcher watcher = new ZKWatcher(conf, node, null, false);
+ List<ACL> aclList = ZKUtil.createACL(watcher, node, false);
+ Assert.assertEquals(aclList.size(), 1);
+ Assert.assertTrue(aclList.contains(Ids.OPEN_ACL_UNSAFE.iterator().next()));
+ }
+
+ @Test
+ public void testSecuritySingleSuperuser() throws ZooKeeperConnectionException, IOException {
+ Configuration conf = HBaseConfiguration.create();
+ conf.set(Superusers.SUPERUSER_CONF_KEY, "user1");
+ String node = "/hbase/testSecuritySingleSuperuser";
+ ZKWatcher watcher = new ZKWatcher(conf, node, null, false);
+ List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
+ Assert.assertEquals(aclList.size(), 2); // 1+1, since ACL will be set for the creator by default
+ Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user1"))));
+ Assert.assertTrue(aclList.contains(Ids.CREATOR_ALL_ACL.iterator().next()));
+ }
+
+ @Test
+ public void testCreateACL() throws ZooKeeperConnectionException, IOException {
+ Configuration conf = HBaseConfiguration.create();
+ conf.set(Superusers.SUPERUSER_CONF_KEY, "user1,@group1,user2,@group2,user3");
+ String node = "/hbase/testCreateACL";
+ ZKWatcher watcher = new ZKWatcher(conf, node, null, false);
+ List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
+ Assert.assertEquals(aclList.size(), 4); // 3+1, since ACL will be set for the creator by default
+ Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group1"))));
+ Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group2"))));
+ Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user1"))));
+ Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user2"))));
+ Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user3"))));
+ }
+
+ @Test
+ public void testCreateACLWithSameUser() throws ZooKeeperConnectionException, IOException {
+ Configuration conf = HBaseConfiguration.create();
+ conf.set(Superusers.SUPERUSER_CONF_KEY, "user4,@group1,user5,user6");
+ UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("user4"));
+ String node = "/hbase/testCreateACL";
+ ZKWatcher watcher = new ZKWatcher(conf, node, null, false);
+ List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
+ Assert.assertEquals(aclList.size(), 3); // 3, since service user the same as one of superuser
+ Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group1"))));
+ Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("auth", ""))));
+ Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user5"))));
+ Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user6"))));
+ }
+
+ @Test(expected = KeeperException.SystemErrorException.class)
+ public void testInterruptedDuringAction()
+ throws ZooKeeperConnectionException, IOException, KeeperException, InterruptedException {
+ final RecoverableZooKeeper recoverableZk = Mockito.mock(RecoverableZooKeeper.class);
+ ZKWatcher zkw = new ZKWatcher(HBaseConfiguration.create(), "unittest", null) {
+ @Override
+ public RecoverableZooKeeper getRecoverableZooKeeper() {
+ return recoverableZk;
+ }
+ };
+ Mockito.doThrow(new InterruptedException()).when(recoverableZk)
+ .getChildren(zkw.znodePaths.baseZNode, null);
+ ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.baseZNode);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKWatcher.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKWatcher.java
new file mode 100644
index 0000000..bd4575d
--- /dev/null
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKWatcher.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ SmallTests.class })
+public class TestZKWatcher {
+
+ @Test
+ public void testIsClientReadable() throws IOException {
+ ZKWatcher watcher =
+ new ZKWatcher(HBaseConfiguration.create(), "testIsClientReadable", null, false);
+
+ assertTrue(watcher.isClientReadable(watcher.znodePaths.baseZNode));
+ assertTrue(watcher.isClientReadable(watcher.znodePaths.getZNodeForReplica(0)));
+ assertTrue(watcher.isClientReadable(watcher.znodePaths.masterAddressZNode));
+ assertTrue(watcher.isClientReadable(watcher.znodePaths.clusterIdZNode));
+ assertTrue(watcher.isClientReadable(watcher.znodePaths.tableZNode));
+ assertTrue(
+ watcher.isClientReadable(ZNodePaths.joinZNode(watcher.znodePaths.tableZNode, "foo")));
+ assertTrue(watcher.isClientReadable(watcher.znodePaths.rsZNode));
+
+ assertFalse(watcher.isClientReadable(watcher.znodePaths.tableLockZNode));
+ assertFalse(watcher.isClientReadable(watcher.znodePaths.balancerZNode));
+ assertFalse(watcher.isClientReadable(watcher.znodePaths.regionNormalizerZNode));
+ assertFalse(watcher.isClientReadable(watcher.znodePaths.clusterStateZNode));
+ assertFalse(watcher.isClientReadable(watcher.znodePaths.drainingZNode));
+ assertFalse(watcher.isClientReadable(watcher.znodePaths.splitLogZNode));
+ assertFalse(watcher.isClientReadable(watcher.znodePaths.backupMasterAddressesZNode));
+
+ watcher.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b423085..ee35212 100755
--- a/pom.xml
+++ b/pom.xml
@@ -92,6 +92,7 @@
<module>hbase-metrics</module>
<module>hbase-spark-it</module>
<module>hbase-backup</module>
+ <module>hbase-zookeeper</module>
</modules>
<!--Add apache snapshots in case we want to use unreleased versions of plugins:
e.g. surefire 2.18-SNAPSHOT-->
@@ -1789,6 +1790,11 @@
<type>test-jar</type>
<scope>test</scope>
</dependency>
+ <dependency>
+ <artifactId>hbase-zookeeper</artifactId>
+ <groupId>org.apache.hbase</groupId>
+ <version>${project.version}</version>
+ </dependency>
<!-- General dependencies -->
<dependency>
<groupId>com.github.stephenc.findbugs</groupId>
[11/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper
from hbase-server and hbase-client
Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
deleted file mode 100644
index 61a5a42..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ /dev/null
@@ -1,2072 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.PrintWriter;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import javax.security.auth.login.AppConfigurationEntry;
-import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.AuthUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-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.Threads;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.DeleteNodeFailSilent;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.SetData;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.util.KerberosUtil;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NoNodeException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.ZooDefs.Perms;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.client.ZooKeeperSaslClient;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Id;
-import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.CreateRequest;
-import org.apache.zookeeper.proto.DeleteRequest;
-import org.apache.zookeeper.proto.SetDataRequest;
-import org.apache.zookeeper.server.ZooKeeperSaslServer;
-
-/**
- * Internal HBase utility class for ZooKeeper.
- *
- * <p>Contains only static methods and constants.
- *
- * <p>Methods all throw {@link KeeperException} if there is an unexpected
- * zookeeper exception, so callers of these methods must handle appropriately.
- * If ZK is required for the operation, the server will need to be aborted.
- */
-@InterfaceAudience.Private
-public class ZKUtil {
- private static final Log LOG = LogFactory.getLog(ZKUtil.class);
-
- private static int zkDumpConnectionTimeOut;
-
- /**
- * Creates a new connection to ZooKeeper, pulling settings and ensemble config
- * from the specified configuration object using methods from {@link ZKConfig}.
- *
- * Sets the connection status monitoring watcher to the specified watcher.
- *
- * @param conf configuration to pull ensemble and other settings from
- * @param watcher watcher to monitor connection changes
- * @return connection to zookeeper
- * @throws IOException if unable to connect to zk or config problem
- */
- public static RecoverableZooKeeper connect(Configuration conf, Watcher watcher)
- throws IOException {
- String ensemble = ZKConfig.getZKQuorumServersString(conf);
- return connect(conf, ensemble, watcher);
- }
-
- public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
- Watcher watcher)
- throws IOException {
- return connect(conf, ensemble, watcher, null);
- }
-
- public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
- Watcher watcher, final String identifier)
- throws IOException {
- if(ensemble == null) {
- throw new IOException("Unable to determine ZooKeeper ensemble");
- }
- int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT,
- HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
- if (LOG.isTraceEnabled()) {
- LOG.trace(identifier + " opening connection to ZooKeeper ensemble=" + ensemble);
- }
- int retry = conf.getInt("zookeeper.recovery.retry", 3);
- int retryIntervalMillis =
- conf.getInt("zookeeper.recovery.retry.intervalmill", 1000);
- int maxSleepTime = conf.getInt("zookeeper.recovery.retry.maxsleeptime", 60000);
- zkDumpConnectionTimeOut = conf.getInt("zookeeper.dump.connection.timeout",
- 1000);
- return new RecoverableZooKeeper(ensemble, timeout, watcher,
- retry, retryIntervalMillis, maxSleepTime, identifier);
- }
-
- /**
- * Log in the current zookeeper server process using the given configuration
- * keys for the credential file and login principal.
- *
- * <p><strong>This is only applicable when running on secure hbase</strong>
- * On regular HBase (without security features), this will safely be ignored.
- * </p>
- *
- * @param conf The configuration data to use
- * @param keytabFileKey Property key used to configure the path to the credential file
- * @param userNameKey Property key used to configure the login principal
- * @param hostname Current hostname to use in any credentials
- * @throws IOException underlying exception from SecurityUtil.login() call
- */
- public static void loginServer(Configuration conf, String keytabFileKey,
- String userNameKey, String hostname) throws IOException {
- login(conf, keytabFileKey, userNameKey, hostname,
- ZooKeeperSaslServer.LOGIN_CONTEXT_NAME_KEY,
- JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME);
- }
-
- /**
- * Log in the current zookeeper client using the given configuration
- * keys for the credential file and login principal.
- *
- * <p><strong>This is only applicable when running on secure hbase</strong>
- * On regular HBase (without security features), this will safely be ignored.
- * </p>
- *
- * @param conf The configuration data to use
- * @param keytabFileKey Property key used to configure the path to the credential file
- * @param userNameKey Property key used to configure the login principal
- * @param hostname Current hostname to use in any credentials
- * @throws IOException underlying exception from SecurityUtil.login() call
- */
- public static void loginClient(Configuration conf, String keytabFileKey,
- String userNameKey, String hostname) throws IOException {
- login(conf, keytabFileKey, userNameKey, hostname,
- ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY,
- JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME);
- }
-
- /**
- * Log in the current process using the given configuration keys for the
- * credential file and login principal.
- *
- * <p><strong>This is only applicable when running on secure hbase</strong>
- * On regular HBase (without security features), this will safely be ignored.
- * </p>
- *
- * @param conf The configuration data to use
- * @param keytabFileKey Property key used to configure the path to the credential file
- * @param userNameKey Property key used to configure the login principal
- * @param hostname Current hostname to use in any credentials
- * @param loginContextProperty property name to expose the entry name
- * @param loginContextName jaas entry name
- * @throws IOException underlying exception from SecurityUtil.login() call
- */
- private static void login(Configuration conf, String keytabFileKey,
- String userNameKey, String hostname,
- String loginContextProperty, String loginContextName)
- throws IOException {
- if (!isSecureZooKeeper(conf))
- return;
-
- // User has specified a jaas.conf, keep this one as the good one.
- // HBASE_OPTS="-Djava.security.auth.login.config=jaas.conf"
- if (System.getProperty("java.security.auth.login.config") != null)
- return;
-
- // No keytab specified, no auth
- String keytabFilename = conf.get(keytabFileKey);
- if (keytabFilename == null) {
- LOG.warn("no keytab specified for: " + keytabFileKey);
- return;
- }
-
- String principalConfig = conf.get(userNameKey, System.getProperty("user.name"));
- String principalName = SecurityUtil.getServerPrincipal(principalConfig, hostname);
-
- // Initialize the "jaas.conf" for keyTab/principal,
- // If keyTab is not specified use the Ticket Cache.
- // and set the zookeeper login context name.
- JaasConfiguration jaasConf = new JaasConfiguration(loginContextName,
- principalName, keytabFilename);
- javax.security.auth.login.Configuration.setConfiguration(jaasConf);
- System.setProperty(loginContextProperty, loginContextName);
- }
-
- /**
- * A JAAS configuration that defines the login modules that we want to use for login.
- */
- private static class JaasConfiguration extends javax.security.auth.login.Configuration {
- private static final String SERVER_KEYTAB_KERBEROS_CONFIG_NAME =
- "zookeeper-server-keytab-kerberos";
- private static final String CLIENT_KEYTAB_KERBEROS_CONFIG_NAME =
- "zookeeper-client-keytab-kerberos";
-
- private static final Map<String, String> BASIC_JAAS_OPTIONS = new HashMap<>();
- static {
- String jaasEnvVar = System.getenv("HBASE_JAAS_DEBUG");
- if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) {
- BASIC_JAAS_OPTIONS.put("debug", "true");
- }
- }
-
- private static final Map<String,String> KEYTAB_KERBEROS_OPTIONS = new HashMap<>();
- static {
- KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true");
- KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true");
- KEYTAB_KERBEROS_OPTIONS.put("refreshKrb5Config", "true");
- KEYTAB_KERBEROS_OPTIONS.putAll(BASIC_JAAS_OPTIONS);
- }
-
- private static final AppConfigurationEntry KEYTAB_KERBEROS_LOGIN =
- new AppConfigurationEntry(KerberosUtil.getKrb5LoginModuleName(),
- LoginModuleControlFlag.REQUIRED,
- KEYTAB_KERBEROS_OPTIONS);
-
- private static final AppConfigurationEntry[] KEYTAB_KERBEROS_CONF =
- new AppConfigurationEntry[]{KEYTAB_KERBEROS_LOGIN};
-
- private javax.security.auth.login.Configuration baseConfig;
- private final String loginContextName;
- private final boolean useTicketCache;
- private final String keytabFile;
- private final String principal;
-
- public JaasConfiguration(String loginContextName, String principal, String keytabFile) {
- this(loginContextName, principal, keytabFile, keytabFile == null || keytabFile.length() == 0);
- }
-
- private JaasConfiguration(String loginContextName, String principal,
- String keytabFile, boolean useTicketCache) {
- try {
- this.baseConfig = javax.security.auth.login.Configuration.getConfiguration();
- } catch (SecurityException e) {
- this.baseConfig = null;
- }
- this.loginContextName = loginContextName;
- this.useTicketCache = useTicketCache;
- this.keytabFile = keytabFile;
- this.principal = principal;
- LOG.info("JaasConfiguration loginContextName=" + loginContextName +
- " principal=" + principal + " useTicketCache=" + useTicketCache +
- " keytabFile=" + keytabFile);
- }
-
- @Override
- public AppConfigurationEntry[] getAppConfigurationEntry(String appName) {
- if (loginContextName.equals(appName)) {
- if (!useTicketCache) {
- KEYTAB_KERBEROS_OPTIONS.put("keyTab", keytabFile);
- KEYTAB_KERBEROS_OPTIONS.put("useKeyTab", "true");
- }
- KEYTAB_KERBEROS_OPTIONS.put("principal", principal);
- KEYTAB_KERBEROS_OPTIONS.put("useTicketCache", useTicketCache ? "true" : "false");
- return KEYTAB_KERBEROS_CONF;
- }
- if (baseConfig != null) return baseConfig.getAppConfigurationEntry(appName);
- return(null);
- }
- }
-
- //
- // Helper methods
- //
- /**
- * Returns the full path of the immediate parent of the specified node.
- * @param node path to get parent of
- * @return parent of path, null if passed the root node or an invalid node
- */
- public static String getParent(String node) {
- int idx = node.lastIndexOf(ZNodePaths.ZNODE_PATH_SEPARATOR);
- return idx <= 0 ? null : node.substring(0, idx);
- }
-
- /**
- * Get the name of the current node from the specified fully-qualified path.
- * @param path fully-qualified path
- * @return name of the current node
- */
- public static String getNodeName(String path) {
- return path.substring(path.lastIndexOf("/")+1);
- }
-
- //
- // Existence checks and watches
- //
-
- /**
- * Watch the specified znode for delete/create/change events. The watcher is
- * set whether or not the node exists. If the node already exists, the method
- * returns true. If the node does not exist, the method returns false.
- *
- * @param zkw zk reference
- * @param znode path of node to watch
- * @return true if znode exists, false if does not exist or error
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static boolean watchAndCheckExists(ZooKeeperWatcher zkw, String znode)
- throws KeeperException {
- try {
- Stat s = zkw.getRecoverableZooKeeper().exists(znode, zkw);
- boolean exists = s != null ? true : false;
- if (exists) {
- LOG.debug(zkw.prefix("Set watcher on existing znode=" + znode));
- } else {
- LOG.debug(zkw.prefix("Set watcher on znode that does not yet exist, " + znode));
- }
- return exists;
- } catch (KeeperException e) {
- LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
- zkw.keeperException(e);
- return false;
- } catch (InterruptedException e) {
- LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
- zkw.interruptedException(e);
- return false;
- }
- }
-
- /**
- * Watch the specified znode, but only if exists. Useful when watching
- * for deletions. Uses .getData() (and handles NoNodeException) instead
- * of .exists() to accomplish this, as .getData() will only set a watch if
- * the znode exists.
- * @param zkw zk reference
- * @param znode path of node to watch
- * @return true if the watch is set, false if node does not exists
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static boolean setWatchIfNodeExists(ZooKeeperWatcher zkw, String znode)
- throws KeeperException {
- try {
- zkw.getRecoverableZooKeeper().getData(znode, true, null);
- return true;
- } catch (NoNodeException e) {
- return false;
- } catch (InterruptedException e) {
- LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
- zkw.interruptedException(e);
- return false;
- }
- }
-
- /**
- * Check if the specified node exists. Sets no watches.
- *
- * @param zkw zk reference
- * @param znode path of node to watch
- * @return version of the node if it exists, -1 if does not exist
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static int checkExists(ZooKeeperWatcher zkw, String znode)
- throws KeeperException {
- try {
- Stat s = zkw.getRecoverableZooKeeper().exists(znode, null);
- return s != null ? s.getVersion() : -1;
- } catch (KeeperException e) {
- LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
- zkw.keeperException(e);
- return -1;
- } catch (InterruptedException e) {
- LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
- zkw.interruptedException(e);
- return -1;
- }
- }
-
- //
- // Znode listings
- //
-
- /**
- * Lists the children znodes of the specified znode. Also sets a watch on
- * the specified znode which will capture a NodeDeleted event on the specified
- * znode as well as NodeChildrenChanged if any children of the specified znode
- * are created or deleted.
- *
- * Returns null if the specified node does not exist. Otherwise returns a
- * list of children of the specified node. If the node exists but it has no
- * children, an empty list will be returned.
- *
- * @param zkw zk reference
- * @param znode path of node to list and watch children of
- * @return list of children of the specified node, an empty list if the node
- * exists but has no children, and null if the node does not exist
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static List<String> listChildrenAndWatchForNewChildren(
- ZooKeeperWatcher zkw, String znode)
- throws KeeperException {
- try {
- List<String> children = zkw.getRecoverableZooKeeper().getChildren(znode, zkw);
- return children;
- } catch(KeeperException.NoNodeException ke) {
- LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
- "because node does not exist (not an error)"));
- return null;
- } catch (KeeperException e) {
- LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
- zkw.keeperException(e);
- return null;
- } catch (InterruptedException e) {
- LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
- zkw.interruptedException(e);
- return null;
- }
- }
-
- /**
- * List all the children of the specified znode, setting a watch for children
- * changes and also setting a watch on every individual child in order to get
- * the NodeCreated and NodeDeleted events.
- * @param zkw zookeeper reference
- * @param znode node to get children of and watch
- * @return list of znode names, null if the node doesn't exist
- * @throws KeeperException
- */
- public static List<String> listChildrenAndWatchThem(ZooKeeperWatcher zkw,
- String znode) throws KeeperException {
- List<String> children = listChildrenAndWatchForNewChildren(zkw, znode);
- if (children == null) {
- return null;
- }
- for (String child : children) {
- watchAndCheckExists(zkw, ZNodePaths.joinZNode(znode, child));
- }
- return children;
- }
-
- /**
- * Lists the children of the specified znode without setting any watches.
- *
- * Sets no watches at all, this method is best effort.
- *
- * Returns an empty list if the node has no children. Returns null if the
- * parent node itself does not exist.
- *
- * @param zkw zookeeper reference
- * @param znode node to get children
- * @return list of data of children of specified znode, empty if no children,
- * null if parent does not exist
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static List<String> listChildrenNoWatch(ZooKeeperWatcher zkw, String znode)
- throws KeeperException {
- List<String> children = null;
- try {
- // List the children without watching
- children = zkw.getRecoverableZooKeeper().getChildren(znode, null);
- } catch(KeeperException.NoNodeException nne) {
- return null;
- } catch(InterruptedException ie) {
- zkw.interruptedException(ie);
- }
- return children;
- }
-
- /**
- * Simple class to hold a node path and node data.
- * @deprecated Unused
- */
- @Deprecated
- public static class NodeAndData {
- private String node;
- private byte [] data;
- public NodeAndData(String node, byte [] data) {
- this.node = node;
- this.data = data;
- }
- public String getNode() {
- return node;
- }
- public byte [] getData() {
- return data;
- }
- @Override
- public String toString() {
- return node;
- }
- public boolean isEmpty() {
- return (data == null || data.length == 0);
- }
- }
-
- /**
- * Checks if the specified znode has any children. Sets no watches.
- *
- * Returns true if the node exists and has children. Returns false if the
- * node does not exist or if the node does not have any children.
- *
- * Used during master initialization to determine if the master is a
- * failed-over-to master or the first master during initial cluster startup.
- * If the directory for regionserver ephemeral nodes is empty then this is
- * a cluster startup, if not then it is not cluster startup.
- *
- * @param zkw zk reference
- * @param znode path of node to check for children of
- * @return true if node has children, false if not or node does not exist
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static boolean nodeHasChildren(ZooKeeperWatcher zkw, String znode)
- throws KeeperException {
- try {
- return !zkw.getRecoverableZooKeeper().getChildren(znode, null).isEmpty();
- } catch(KeeperException.NoNodeException ke) {
- LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
- "because node does not exist (not an error)"));
- return false;
- } catch (KeeperException e) {
- LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
- zkw.keeperException(e);
- return false;
- } catch (InterruptedException e) {
- LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
- zkw.interruptedException(e);
- return false;
- }
- }
-
- /**
- * Get the number of children of the specified node.
- *
- * If the node does not exist or has no children, returns 0.
- *
- * Sets no watches at all.
- *
- * @param zkw zk reference
- * @param znode path of node to count children of
- * @return number of children of specified node, 0 if none or parent does not
- * exist
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static int getNumberOfChildren(ZooKeeperWatcher zkw, String znode)
- throws KeeperException {
- try {
- Stat stat = zkw.getRecoverableZooKeeper().exists(znode, null);
- return stat == null ? 0 : stat.getNumChildren();
- } catch(KeeperException e) {
- LOG.warn(zkw.prefix("Unable to get children of node " + znode));
- zkw.keeperException(e);
- } catch(InterruptedException e) {
- zkw.interruptedException(e);
- }
- return 0;
- }
-
- //
- // Data retrieval
- //
-
- /**
- * Get znode data. Does not set a watcher.
- * @return ZNode data, null if the node does not exist or if there is an
- * error.
- */
- public static byte [] getData(ZooKeeperWatcher zkw, String znode)
- throws KeeperException, InterruptedException {
- try {
- byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, null);
- logRetrievedMsg(zkw, znode, data, false);
- return data;
- } catch (KeeperException.NoNodeException e) {
- LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
- "because node does not exist (not an error)"));
- return null;
- } catch (KeeperException e) {
- LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
- zkw.keeperException(e);
- return null;
- }
- }
-
- /**
- * Get the data at the specified znode and set a watch.
- *
- * Returns the data and sets a watch if the node exists. Returns null and no
- * watch is set if the node does not exist or there is an exception.
- *
- * @param zkw zk reference
- * @param znode path of node
- * @return data of the specified znode, or null
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static byte [] getDataAndWatch(ZooKeeperWatcher zkw, String znode)
- throws KeeperException {
- return getDataInternal(zkw, znode, null, true);
- }
-
- /**
- * Get the data at the specified znode and set a watch.
- *
- * Returns the data and sets a watch if the node exists. Returns null and no
- * watch is set if the node does not exist or there is an exception.
- *
- * @param zkw zk reference
- * @param znode path of node
- * @param stat object to populate the version of the znode
- * @return data of the specified znode, or null
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static byte[] getDataAndWatch(ZooKeeperWatcher zkw, String znode,
- Stat stat) throws KeeperException {
- return getDataInternal(zkw, znode, stat, true);
- }
-
- private static byte[] getDataInternal(ZooKeeperWatcher zkw, String znode, Stat stat,
- boolean watcherSet)
- throws KeeperException {
- try {
- byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat);
- logRetrievedMsg(zkw, znode, data, watcherSet);
- return data;
- } catch (KeeperException.NoNodeException e) {
- // This log can get pretty annoying when we cycle on 100ms waits.
- // Enable trace if you really want to see it.
- LOG.trace(zkw.prefix("Unable to get data of znode " + znode + " " +
- "because node does not exist (not an error)"));
- return null;
- } catch (KeeperException e) {
- LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
- zkw.keeperException(e);
- return null;
- } catch (InterruptedException e) {
- LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
- zkw.interruptedException(e);
- return null;
- }
- }
-
- /**
- * Get the data at the specified znode without setting a watch.
- *
- * Returns the data if the node exists. Returns null if the node does not
- * exist.
- *
- * Sets the stats of the node in the passed Stat object. Pass a null stat if
- * not interested.
- *
- * @param zkw zk reference
- * @param znode path of node
- * @param stat node status to get if node exists
- * @return data of the specified znode, or null if node does not exist
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static byte [] getDataNoWatch(ZooKeeperWatcher zkw, String znode,
- Stat stat)
- throws KeeperException {
- try {
- byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, stat);
- logRetrievedMsg(zkw, znode, data, false);
- return data;
- } catch (KeeperException.NoNodeException e) {
- LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
- "because node does not exist (not necessarily an error)"));
- return null;
- } catch (KeeperException e) {
- LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
- zkw.keeperException(e);
- return null;
- } catch (InterruptedException e) {
- LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
- zkw.interruptedException(e);
- return null;
- }
- }
-
- /**
- * Returns the date of child znodes of the specified znode. Also sets a watch on
- * the specified znode which will capture a NodeDeleted event on the specified
- * znode as well as NodeChildrenChanged if any children of the specified znode
- * are created or deleted.
- *
- * Returns null if the specified node does not exist. Otherwise returns a
- * list of children of the specified node. If the node exists but it has no
- * children, an empty list will be returned.
- *
- * @param zkw zk reference
- * @param baseNode path of node to list and watch children of
- * @return list of data of children of the specified node, an empty list if the node
- * exists but has no children, and null if the node does not exist
- * @throws KeeperException if unexpected zookeeper exception
- * @deprecated Unused
- */
- @Deprecated
- public static List<NodeAndData> getChildDataAndWatchForNewChildren(
- ZooKeeperWatcher zkw, String baseNode) throws KeeperException {
- List<String> nodes =
- ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode);
- if (nodes != null) {
- List<NodeAndData> newNodes = new ArrayList<>();
- for (String node : nodes) {
- String nodePath = ZNodePaths.joinZNode(baseNode, node);
- byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath);
- newNodes.add(new NodeAndData(nodePath, data));
- }
- return newNodes;
- }
- return null;
- }
-
- /**
- * Update the data of an existing node with the expected version to have the
- * specified data.
- *
- * Throws an exception if there is a version mismatch or some other problem.
- *
- * Sets no watches under any conditions.
- *
- * @param zkw zk reference
- * @param znode
- * @param data
- * @param expectedVersion
- * @throws KeeperException if unexpected zookeeper exception
- * @throws KeeperException.BadVersionException if version mismatch
- * @deprecated Unused
- */
- @Deprecated
- public static void updateExistingNodeData(ZooKeeperWatcher zkw, String znode,
- byte [] data, int expectedVersion)
- throws KeeperException {
- try {
- zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion);
- } catch(InterruptedException ie) {
- zkw.interruptedException(ie);
- }
- }
-
- //
- // Data setting
- //
-
- /**
- * Sets the data of the existing znode to be the specified data. Ensures that
- * the current data has the specified expected version.
- *
- * <p>If the node does not exist, a {@link NoNodeException} will be thrown.
- *
- * <p>If their is a version mismatch, method returns null.
- *
- * <p>No watches are set but setting data will trigger other watchers of this
- * node.
- *
- * <p>If there is another problem, a KeeperException will be thrown.
- *
- * @param zkw zk reference
- * @param znode path of node
- * @param data data to set for node
- * @param expectedVersion version expected when setting data
- * @return true if data set, false if version mismatch
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static boolean setData(ZooKeeperWatcher zkw, String znode,
- byte [] data, int expectedVersion)
- throws KeeperException, KeeperException.NoNodeException {
- try {
- return zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion) != null;
- } catch (InterruptedException e) {
- zkw.interruptedException(e);
- return false;
- }
- }
-
- /**
- * Set data into node creating node if it doesn't yet exist.
- * Does not set watch.
- *
- * @param zkw zk reference
- * @param znode path of node
- * @param data data to set for node
- * @throws KeeperException
- */
- public static void createSetData(final ZooKeeperWatcher zkw, final String znode,
- final byte [] data)
- throws KeeperException {
- if (checkExists(zkw, znode) == -1) {
- ZKUtil.createWithParents(zkw, znode, data);
- } else {
- ZKUtil.setData(zkw, znode, data);
- }
- }
-
- /**
- * Sets the data of the existing znode to be the specified data. The node
- * must exist but no checks are done on the existing data or version.
- *
- * <p>If the node does not exist, a {@link NoNodeException} will be thrown.
- *
- * <p>No watches are set but setting data will trigger other watchers of this
- * node.
- *
- * <p>If there is another problem, a KeeperException will be thrown.
- *
- * @param zkw zk reference
- * @param znode path of node
- * @param data data to set for node
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static void setData(ZooKeeperWatcher zkw, String znode, byte [] data)
- throws KeeperException, KeeperException.NoNodeException {
- setData(zkw, (SetData)ZKUtilOp.setData(znode, data));
- }
-
- private static void setData(ZooKeeperWatcher zkw, SetData setData)
- throws KeeperException, KeeperException.NoNodeException {
- SetDataRequest sd = (SetDataRequest)toZooKeeperOp(zkw, setData).toRequestRecord();
- setData(zkw, sd.getPath(), sd.getData(), sd.getVersion());
- }
-
- /**
- * Returns whether or not secure authentication is enabled
- * (whether <code>hbase.security.authentication</code> is set to
- * <code>kerberos</code>.
- */
- public static boolean isSecureZooKeeper(Configuration conf) {
- // Detection for embedded HBase client with jaas configuration
- // defined for third party programs.
- try {
- javax.security.auth.login.Configuration testConfig =
- javax.security.auth.login.Configuration.getConfiguration();
- if (testConfig.getAppConfigurationEntry("Client") == null
- && testConfig.getAppConfigurationEntry(
- JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME) == null
- && testConfig.getAppConfigurationEntry(
- JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME) == null
- && conf.get(HConstants.ZK_CLIENT_KERBEROS_PRINCIPAL) == null
- && conf.get(HConstants.ZK_SERVER_KERBEROS_PRINCIPAL) == null) {
-
- return false;
- }
- } catch(Exception e) {
- // No Jaas configuration defined.
- return false;
- }
-
- // Master & RSs uses hbase.zookeeper.client.*
- return "kerberos".equalsIgnoreCase(conf.get("hbase.security.authentication"));
- }
-
- private static ArrayList<ACL> createACL(ZooKeeperWatcher zkw, String node) {
- return createACL(zkw, node, isSecureZooKeeper(zkw.getConfiguration()));
- }
-
- public static ArrayList<ACL> createACL(ZooKeeperWatcher zkw, String node,
- boolean isSecureZooKeeper) {
- if (!node.startsWith(zkw.znodePaths.baseZNode)) {
- return Ids.OPEN_ACL_UNSAFE;
- }
- if (isSecureZooKeeper) {
- ArrayList<ACL> acls = new ArrayList<>();
- // add permission to hbase supper user
- String[] superUsers = zkw.getConfiguration().getStrings(Superusers.SUPERUSER_CONF_KEY);
- String hbaseUser = null;
- try {
- hbaseUser = UserGroupInformation.getCurrentUser().getShortUserName();
- } catch (IOException e) {
- LOG.debug("Could not acquire current User.", e);
- }
- if (superUsers != null) {
- List<String> groups = new ArrayList<>();
- for (String user : superUsers) {
- if (AuthUtil.isGroupPrincipal(user)) {
- // TODO: Set node ACL for groups when ZK supports this feature
- groups.add(user);
- } else {
- if(!user.equals(hbaseUser)) {
- acls.add(new ACL(Perms.ALL, new Id("sasl", user)));
- }
- }
- }
- if (!groups.isEmpty()) {
- LOG.warn("Znode ACL setting for group " + groups
- + " is skipped, ZooKeeper doesn't support this feature presently.");
- }
- }
- // Certain znodes are accessed directly by the client,
- // so they must be readable by non-authenticated clients
- if (zkw.isClientReadable(node)) {
- acls.addAll(Ids.CREATOR_ALL_ACL);
- acls.addAll(Ids.READ_ACL_UNSAFE);
- } else {
- acls.addAll(Ids.CREATOR_ALL_ACL);
- }
- return acls;
- } else {
- return Ids.OPEN_ACL_UNSAFE;
- }
- }
-
- //
- // Node creation
- //
-
- /**
- *
- * Set the specified znode to be an ephemeral node carrying the specified
- * data.
- *
- * If the node is created successfully, a watcher is also set on the node.
- *
- * If the node is not created successfully because it already exists, this
- * method will also set a watcher on the node.
- *
- * If there is another problem, a KeeperException will be thrown.
- *
- * @param zkw zk reference
- * @param znode path of node
- * @param data data of node
- * @return true if node created, false if not, watch set in both cases
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static boolean createEphemeralNodeAndWatch(ZooKeeperWatcher zkw,
- String znode, byte [] data)
- throws KeeperException {
- boolean ret = true;
- try {
- zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
- CreateMode.EPHEMERAL);
- } catch (KeeperException.NodeExistsException nee) {
- ret = false;
- } catch (InterruptedException e) {
- LOG.info("Interrupted", e);
- Thread.currentThread().interrupt();
- }
- if(!watchAndCheckExists(zkw, znode)) {
- // It did exist but now it doesn't, try again
- return createEphemeralNodeAndWatch(zkw, znode, data);
- }
- return ret;
- }
-
- /**
- * Creates the specified znode to be a persistent node carrying the specified
- * data.
- *
- * Returns true if the node was successfully created, false if the node
- * already existed.
- *
- * If the node is created successfully, a watcher is also set on the node.
- *
- * If the node is not created successfully because it already exists, this
- * method will also set a watcher on the node but return false.
- *
- * If there is another problem, a KeeperException will be thrown.
- *
- * @param zkw zk reference
- * @param znode path of node
- * @param data data of node
- * @return true if node created, false if not, watch set in both cases
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static boolean createNodeIfNotExistsAndWatch(
- ZooKeeperWatcher zkw, String znode, byte [] data)
- throws KeeperException {
- boolean ret = true;
- try {
- zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
- CreateMode.PERSISTENT);
- } catch (KeeperException.NodeExistsException nee) {
- ret = false;
- } catch (InterruptedException e) {
- zkw.interruptedException(e);
- return false;
- }
- try {
- zkw.getRecoverableZooKeeper().exists(znode, zkw);
- } catch (InterruptedException e) {
- zkw.interruptedException(e);
- return false;
- }
- return ret;
- }
-
- /**
- * Creates the specified znode with the specified data but does not watch it.
- *
- * Returns the znode of the newly created node
- *
- * If there is another problem, a KeeperException will be thrown.
- *
- * @param zkw zk reference
- * @param znode path of node
- * @param data data of node
- * @param createMode specifying whether the node to be created is ephemeral and/or sequential
- * @return true name of the newly created znode or null
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static String createNodeIfNotExistsNoWatch(ZooKeeperWatcher zkw, String znode,
- byte[] data, CreateMode createMode) throws KeeperException {
-
- String createdZNode = null;
- try {
- createdZNode = zkw.getRecoverableZooKeeper().create(znode, data,
- createACL(zkw, znode), createMode);
- } catch (KeeperException.NodeExistsException nee) {
- return znode;
- } catch (InterruptedException e) {
- zkw.interruptedException(e);
- return null;
- }
- return createdZNode;
- }
-
- /**
- * Creates the specified node with the specified data and watches it.
- *
- * <p>Throws an exception if the node already exists.
- *
- * <p>The node created is persistent and open access.
- *
- * <p>Returns the version number of the created node if successful.
- *
- * @param zkw zk reference
- * @param znode path of node to create
- * @param data data of node to create
- * @return version of node created
- * @throws KeeperException if unexpected zookeeper exception
- * @throws KeeperException.NodeExistsException if node already exists
- */
- public static int createAndWatch(ZooKeeperWatcher zkw,
- String znode, byte [] data)
- throws KeeperException, KeeperException.NodeExistsException {
- try {
- zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
- CreateMode.PERSISTENT);
- Stat stat = zkw.getRecoverableZooKeeper().exists(znode, zkw);
- if (stat == null){
- // Likely a race condition. Someone deleted the znode.
- throw KeeperException.create(KeeperException.Code.SYSTEMERROR,
- "ZK.exists returned null (i.e.: znode does not exist) for znode=" + znode);
- }
- return stat.getVersion();
- } catch (InterruptedException e) {
- zkw.interruptedException(e);
- return -1;
- }
- }
-
- /**
- * Async creates the specified node with the specified data.
- *
- * <p>Throws an exception if the node already exists.
- *
- * <p>The node created is persistent and open access.
- *
- * @param zkw zk reference
- * @param znode path of node to create
- * @param data data of node to create
- * @param cb
- * @param ctx
- */
- public static void asyncCreate(ZooKeeperWatcher zkw,
- String znode, byte [] data, final AsyncCallback.StringCallback cb,
- final Object ctx) {
- zkw.getRecoverableZooKeeper().getZooKeeper().create(znode, data,
- createACL(zkw, znode), CreateMode.PERSISTENT, cb, ctx);
- }
-
- /**
- * Creates the specified node, iff the node does not exist. Does not set a
- * watch and fails silently if the node already exists.
- *
- * The node created is persistent and open access.
- *
- * @param zkw zk reference
- * @param znode path of node
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static void createAndFailSilent(ZooKeeperWatcher zkw,
- String znode) throws KeeperException {
- createAndFailSilent(zkw, znode, new byte[0]);
- }
-
- /**
- * Creates the specified node containing specified data, iff the node does not exist. Does
- * not set a watch and fails silently if the node already exists.
- *
- * The node created is persistent and open access.
- *
- * @param zkw zk reference
- * @param znode path of node
- * @param data a byte array data to store in the znode
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static void createAndFailSilent(ZooKeeperWatcher zkw,
- String znode, byte[] data)
- throws KeeperException {
- createAndFailSilent(zkw,
- (CreateAndFailSilent)ZKUtilOp.createAndFailSilent(znode, data));
- }
-
- private static void createAndFailSilent(ZooKeeperWatcher zkw, CreateAndFailSilent cafs)
- throws KeeperException {
- CreateRequest create = (CreateRequest)toZooKeeperOp(zkw, cafs).toRequestRecord();
- String znode = create.getPath();
- try {
- RecoverableZooKeeper zk = zkw.getRecoverableZooKeeper();
- if (zk.exists(znode, false) == null) {
- zk.create(znode, create.getData(), create.getAcl(), CreateMode.fromFlag(create.getFlags()));
- }
- } catch(KeeperException.NodeExistsException nee) {
- } catch(KeeperException.NoAuthException nee){
- try {
- if (null == zkw.getRecoverableZooKeeper().exists(znode, false)) {
- // If we failed to create the file and it does not already exist.
- throw(nee);
- }
- } catch (InterruptedException ie) {
- zkw.interruptedException(ie);
- }
- } catch(InterruptedException ie) {
- zkw.interruptedException(ie);
- }
- }
-
- /**
- * Creates the specified node and all parent nodes required for it to exist.
- *
- * No watches are set and no errors are thrown if the node already exists.
- *
- * The nodes created are persistent and open access.
- *
- * @param zkw zk reference
- * @param znode path of node
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static void createWithParents(ZooKeeperWatcher zkw, String znode)
- throws KeeperException {
- createWithParents(zkw, znode, new byte[0]);
- }
-
- /**
- * Creates the specified node and all parent nodes required for it to exist. The creation of
- * parent znodes is not atomic with the leafe znode creation but the data is written atomically
- * when the leaf node is created.
- *
- * No watches are set and no errors are thrown if the node already exists.
- *
- * The nodes created are persistent and open access.
- *
- * @param zkw zk reference
- * @param znode path of node
- * @throws KeeperException if unexpected zookeeper exception
- */
- public static void createWithParents(ZooKeeperWatcher zkw, String znode, byte[] data)
- throws KeeperException {
- try {
- if(znode == null) {
- return;
- }
- zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
- CreateMode.PERSISTENT);
- } catch(KeeperException.NodeExistsException nee) {
- return;
- } catch(KeeperException.NoNodeException nne) {
- createWithParents(zkw, getParent(znode));
- createWithParents(zkw, znode, data);
- } catch(InterruptedException ie) {
- zkw.interruptedException(ie);
- }
- }
-
- //
- // Deletes
- //
-
- /**
- * Delete the specified node. Sets no watches. Throws all exceptions.
- */
- public static void deleteNode(ZooKeeperWatcher zkw, String node)
- throws KeeperException {
- deleteNode(zkw, node, -1);
- }
-
- /**
- * Delete the specified node with the specified version. Sets no watches.
- * Throws all exceptions.
- */
- public static boolean deleteNode(ZooKeeperWatcher zkw, String node,
- int version)
- throws KeeperException {
- try {
- zkw.getRecoverableZooKeeper().delete(node, version);
- return true;
- } catch(KeeperException.BadVersionException bve) {
- return false;
- } catch(InterruptedException ie) {
- zkw.interruptedException(ie);
- return false;
- }
- }
-
- /**
- * Deletes the specified node. Fails silent if the node does not exist.
- * @param zkw
- * @param node
- * @throws KeeperException
- */
- public static void deleteNodeFailSilent(ZooKeeperWatcher zkw, String node)
- throws KeeperException {
- deleteNodeFailSilent(zkw,
- (DeleteNodeFailSilent)ZKUtilOp.deleteNodeFailSilent(node));
- }
-
- private static void deleteNodeFailSilent(ZooKeeperWatcher zkw,
- DeleteNodeFailSilent dnfs) throws KeeperException {
- DeleteRequest delete = (DeleteRequest)toZooKeeperOp(zkw, dnfs).toRequestRecord();
- try {
- zkw.getRecoverableZooKeeper().delete(delete.getPath(), delete.getVersion());
- } catch(KeeperException.NoNodeException nne) {
- } catch(InterruptedException ie) {
- zkw.interruptedException(ie);
- }
- }
-
-
- /**
- * Delete the specified node and all of it's children.
- * <p>
- * If the node does not exist, just returns.
- * <p>
- * Sets no watches. Throws all exceptions besides dealing with deletion of
- * children.
- */
- public static void deleteNodeRecursively(ZooKeeperWatcher zkw, String node)
- throws KeeperException {
- deleteNodeRecursivelyMultiOrSequential(zkw, true, node);
- }
-
- /**
- * Delete all the children of the specified node but not the node itself.
- *
- * Sets no watches. Throws all exceptions besides dealing with deletion of
- * children.
- *
- * @throws KeeperException
- */
- public static void deleteChildrenRecursively(ZooKeeperWatcher zkw, String node)
- throws KeeperException {
- deleteChildrenRecursivelyMultiOrSequential(zkw, true, node);
- }
-
- /**
- * Delete all the children of the specified node but not the node itself. This
- * will first traverse the znode tree for listing the children and then delete
- * these znodes using multi-update api or sequential based on the specified
- * configurations.
- * <p>
- * Sets no watches. Throws all exceptions besides dealing with deletion of
- * children.
- * <p>
- * If the following is true:
- * <ul>
- * <li>runSequentialOnMultiFailure is true
- * </ul>
- * on calling multi, we get a ZooKeeper exception that can be handled by a
- * sequential call(*), we retry the operations one-by-one (sequentially).
- *
- * @param zkw
- * - zk reference
- * @param runSequentialOnMultiFailure
- * - if true when we get a ZooKeeper exception that could retry the
- * operations one-by-one (sequentially)
- * @param pathRoots
- * - path of the parent node(s)
- * @throws KeeperException.NotEmptyException
- * if node has children while deleting
- * @throws KeeperException
- * if unexpected ZooKeeper exception
- * @throws IllegalArgumentException
- * if an invalid path is specified
- */
- public static void deleteChildrenRecursivelyMultiOrSequential(
- ZooKeeperWatcher zkw, boolean runSequentialOnMultiFailure,
- String... pathRoots) throws KeeperException {
- if (pathRoots == null || pathRoots.length <= 0) {
- LOG.warn("Given path is not valid!");
- return;
- }
- List<ZKUtilOp> ops = new ArrayList<>();
- for (String eachRoot : pathRoots) {
- List<String> children = listChildrenBFSNoWatch(zkw, eachRoot);
- // Delete the leaves first and eventually get rid of the root
- for (int i = children.size() - 1; i >= 0; --i) {
- ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i)));
- }
- }
- // atleast one element should exist
- if (ops.size() > 0) {
- multiOrSequential(zkw, ops, runSequentialOnMultiFailure);
- }
- }
-
- /**
- * Delete the specified node and its children. This traverse the
- * znode tree for listing the children and then delete
- * these znodes including the parent using multi-update api or
- * sequential based on the specified configurations.
- * <p>
- * Sets no watches. Throws all exceptions besides dealing with deletion of
- * children.
- * <p>
- * If the following is true:
- * <ul>
- * <li>runSequentialOnMultiFailure is true
- * </ul>
- * on calling multi, we get a ZooKeeper exception that can be handled by a
- * sequential call(*), we retry the operations one-by-one (sequentially).
- *
- * @param zkw
- * - zk reference
- * @param runSequentialOnMultiFailure
- * - if true when we get a ZooKeeper exception that could retry the
- * operations one-by-one (sequentially)
- * @param pathRoots
- * - path of the parent node(s)
- * @throws KeeperException.NotEmptyException
- * if node has children while deleting
- * @throws KeeperException
- * if unexpected ZooKeeper exception
- * @throws IllegalArgumentException
- * if an invalid path is specified
- */
- public static void deleteNodeRecursivelyMultiOrSequential(ZooKeeperWatcher zkw,
- boolean runSequentialOnMultiFailure, String... pathRoots) throws KeeperException {
- if (pathRoots == null || pathRoots.length <= 0) {
- LOG.warn("Given path is not valid!");
- return;
- }
- List<ZKUtilOp> ops = new ArrayList<>();
- for (String eachRoot : pathRoots) {
- // ZooKeeper Watches are one time triggers; When children of parent nodes are deleted
- // recursively, must set another watch, get notified of delete node
- List<String> children = listChildrenBFSAndWatchThem(zkw, eachRoot);
- // Delete the leaves first and eventually get rid of the root
- for (int i = children.size() - 1; i >= 0; --i) {
- ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i)));
- }
- try {
- if (zkw.getRecoverableZooKeeper().exists(eachRoot, zkw) != null) {
- ops.add(ZKUtilOp.deleteNodeFailSilent(eachRoot));
- }
- } catch (InterruptedException e) {
- zkw.interruptedException(e);
- }
- }
- // atleast one element should exist
- if (ops.size() > 0) {
- multiOrSequential(zkw, ops, runSequentialOnMultiFailure);
- }
- }
-
- /**
- * BFS Traversal of all the children under path, with the entries in the list,
- * in the same order as that of the traversal. Lists all the children without
- * setting any watches.
- *
- * @param zkw
- * - zk reference
- * @param znode
- * - path of node
- * @return list of children znodes under the path
- * @throws KeeperException
- * if unexpected ZooKeeper exception
- */
- private static List<String> listChildrenBFSNoWatch(ZooKeeperWatcher zkw,
- final String znode) throws KeeperException {
- Deque<String> queue = new LinkedList<>();
- List<String> tree = new ArrayList<>();
- queue.add(znode);
- while (true) {
- String node = queue.pollFirst();
- if (node == null) {
- break;
- }
- List<String> children = listChildrenNoWatch(zkw, node);
- if (children == null) {
- continue;
- }
- for (final String child : children) {
- final String childPath = node + "/" + child;
- queue.add(childPath);
- tree.add(childPath);
- }
- }
- return tree;
- }
-
- /**
- * BFS Traversal of all the children under path, with the entries in the list,
- * in the same order as that of the traversal.
- * Lists all the children and set watches on to them.
- *
- * @param zkw
- * - zk reference
- * @param znode
- * - path of node
- * @return list of children znodes under the path
- * @throws KeeperException
- * if unexpected ZooKeeper exception
- */
- private static List<String> listChildrenBFSAndWatchThem(ZooKeeperWatcher zkw, final String znode)
- throws KeeperException {
- Deque<String> queue = new LinkedList<>();
- List<String> tree = new ArrayList<>();
- queue.add(znode);
- while (true) {
- String node = queue.pollFirst();
- if (node == null) {
- break;
- }
- List<String> children = listChildrenAndWatchThem(zkw, node);
- if (children == null) {
- continue;
- }
- for (final String child : children) {
- final String childPath = node + "/" + child;
- queue.add(childPath);
- tree.add(childPath);
- }
- }
- return tree;
- }
-
- /**
- * Represents an action taken by ZKUtil, e.g. createAndFailSilent.
- * These actions are higher-level than ZKOp actions, which represent
- * individual actions in the ZooKeeper API, like create.
- */
- public abstract static class ZKUtilOp {
- private String path;
-
- private ZKUtilOp(String path) {
- this.path = path;
- }
-
- /**
- * @return a createAndFailSilent ZKUtilOp
- */
- public static ZKUtilOp createAndFailSilent(String path, byte[] data) {
- return new CreateAndFailSilent(path, data);
- }
-
- /**
- * @return a deleteNodeFailSilent ZKUtilOP
- */
- public static ZKUtilOp deleteNodeFailSilent(String path) {
- return new DeleteNodeFailSilent(path);
- }
-
- /**
- * @return a setData ZKUtilOp
- */
- public static ZKUtilOp setData(String path, byte [] data) {
- return new SetData(path, data);
- }
-
- /**
- * @return path to znode where the ZKOp will occur
- */
- public String getPath() {
- return path;
- }
-
- /**
- * ZKUtilOp representing createAndFailSilent in ZooKeeper
- * (attempt to create node, ignore error if already exists)
- */
- public static class CreateAndFailSilent extends ZKUtilOp {
- private byte [] data;
-
- private CreateAndFailSilent(String path, byte [] data) {
- super(path);
- this.data = data;
- }
-
- public byte[] getData() {
- return data;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (!(o instanceof CreateAndFailSilent)) return false;
-
- CreateAndFailSilent op = (CreateAndFailSilent) o;
- return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
- }
-
- @Override
- public int hashCode() {
- int ret = 17 + getPath().hashCode() * 31;
- return ret * 31 + Bytes.hashCode(data);
- }
- }
-
- /**
- * ZKUtilOp representing deleteNodeFailSilent in ZooKeeper
- * (attempt to delete node, ignore error if node doesn't exist)
- */
- public static class DeleteNodeFailSilent extends ZKUtilOp {
- private DeleteNodeFailSilent(String path) {
- super(path);
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (!(o instanceof DeleteNodeFailSilent)) return false;
-
- return super.equals(o);
- }
-
- @Override
- public int hashCode() {
- return getPath().hashCode();
- }
- }
-
- /**
- * ZKUtilOp representing setData in ZooKeeper
- */
- public static class SetData extends ZKUtilOp {
- private byte [] data;
-
- private SetData(String path, byte [] data) {
- super(path);
- this.data = data;
- }
-
- public byte[] getData() {
- return data;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (!(o instanceof SetData)) return false;
-
- SetData op = (SetData) o;
- return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
- }
-
- @Override
- public int hashCode() {
- int ret = getPath().hashCode();
- return ret * 31 + Bytes.hashCode(data);
- }
- }
- }
-
- /**
- * Convert from ZKUtilOp to ZKOp
- */
- private static Op toZooKeeperOp(ZooKeeperWatcher zkw, ZKUtilOp op)
- throws UnsupportedOperationException {
- if(op == null) return null;
-
- if (op instanceof CreateAndFailSilent) {
- CreateAndFailSilent cafs = (CreateAndFailSilent)op;
- return Op.create(cafs.getPath(), cafs.getData(), createACL(zkw, cafs.getPath()),
- CreateMode.PERSISTENT);
- } else if (op instanceof DeleteNodeFailSilent) {
- DeleteNodeFailSilent dnfs = (DeleteNodeFailSilent)op;
- return Op.delete(dnfs.getPath(), -1);
- } else if (op instanceof SetData) {
- SetData sd = (SetData)op;
- return Op.setData(sd.getPath(), sd.getData(), -1);
- } else {
- throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
- + op.getClass().getName());
- }
- }
-
- /**
- * Use ZooKeeper's multi-update functionality.
- *
- * If all of the following are true:
- * - runSequentialOnMultiFailure is true
- * - on calling multi, we get a ZooKeeper exception that can be handled by a sequential call(*)
- * Then:
- * - we retry the operations one-by-one (sequentially)
- *
- * Note *: an example is receiving a NodeExistsException from a "create" call. Without multi,
- * a user could call "createAndFailSilent" to ensure that a node exists if they don't care who
- * actually created the node (i.e. the NodeExistsException from ZooKeeper is caught).
- * This will cause all operations in the multi to fail, however, because
- * the NodeExistsException that zk.create throws will fail the multi transaction.
- * 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.
- *
- * @throws KeeperException
- */
- public static void multiOrSequential(ZooKeeperWatcher zkw, List<ZKUtilOp> ops,
- boolean runSequentialOnMultiFailure) throws KeeperException {
- if (zkw.getConfiguration().get("hbase.zookeeper.useMulti") != null) {
- LOG.warn("hbase.zookeeper.useMulti is deprecated. Default to true always.");
- }
- if (ops == null) return;
-
- List<Op> zkOps = new LinkedList<>();
- for (ZKUtilOp op : ops) {
- zkOps.add(toZooKeeperOp(zkw, op));
- }
- try {
- zkw.getRecoverableZooKeeper().multi(zkOps);
- } catch (KeeperException ke) {
- switch (ke.code()) {
- case NODEEXISTS:
- case NONODE:
- case BADVERSION:
- case NOAUTH:
- // if we get an exception that could be solved by running sequentially
- // (and the client asked us to), then break out and run sequentially
- if (runSequentialOnMultiFailure) {
- LOG.info("On call to ZK.multi, received exception: " + ke.toString() + "."
- + " Attempting to run operations sequentially because"
- + " runSequentialOnMultiFailure is: " + runSequentialOnMultiFailure + ".");
- processSequentially(zkw, ops);
- break;
- }
- default:
- throw ke;
- }
- } catch (InterruptedException ie) {
- zkw.interruptedException(ie);
- }
- }
-
- private static void processSequentially(ZooKeeperWatcher zkw, List<ZKUtilOp> ops)
- throws KeeperException, NoNodeException {
- for (ZKUtilOp op : ops) {
- if (op instanceof CreateAndFailSilent) {
- createAndFailSilent(zkw, (CreateAndFailSilent) op);
- } else if (op instanceof DeleteNodeFailSilent) {
- deleteNodeFailSilent(zkw, (DeleteNodeFailSilent) op);
- } else if (op instanceof SetData) {
- setData(zkw, (SetData) op);
- } else {
- throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
- + op.getClass().getName());
- }
- }
- }
-
- //
- // ZooKeeper cluster information
- //
-
- /** @return String dump of everything in ZooKeeper. */
- public static String dump(ZooKeeperWatcher zkw) {
- StringBuilder sb = new StringBuilder();
- try {
- sb.append("HBase is rooted at ").append(zkw.znodePaths.baseZNode);
- sb.append("\nActive master address: ");
- try {
- sb.append(MasterAddressTracker.getMasterAddress(zkw));
- } catch (IOException e) {
- sb.append("<<FAILED LOOKUP: " + e.getMessage() + ">>");
- }
- sb.append("\nBackup master addresses:");
- for (String child : listChildrenNoWatch(zkw, zkw.znodePaths.backupMasterAddressesZNode)) {
- sb.append("\n ").append(child);
- }
- sb.append("\nRegion server holding hbase:meta: "
- + new MetaTableLocator().getMetaRegionLocation(zkw));
- Configuration conf = HBaseConfiguration.create();
- int numMetaReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
- HConstants.DEFAULT_META_REPLICA_NUM);
- for (int i = 1; i < numMetaReplicas; i++) {
- sb.append("\nRegion server holding hbase:meta, replicaId " + i + " "
- + new MetaTableLocator().getMetaRegionLocation(zkw, i));
- }
- sb.append("\nRegion servers:");
- for (String child : listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode)) {
- sb.append("\n ").append(child);
- }
- try {
- getReplicationZnodesDump(zkw, sb);
- } catch (KeeperException ke) {
- LOG.warn("Couldn't get the replication znode dump", ke);
- }
- sb.append("\nQuorum Server Statistics:");
- String[] servers = zkw.getQuorum().split(",");
- for (String server : servers) {
- sb.append("\n ").append(server);
- try {
- String[] stat = getServerStats(server, ZKUtil.zkDumpConnectionTimeOut);
-
- if (stat == null) {
- sb.append("[Error] invalid quorum server: " + server);
- break;
- }
-
- for (String s : stat) {
- sb.append("\n ").append(s);
- }
- } catch (Exception e) {
- sb.append("\n ERROR: ").append(e.getMessage());
- }
- }
- } catch (KeeperException ke) {
- sb.append("\nFATAL ZooKeeper Exception!\n");
- sb.append("\n" + ke.getMessage());
- }
- return sb.toString();
- }
-
- /**
- * Appends replication znodes to the passed StringBuilder.
- * @param zkw
- * @param sb
- * @throws KeeperException
- */
- private static void getReplicationZnodesDump(ZooKeeperWatcher zkw, StringBuilder sb)
- throws KeeperException {
- String replicationZnode = zkw.znodePaths.replicationZNode;
- if (ZKUtil.checkExists(zkw, replicationZnode) == -1) return;
- // do a ls -r on this znode
- sb.append("\n").append(replicationZnode).append(": ");
- List<String> children = ZKUtil.listChildrenNoWatch(zkw, replicationZnode);
- for (String child : children) {
- String znode = ZNodePaths.joinZNode(replicationZnode, child);
- if (znode.equals(zkw.znodePaths.peersZNode)) {
- appendPeersZnodes(zkw, znode, sb);
- } else if (znode.equals(zkw.znodePaths.queuesZNode)) {
- appendRSZnodes(zkw, znode, sb);
- } else if (znode.equals(zkw.znodePaths.hfileRefsZNode)) {
- appendHFileRefsZnodes(zkw, znode, sb);
- }
- }
- }
-
- private static void appendHFileRefsZnodes(ZooKeeperWatcher zkw, String hfileRefsZnode,
- StringBuilder sb) throws KeeperException {
- sb.append("\n").append(hfileRefsZnode).append(": ");
- for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, hfileRefsZnode)) {
- String znodeToProcess = ZNodePaths.joinZNode(hfileRefsZnode, peerIdZnode);
- sb.append("\n").append(znodeToProcess).append(": ");
- List<String> peerHFileRefsZnodes = ZKUtil.listChildrenNoWatch(zkw, znodeToProcess);
- int size = peerHFileRefsZnodes.size();
- for (int i = 0; i < size; i++) {
- sb.append(peerHFileRefsZnodes.get(i));
- if (i != size - 1) {
- sb.append(", ");
- }
- }
- }
- }
-
- /**
- * Returns a string with replication znodes and position of the replication log
- * @param zkw
- * @return aq string of replication znodes and log positions
- */
- public static String getReplicationZnodesDump(ZooKeeperWatcher zkw) throws KeeperException {
- StringBuilder sb = new StringBuilder();
- getReplicationZnodesDump(zkw, sb);
- return sb.toString();
- }
-
- private static void appendRSZnodes(ZooKeeperWatcher zkw, String znode, StringBuilder sb)
- throws KeeperException {
- List<String> stack = new LinkedList<>();
- stack.add(znode);
- do {
- String znodeToProcess = stack.remove(stack.size() - 1);
- sb.append("\n").append(znodeToProcess).append(": ");
- byte[] data;
- try {
- data = ZKUtil.getData(zkw, znodeToProcess);
- } catch (InterruptedException e) {
- zkw.interruptedException(e);
- return;
- }
- if (data != null && data.length > 0) { // log position
- long position = 0;
- try {
- position = ZKUtil.parseWALPositionFrom(ZKUtil.getData(zkw, znodeToProcess));
- sb.append(position);
- } catch (DeserializationException ignored) {
- } catch (InterruptedException e) {
- zkw.interruptedException(e);
- return;
- }
- }
- for (String zNodeChild : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
- stack.add(ZNodePaths.joinZNode(znodeToProcess, zNodeChild));
- }
- } while (stack.size() > 0);
- }
-
- private static void appendPeersZnodes(ZooKeeperWatcher zkw, String peersZnode,
- StringBuilder sb) throws KeeperException {
- int pblen = ProtobufUtil.lengthOfPBMagic();
- sb.append("\n").append(peersZnode).append(": ");
- for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, peersZnode)) {
- String znodeToProcess = ZNodePaths.joinZNode(peersZnode, peerIdZnode);
- byte[] data;
- try {
- data = ZKUtil.getData(zkw, znodeToProcess);
- } catch (InterruptedException e) {
- zkw.interruptedException(e);
- return;
- }
- // parse the data of the above peer znode.
- try {
- ReplicationProtos.ReplicationPeer.Builder builder =
- ReplicationProtos.ReplicationPeer.newBuilder();
- ProtobufUtil.mergeFrom(builder, data, pblen, data.length - pblen);
- String clusterKey = builder.getClusterkey();
- sb.append("\n").append(znodeToProcess).append(": ").append(clusterKey);
- // add the peer-state.
- appendPeerState(zkw, znodeToProcess, sb);
- } catch (IOException ipbe) {
- LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe);
- }
- }
- }
-
- private static void appendPeerState(ZooKeeperWatcher zkw, String znodeToProcess,
- StringBuilder sb) throws KeeperException, InvalidProtocolBufferException {
- String peerState = zkw.getConfiguration().get("zookeeper.znode.replication.peers.state",
- "peer-state");
- int pblen = ProtobufUtil.lengthOfPBMagic();
- for (String child : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
- if (!child.equals(peerState)) continue;
- String peerStateZnode = ZNodePaths.joinZNode(znodeToProcess, child);
- sb.append("\n").append(peerStateZnode).append(": ");
- byte[] peerStateData;
- try {
- peerStateData = ZKUtil.getData(zkw, peerStateZnode);
- ReplicationProtos.ReplicationState.Builder builder =
- ReplicationProtos.ReplicationState.newBuilder();
- ProtobufUtil.mergeFrom(builder, peerStateData, pblen, peerStateData.length - pblen);
- sb.append(builder.getState().name());
- } catch (IOException ipbe) {
- LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe);
- } catch (InterruptedException e) {
- zkw.interruptedException(e);
- return;
- }
- }
- }
-
- /**
- * Gets the statistics from the given server.
- *
- * @param server The server to get the statistics from.
- * @param timeout The socket timeout to use.
- * @return The array of response strings.
- * @throws IOException When the socket communication fails.
- */
- public static String[] getServerStats(String server, int timeout)
- throws IOException {
- String[] sp = server.split(":");
- if (sp == null || sp.length == 0) {
- return null;
- }
-
- String host = sp[0];
- int port = sp.length > 1 ? Integer.parseInt(sp[1])
- : HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT;
-
- InetSocketAddress sockAddr = new InetSocketAddress(host, port);
- try (Socket socket = new Socket()) {
- socket.connect(sockAddr, timeout);
-
- socket.setSoTimeout(timeout);
- try (PrintWriter out = new PrintWriter(socket.getOutputStream(), true);
- BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream()))) {
- out.println("stat");
- out.flush();
- ArrayList<String> res = new ArrayList<>();
- while (true) {
- String line = in.readLine();
- if (line != null) {
- res.add(line);
- } else {
- break;
- }
- }
- return res.toArray(new String[res.size()]);
- }
- }
- }
-
- private static void logRetrievedMsg(final ZooKeeperWatcher zkw,
- final String znode, final byte [] data, final boolean watcherSet) {
- if (!LOG.isTraceEnabled()) return;
- LOG.trace(zkw.prefix("Retrieved " + ((data == null)? 0: data.length) +
- " byte(s) of data from znode " + znode +
- (watcherSet? " and set watcher; ": "; data=") +
- (data == null? "null": data.length == 0? "empty": (
- znode.startsWith(zkw.znodePaths.metaZNodePrefix)?
- getServerNameOrEmptyString(data):
- znode.startsWith(zkw.znodePaths.backupMasterAddressesZNode)?
- getServerNameOrEmptyString(data):
- StringUtils.abbreviate(Bytes.toStringBinary(data), 32)))));
- }
-
- private static String getServerNameOrEmptyString(final byte [] data) {
- try {
- return ProtobufUtil.parseServerNameFrom(data).toString();
- } catch (DeserializationException e) {
- return "";
- }
- }
-
- /**
- * Waits for HBase installation's base (parent) znode to become available.
- * @throws IOException on ZK errors
- */
- public static void waitForBaseZNode(Configuration conf) throws IOException {
- LOG.info("Waiting until the base znode is available");
- String parentZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
- HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
- ZooKeeper zk = new ZooKeeper(ZKConfig.getZKQuorumServersString(conf),
- conf.getInt(HConstants.ZK_SESSION_TIMEOUT,
- HConstants.DEFAULT_ZK_SESSION_TIMEOUT), EmptyWatcher.instance);
-
- final int maxTimeMs = 10000;
- final int maxNumAttempts = maxTimeMs / HConstants.SOCKET_RETRY_WAIT_MS;
-
- KeeperException keeperEx = null;
- try {
- try {
- for (int attempt = 0; attempt < maxNumAttempts; ++attempt) {
- try {
- if (zk.exists(parentZNode, false) != null) {
- LOG.info("Parent znode exists: " + parentZNode);
- keeperEx = null;
- break;
- }
- } catch (KeeperException e) {
- keeperEx = e;
- }
- Threads.sleepWithoutInterrupt(HConstants.SOCKET_RETRY_WAIT_MS);
- }
- } finally {
- zk.close();
- }
- } catch (InterruptedException ex) {
- Thread.currentThread().interrupt();
- }
-
- if (keeperEx != null) {
- throw new IOException(keeperEx);
- }
- }
-
- /**
- * Convert a {@link DeserializationException} to a more palatable {@link KeeperException}.
- * Used when can't let a {@link DeserializationException} out w/o changing public API.
- * @param e Exception to convert
- * @return Converted exception
- */
- public static KeeperException convert(final DeserializationException e) {
- KeeperException ke = new KeeperException.DataInconsistencyException();
- ke.initCause(e);
- return ke;
- }
-
- /**
- * Recursively print the current state of ZK (non-transactional)
- * @param root name of the root directory in zk to print
- */
- public static void logZKTree(ZooKeeperWatcher zkw, String root) {
- if (!LOG.isDebugEnabled()) return;
- LOG.debug("Current zk system:");
- String prefix = "|-";
- LOG.debug(prefix + root);
- try {
- logZKTree(zkw, root, prefix);
- } catch (KeeperException e) {
- throw new RuntimeException(e);
- }
- }
-
- /**
- * Helper method to print the current state of the ZK tree.
- * @see #logZKTree(ZooKeeperWatcher, String)
- * @throws KeeperException if an unexpected exception occurs
- */
- protected static void logZKTree(ZooKeeperWatcher zkw, String root, String prefix)
- throws KeeperException {
- List<String> children = ZKUtil.listChildrenNoWatch(zkw, root);
- if (children == null) return;
- for (String child : children) {
- LOG.debug(prefix + child);
- String node = ZNodePaths.joinZNode(root.equals("/") ? "" : root, child);
- logZKTree(zkw, node, prefix + "---");
- }
- }
-
- /**
- * @param position
- * @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.
- */
- public static byte[] positionToByteArray(final long position) {
- byte[] bytes = ReplicationProtos.ReplicationHLogPosition.newBuilder().setPosition(position)
- .build().toByteArray();
- return ProtobufUtil.prependPBMagic(bytes);
- }
-
- /**
- * @param bytes - Content of a WAL position znode.
- * @return long - The current WAL position.
- * @throws DeserializationException
- */
- public static long parseWALPositionFrom(final byte[] bytes) throws DeserializationException {
- if (bytes == null) {
- throw new DeserializationException("Unable to parse null WAL position.");
- }
- if (ProtobufUtil.isPBMagicPrefix(bytes)) {
- int pblen = ProtobufUtil.lengthOfPBMagic();
- ReplicationProtos.ReplicationHLogPosition.Builder builder =
- ReplicationProtos.ReplicationHLogPosition.newBuilder();
- ReplicationProtos.ReplicationHLogPosition position;
- try {
- ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
- position = builder.build();
- } catch (IOException e) {
- throw new DeserializationException(e);
- }
- return position.getPosition();
- } else {
- if (bytes.length > 0) {
- return Bytes.toLong(bytes);
- }
- return 0;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java
deleted file mode 100644
index 045fd97..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.ZooKeeper;
-
-/**
- * You may add the jaas.conf option
- * -Djava.security.auth.login.config=/PATH/jaas.conf
- *
- * You may also specify -D to set options
- * "hbase.zookeeper.quorum" (it should be in hbase-site.xml)
- * "zookeeper.znode.parent" (it should be in hbase-site.xml)
- *
- * Use -set-acls to set the ACLs, no option to erase ACLs
- */
-@InterfaceAudience.Private
-public class ZkAclReset extends Configured implements Tool {
- private static final Log LOG = LogFactory.getLog(ZkAclReset.class);
-
- private static void resetAcls(final ZooKeeperWatcher zkw, final String znode,
- final boolean eraseAcls) throws Exception {
- List<String> children = ZKUtil.listChildrenNoWatch(zkw, znode);
- if (children != null) {
- for (String child: children) {
- resetAcls(zkw, ZNodePaths.joinZNode(znode, child), eraseAcls);
- }
- }
-
- ZooKeeper zk = zkw.getRecoverableZooKeeper().getZooKeeper();
- if (eraseAcls) {
- LOG.info(" - erase ACLs for " + znode);
- zk.setACL(znode, ZooDefs.Ids.OPEN_ACL_UNSAFE, -1);
- } else {
- LOG.info(" - set ACLs for " + znode);
- zk.setACL(znode, ZKUtil.createACL(zkw, znode, true), -1);
- }
- }
-
- private static void resetAcls(final Configuration conf, boolean eraseAcls)
- throws Exception {
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "ZkAclReset", null);
- try {
- LOG.info((eraseAcls ? "Erase" : "Set") + " HBase ACLs for " +
- zkw.getQuorum() + " " + zkw.znodePaths.baseZNode);
- resetAcls(zkw, zkw.znodePaths.baseZNode, eraseAcls);
- } finally {
- zkw.close();
- }
- }
-
- private void printUsageAndExit() {
- System.err.printf("Usage: hbase %s [options]%n", getClass().getName());
- System.err.println(" where [options] are:");
- System.err.println(" -h|-help Show this help and exit.");
- System.err.println(" -set-acls Setup the hbase znode ACLs for a secure cluster");
- System.err.println();
- System.err.println("Examples:");
- System.err.println(" To reset the ACLs to the unsecure cluster behavior:");
- System.err.println(" hbase " + getClass().getName());
- System.err.println();
- System.err.println(" To reset the ACLs to the secure cluster behavior:");
- System.err.println(" hbase " + getClass().getName() + " -set-acls");
- System.exit(1);
- }
-
- @Override
- public int run(String[] args) throws Exception {
- boolean eraseAcls = true;
-
- for (int i = 0; i < args.length; ++i) {
- if (args[i].equals("-help")) {
- printUsageAndExit();
- } else if (args[i].equals("-set-acls")) {
- eraseAcls = false;
- } else {
- printUsageAndExit();
- }
- }
-
- resetAcls(getConf(), eraseAcls);
- return(0);
- }
-
- public static void main(String[] args) throws Exception {
- System.exit(ToolRunner.run(HBaseConfiguration.create(), new ZkAclReset(), args));
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java
deleted file mode 100644
index f78f1d8..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Base class for internal listeners of ZooKeeper events.
- *
- * The {@link ZooKeeperWatcher} for a process will execute the appropriate
- * methods of implementations of this class. In order to receive events from
- * the watcher, every listener must register itself via {@link ZooKeeperWatcher#registerListener}.
- *
- * Subclasses need only override those methods in which they are interested.
- *
- * Note that the watcher will be blocked when invoking methods in listeners so
- * they must not be long-running.
- */
-@InterfaceAudience.Private
-public abstract class ZooKeeperListener {
-
- // Reference to the zk watcher which also contains configuration and constants
- protected ZooKeeperWatcher watcher;
-
- /**
- * Construct a ZooKeeper event listener.
- */
- public ZooKeeperListener(ZooKeeperWatcher watcher) {
- this.watcher = watcher;
- }
-
- /**
- * Called when a new node has been created.
- * @param path full path of the new node
- */
- public void nodeCreated(String path) {
- // no-op
- }
-
- /**
- * Called when a node has been deleted
- * @param path full path of the deleted node
- */
- public void nodeDeleted(String path) {
- // no-op
- }
-
- /**
- * Called when an existing node has changed data.
- * @param path full path of the updated node
- */
- public void nodeDataChanged(String path) {
- // no-op
- }
-
- /**
- * Called when an existing node has a child node added or removed.
- * @param path full path of the node whose children have changed
- */
- public void nodeChildrenChanged(String path) {
- // no-op
- }
-
- /**
- * @return The watcher associated with this listener
- */
- public ZooKeeperWatcher getWatcher() {
- return this.watcher;
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java
deleted file mode 100644
index 12ce2cf..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.zookeeper;
-
-import org.apache.yetus.audience.InterfaceAudience;
-
-@InterfaceAudience.Private
-public interface ZooKeeperMetricsListener {
-
- /**
- * An AUTHFAILED Exception was seen.
- */
- void registerAuthFailedException();
-
- /**
- * A CONNECTIONLOSS Exception was seen.
- */
- void registerConnectionLossException();
-
- /**
- * A DATAINCONSISTENCY Exception was seen.
- */
- void registerDataInconsistencyException();
-
- /**
- * An INVALIDACL Exception was seen.
- */
- void registerInvalidACLException();
-
- /**
- * A NOAUTH Exception was seen.
- */
- void registerNoAuthException();
-
- /**
- * A OPERATIONTIMEOUT Exception was seen.
- */
- void registerOperationTimeoutException();
-
- /**
- * A RUNTIMEINCONSISTENCY Exception was seen.
- */
- void registerRuntimeInconsistencyException();
-
- /**
- * A SESSIONEXPIRED Exception was seen.
- */
- void registerSessionExpiredException();
-
- /**
- * A SYSTEMERROR Exception was seen.
- */
- void registerSystemErrorException();
-
- /**
- * A ZooKeeper API Call failed.
- */
- void registerFailedZKCall();
-
- /**
- * Register the latency incurred for read operations.
- */
- void registerReadOperationLatency(long latency);
-
- /**
- * Register the latency incurred for write operations.
- */
- void registerWriteOperationLatency(long latency);
-
- /**
- * Register the latency incurred for sync operations.
- */
- void registerSyncOperationLatency(long latency);
-}
[02/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper
from hbase-server and hbase-client
Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
new file mode 100644
index 0000000..d8472a9
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -0,0 +1,2072 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+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.Threads;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.DeleteNodeFailSilent;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.SetData;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.util.KerberosUtil;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooDefs.Perms;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.proto.CreateRequest;
+import org.apache.zookeeper.proto.DeleteRequest;
+import org.apache.zookeeper.proto.SetDataRequest;
+import org.apache.zookeeper.server.ZooKeeperSaslServer;
+
+/**
+ * Internal HBase utility class for ZooKeeper.
+ *
+ * <p>Contains only static methods and constants.
+ *
+ * <p>Methods all throw {@link KeeperException} if there is an unexpected
+ * zookeeper exception, so callers of these methods must handle appropriately.
+ * If ZK is required for the operation, the server will need to be aborted.
+ */
+@InterfaceAudience.Private
+public class ZKUtil {
+ private static final Log LOG = LogFactory.getLog(ZKUtil.class);
+
+ private static int zkDumpConnectionTimeOut;
+
+ /**
+ * Creates a new connection to ZooKeeper, pulling settings and ensemble config
+ * from the specified configuration object using methods from {@link ZKConfig}.
+ *
+ * Sets the connection status monitoring watcher to the specified watcher.
+ *
+ * @param conf configuration to pull ensemble and other settings from
+ * @param watcher watcher to monitor connection changes
+ * @return connection to zookeeper
+ * @throws IOException if unable to connect to zk or config problem
+ */
+ public static RecoverableZooKeeper connect(Configuration conf, Watcher watcher)
+ throws IOException {
+ String ensemble = ZKConfig.getZKQuorumServersString(conf);
+ return connect(conf, ensemble, watcher);
+ }
+
+ public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
+ Watcher watcher)
+ throws IOException {
+ return connect(conf, ensemble, watcher, null);
+ }
+
+ public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
+ Watcher watcher, final String identifier)
+ throws IOException {
+ if(ensemble == null) {
+ throw new IOException("Unable to determine ZooKeeper ensemble");
+ }
+ int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT,
+ HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(identifier + " opening connection to ZooKeeper ensemble=" + ensemble);
+ }
+ int retry = conf.getInt("zookeeper.recovery.retry", 3);
+ int retryIntervalMillis =
+ conf.getInt("zookeeper.recovery.retry.intervalmill", 1000);
+ int maxSleepTime = conf.getInt("zookeeper.recovery.retry.maxsleeptime", 60000);
+ zkDumpConnectionTimeOut = conf.getInt("zookeeper.dump.connection.timeout",
+ 1000);
+ return new RecoverableZooKeeper(ensemble, timeout, watcher,
+ retry, retryIntervalMillis, maxSleepTime, identifier);
+ }
+
+ /**
+ * Log in the current zookeeper server process using the given configuration
+ * keys for the credential file and login principal.
+ *
+ * <p><strong>This is only applicable when running on secure hbase</strong>
+ * On regular HBase (without security features), this will safely be ignored.
+ * </p>
+ *
+ * @param conf The configuration data to use
+ * @param keytabFileKey Property key used to configure the path to the credential file
+ * @param userNameKey Property key used to configure the login principal
+ * @param hostname Current hostname to use in any credentials
+ * @throws IOException underlying exception from SecurityUtil.login() call
+ */
+ public static void loginServer(Configuration conf, String keytabFileKey,
+ String userNameKey, String hostname) throws IOException {
+ login(conf, keytabFileKey, userNameKey, hostname,
+ ZooKeeperSaslServer.LOGIN_CONTEXT_NAME_KEY,
+ JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME);
+ }
+
+ /**
+ * Log in the current zookeeper client using the given configuration
+ * keys for the credential file and login principal.
+ *
+ * <p><strong>This is only applicable when running on secure hbase</strong>
+ * On regular HBase (without security features), this will safely be ignored.
+ * </p>
+ *
+ * @param conf The configuration data to use
+ * @param keytabFileKey Property key used to configure the path to the credential file
+ * @param userNameKey Property key used to configure the login principal
+ * @param hostname Current hostname to use in any credentials
+ * @throws IOException underlying exception from SecurityUtil.login() call
+ */
+ public static void loginClient(Configuration conf, String keytabFileKey,
+ String userNameKey, String hostname) throws IOException {
+ login(conf, keytabFileKey, userNameKey, hostname,
+ ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY,
+ JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME);
+ }
+
+ /**
+ * Log in the current process using the given configuration keys for the
+ * credential file and login principal.
+ *
+ * <p><strong>This is only applicable when running on secure hbase</strong>
+ * On regular HBase (without security features), this will safely be ignored.
+ * </p>
+ *
+ * @param conf The configuration data to use
+ * @param keytabFileKey Property key used to configure the path to the credential file
+ * @param userNameKey Property key used to configure the login principal
+ * @param hostname Current hostname to use in any credentials
+ * @param loginContextProperty property name to expose the entry name
+ * @param loginContextName jaas entry name
+ * @throws IOException underlying exception from SecurityUtil.login() call
+ */
+ private static void login(Configuration conf, String keytabFileKey,
+ String userNameKey, String hostname,
+ String loginContextProperty, String loginContextName)
+ throws IOException {
+ if (!isSecureZooKeeper(conf))
+ return;
+
+ // User has specified a jaas.conf, keep this one as the good one.
+ // HBASE_OPTS="-Djava.security.auth.login.config=jaas.conf"
+ if (System.getProperty("java.security.auth.login.config") != null)
+ return;
+
+ // No keytab specified, no auth
+ String keytabFilename = conf.get(keytabFileKey);
+ if (keytabFilename == null) {
+ LOG.warn("no keytab specified for: " + keytabFileKey);
+ return;
+ }
+
+ String principalConfig = conf.get(userNameKey, System.getProperty("user.name"));
+ String principalName = SecurityUtil.getServerPrincipal(principalConfig, hostname);
+
+ // Initialize the "jaas.conf" for keyTab/principal,
+ // If keyTab is not specified use the Ticket Cache.
+ // and set the zookeeper login context name.
+ JaasConfiguration jaasConf = new JaasConfiguration(loginContextName,
+ principalName, keytabFilename);
+ javax.security.auth.login.Configuration.setConfiguration(jaasConf);
+ System.setProperty(loginContextProperty, loginContextName);
+ }
+
+ /**
+ * A JAAS configuration that defines the login modules that we want to use for login.
+ */
+ private static class JaasConfiguration extends javax.security.auth.login.Configuration {
+ private static final String SERVER_KEYTAB_KERBEROS_CONFIG_NAME =
+ "zookeeper-server-keytab-kerberos";
+ private static final String CLIENT_KEYTAB_KERBEROS_CONFIG_NAME =
+ "zookeeper-client-keytab-kerberos";
+
+ private static final Map<String, String> BASIC_JAAS_OPTIONS = new HashMap<>();
+ static {
+ String jaasEnvVar = System.getenv("HBASE_JAAS_DEBUG");
+ if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) {
+ BASIC_JAAS_OPTIONS.put("debug", "true");
+ }
+ }
+
+ private static final Map<String,String> KEYTAB_KERBEROS_OPTIONS = new HashMap<>();
+ static {
+ KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true");
+ KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true");
+ KEYTAB_KERBEROS_OPTIONS.put("refreshKrb5Config", "true");
+ KEYTAB_KERBEROS_OPTIONS.putAll(BASIC_JAAS_OPTIONS);
+ }
+
+ private static final AppConfigurationEntry KEYTAB_KERBEROS_LOGIN =
+ new AppConfigurationEntry(KerberosUtil.getKrb5LoginModuleName(),
+ LoginModuleControlFlag.REQUIRED,
+ KEYTAB_KERBEROS_OPTIONS);
+
+ private static final AppConfigurationEntry[] KEYTAB_KERBEROS_CONF =
+ new AppConfigurationEntry[]{KEYTAB_KERBEROS_LOGIN};
+
+ private javax.security.auth.login.Configuration baseConfig;
+ private final String loginContextName;
+ private final boolean useTicketCache;
+ private final String keytabFile;
+ private final String principal;
+
+ public JaasConfiguration(String loginContextName, String principal, String keytabFile) {
+ this(loginContextName, principal, keytabFile, keytabFile == null || keytabFile.length() == 0);
+ }
+
+ private JaasConfiguration(String loginContextName, String principal,
+ String keytabFile, boolean useTicketCache) {
+ try {
+ this.baseConfig = javax.security.auth.login.Configuration.getConfiguration();
+ } catch (SecurityException e) {
+ this.baseConfig = null;
+ }
+ this.loginContextName = loginContextName;
+ this.useTicketCache = useTicketCache;
+ this.keytabFile = keytabFile;
+ this.principal = principal;
+ LOG.info("JaasConfiguration loginContextName=" + loginContextName +
+ " principal=" + principal + " useTicketCache=" + useTicketCache +
+ " keytabFile=" + keytabFile);
+ }
+
+ @Override
+ public AppConfigurationEntry[] getAppConfigurationEntry(String appName) {
+ if (loginContextName.equals(appName)) {
+ if (!useTicketCache) {
+ KEYTAB_KERBEROS_OPTIONS.put("keyTab", keytabFile);
+ KEYTAB_KERBEROS_OPTIONS.put("useKeyTab", "true");
+ }
+ KEYTAB_KERBEROS_OPTIONS.put("principal", principal);
+ KEYTAB_KERBEROS_OPTIONS.put("useTicketCache", useTicketCache ? "true" : "false");
+ return KEYTAB_KERBEROS_CONF;
+ }
+ if (baseConfig != null) return baseConfig.getAppConfigurationEntry(appName);
+ return(null);
+ }
+ }
+
+ //
+ // Helper methods
+ //
+ /**
+ * Returns the full path of the immediate parent of the specified node.
+ * @param node path to get parent of
+ * @return parent of path, null if passed the root node or an invalid node
+ */
+ public static String getParent(String node) {
+ int idx = node.lastIndexOf(ZNodePaths.ZNODE_PATH_SEPARATOR);
+ return idx <= 0 ? null : node.substring(0, idx);
+ }
+
+ /**
+ * Get the name of the current node from the specified fully-qualified path.
+ * @param path fully-qualified path
+ * @return name of the current node
+ */
+ public static String getNodeName(String path) {
+ return path.substring(path.lastIndexOf("/")+1);
+ }
+
+ //
+ // Existence checks and watches
+ //
+
+ /**
+ * Watch the specified znode for delete/create/change events. The watcher is
+ * set whether or not the node exists. If the node already exists, the method
+ * returns true. If the node does not exist, the method returns false.
+ *
+ * @param zkw zk reference
+ * @param znode path of node to watch
+ * @return true if znode exists, false if does not exist or error
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static boolean watchAndCheckExists(ZKWatcher zkw, String znode)
+ throws KeeperException {
+ try {
+ Stat s = zkw.getRecoverableZooKeeper().exists(znode, zkw);
+ boolean exists = s != null ? true : false;
+ if (exists) {
+ LOG.debug(zkw.prefix("Set watcher on existing znode=" + znode));
+ } else {
+ LOG.debug(zkw.prefix("Set watcher on znode that does not yet exist, " + znode));
+ }
+ return exists;
+ } catch (KeeperException e) {
+ LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
+ zkw.keeperException(e);
+ return false;
+ } catch (InterruptedException e) {
+ LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
+ zkw.interruptedException(e);
+ return false;
+ }
+ }
+
+ /**
+ * Watch the specified znode, but only if exists. Useful when watching
+ * for deletions. Uses .getData() (and handles NoNodeException) instead
+ * of .exists() to accomplish this, as .getData() will only set a watch if
+ * the znode exists.
+ * @param zkw zk reference
+ * @param znode path of node to watch
+ * @return true if the watch is set, false if node does not exists
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static boolean setWatchIfNodeExists(ZKWatcher zkw, String znode)
+ throws KeeperException {
+ try {
+ zkw.getRecoverableZooKeeper().getData(znode, true, null);
+ return true;
+ } catch (NoNodeException e) {
+ return false;
+ } catch (InterruptedException e) {
+ LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
+ zkw.interruptedException(e);
+ return false;
+ }
+ }
+
+ /**
+ * Check if the specified node exists. Sets no watches.
+ *
+ * @param zkw zk reference
+ * @param znode path of node to watch
+ * @return version of the node if it exists, -1 if does not exist
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static int checkExists(ZKWatcher zkw, String znode)
+ throws KeeperException {
+ try {
+ Stat s = zkw.getRecoverableZooKeeper().exists(znode, null);
+ return s != null ? s.getVersion() : -1;
+ } catch (KeeperException e) {
+ LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
+ zkw.keeperException(e);
+ return -1;
+ } catch (InterruptedException e) {
+ LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
+ zkw.interruptedException(e);
+ return -1;
+ }
+ }
+
+ //
+ // Znode listings
+ //
+
+ /**
+ * Lists the children znodes of the specified znode. Also sets a watch on
+ * the specified znode which will capture a NodeDeleted event on the specified
+ * znode as well as NodeChildrenChanged if any children of the specified znode
+ * are created or deleted.
+ *
+ * Returns null if the specified node does not exist. Otherwise returns a
+ * list of children of the specified node. If the node exists but it has no
+ * children, an empty list will be returned.
+ *
+ * @param zkw zk reference
+ * @param znode path of node to list and watch children of
+ * @return list of children of the specified node, an empty list if the node
+ * exists but has no children, and null if the node does not exist
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static List<String> listChildrenAndWatchForNewChildren(
+ ZKWatcher zkw, String znode)
+ throws KeeperException {
+ try {
+ List<String> children = zkw.getRecoverableZooKeeper().getChildren(znode, zkw);
+ return children;
+ } catch(KeeperException.NoNodeException ke) {
+ LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
+ "because node does not exist (not an error)"));
+ return null;
+ } catch (KeeperException e) {
+ LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
+ zkw.keeperException(e);
+ return null;
+ } catch (InterruptedException e) {
+ LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
+ zkw.interruptedException(e);
+ return null;
+ }
+ }
+
+ /**
+ * List all the children of the specified znode, setting a watch for children
+ * changes and also setting a watch on every individual child in order to get
+ * the NodeCreated and NodeDeleted events.
+ * @param zkw zookeeper reference
+ * @param znode node to get children of and watch
+ * @return list of znode names, null if the node doesn't exist
+ * @throws KeeperException
+ */
+ public static List<String> listChildrenAndWatchThem(ZKWatcher zkw,
+ String znode) throws KeeperException {
+ List<String> children = listChildrenAndWatchForNewChildren(zkw, znode);
+ if (children == null) {
+ return null;
+ }
+ for (String child : children) {
+ watchAndCheckExists(zkw, ZNodePaths.joinZNode(znode, child));
+ }
+ return children;
+ }
+
+ /**
+ * Lists the children of the specified znode without setting any watches.
+ *
+ * Sets no watches at all, this method is best effort.
+ *
+ * Returns an empty list if the node has no children. Returns null if the
+ * parent node itself does not exist.
+ *
+ * @param zkw zookeeper reference
+ * @param znode node to get children
+ * @return list of data of children of specified znode, empty if no children,
+ * null if parent does not exist
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static List<String> listChildrenNoWatch(ZKWatcher zkw, String znode)
+ throws KeeperException {
+ List<String> children = null;
+ try {
+ // List the children without watching
+ children = zkw.getRecoverableZooKeeper().getChildren(znode, null);
+ } catch(KeeperException.NoNodeException nne) {
+ return null;
+ } catch(InterruptedException ie) {
+ zkw.interruptedException(ie);
+ }
+ return children;
+ }
+
+ /**
+ * Simple class to hold a node path and node data.
+ * @deprecated Unused
+ */
+ @Deprecated
+ public static class NodeAndData {
+ private String node;
+ private byte [] data;
+ public NodeAndData(String node, byte [] data) {
+ this.node = node;
+ this.data = data;
+ }
+ public String getNode() {
+ return node;
+ }
+ public byte [] getData() {
+ return data;
+ }
+ @Override
+ public String toString() {
+ return node;
+ }
+ public boolean isEmpty() {
+ return (data == null || data.length == 0);
+ }
+ }
+
+ /**
+ * Checks if the specified znode has any children. Sets no watches.
+ *
+ * Returns true if the node exists and has children. Returns false if the
+ * node does not exist or if the node does not have any children.
+ *
+ * Used during master initialization to determine if the master is a
+ * failed-over-to master or the first master during initial cluster startup.
+ * If the directory for regionserver ephemeral nodes is empty then this is
+ * a cluster startup, if not then it is not cluster startup.
+ *
+ * @param zkw zk reference
+ * @param znode path of node to check for children of
+ * @return true if node has children, false if not or node does not exist
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static boolean nodeHasChildren(ZKWatcher zkw, String znode)
+ throws KeeperException {
+ try {
+ return !zkw.getRecoverableZooKeeper().getChildren(znode, null).isEmpty();
+ } catch(KeeperException.NoNodeException ke) {
+ LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
+ "because node does not exist (not an error)"));
+ return false;
+ } catch (KeeperException e) {
+ LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
+ zkw.keeperException(e);
+ return false;
+ } catch (InterruptedException e) {
+ LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
+ zkw.interruptedException(e);
+ return false;
+ }
+ }
+
+ /**
+ * Get the number of children of the specified node.
+ *
+ * If the node does not exist or has no children, returns 0.
+ *
+ * Sets no watches at all.
+ *
+ * @param zkw zk reference
+ * @param znode path of node to count children of
+ * @return number of children of specified node, 0 if none or parent does not
+ * exist
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static int getNumberOfChildren(ZKWatcher zkw, String znode)
+ throws KeeperException {
+ try {
+ Stat stat = zkw.getRecoverableZooKeeper().exists(znode, null);
+ return stat == null ? 0 : stat.getNumChildren();
+ } catch(KeeperException e) {
+ LOG.warn(zkw.prefix("Unable to get children of node " + znode));
+ zkw.keeperException(e);
+ } catch(InterruptedException e) {
+ zkw.interruptedException(e);
+ }
+ return 0;
+ }
+
+ //
+ // Data retrieval
+ //
+
+ /**
+ * Get znode data. Does not set a watcher.
+ * @return ZNode data, null if the node does not exist or if there is an
+ * error.
+ */
+ public static byte [] getData(ZKWatcher zkw, String znode)
+ throws KeeperException, InterruptedException {
+ try {
+ byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, null);
+ logRetrievedMsg(zkw, znode, data, false);
+ return data;
+ } catch (KeeperException.NoNodeException e) {
+ LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
+ "because node does not exist (not an error)"));
+ return null;
+ } catch (KeeperException e) {
+ LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
+ zkw.keeperException(e);
+ return null;
+ }
+ }
+
+ /**
+ * Get the data at the specified znode and set a watch.
+ *
+ * Returns the data and sets a watch if the node exists. Returns null and no
+ * watch is set if the node does not exist or there is an exception.
+ *
+ * @param zkw zk reference
+ * @param znode path of node
+ * @return data of the specified znode, or null
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static byte [] getDataAndWatch(ZKWatcher zkw, String znode)
+ throws KeeperException {
+ return getDataInternal(zkw, znode, null, true);
+ }
+
+ /**
+ * Get the data at the specified znode and set a watch.
+ *
+ * Returns the data and sets a watch if the node exists. Returns null and no
+ * watch is set if the node does not exist or there is an exception.
+ *
+ * @param zkw zk reference
+ * @param znode path of node
+ * @param stat object to populate the version of the znode
+ * @return data of the specified znode, or null
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static byte[] getDataAndWatch(ZKWatcher zkw, String znode,
+ Stat stat) throws KeeperException {
+ return getDataInternal(zkw, znode, stat, true);
+ }
+
+ private static byte[] getDataInternal(ZKWatcher zkw, String znode, Stat stat,
+ boolean watcherSet)
+ throws KeeperException {
+ try {
+ byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat);
+ logRetrievedMsg(zkw, znode, data, watcherSet);
+ return data;
+ } catch (KeeperException.NoNodeException e) {
+ // This log can get pretty annoying when we cycle on 100ms waits.
+ // Enable trace if you really want to see it.
+ LOG.trace(zkw.prefix("Unable to get data of znode " + znode + " " +
+ "because node does not exist (not an error)"));
+ return null;
+ } catch (KeeperException e) {
+ LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
+ zkw.keeperException(e);
+ return null;
+ } catch (InterruptedException e) {
+ LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
+ zkw.interruptedException(e);
+ return null;
+ }
+ }
+
+ /**
+ * Get the data at the specified znode without setting a watch.
+ *
+ * Returns the data if the node exists. Returns null if the node does not
+ * exist.
+ *
+ * Sets the stats of the node in the passed Stat object. Pass a null stat if
+ * not interested.
+ *
+ * @param zkw zk reference
+ * @param znode path of node
+ * @param stat node status to get if node exists
+ * @return data of the specified znode, or null if node does not exist
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static byte [] getDataNoWatch(ZKWatcher zkw, String znode,
+ Stat stat)
+ throws KeeperException {
+ try {
+ byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, stat);
+ logRetrievedMsg(zkw, znode, data, false);
+ return data;
+ } catch (KeeperException.NoNodeException e) {
+ LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
+ "because node does not exist (not necessarily an error)"));
+ return null;
+ } catch (KeeperException e) {
+ LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
+ zkw.keeperException(e);
+ return null;
+ } catch (InterruptedException e) {
+ LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
+ zkw.interruptedException(e);
+ return null;
+ }
+ }
+
+ /**
+ * Returns the date of child znodes of the specified znode. Also sets a watch on
+ * the specified znode which will capture a NodeDeleted event on the specified
+ * znode as well as NodeChildrenChanged if any children of the specified znode
+ * are created or deleted.
+ *
+ * Returns null if the specified node does not exist. Otherwise returns a
+ * list of children of the specified node. If the node exists but it has no
+ * children, an empty list will be returned.
+ *
+ * @param zkw zk reference
+ * @param baseNode path of node to list and watch children of
+ * @return list of data of children of the specified node, an empty list if the node
+ * exists but has no children, and null if the node does not exist
+ * @throws KeeperException if unexpected zookeeper exception
+ * @deprecated Unused
+ */
+ @Deprecated
+ public static List<NodeAndData> getChildDataAndWatchForNewChildren(
+ ZKWatcher zkw, String baseNode) throws KeeperException {
+ List<String> nodes =
+ ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode);
+ if (nodes != null) {
+ List<NodeAndData> newNodes = new ArrayList<>();
+ for (String node : nodes) {
+ String nodePath = ZNodePaths.joinZNode(baseNode, node);
+ byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath);
+ newNodes.add(new NodeAndData(nodePath, data));
+ }
+ return newNodes;
+ }
+ return null;
+ }
+
+ /**
+ * Update the data of an existing node with the expected version to have the
+ * specified data.
+ *
+ * Throws an exception if there is a version mismatch or some other problem.
+ *
+ * Sets no watches under any conditions.
+ *
+ * @param zkw zk reference
+ * @param znode
+ * @param data
+ * @param expectedVersion
+ * @throws KeeperException if unexpected zookeeper exception
+ * @throws KeeperException.BadVersionException if version mismatch
+ * @deprecated Unused
+ */
+ @Deprecated
+ public static void updateExistingNodeData(ZKWatcher zkw, String znode,
+ byte [] data, int expectedVersion)
+ throws KeeperException {
+ try {
+ zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion);
+ } catch(InterruptedException ie) {
+ zkw.interruptedException(ie);
+ }
+ }
+
+ //
+ // Data setting
+ //
+
+ /**
+ * Sets the data of the existing znode to be the specified data. Ensures that
+ * the current data has the specified expected version.
+ *
+ * <p>If the node does not exist, a {@link NoNodeException} will be thrown.
+ *
+ * <p>If their is a version mismatch, method returns null.
+ *
+ * <p>No watches are set but setting data will trigger other watchers of this
+ * node.
+ *
+ * <p>If there is another problem, a KeeperException will be thrown.
+ *
+ * @param zkw zk reference
+ * @param znode path of node
+ * @param data data to set for node
+ * @param expectedVersion version expected when setting data
+ * @return true if data set, false if version mismatch
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static boolean setData(ZKWatcher zkw, String znode,
+ byte [] data, int expectedVersion)
+ throws KeeperException, KeeperException.NoNodeException {
+ try {
+ return zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion) != null;
+ } catch (InterruptedException e) {
+ zkw.interruptedException(e);
+ return false;
+ }
+ }
+
+ /**
+ * Set data into node creating node if it doesn't yet exist.
+ * Does not set watch.
+ *
+ * @param zkw zk reference
+ * @param znode path of node
+ * @param data data to set for node
+ * @throws KeeperException
+ */
+ public static void createSetData(final ZKWatcher zkw, final String znode,
+ final byte [] data)
+ throws KeeperException {
+ if (checkExists(zkw, znode) == -1) {
+ ZKUtil.createWithParents(zkw, znode, data);
+ } else {
+ ZKUtil.setData(zkw, znode, data);
+ }
+ }
+
+ /**
+ * Sets the data of the existing znode to be the specified data. The node
+ * must exist but no checks are done on the existing data or version.
+ *
+ * <p>If the node does not exist, a {@link NoNodeException} will be thrown.
+ *
+ * <p>No watches are set but setting data will trigger other watchers of this
+ * node.
+ *
+ * <p>If there is another problem, a KeeperException will be thrown.
+ *
+ * @param zkw zk reference
+ * @param znode path of node
+ * @param data data to set for node
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static void setData(ZKWatcher zkw, String znode, byte [] data)
+ throws KeeperException, KeeperException.NoNodeException {
+ setData(zkw, (SetData)ZKUtilOp.setData(znode, data));
+ }
+
+ private static void setData(ZKWatcher zkw, SetData setData)
+ throws KeeperException, KeeperException.NoNodeException {
+ SetDataRequest sd = (SetDataRequest)toZooKeeperOp(zkw, setData).toRequestRecord();
+ setData(zkw, sd.getPath(), sd.getData(), sd.getVersion());
+ }
+
+ /**
+ * Returns whether or not secure authentication is enabled
+ * (whether <code>hbase.security.authentication</code> is set to
+ * <code>kerberos</code>.
+ */
+ public static boolean isSecureZooKeeper(Configuration conf) {
+ // Detection for embedded HBase client with jaas configuration
+ // defined for third party programs.
+ try {
+ javax.security.auth.login.Configuration testConfig =
+ javax.security.auth.login.Configuration.getConfiguration();
+ if (testConfig.getAppConfigurationEntry("Client") == null
+ && testConfig.getAppConfigurationEntry(
+ JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME) == null
+ && testConfig.getAppConfigurationEntry(
+ JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME) == null
+ && conf.get(HConstants.ZK_CLIENT_KERBEROS_PRINCIPAL) == null
+ && conf.get(HConstants.ZK_SERVER_KERBEROS_PRINCIPAL) == null) {
+
+ return false;
+ }
+ } catch(Exception e) {
+ // No Jaas configuration defined.
+ return false;
+ }
+
+ // Master & RSs uses hbase.zookeeper.client.*
+ return "kerberos".equalsIgnoreCase(conf.get("hbase.security.authentication"));
+ }
+
+ private static ArrayList<ACL> createACL(ZKWatcher zkw, String node) {
+ return createACL(zkw, node, isSecureZooKeeper(zkw.getConfiguration()));
+ }
+
+ public static ArrayList<ACL> createACL(ZKWatcher zkw, String node,
+ boolean isSecureZooKeeper) {
+ if (!node.startsWith(zkw.znodePaths.baseZNode)) {
+ return Ids.OPEN_ACL_UNSAFE;
+ }
+ if (isSecureZooKeeper) {
+ ArrayList<ACL> acls = new ArrayList<>();
+ // add permission to hbase supper user
+ String[] superUsers = zkw.getConfiguration().getStrings(Superusers.SUPERUSER_CONF_KEY);
+ String hbaseUser = null;
+ try {
+ hbaseUser = UserGroupInformation.getCurrentUser().getShortUserName();
+ } catch (IOException e) {
+ LOG.debug("Could not acquire current User.", e);
+ }
+ if (superUsers != null) {
+ List<String> groups = new ArrayList<>();
+ for (String user : superUsers) {
+ if (AuthUtil.isGroupPrincipal(user)) {
+ // TODO: Set node ACL for groups when ZK supports this feature
+ groups.add(user);
+ } else {
+ if(!user.equals(hbaseUser)) {
+ acls.add(new ACL(Perms.ALL, new Id("sasl", user)));
+ }
+ }
+ }
+ if (!groups.isEmpty()) {
+ LOG.warn("Znode ACL setting for group " + groups
+ + " is skipped, ZooKeeper doesn't support this feature presently.");
+ }
+ }
+ // Certain znodes are accessed directly by the client,
+ // so they must be readable by non-authenticated clients
+ if (zkw.isClientReadable(node)) {
+ acls.addAll(Ids.CREATOR_ALL_ACL);
+ acls.addAll(Ids.READ_ACL_UNSAFE);
+ } else {
+ acls.addAll(Ids.CREATOR_ALL_ACL);
+ }
+ return acls;
+ } else {
+ return Ids.OPEN_ACL_UNSAFE;
+ }
+ }
+
+ //
+ // Node creation
+ //
+
+ /**
+ *
+ * Set the specified znode to be an ephemeral node carrying the specified
+ * data.
+ *
+ * If the node is created successfully, a watcher is also set on the node.
+ *
+ * If the node is not created successfully because it already exists, this
+ * method will also set a watcher on the node.
+ *
+ * If there is another problem, a KeeperException will be thrown.
+ *
+ * @param zkw zk reference
+ * @param znode path of node
+ * @param data data of node
+ * @return true if node created, false if not, watch set in both cases
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static boolean createEphemeralNodeAndWatch(ZKWatcher zkw,
+ String znode, byte [] data)
+ throws KeeperException {
+ boolean ret = true;
+ try {
+ zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
+ CreateMode.EPHEMERAL);
+ } catch (KeeperException.NodeExistsException nee) {
+ ret = false;
+ } catch (InterruptedException e) {
+ LOG.info("Interrupted", e);
+ Thread.currentThread().interrupt();
+ }
+ if(!watchAndCheckExists(zkw, znode)) {
+ // It did exist but now it doesn't, try again
+ return createEphemeralNodeAndWatch(zkw, znode, data);
+ }
+ return ret;
+ }
+
+ /**
+ * Creates the specified znode to be a persistent node carrying the specified
+ * data.
+ *
+ * Returns true if the node was successfully created, false if the node
+ * already existed.
+ *
+ * If the node is created successfully, a watcher is also set on the node.
+ *
+ * If the node is not created successfully because it already exists, this
+ * method will also set a watcher on the node but return false.
+ *
+ * If there is another problem, a KeeperException will be thrown.
+ *
+ * @param zkw zk reference
+ * @param znode path of node
+ * @param data data of node
+ * @return true if node created, false if not, watch set in both cases
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static boolean createNodeIfNotExistsAndWatch(
+ ZKWatcher zkw, String znode, byte [] data)
+ throws KeeperException {
+ boolean ret = true;
+ try {
+ zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
+ CreateMode.PERSISTENT);
+ } catch (KeeperException.NodeExistsException nee) {
+ ret = false;
+ } catch (InterruptedException e) {
+ zkw.interruptedException(e);
+ return false;
+ }
+ try {
+ zkw.getRecoverableZooKeeper().exists(znode, zkw);
+ } catch (InterruptedException e) {
+ zkw.interruptedException(e);
+ return false;
+ }
+ return ret;
+ }
+
+ /**
+ * Creates the specified znode with the specified data but does not watch it.
+ *
+ * Returns the znode of the newly created node
+ *
+ * If there is another problem, a KeeperException will be thrown.
+ *
+ * @param zkw zk reference
+ * @param znode path of node
+ * @param data data of node
+ * @param createMode specifying whether the node to be created is ephemeral and/or sequential
+ * @return true name of the newly created znode or null
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static String createNodeIfNotExistsNoWatch(ZKWatcher zkw, String znode,
+ byte[] data, CreateMode createMode) throws KeeperException {
+
+ String createdZNode = null;
+ try {
+ createdZNode = zkw.getRecoverableZooKeeper().create(znode, data,
+ createACL(zkw, znode), createMode);
+ } catch (KeeperException.NodeExistsException nee) {
+ return znode;
+ } catch (InterruptedException e) {
+ zkw.interruptedException(e);
+ return null;
+ }
+ return createdZNode;
+ }
+
+ /**
+ * Creates the specified node with the specified data and watches it.
+ *
+ * <p>Throws an exception if the node already exists.
+ *
+ * <p>The node created is persistent and open access.
+ *
+ * <p>Returns the version number of the created node if successful.
+ *
+ * @param zkw zk reference
+ * @param znode path of node to create
+ * @param data data of node to create
+ * @return version of node created
+ * @throws KeeperException if unexpected zookeeper exception
+ * @throws KeeperException.NodeExistsException if node already exists
+ */
+ public static int createAndWatch(ZKWatcher zkw,
+ String znode, byte [] data)
+ throws KeeperException, KeeperException.NodeExistsException {
+ try {
+ zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
+ CreateMode.PERSISTENT);
+ Stat stat = zkw.getRecoverableZooKeeper().exists(znode, zkw);
+ if (stat == null){
+ // Likely a race condition. Someone deleted the znode.
+ throw KeeperException.create(KeeperException.Code.SYSTEMERROR,
+ "ZK.exists returned null (i.e.: znode does not exist) for znode=" + znode);
+ }
+ return stat.getVersion();
+ } catch (InterruptedException e) {
+ zkw.interruptedException(e);
+ return -1;
+ }
+ }
+
+ /**
+ * Async creates the specified node with the specified data.
+ *
+ * <p>Throws an exception if the node already exists.
+ *
+ * <p>The node created is persistent and open access.
+ *
+ * @param zkw zk reference
+ * @param znode path of node to create
+ * @param data data of node to create
+ * @param cb
+ * @param ctx
+ */
+ public static void asyncCreate(ZKWatcher zkw,
+ String znode, byte [] data, final AsyncCallback.StringCallback cb,
+ final Object ctx) {
+ zkw.getRecoverableZooKeeper().getZooKeeper().create(znode, data,
+ createACL(zkw, znode), CreateMode.PERSISTENT, cb, ctx);
+ }
+
+ /**
+ * Creates the specified node, iff the node does not exist. Does not set a
+ * watch and fails silently if the node already exists.
+ *
+ * The node created is persistent and open access.
+ *
+ * @param zkw zk reference
+ * @param znode path of node
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static void createAndFailSilent(ZKWatcher zkw,
+ String znode) throws KeeperException {
+ createAndFailSilent(zkw, znode, new byte[0]);
+ }
+
+ /**
+ * Creates the specified node containing specified data, iff the node does not exist. Does
+ * not set a watch and fails silently if the node already exists.
+ *
+ * The node created is persistent and open access.
+ *
+ * @param zkw zk reference
+ * @param znode path of node
+ * @param data a byte array data to store in the znode
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static void createAndFailSilent(ZKWatcher zkw,
+ String znode, byte[] data)
+ throws KeeperException {
+ createAndFailSilent(zkw,
+ (CreateAndFailSilent)ZKUtilOp.createAndFailSilent(znode, data));
+ }
+
+ private static void createAndFailSilent(ZKWatcher zkw, CreateAndFailSilent cafs)
+ throws KeeperException {
+ CreateRequest create = (CreateRequest)toZooKeeperOp(zkw, cafs).toRequestRecord();
+ String znode = create.getPath();
+ try {
+ RecoverableZooKeeper zk = zkw.getRecoverableZooKeeper();
+ if (zk.exists(znode, false) == null) {
+ zk.create(znode, create.getData(), create.getAcl(), CreateMode.fromFlag(create.getFlags()));
+ }
+ } catch(KeeperException.NodeExistsException nee) {
+ } catch(KeeperException.NoAuthException nee){
+ try {
+ if (null == zkw.getRecoverableZooKeeper().exists(znode, false)) {
+ // If we failed to create the file and it does not already exist.
+ throw(nee);
+ }
+ } catch (InterruptedException ie) {
+ zkw.interruptedException(ie);
+ }
+ } catch(InterruptedException ie) {
+ zkw.interruptedException(ie);
+ }
+ }
+
+ /**
+ * Creates the specified node and all parent nodes required for it to exist.
+ *
+ * No watches are set and no errors are thrown if the node already exists.
+ *
+ * The nodes created are persistent and open access.
+ *
+ * @param zkw zk reference
+ * @param znode path of node
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static void createWithParents(ZKWatcher zkw, String znode)
+ throws KeeperException {
+ createWithParents(zkw, znode, new byte[0]);
+ }
+
+ /**
+ * Creates the specified node and all parent nodes required for it to exist. The creation of
+ * parent znodes is not atomic with the leafe znode creation but the data is written atomically
+ * when the leaf node is created.
+ *
+ * No watches are set and no errors are thrown if the node already exists.
+ *
+ * The nodes created are persistent and open access.
+ *
+ * @param zkw zk reference
+ * @param znode path of node
+ * @throws KeeperException if unexpected zookeeper exception
+ */
+ public static void createWithParents(ZKWatcher zkw, String znode, byte[] data)
+ throws KeeperException {
+ try {
+ if(znode == null) {
+ return;
+ }
+ zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
+ CreateMode.PERSISTENT);
+ } catch(KeeperException.NodeExistsException nee) {
+ return;
+ } catch(KeeperException.NoNodeException nne) {
+ createWithParents(zkw, getParent(znode));
+ createWithParents(zkw, znode, data);
+ } catch(InterruptedException ie) {
+ zkw.interruptedException(ie);
+ }
+ }
+
+ //
+ // Deletes
+ //
+
+ /**
+ * Delete the specified node. Sets no watches. Throws all exceptions.
+ */
+ public static void deleteNode(ZKWatcher zkw, String node)
+ throws KeeperException {
+ deleteNode(zkw, node, -1);
+ }
+
+ /**
+ * Delete the specified node with the specified version. Sets no watches.
+ * Throws all exceptions.
+ */
+ public static boolean deleteNode(ZKWatcher zkw, String node,
+ int version)
+ throws KeeperException {
+ try {
+ zkw.getRecoverableZooKeeper().delete(node, version);
+ return true;
+ } catch(KeeperException.BadVersionException bve) {
+ return false;
+ } catch(InterruptedException ie) {
+ zkw.interruptedException(ie);
+ return false;
+ }
+ }
+
+ /**
+ * Deletes the specified node. Fails silent if the node does not exist.
+ * @param zkw
+ * @param node
+ * @throws KeeperException
+ */
+ public static void deleteNodeFailSilent(ZKWatcher zkw, String node)
+ throws KeeperException {
+ deleteNodeFailSilent(zkw,
+ (DeleteNodeFailSilent)ZKUtilOp.deleteNodeFailSilent(node));
+ }
+
+ private static void deleteNodeFailSilent(ZKWatcher zkw,
+ DeleteNodeFailSilent dnfs) throws KeeperException {
+ DeleteRequest delete = (DeleteRequest)toZooKeeperOp(zkw, dnfs).toRequestRecord();
+ try {
+ zkw.getRecoverableZooKeeper().delete(delete.getPath(), delete.getVersion());
+ } catch(KeeperException.NoNodeException nne) {
+ } catch(InterruptedException ie) {
+ zkw.interruptedException(ie);
+ }
+ }
+
+
+ /**
+ * Delete the specified node and all of it's children.
+ * <p>
+ * If the node does not exist, just returns.
+ * <p>
+ * Sets no watches. Throws all exceptions besides dealing with deletion of
+ * children.
+ */
+ public static void deleteNodeRecursively(ZKWatcher zkw, String node)
+ throws KeeperException {
+ deleteNodeRecursivelyMultiOrSequential(zkw, true, node);
+ }
+
+ /**
+ * Delete all the children of the specified node but not the node itself.
+ *
+ * Sets no watches. Throws all exceptions besides dealing with deletion of
+ * children.
+ *
+ * @throws KeeperException
+ */
+ public static void deleteChildrenRecursively(ZKWatcher zkw, String node)
+ throws KeeperException {
+ deleteChildrenRecursivelyMultiOrSequential(zkw, true, node);
+ }
+
+ /**
+ * Delete all the children of the specified node but not the node itself. This
+ * will first traverse the znode tree for listing the children and then delete
+ * these znodes using multi-update api or sequential based on the specified
+ * configurations.
+ * <p>
+ * Sets no watches. Throws all exceptions besides dealing with deletion of
+ * children.
+ * <p>
+ * If the following is true:
+ * <ul>
+ * <li>runSequentialOnMultiFailure is true
+ * </ul>
+ * on calling multi, we get a ZooKeeper exception that can be handled by a
+ * sequential call(*), we retry the operations one-by-one (sequentially).
+ *
+ * @param zkw
+ * - zk reference
+ * @param runSequentialOnMultiFailure
+ * - if true when we get a ZooKeeper exception that could retry the
+ * operations one-by-one (sequentially)
+ * @param pathRoots
+ * - path of the parent node(s)
+ * @throws KeeperException.NotEmptyException
+ * if node has children while deleting
+ * @throws KeeperException
+ * if unexpected ZooKeeper exception
+ * @throws IllegalArgumentException
+ * if an invalid path is specified
+ */
+ public static void deleteChildrenRecursivelyMultiOrSequential(
+ ZKWatcher zkw, boolean runSequentialOnMultiFailure,
+ String... pathRoots) throws KeeperException {
+ if (pathRoots == null || pathRoots.length <= 0) {
+ LOG.warn("Given path is not valid!");
+ return;
+ }
+ List<ZKUtilOp> ops = new ArrayList<>();
+ for (String eachRoot : pathRoots) {
+ List<String> children = listChildrenBFSNoWatch(zkw, eachRoot);
+ // Delete the leaves first and eventually get rid of the root
+ for (int i = children.size() - 1; i >= 0; --i) {
+ ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i)));
+ }
+ }
+ // atleast one element should exist
+ if (ops.size() > 0) {
+ multiOrSequential(zkw, ops, runSequentialOnMultiFailure);
+ }
+ }
+
+ /**
+ * Delete the specified node and its children. This traverse the
+ * znode tree for listing the children and then delete
+ * these znodes including the parent using multi-update api or
+ * sequential based on the specified configurations.
+ * <p>
+ * Sets no watches. Throws all exceptions besides dealing with deletion of
+ * children.
+ * <p>
+ * If the following is true:
+ * <ul>
+ * <li>runSequentialOnMultiFailure is true
+ * </ul>
+ * on calling multi, we get a ZooKeeper exception that can be handled by a
+ * sequential call(*), we retry the operations one-by-one (sequentially).
+ *
+ * @param zkw
+ * - zk reference
+ * @param runSequentialOnMultiFailure
+ * - if true when we get a ZooKeeper exception that could retry the
+ * operations one-by-one (sequentially)
+ * @param pathRoots
+ * - path of the parent node(s)
+ * @throws KeeperException.NotEmptyException
+ * if node has children while deleting
+ * @throws KeeperException
+ * if unexpected ZooKeeper exception
+ * @throws IllegalArgumentException
+ * if an invalid path is specified
+ */
+ public static void deleteNodeRecursivelyMultiOrSequential(ZKWatcher zkw,
+ boolean runSequentialOnMultiFailure, String... pathRoots) throws KeeperException {
+ if (pathRoots == null || pathRoots.length <= 0) {
+ LOG.warn("Given path is not valid!");
+ return;
+ }
+ List<ZKUtilOp> ops = new ArrayList<>();
+ for (String eachRoot : pathRoots) {
+ // ZooKeeper Watches are one time triggers; When children of parent nodes are deleted
+ // recursively, must set another watch, get notified of delete node
+ List<String> children = listChildrenBFSAndWatchThem(zkw, eachRoot);
+ // Delete the leaves first and eventually get rid of the root
+ for (int i = children.size() - 1; i >= 0; --i) {
+ ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i)));
+ }
+ try {
+ if (zkw.getRecoverableZooKeeper().exists(eachRoot, zkw) != null) {
+ ops.add(ZKUtilOp.deleteNodeFailSilent(eachRoot));
+ }
+ } catch (InterruptedException e) {
+ zkw.interruptedException(e);
+ }
+ }
+ // atleast one element should exist
+ if (ops.size() > 0) {
+ multiOrSequential(zkw, ops, runSequentialOnMultiFailure);
+ }
+ }
+
+ /**
+ * BFS Traversal of all the children under path, with the entries in the list,
+ * in the same order as that of the traversal. Lists all the children without
+ * setting any watches.
+ *
+ * @param zkw
+ * - zk reference
+ * @param znode
+ * - path of node
+ * @return list of children znodes under the path
+ * @throws KeeperException
+ * if unexpected ZooKeeper exception
+ */
+ private static List<String> listChildrenBFSNoWatch(ZKWatcher zkw,
+ final String znode) throws KeeperException {
+ Deque<String> queue = new LinkedList<>();
+ List<String> tree = new ArrayList<>();
+ queue.add(znode);
+ while (true) {
+ String node = queue.pollFirst();
+ if (node == null) {
+ break;
+ }
+ List<String> children = listChildrenNoWatch(zkw, node);
+ if (children == null) {
+ continue;
+ }
+ for (final String child : children) {
+ final String childPath = node + "/" + child;
+ queue.add(childPath);
+ tree.add(childPath);
+ }
+ }
+ return tree;
+ }
+
+ /**
+ * BFS Traversal of all the children under path, with the entries in the list,
+ * in the same order as that of the traversal.
+ * Lists all the children and set watches on to them.
+ *
+ * @param zkw
+ * - zk reference
+ * @param znode
+ * - path of node
+ * @return list of children znodes under the path
+ * @throws KeeperException
+ * if unexpected ZooKeeper exception
+ */
+ private static List<String> listChildrenBFSAndWatchThem(ZKWatcher zkw, final String znode)
+ throws KeeperException {
+ Deque<String> queue = new LinkedList<>();
+ List<String> tree = new ArrayList<>();
+ queue.add(znode);
+ while (true) {
+ String node = queue.pollFirst();
+ if (node == null) {
+ break;
+ }
+ List<String> children = listChildrenAndWatchThem(zkw, node);
+ if (children == null) {
+ continue;
+ }
+ for (final String child : children) {
+ final String childPath = node + "/" + child;
+ queue.add(childPath);
+ tree.add(childPath);
+ }
+ }
+ return tree;
+ }
+
+ /**
+ * Represents an action taken by ZKUtil, e.g. createAndFailSilent.
+ * These actions are higher-level than ZKOp actions, which represent
+ * individual actions in the ZooKeeper API, like create.
+ */
+ public abstract static class ZKUtilOp {
+ private String path;
+
+ private ZKUtilOp(String path) {
+ this.path = path;
+ }
+
+ /**
+ * @return a createAndFailSilent ZKUtilOp
+ */
+ public static ZKUtilOp createAndFailSilent(String path, byte[] data) {
+ return new CreateAndFailSilent(path, data);
+ }
+
+ /**
+ * @return a deleteNodeFailSilent ZKUtilOP
+ */
+ public static ZKUtilOp deleteNodeFailSilent(String path) {
+ return new DeleteNodeFailSilent(path);
+ }
+
+ /**
+ * @return a setData ZKUtilOp
+ */
+ public static ZKUtilOp setData(String path, byte [] data) {
+ return new SetData(path, data);
+ }
+
+ /**
+ * @return path to znode where the ZKOp will occur
+ */
+ public String getPath() {
+ return path;
+ }
+
+ /**
+ * ZKUtilOp representing createAndFailSilent in ZooKeeper
+ * (attempt to create node, ignore error if already exists)
+ */
+ public static class CreateAndFailSilent extends ZKUtilOp {
+ private byte [] data;
+
+ private CreateAndFailSilent(String path, byte [] data) {
+ super(path);
+ this.data = data;
+ }
+
+ public byte[] getData() {
+ return data;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (!(o instanceof CreateAndFailSilent)) return false;
+
+ CreateAndFailSilent op = (CreateAndFailSilent) o;
+ return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
+ }
+
+ @Override
+ public int hashCode() {
+ int ret = 17 + getPath().hashCode() * 31;
+ return ret * 31 + Bytes.hashCode(data);
+ }
+ }
+
+ /**
+ * ZKUtilOp representing deleteNodeFailSilent in ZooKeeper
+ * (attempt to delete node, ignore error if node doesn't exist)
+ */
+ public static class DeleteNodeFailSilent extends ZKUtilOp {
+ private DeleteNodeFailSilent(String path) {
+ super(path);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (!(o instanceof DeleteNodeFailSilent)) return false;
+
+ return super.equals(o);
+ }
+
+ @Override
+ public int hashCode() {
+ return getPath().hashCode();
+ }
+ }
+
+ /**
+ * ZKUtilOp representing setData in ZooKeeper
+ */
+ public static class SetData extends ZKUtilOp {
+ private byte [] data;
+
+ private SetData(String path, byte [] data) {
+ super(path);
+ this.data = data;
+ }
+
+ public byte[] getData() {
+ return data;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (!(o instanceof SetData)) return false;
+
+ SetData op = (SetData) o;
+ return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
+ }
+
+ @Override
+ public int hashCode() {
+ int ret = getPath().hashCode();
+ return ret * 31 + Bytes.hashCode(data);
+ }
+ }
+ }
+
+ /**
+ * Convert from ZKUtilOp to ZKOp
+ */
+ private static Op toZooKeeperOp(ZKWatcher zkw, ZKUtilOp op)
+ throws UnsupportedOperationException {
+ if(op == null) return null;
+
+ if (op instanceof CreateAndFailSilent) {
+ CreateAndFailSilent cafs = (CreateAndFailSilent)op;
+ return Op.create(cafs.getPath(), cafs.getData(), createACL(zkw, cafs.getPath()),
+ CreateMode.PERSISTENT);
+ } else if (op instanceof DeleteNodeFailSilent) {
+ DeleteNodeFailSilent dnfs = (DeleteNodeFailSilent)op;
+ return Op.delete(dnfs.getPath(), -1);
+ } else if (op instanceof SetData) {
+ SetData sd = (SetData)op;
+ return Op.setData(sd.getPath(), sd.getData(), -1);
+ } else {
+ throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
+ + op.getClass().getName());
+ }
+ }
+
+ /**
+ * Use ZooKeeper's multi-update functionality.
+ *
+ * If all of the following are true:
+ * - runSequentialOnMultiFailure is true
+ * - on calling multi, we get a ZooKeeper exception that can be handled by a sequential call(*)
+ * Then:
+ * - we retry the operations one-by-one (sequentially)
+ *
+ * Note *: an example is receiving a NodeExistsException from a "create" call. Without multi,
+ * a user could call "createAndFailSilent" to ensure that a node exists if they don't care who
+ * actually created the node (i.e. the NodeExistsException from ZooKeeper is caught).
+ * This will cause all operations in the multi to fail, however, because
+ * the NodeExistsException that zk.create throws will fail the multi transaction.
+ * 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.
+ *
+ * @throws KeeperException
+ */
+ public static void multiOrSequential(ZKWatcher zkw, List<ZKUtilOp> ops,
+ boolean runSequentialOnMultiFailure) throws KeeperException {
+ if (zkw.getConfiguration().get("hbase.zookeeper.useMulti") != null) {
+ LOG.warn("hbase.zookeeper.useMulti is deprecated. Default to true always.");
+ }
+ if (ops == null) return;
+
+ List<Op> zkOps = new LinkedList<>();
+ for (ZKUtilOp op : ops) {
+ zkOps.add(toZooKeeperOp(zkw, op));
+ }
+ try {
+ zkw.getRecoverableZooKeeper().multi(zkOps);
+ } catch (KeeperException ke) {
+ switch (ke.code()) {
+ case NODEEXISTS:
+ case NONODE:
+ case BADVERSION:
+ case NOAUTH:
+ // if we get an exception that could be solved by running sequentially
+ // (and the client asked us to), then break out and run sequentially
+ if (runSequentialOnMultiFailure) {
+ LOG.info("On call to ZK.multi, received exception: " + ke.toString() + "."
+ + " Attempting to run operations sequentially because"
+ + " runSequentialOnMultiFailure is: " + runSequentialOnMultiFailure + ".");
+ processSequentially(zkw, ops);
+ break;
+ }
+ default:
+ throw ke;
+ }
+ } catch (InterruptedException ie) {
+ zkw.interruptedException(ie);
+ }
+ }
+
+ private static void processSequentially(ZKWatcher zkw, List<ZKUtilOp> ops)
+ throws KeeperException, NoNodeException {
+ for (ZKUtilOp op : ops) {
+ if (op instanceof CreateAndFailSilent) {
+ createAndFailSilent(zkw, (CreateAndFailSilent) op);
+ } else if (op instanceof DeleteNodeFailSilent) {
+ deleteNodeFailSilent(zkw, (DeleteNodeFailSilent) op);
+ } else if (op instanceof SetData) {
+ setData(zkw, (SetData) op);
+ } else {
+ throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
+ + op.getClass().getName());
+ }
+ }
+ }
+
+ //
+ // ZooKeeper cluster information
+ //
+
+ /** @return String dump of everything in ZooKeeper. */
+ public static String dump(ZKWatcher zkw) {
+ StringBuilder sb = new StringBuilder();
+ try {
+ sb.append("HBase is rooted at ").append(zkw.znodePaths.baseZNode);
+ sb.append("\nActive master address: ");
+ try {
+ sb.append(MasterAddressTracker.getMasterAddress(zkw));
+ } catch (IOException e) {
+ sb.append("<<FAILED LOOKUP: " + e.getMessage() + ">>");
+ }
+ sb.append("\nBackup master addresses:");
+ for (String child : listChildrenNoWatch(zkw, zkw.znodePaths.backupMasterAddressesZNode)) {
+ sb.append("\n ").append(child);
+ }
+ sb.append("\nRegion server holding hbase:meta: "
+ + new MetaTableLocator().getMetaRegionLocation(zkw));
+ Configuration conf = HBaseConfiguration.create();
+ int numMetaReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
+ HConstants.DEFAULT_META_REPLICA_NUM);
+ for (int i = 1; i < numMetaReplicas; i++) {
+ sb.append("\nRegion server holding hbase:meta, replicaId " + i + " "
+ + new MetaTableLocator().getMetaRegionLocation(zkw, i));
+ }
+ sb.append("\nRegion servers:");
+ for (String child : listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode)) {
+ sb.append("\n ").append(child);
+ }
+ try {
+ getReplicationZnodesDump(zkw, sb);
+ } catch (KeeperException ke) {
+ LOG.warn("Couldn't get the replication znode dump", ke);
+ }
+ sb.append("\nQuorum Server Statistics:");
+ String[] servers = zkw.getQuorum().split(",");
+ for (String server : servers) {
+ sb.append("\n ").append(server);
+ try {
+ String[] stat = getServerStats(server, ZKUtil.zkDumpConnectionTimeOut);
+
+ if (stat == null) {
+ sb.append("[Error] invalid quorum server: " + server);
+ break;
+ }
+
+ for (String s : stat) {
+ sb.append("\n ").append(s);
+ }
+ } catch (Exception e) {
+ sb.append("\n ERROR: ").append(e.getMessage());
+ }
+ }
+ } catch (KeeperException ke) {
+ sb.append("\nFATAL ZooKeeper Exception!\n");
+ sb.append("\n" + ke.getMessage());
+ }
+ return sb.toString();
+ }
+
+ /**
+ * Appends replication znodes to the passed StringBuilder.
+ * @param zkw
+ * @param sb
+ * @throws KeeperException
+ */
+ private static void getReplicationZnodesDump(ZKWatcher zkw, StringBuilder sb)
+ throws KeeperException {
+ String replicationZnode = zkw.znodePaths.replicationZNode;
+ if (ZKUtil.checkExists(zkw, replicationZnode) == -1) return;
+ // do a ls -r on this znode
+ sb.append("\n").append(replicationZnode).append(": ");
+ List<String> children = ZKUtil.listChildrenNoWatch(zkw, replicationZnode);
+ for (String child : children) {
+ String znode = ZNodePaths.joinZNode(replicationZnode, child);
+ if (znode.equals(zkw.znodePaths.peersZNode)) {
+ appendPeersZnodes(zkw, znode, sb);
+ } else if (znode.equals(zkw.znodePaths.queuesZNode)) {
+ appendRSZnodes(zkw, znode, sb);
+ } else if (znode.equals(zkw.znodePaths.hfileRefsZNode)) {
+ appendHFileRefsZnodes(zkw, znode, sb);
+ }
+ }
+ }
+
+ private static void appendHFileRefsZnodes(ZKWatcher zkw, String hfileRefsZnode,
+ StringBuilder sb) throws KeeperException {
+ sb.append("\n").append(hfileRefsZnode).append(": ");
+ for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, hfileRefsZnode)) {
+ String znodeToProcess = ZNodePaths.joinZNode(hfileRefsZnode, peerIdZnode);
+ sb.append("\n").append(znodeToProcess).append(": ");
+ List<String> peerHFileRefsZnodes = ZKUtil.listChildrenNoWatch(zkw, znodeToProcess);
+ int size = peerHFileRefsZnodes.size();
+ for (int i = 0; i < size; i++) {
+ sb.append(peerHFileRefsZnodes.get(i));
+ if (i != size - 1) {
+ sb.append(", ");
+ }
+ }
+ }
+ }
+
+ /**
+ * Returns a string with replication znodes and position of the replication log
+ * @param zkw
+ * @return aq string of replication znodes and log positions
+ */
+ public static String getReplicationZnodesDump(ZKWatcher zkw) throws KeeperException {
+ StringBuilder sb = new StringBuilder();
+ getReplicationZnodesDump(zkw, sb);
+ return sb.toString();
+ }
+
+ private static void appendRSZnodes(ZKWatcher zkw, String znode, StringBuilder sb)
+ throws KeeperException {
+ List<String> stack = new LinkedList<>();
+ stack.add(znode);
+ do {
+ String znodeToProcess = stack.remove(stack.size() - 1);
+ sb.append("\n").append(znodeToProcess).append(": ");
+ byte[] data;
+ try {
+ data = ZKUtil.getData(zkw, znodeToProcess);
+ } catch (InterruptedException e) {
+ zkw.interruptedException(e);
+ return;
+ }
+ if (data != null && data.length > 0) { // log position
+ long position = 0;
+ try {
+ position = ZKUtil.parseWALPositionFrom(ZKUtil.getData(zkw, znodeToProcess));
+ sb.append(position);
+ } catch (DeserializationException ignored) {
+ } catch (InterruptedException e) {
+ zkw.interruptedException(e);
+ return;
+ }
+ }
+ for (String zNodeChild : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
+ stack.add(ZNodePaths.joinZNode(znodeToProcess, zNodeChild));
+ }
+ } while (stack.size() > 0);
+ }
+
+ private static void appendPeersZnodes(ZKWatcher zkw, String peersZnode,
+ StringBuilder sb) throws KeeperException {
+ int pblen = ProtobufUtil.lengthOfPBMagic();
+ sb.append("\n").append(peersZnode).append(": ");
+ for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, peersZnode)) {
+ String znodeToProcess = ZNodePaths.joinZNode(peersZnode, peerIdZnode);
+ byte[] data;
+ try {
+ data = ZKUtil.getData(zkw, znodeToProcess);
+ } catch (InterruptedException e) {
+ zkw.interruptedException(e);
+ return;
+ }
+ // parse the data of the above peer znode.
+ try {
+ ReplicationProtos.ReplicationPeer.Builder builder =
+ ReplicationProtos.ReplicationPeer.newBuilder();
+ ProtobufUtil.mergeFrom(builder, data, pblen, data.length - pblen);
+ String clusterKey = builder.getClusterkey();
+ sb.append("\n").append(znodeToProcess).append(": ").append(clusterKey);
+ // add the peer-state.
+ appendPeerState(zkw, znodeToProcess, sb);
+ } catch (IOException ipbe) {
+ LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe);
+ }
+ }
+ }
+
+ private static void appendPeerState(ZKWatcher zkw, String znodeToProcess,
+ StringBuilder sb) throws KeeperException, InvalidProtocolBufferException {
+ String peerState = zkw.getConfiguration().get("zookeeper.znode.replication.peers.state",
+ "peer-state");
+ int pblen = ProtobufUtil.lengthOfPBMagic();
+ for (String child : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
+ if (!child.equals(peerState)) continue;
+ String peerStateZnode = ZNodePaths.joinZNode(znodeToProcess, child);
+ sb.append("\n").append(peerStateZnode).append(": ");
+ byte[] peerStateData;
+ try {
+ peerStateData = ZKUtil.getData(zkw, peerStateZnode);
+ ReplicationProtos.ReplicationState.Builder builder =
+ ReplicationProtos.ReplicationState.newBuilder();
+ ProtobufUtil.mergeFrom(builder, peerStateData, pblen, peerStateData.length - pblen);
+ sb.append(builder.getState().name());
+ } catch (IOException ipbe) {
+ LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe);
+ } catch (InterruptedException e) {
+ zkw.interruptedException(e);
+ return;
+ }
+ }
+ }
+
+ /**
+ * Gets the statistics from the given server.
+ *
+ * @param server The server to get the statistics from.
+ * @param timeout The socket timeout to use.
+ * @return The array of response strings.
+ * @throws IOException When the socket communication fails.
+ */
+ public static String[] getServerStats(String server, int timeout)
+ throws IOException {
+ String[] sp = server.split(":");
+ if (sp == null || sp.length == 0) {
+ return null;
+ }
+
+ String host = sp[0];
+ int port = sp.length > 1 ? Integer.parseInt(sp[1])
+ : HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT;
+
+ InetSocketAddress sockAddr = new InetSocketAddress(host, port);
+ try (Socket socket = new Socket()) {
+ socket.connect(sockAddr, timeout);
+
+ socket.setSoTimeout(timeout);
+ try (PrintWriter out = new PrintWriter(socket.getOutputStream(), true);
+ BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream()))) {
+ out.println("stat");
+ out.flush();
+ ArrayList<String> res = new ArrayList<>();
+ while (true) {
+ String line = in.readLine();
+ if (line != null) {
+ res.add(line);
+ } else {
+ break;
+ }
+ }
+ return res.toArray(new String[res.size()]);
+ }
+ }
+ }
+
+ private static void logRetrievedMsg(final ZKWatcher zkw,
+ final String znode, final byte [] data, final boolean watcherSet) {
+ if (!LOG.isTraceEnabled()) return;
+ LOG.trace(zkw.prefix("Retrieved " + ((data == null)? 0: data.length) +
+ " byte(s) of data from znode " + znode +
+ (watcherSet? " and set watcher; ": "; data=") +
+ (data == null? "null": data.length == 0? "empty": (
+ znode.startsWith(zkw.znodePaths.metaZNodePrefix)?
+ getServerNameOrEmptyString(data):
+ znode.startsWith(zkw.znodePaths.backupMasterAddressesZNode)?
+ getServerNameOrEmptyString(data):
+ StringUtils.abbreviate(Bytes.toStringBinary(data), 32)))));
+ }
+
+ private static String getServerNameOrEmptyString(final byte [] data) {
+ try {
+ return ProtobufUtil.parseServerNameFrom(data).toString();
+ } catch (DeserializationException e) {
+ return "";
+ }
+ }
+
+ /**
+ * Waits for HBase installation's base (parent) znode to become available.
+ * @throws IOException on ZK errors
+ */
+ public static void waitForBaseZNode(Configuration conf) throws IOException {
+ LOG.info("Waiting until the base znode is available");
+ String parentZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
+ HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
+ ZooKeeper zk = new ZooKeeper(ZKConfig.getZKQuorumServersString(conf),
+ conf.getInt(HConstants.ZK_SESSION_TIMEOUT,
+ HConstants.DEFAULT_ZK_SESSION_TIMEOUT), EmptyWatcher.instance);
+
+ final int maxTimeMs = 10000;
+ final int maxNumAttempts = maxTimeMs / HConstants.SOCKET_RETRY_WAIT_MS;
+
+ KeeperException keeperEx = null;
+ try {
+ try {
+ for (int attempt = 0; attempt < maxNumAttempts; ++attempt) {
+ try {
+ if (zk.exists(parentZNode, false) != null) {
+ LOG.info("Parent znode exists: " + parentZNode);
+ keeperEx = null;
+ break;
+ }
+ } catch (KeeperException e) {
+ keeperEx = e;
+ }
+ Threads.sleepWithoutInterrupt(HConstants.SOCKET_RETRY_WAIT_MS);
+ }
+ } finally {
+ zk.close();
+ }
+ } catch (InterruptedException ex) {
+ Thread.currentThread().interrupt();
+ }
+
+ if (keeperEx != null) {
+ throw new IOException(keeperEx);
+ }
+ }
+
+ /**
+ * Convert a {@link DeserializationException} to a more palatable {@link KeeperException}.
+ * Used when can't let a {@link DeserializationException} out w/o changing public API.
+ * @param e Exception to convert
+ * @return Converted exception
+ */
+ public static KeeperException convert(final DeserializationException e) {
+ KeeperException ke = new KeeperException.DataInconsistencyException();
+ ke.initCause(e);
+ return ke;
+ }
+
+ /**
+ * Recursively print the current state of ZK (non-transactional)
+ * @param root name of the root directory in zk to print
+ */
+ public static void logZKTree(ZKWatcher zkw, String root) {
+ if (!LOG.isDebugEnabled()) return;
+ LOG.debug("Current zk system:");
+ String prefix = "|-";
+ LOG.debug(prefix + root);
+ try {
+ logZKTree(zkw, root, prefix);
+ } catch (KeeperException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * Helper method to print the current state of the ZK tree.
+ * @see #logZKTree(ZKWatcher, String)
+ * @throws KeeperException if an unexpected exception occurs
+ */
+ protected static void logZKTree(ZKWatcher zkw, String root, String prefix)
+ throws KeeperException {
+ List<String> children = ZKUtil.listChildrenNoWatch(zkw, root);
+ if (children == null) return;
+ for (String child : children) {
+ LOG.debug(prefix + child);
+ String node = ZNodePaths.joinZNode(root.equals("/") ? "" : root, child);
+ logZKTree(zkw, node, prefix + "---");
+ }
+ }
+
+ /**
+ * @param position
+ * @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.
+ */
+ public static byte[] positionToByteArray(final long position) {
+ byte[] bytes = ReplicationProtos.ReplicationHLogPosition.newBuilder().setPosition(position)
+ .build().toByteArray();
+ return ProtobufUtil.prependPBMagic(bytes);
+ }
+
+ /**
+ * @param bytes - Content of a WAL position znode.
+ * @return long - The current WAL position.
+ * @throws DeserializationException
+ */
+ public static long parseWALPositionFrom(final byte[] bytes) throws DeserializationException {
+ if (bytes == null) {
+ throw new DeserializationException("Unable to parse null WAL position.");
+ }
+ if (ProtobufUtil.isPBMagicPrefix(bytes)) {
+ int pblen = ProtobufUtil.lengthOfPBMagic();
+ ReplicationProtos.ReplicationHLogPosition.Builder builder =
+ ReplicationProtos.ReplicationHLogPosition.newBuilder();
+ ReplicationProtos.ReplicationHLogPosition position;
+ try {
+ ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
+ position = builder.build();
+ } catch (IOException e) {
+ throw new DeserializationException(e);
+ }
+ return position.getPosition();
+ } else {
+ if (bytes.length > 0) {
+ return Bytes.toLong(bytes);
+ }
+ return 0;
+ }
+ }
+}
[05/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper
from hbase-server and hbase-client
Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java
deleted file mode 100644
index 951c6ba..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java
+++ /dev/null
@@ -1,351 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.Random;
-import java.util.concurrent.Semaphore;
-
-import junit.framework.Assert;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.master.TestActiveMasterManager.NodeDeletionListener;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.ZooKeeper;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({MiscTests.class, MediumTests.class})
-public class TestZooKeeperNodeTracker {
- private static final Log LOG = LogFactory.getLog(TestZooKeeperNodeTracker.class);
- private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
- private final static Random rand = new Random();
-
- @BeforeClass
- public static void setUpBeforeClass() throws Exception {
- TEST_UTIL.startMiniZKCluster();
- }
-
- @AfterClass
- public static void tearDownAfterClass() throws Exception {
- TEST_UTIL.shutdownMiniZKCluster();
- }
-
- /**
- * Test that we can interrupt a node that is blocked on a wait.
- * @throws IOException
- * @throws InterruptedException
- */
- @Test public void testInterruptible() throws IOException, InterruptedException {
- Abortable abortable = new StubAbortable();
- ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
- "testInterruptible", abortable);
- final TestTracker tracker = new TestTracker(zk, "/xyz", abortable);
- tracker.start();
- Thread t = new Thread() {
- @Override
- public void run() {
- try {
- tracker.blockUntilAvailable();
- } catch (InterruptedException e) {
- throw new RuntimeException("Interrupted", e);
- }
- }
- };
- t.start();
- while (!t.isAlive()) Threads.sleep(1);
- tracker.stop();
- t.join();
- // If it wasn't interruptible, we'd never get to here.
- }
-
- @Test
- public void testNodeTracker() throws Exception {
- Abortable abortable = new StubAbortable();
- ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
- "testNodeTracker", abortable);
- ZKUtil.createAndFailSilent(zk, zk.znodePaths.baseZNode);
-
- final String node =
- ZNodePaths.joinZNode(zk.znodePaths.baseZNode, new Long(rand.nextLong()).toString());
-
- final byte [] dataOne = Bytes.toBytes("dataOne");
- final byte [] dataTwo = Bytes.toBytes("dataTwo");
-
- // Start a ZKNT with no node currently available
- TestTracker localTracker = new TestTracker(zk, node, abortable);
- localTracker.start();
- zk.registerListener(localTracker);
-
- // Make sure we don't have a node
- assertNull(localTracker.getData(false));
-
- // Spin up a thread with another ZKNT and have it block
- WaitToGetDataThread thread = new WaitToGetDataThread(zk, node);
- thread.start();
-
- // Verify the thread doesn't have a node
- assertFalse(thread.hasData);
-
- // Now, start a new ZKNT with the node already available
- TestTracker secondTracker = new TestTracker(zk, node, null);
- secondTracker.start();
- zk.registerListener(secondTracker);
-
- // Put up an additional zk listener so we know when zk event is done
- TestingZKListener zkListener = new TestingZKListener(zk, node);
- zk.registerListener(zkListener);
- assertEquals(0, zkListener.createdLock.availablePermits());
-
- // Create a completely separate zk connection for test triggers and avoid
- // any weird watcher interactions from the test
- final ZooKeeper zkconn = new ZooKeeper(
- ZKConfig.getZKQuorumServersString(TEST_UTIL.getConfiguration()), 60000,
- new StubWatcher());
-
- // Add the node with data one
- zkconn.create(node, dataOne, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-
- // Wait for the zk event to be processed
- zkListener.waitForCreation();
- thread.join();
-
- // Both trackers should have the node available with data one
- assertNotNull(localTracker.getData(false));
- assertNotNull(localTracker.blockUntilAvailable());
- assertTrue(Bytes.equals(localTracker.getData(false), dataOne));
- assertTrue(thread.hasData);
- assertTrue(Bytes.equals(thread.tracker.getData(false), dataOne));
- LOG.info("Successfully got data one");
-
- // Make sure it's available and with the expected data
- assertNotNull(secondTracker.getData(false));
- assertNotNull(secondTracker.blockUntilAvailable());
- assertTrue(Bytes.equals(secondTracker.getData(false), dataOne));
- LOG.info("Successfully got data one with the second tracker");
-
- // Drop the node
- zkconn.delete(node, -1);
- zkListener.waitForDeletion();
-
- // Create a new thread but with the existing thread's tracker to wait
- TestTracker threadTracker = thread.tracker;
- thread = new WaitToGetDataThread(zk, node, threadTracker);
- thread.start();
-
- // Verify other guys don't have data
- assertFalse(thread.hasData);
- assertNull(secondTracker.getData(false));
- assertNull(localTracker.getData(false));
- LOG.info("Successfully made unavailable");
-
- // Create with second data
- zkconn.create(node, dataTwo, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-
- // Wait for the zk event to be processed
- zkListener.waitForCreation();
- thread.join();
-
- // All trackers should have the node available with data two
- assertNotNull(localTracker.getData(false));
- assertNotNull(localTracker.blockUntilAvailable());
- assertTrue(Bytes.equals(localTracker.getData(false), dataTwo));
- assertNotNull(secondTracker.getData(false));
- assertNotNull(secondTracker.blockUntilAvailable());
- assertTrue(Bytes.equals(secondTracker.getData(false), dataTwo));
- assertTrue(thread.hasData);
- assertTrue(Bytes.equals(thread.tracker.getData(false), dataTwo));
- LOG.info("Successfully got data two on all trackers and threads");
-
- // Change the data back to data one
- zkconn.setData(node, dataOne, -1);
-
- // Wait for zk event to be processed
- zkListener.waitForDataChange();
-
- // All trackers should have the node available with data one
- assertNotNull(localTracker.getData(false));
- assertNotNull(localTracker.blockUntilAvailable());
- assertTrue(Bytes.equals(localTracker.getData(false), dataOne));
- assertNotNull(secondTracker.getData(false));
- assertNotNull(secondTracker.blockUntilAvailable());
- assertTrue(Bytes.equals(secondTracker.getData(false), dataOne));
- assertTrue(thread.hasData);
- assertTrue(Bytes.equals(thread.tracker.getData(false), dataOne));
- LOG.info("Successfully got data one following a data change on all trackers and threads");
- }
-
- public static class WaitToGetDataThread extends Thread {
-
- TestTracker tracker;
- boolean hasData;
-
- public WaitToGetDataThread(ZooKeeperWatcher zk, String node) {
- tracker = new TestTracker(zk, node, null);
- tracker.start();
- zk.registerListener(tracker);
- hasData = false;
- }
-
- public WaitToGetDataThread(ZooKeeperWatcher zk, String node,
- TestTracker tracker) {
- this.tracker = tracker;
- hasData = false;
- }
-
- @Override
- public void run() {
- LOG.info("Waiting for data to be available in WaitToGetDataThread");
- try {
- tracker.blockUntilAvailable();
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- LOG.info("Data now available in tracker from WaitToGetDataThread");
- hasData = true;
- }
- }
-
- public static class TestTracker extends ZooKeeperNodeTracker {
- public TestTracker(ZooKeeperWatcher watcher, String node,
- Abortable abortable) {
- super(watcher, node, abortable);
- }
- }
-
- public static class TestingZKListener extends ZooKeeperListener {
- private static final Log LOG = LogFactory.getLog(NodeDeletionListener.class);
-
- private Semaphore deletedLock;
- private Semaphore createdLock;
- private Semaphore changedLock;
- private String node;
-
- public TestingZKListener(ZooKeeperWatcher watcher, String node) {
- super(watcher);
- deletedLock = new Semaphore(0);
- createdLock = new Semaphore(0);
- changedLock = new Semaphore(0);
- this.node = node;
- }
-
- @Override
- public void nodeDeleted(String path) {
- if(path.equals(node)) {
- LOG.debug("nodeDeleted(" + path + ")");
- deletedLock.release();
- }
- }
-
- @Override
- public void nodeCreated(String path) {
- if(path.equals(node)) {
- LOG.debug("nodeCreated(" + path + ")");
- createdLock.release();
- }
- }
-
- @Override
- public void nodeDataChanged(String path) {
- if(path.equals(node)) {
- LOG.debug("nodeDataChanged(" + path + ")");
- changedLock.release();
- }
- }
-
- public void waitForDeletion() throws InterruptedException {
- deletedLock.acquire();
- }
-
- public void waitForCreation() throws InterruptedException {
- createdLock.acquire();
- }
-
- public void waitForDataChange() throws InterruptedException {
- changedLock.acquire();
- }
- }
-
- public static class StubAbortable implements Abortable {
- @Override
- public void abort(final String msg, final Throwable t) {}
-
- @Override
- public boolean isAborted() {
- return false;
- }
-
- }
-
- public static class StubWatcher implements Watcher {
- @Override
- public void process(WatchedEvent event) {}
- }
-
- @Test
- public void testCleanZNode() throws Exception {
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
- "testNodeTracker", new TestZooKeeperNodeTracker.StubAbortable());
-
- final ServerName sn = ServerName.valueOf("127.0.0.1:52", 45L);
-
- ZKUtil.createAndFailSilent(zkw,
- TEST_UTIL.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT,
- HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT));
-
- final String nodeName = zkw.znodePaths.masterAddressZNode;
-
- // Check that we manage the case when there is no data
- ZKUtil.createAndFailSilent(zkw, nodeName);
- MasterAddressTracker.deleteIfEquals(zkw, sn.toString());
- Assert.assertFalse(ZKUtil.getData(zkw, nodeName) == null);
-
- // Check that we don't delete if we're not supposed to
- ZKUtil.setData(zkw, nodeName, MasterAddressTracker.toByteArray(sn, 0));
- MasterAddressTracker.deleteIfEquals(zkw, ServerName.valueOf("127.0.0.2:52", 45L).toString());
- Assert.assertFalse(ZKUtil.getData(zkw, nodeName) == null);
-
- // Check that we delete when we're supposed to
- ZKUtil.setData(zkw, nodeName,MasterAddressTracker.toByteArray(sn, 0));
- MasterAddressTracker.deleteIfEquals(zkw, sn.toString());
- Assert.assertTrue( ZKUtil.getData(zkw, nodeName)== null );
-
- // Check that we support the case when the znode does not exist
- MasterAddressTracker.deleteIfEquals(zkw, sn.toString()); // must not throw an exception
- }
-
-}
-
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 0ce38cc..9f8551c 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -328,7 +328,7 @@ module Hbase
#----------------------------------------------------------------------------------------------
# Returns ZooKeeper status dump
def zk_dump
- @zk_wrapper = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(
+ @zk_wrapper = org.apache.hadoop.hbase.zookeeper.ZKWatcher.new(
@admin.getConfiguration,
'admin',
nil
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/pom.xml b/hbase-zookeeper/pom.xml
new file mode 100644
index 0000000..06b7dff
--- /dev/null
+++ b/hbase-zookeeper/pom.xml
@@ -0,0 +1,412 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!--
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+-->
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <artifactId>hbase-build-configuration</artifactId>
+ <groupId>org.apache.hbase</groupId>
+ <version>3.0.0-SNAPSHOT</version>
+ <relativePath>../hbase-build-configuration</relativePath>
+ </parent>
+ <artifactId>hbase-zookeeper</artifactId>
+ <name>Apache HBase - Zookeeper</name>
+ <description>Zookeeper Helpers for HBase</description>
+
+ <build>
+ <!-- Makes sure the resources get added before they are processed
+ by placing this first -->
+ <testResources>
+ <!-- Our test artifact has different license info than our source/bin ones -->
+ <testResource>
+ <directory>src/test/resources/META-INF/</directory>
+ <targetPath>META-INF/</targetPath>
+ <includes>
+ <include>NOTICE</include>
+ </includes>
+ <filtering>true</filtering>
+ </testResource>
+ </testResources>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-site-plugin</artifactId>
+ <configuration>
+ <skip>true</skip>
+ </configuration>
+ </plugin>
+ <!-- Run with -Dmaven.test.skip.exec=true to build -tests.jar without running
+ tests (this is needed for upstream projects whose tests need this jar simply for
+ compilation) -->
+ <plugin>
+ <!--Make it so assembly:single does nothing in here-->
+ <artifactId>maven-assembly-plugin</artifactId>
+ <configuration>
+ <skipAssembly>true</skipAssembly>
+ </configuration>
+ </plugin>
+ <!-- Make a jar and put the sources in the jar -->
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-source-plugin</artifactId>
+ <executions>
+ <execution>
+ <phase>package</phase>
+ <goals>
+ <goal>jar</goal>
+ <goal>test-jar</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ <!-- General plugins -->
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-eclipse-plugin</artifactId>
+ <configuration>
+ <additionalProjectnatures>
+ <projectnature>org.jamon.project.jamonnature</projectnature>
+ </additionalProjectnatures>
+ <buildcommands>
+ <buildcommand>org.jamon.project.templateBuilder</buildcommand>
+ <buildcommand>org.eclipse.jdt.core.javabuilder</buildcommand>
+ <buildcommand>org.jamon.project.markerUpdater</buildcommand>
+ </buildcommands>
+ <additionalConfig>
+ <file>
+ <name>.settings/org.jamon.prefs</name>
+ <content># now
+ eclipse.preferences.version=1
+ templateSourceDir=src/main/jamon
+ templateOutputDir=target/generated-jamon
+ </content>
+ </file>
+ </additionalConfig>
+ </configuration>
+ </plugin>
+ <!-- Run findbugs -->
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>findbugs-maven-plugin</artifactId>
+ </plugin>
+ <!-- Testing plugins -->
+ <plugin>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <configuration>
+ <properties>
+ <property>
+ <name>listener</name>
+ <value>org.apache.hadoop.hbase.ResourceCheckerJUnitListener</value>
+ </property>
+ </properties>
+ </configuration>
+ </plugin>
+ </plugins>
+ <!-- General Resources -->
+ <pluginManagement>
+ <plugins>
+ <!--This plugin's configuration is used to store Eclipse m2e settings
+ only. It has no influence on the Maven build itself and needs to
+ be kept in plugin management, not in the actual plugins. -->
+ <plugin>
+ <groupId>org.eclipse.m2e</groupId>
+ <artifactId>lifecycle-mapping</artifactId>
+ <configuration>
+ <lifecycleMappingMetadata>
+ <pluginExecutions>
+ <pluginExecution>
+ <pluginExecutionFilter>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-antrun-plugin</artifactId>
+ <versionRange>[1.6,)</versionRange>
+ <goals>
+ <goal>run</goal>
+ </goals>
+ </pluginExecutionFilter>
+ <action>
+ <execute>
+ <runOnIncremental>false</runOnIncremental>
+ <runOnConfiguration>true</runOnConfiguration>
+ </execute>
+ </action>
+ </pluginExecution>
+ <pluginExecution>
+ <pluginExecutionFilter>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-dependency-plugin</artifactId>
+ <versionRange>[2.8,)</versionRange>
+ <goals>
+ <goal>build-classpath</goal>
+ </goals>
+ </pluginExecutionFilter>
+ <action>
+ <ignore></ignore>
+ </action>
+ </pluginExecution>
+ <pluginExecution>
+ <pluginExecutionFilter>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <versionRange>[3.2,)</versionRange>
+ <goals>
+ <goal>compile</goal>
+ </goals>
+ </pluginExecutionFilter>
+ <action>
+ <ignore></ignore>
+ </action>
+ </pluginExecution>
+ </pluginExecutions>
+ </lifecycleMappingMetadata>
+ </configuration>
+ </plugin>
+ </plugins>
+ </pluginManagement>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hbase.thirdparty</groupId>
+ <artifactId>hbase-shaded-protobuf</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase.thirdparty</groupId>
+ <artifactId>hbase-shaded-miscellaneous</artifactId>
+ </dependency>
+ <!-- Intra-project dependencies -->
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-common</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-client</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-annotations</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-protocol-shaded</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-hadoop-compat</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-hadoop2-compat</artifactId>
+ <exclusions>
+ <!-- We don't need MR support classes here. -->
+ <exclusion>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-core</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <!-- General dependencies -->
+ <dependency>
+ <groupId>com.github.stephenc.findbugs</groupId>
+ <artifactId>findbugs-annotations</artifactId>
+ <optional>true</optional>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-lang3</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>commons-logging</groupId>
+ <artifactId>commons-logging</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>log4j</groupId>
+ <artifactId>log4j</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.zookeeper</groupId>
+ <artifactId>zookeeper</artifactId>
+ </dependency>
+ <!-- Test dependencies -->
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-core</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <profiles>
+ <!-- Needs to make the profile in apache parent pom -->
+ <profile>
+ <id>apache-release</id>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-resources-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>license-javadocs</id>
+ <phase>prepare-package</phase>
+ <goals>
+ <goal>copy-resources</goal>
+ </goals>
+ <configuration>
+ <outputDirectory>${project.build.directory}/apidocs</outputDirectory>
+ <resources>
+ <resource>
+ <directory>src/main/javadoc/META-INF/</directory>
+ <targetPath>META-INF/</targetPath>
+ <includes>
+ <include>LICENSE</include>
+ <include>NOTICE</include>
+ </includes>
+ <filtering>true</filtering>
+ </resource>
+ </resources>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ </profile>
+ <!-- Skip the tests in this module -->
+ <profile>
+ <id>skipZooKeeperTests</id>
+ <activation>
+ <property>
+ <name>skipZooKeeperTests</name>
+ </property>
+ </activation>
+ <properties>
+ <surefire.skipFirstPart>true</surefire.skipFirstPart>
+ <surefire.skipSecondPart>true</surefire.skipSecondPart>
+ </properties>
+ </profile>
+ <!-- Profiles for building against different hadoop versions -->
+ <!-- There are a lot of common dependencies used here, should investigate
+ if we can combine these profiles somehow -->
+
+ <!-- profile for building against Hadoop 2.x. This is the default. -->
+ <profile>
+ <id>hadoop-2.0</id>
+ <activation>
+ <property>
+ <!--Below formatting for dev-support/generate-hadoopX-poms.sh-->
+ <!--h2--><name>!hadoop.profile</name>
+ </property>
+ </activation>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-auth</artifactId>
+ </dependency>
+ </dependencies>
+ <build>
+ <plugins>
+ <plugin>
+ <artifactId>maven-dependency-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>create-mrapp-generated-classpath</id>
+ <phase>generate-test-resources</phase>
+ <goals>
+ <goal>build-classpath</goal>
+ </goals>
+ <configuration>
+ <!-- needed to run the unit test for DS to generate
+ the required classpath that is required in the env
+ of the launch container in the mini mr/yarn cluster
+ -->
+ <outputFile>${project.build.directory}/test-classes/mrapp-generated-classpath</outputFile>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ </profile>
+ <!--
+ profile for building against Hadoop 3.0.x. Activate using:
+ mvn -Dhadoop.profile=3.0
+ -->
+ <profile>
+ <id>hadoop-3.0</id>
+ <activation>
+ <property>
+ <name>hadoop.profile</name>
+ <value>3.0</value>
+ </property>
+ </activation>
+ <properties>
+ <hadoop.version>${hadoop-three.version}</hadoop.version>
+ </properties>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-auth</artifactId>
+ </dependency>
+ </dependencies>
+ <build>
+ <plugins>
+ <plugin>
+ <artifactId>maven-dependency-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>create-mrapp-generated-classpath</id>
+ <phase>generate-test-resources</phase>
+ <goals>
+ <goal>build-classpath</goal>
+ </goals>
+ <configuration>
+ <!-- needed to run the unit test for DS to generate
+ the required classpath that is required in the env
+ of the launch container in the mini mr/yarn cluster
+ -->
+ <outputFile>${project.build.directory}/test-classes/mrapp-generated-classpath</outputFile>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ </profile>
+ </profiles>
+</project>
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
new file mode 100644
index 0000000..d145d08
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
@@ -0,0 +1,99 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Tracker on cluster settings up in zookeeper.
+ * This is not related to {@link org.apache.hadoop.hbase.ClusterStatus}. That class
+ * is a data structure that holds snapshot of current view on cluster. This class
+ * is about tracking cluster attributes up in zookeeper.
+ *
+ */
+@InterfaceAudience.Private
+public class ClusterStatusTracker extends ZKNodeTracker {
+ private static final Log LOG = LogFactory.getLog(ClusterStatusTracker.class);
+
+ /**
+ * Creates a cluster status tracker.
+ *
+ * <p>After construction, use {@link #start} to kick off tracking.
+ *
+ * @param watcher
+ * @param abortable
+ */
+ public ClusterStatusTracker(ZKWatcher watcher, Abortable abortable) {
+ super(watcher, watcher.znodePaths.clusterStateZNode, abortable);
+ }
+
+ /**
+ * Checks if cluster is up.
+ * @return true if the cluster up ('shutdown' is its name up in zk) znode
+ * exists with data, false if not
+ */
+ public boolean isClusterUp() {
+ return super.getData(false) != null;
+ }
+
+ /**
+ * Sets the cluster as up.
+ * @throws KeeperException unexpected zk exception
+ */
+ public void setClusterUp()
+ throws KeeperException {
+ byte [] upData = toByteArray();
+ try {
+ ZKUtil.createAndWatch(watcher, watcher.znodePaths.clusterStateZNode, upData);
+ } catch(KeeperException.NodeExistsException nee) {
+ ZKUtil.setData(watcher, watcher.znodePaths.clusterStateZNode, upData);
+ }
+ }
+
+ /**
+ * Sets the cluster as down by deleting the znode.
+ * @throws KeeperException unexpected zk exception
+ */
+ public void setClusterDown()
+ throws KeeperException {
+ try {
+ ZKUtil.deleteNode(watcher, watcher.znodePaths.clusterStateZNode);
+ } catch(KeeperException.NoNodeException nne) {
+ LOG.warn("Attempted to set cluster as down but already down, cluster " +
+ "state node (" + watcher.znodePaths.clusterStateZNode + ") not found");
+ }
+ }
+
+ /**
+ * @return Content of the clusterup znode as a serialized pb with the pb
+ * magic as prefix.
+ */
+ static byte [] toByteArray() {
+ ZooKeeperProtos.ClusterUp.Builder builder =
+ ZooKeeperProtos.ClusterUp.newBuilder();
+ builder.setStartDate(new java.util.Date().toString());
+ return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java
new file mode 100644
index 0000000..7c02891
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * A ZooKeeper watcher meant to detect deletions of ZNodes.
+ */
+@InterfaceAudience.Private
+public class DeletionListener extends ZKListener {
+
+ private static final Log LOG = LogFactory.getLog(DeletionListener.class);
+
+ private final String pathToWatch;
+ private final CountDownLatch deletedLatch;
+
+ private volatile Throwable exception;
+
+ /**
+ * Create a new instance of the deletion watcher.
+ * @param zkWatcher ZookeeperWatcher instance
+ * @param pathToWatch (Fully qualified) ZNode path that we are waiting to
+ * be deleted.
+ * @param deletedLatch Count down on this latch when deletion has occurred.
+ */
+ public DeletionListener(ZKWatcher zkWatcher, String pathToWatch,
+ CountDownLatch deletedLatch) {
+ super(zkWatcher);
+ this.pathToWatch = pathToWatch;
+ this.deletedLatch = deletedLatch;
+ exception = null;
+ }
+
+ /**
+ * Check if an exception has occurred when re-setting the watch.
+ * @return True if we were unable to re-set a watch on a ZNode due to
+ * an exception.
+ */
+ public boolean hasException() {
+ return exception != null;
+ }
+
+ /**
+ * Get the last exception which has occurred when re-setting the watch.
+ * Use hasException() to check whether or not an exception has occurred.
+ * @return The last exception observed when re-setting the watch.
+ */
+ public Throwable getException() {
+ return exception;
+ }
+
+ @Override
+ public void nodeDataChanged(String path) {
+ if (!path.equals(pathToWatch)) {
+ return;
+ }
+ try {
+ if (!(ZKUtil.setWatchIfNodeExists(watcher, pathToWatch))) {
+ deletedLatch.countDown();
+ }
+ } catch (KeeperException ex) {
+ exception = ex;
+ deletedLatch.countDown();
+ LOG.error("Error when re-setting the watch on " + pathToWatch, ex);
+ }
+ }
+
+ @Override
+ public void nodeDeleted(String path) {
+ if (!path.equals(pathToWatch)) {
+ return;
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Processing delete on " + pathToWatch);
+ }
+ deletedLatch.countDown();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java
new file mode 100644
index 0000000..6470faa
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+
+/**
+ * An empty ZooKeeper watcher
+ */
+@InterfaceAudience.Private
+public class EmptyWatcher implements Watcher {
+ // Used in this package but also by tests so needs to be public
+ public static final EmptyWatcher instance = new EmptyWatcher();
+ private EmptyWatcher() {}
+
+ public void process(WatchedEvent event) {}
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
new file mode 100644
index 0000000..f07b841
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
@@ -0,0 +1,165 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT;
+import static org.apache.hadoop.hbase.HConstants.ZK_SESSION_TIMEOUT;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+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.Strings;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.zookeeper.server.ServerConfig;
+import org.apache.zookeeper.server.ZooKeeperServerMain;
+import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
+import org.apache.zookeeper.server.quorum.QuorumPeerMain;
+
+/**
+ * HBase's version of ZooKeeper's QuorumPeer. When HBase is set to manage
+ * ZooKeeper, this class is used to start up QuorumPeer instances. By doing
+ * things in here rather than directly calling to ZooKeeper, we have more
+ * control over the process. This class uses {@link ZKConfig} to get settings
+ * from the hbase-site.xml file.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+@InterfaceStability.Evolving
+public class HQuorumPeer {
+
+ /**
+ * Parse ZooKeeper configuration from HBase XML config and run a QuorumPeer.
+ * @param args String[] of command line arguments. Not used.
+ */
+ public static void main(String[] args) {
+ Configuration conf = HBaseConfiguration.create();
+ try {
+ Properties zkProperties = ZKConfig.makeZKProps(conf);
+ writeMyID(zkProperties);
+ QuorumPeerConfig zkConfig = new QuorumPeerConfig();
+ zkConfig.parseProperties(zkProperties);
+
+ // login the zookeeper server principal (if using security)
+ ZKUtil.loginServer(conf, HConstants.ZK_SERVER_KEYTAB_FILE,
+ HConstants.ZK_SERVER_KERBEROS_PRINCIPAL,
+ zkConfig.getClientPortAddress().getHostName());
+
+ runZKServer(zkConfig);
+ } catch (Exception e) {
+ e.printStackTrace();
+ System.exit(-1);
+ }
+ }
+
+ private static void runZKServer(QuorumPeerConfig zkConfig) throws UnknownHostException, IOException {
+ if (zkConfig.isDistributed()) {
+ QuorumPeerMain qp = new QuorumPeerMain();
+ qp.runFromConfig(zkConfig);
+ } else {
+ ZooKeeperServerMain zk = new ZooKeeperServerMain();
+ ServerConfig serverConfig = new ServerConfig();
+ serverConfig.readFrom(zkConfig);
+ zk.runFromConfig(serverConfig);
+ }
+ }
+
+ private static boolean addressIsLocalHost(String address) {
+ return address.equals("localhost") || address.equals("127.0.0.1");
+ }
+
+ static void writeMyID(Properties properties) throws IOException {
+ long myId = -1;
+
+ Configuration conf = HBaseConfiguration.create();
+ String myAddress = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
+ conf.get("hbase.zookeeper.dns.interface","default"),
+ conf.get("hbase.zookeeper.dns.nameserver","default")));
+
+ List<String> ips = new ArrayList<>();
+
+ // Add what could be the best (configured) match
+ ips.add(myAddress.contains(".") ?
+ myAddress :
+ StringUtils.simpleHostname(myAddress));
+
+ // For all nics get all hostnames and IPs
+ Enumeration<?> nics = NetworkInterface.getNetworkInterfaces();
+ while(nics.hasMoreElements()) {
+ Enumeration<?> rawAdrs =
+ ((NetworkInterface)nics.nextElement()).getInetAddresses();
+ while(rawAdrs.hasMoreElements()) {
+ InetAddress inet = (InetAddress) rawAdrs.nextElement();
+ ips.add(StringUtils.simpleHostname(inet.getHostName()));
+ ips.add(inet.getHostAddress());
+ }
+ }
+
+ for (Entry<Object, Object> entry : properties.entrySet()) {
+ String key = entry.getKey().toString().trim();
+ String value = entry.getValue().toString().trim();
+ if (key.startsWith("server.")) {
+ int dot = key.indexOf('.');
+ long id = Long.parseLong(key.substring(dot + 1));
+ String[] parts = value.split(":");
+ String address = parts[0];
+ if (addressIsLocalHost(address) || ips.contains(address)) {
+ myId = id;
+ break;
+ }
+ }
+ }
+
+ // Set the max session timeout from the provided client-side timeout
+ properties.setProperty("maxSessionTimeout",
+ conf.get(HConstants.ZK_SESSION_TIMEOUT, Integer.toString(HConstants.DEFAULT_ZK_SESSION_TIMEOUT)));
+
+ if (myId == -1) {
+ throw new IOException("Could not find my address: " + myAddress +
+ " in list of ZooKeeper quorum servers");
+ }
+
+ String dataDirStr = properties.get("dataDir").toString().trim();
+ File dataDir = new File(dataDirStr);
+ if (!dataDir.isDirectory()) {
+ if (!dataDir.mkdirs()) {
+ throw new IOException("Unable to create data dir " + dataDir);
+ }
+ }
+
+ File myIdFile = new File(dataDir, "myid");
+ PrintWriter w = new PrintWriter(myIdFile);
+ w.println(myId);
+ w.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java
new file mode 100644
index 0000000..e63bfc5
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.util.concurrent.CountDownLatch;
+
+/**
+ * Placeholder of an instance which will be accessed by other threads
+ * but is not yet created. Thread safe.
+ */
+class InstancePending<T> {
+ // Based on a subtle part of the Java Language Specification,
+ // in order to avoid a slight overhead of synchronization for each access.
+
+ private final CountDownLatch pendingLatch = new CountDownLatch(1);
+
+ /** Piggybacking on {@code pendingLatch}. */
+ private InstanceHolder<T> instanceHolder;
+
+ private static class InstanceHolder<T> {
+ // The JLS ensures the visibility of a final field and its contents
+ // unless they are exposed to another thread while the construction.
+ final T instance;
+
+ InstanceHolder(T instance) {
+ this.instance = instance;
+ }
+ }
+
+ /**
+ * Returns the instance given by the method {@link #prepare}.
+ * This is an uninterruptible blocking method
+ * and the interruption flag will be set just before returning if any.
+ */
+ T get() {
+ InstanceHolder<T> instanceHolder;
+ boolean interrupted = false;
+
+ while ((instanceHolder = this.instanceHolder) == null) {
+ try {
+ pendingLatch.await();
+ } catch (InterruptedException e) {
+ interrupted = true;
+ }
+ }
+
+ if (interrupted) {
+ Thread.currentThread().interrupt();
+ }
+ return instanceHolder.instance;
+ }
+
+ /**
+ * Associates the given instance for the method {@link #get}.
+ * This method should be called once, and {@code instance} should be non-null.
+ * This method is expected to call as soon as possible
+ * because the method {@code get} is uninterruptibly blocked until this method is called.
+ */
+ void prepare(T instance) {
+ assert instance != null;
+ instanceHolder = new InstanceHolder<>(instance);
+ pendingLatch.countDown();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java
new file mode 100644
index 0000000..55dafcb
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+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
+ */
+@InterfaceAudience.Private
+public class LoadBalancerTracker extends ZKNodeTracker {
+ private static final Log LOG = LogFactory.getLog(LoadBalancerTracker.class);
+
+ public LoadBalancerTracker(ZKWatcher watcher,
+ Abortable abortable) {
+ super(watcher, watcher.znodePaths.balancerZNode, abortable);
+ }
+
+ /**
+ * Return true if the balance switch is on, false otherwise
+ */
+ public boolean isBalancerOn() {
+ byte [] upData = super.getData(false);
+ try {
+ // if data in ZK is null, use default of on.
+ return upData == null || parseFrom(upData).getBalancerOn();
+ } catch (DeserializationException dex) {
+ LOG.error("ZK state for LoadBalancer could not be parsed " + Bytes.toStringBinary(upData));
+ // return false to be safe.
+ return false;
+ }
+ }
+
+ /**
+ * Set the balancer on/off
+ * @param balancerOn
+ * @throws KeeperException
+ */
+ public void setBalancerOn(boolean balancerOn) throws KeeperException {
+ byte [] upData = toByteArray(balancerOn);
+ try {
+ ZKUtil.setData(watcher, watcher.znodePaths.balancerZNode, upData);
+ } catch(KeeperException.NoNodeException nne) {
+ ZKUtil.createAndWatch(watcher, watcher.znodePaths.balancerZNode, upData);
+ }
+ super.nodeDataChanged(watcher.znodePaths.balancerZNode);
+ }
+
+ private byte [] toByteArray(boolean isBalancerOn) {
+ LoadBalancerProtos.LoadBalancerState.Builder builder =
+ LoadBalancerProtos.LoadBalancerState.newBuilder();
+ builder.setBalancerOn(isBalancerOn);
+ return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
+ }
+
+ private LoadBalancerProtos.LoadBalancerState parseFrom(byte [] pbBytes)
+ throws DeserializationException {
+ ProtobufUtil.expectPBMagicPrefix(pbBytes);
+ LoadBalancerProtos.LoadBalancerState.Builder builder =
+ LoadBalancerProtos.LoadBalancerState.newBuilder();
+ try {
+ int magicLen = ProtobufUtil.lengthOfPBMagic();
+ ProtobufUtil.mergeFrom(builder, pbBytes, magicLen, pbBytes.length - magicLen);
+ } catch (IOException e) {
+ throw new DeserializationException(e);
+ }
+ return builder.build();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
new file mode 100644
index 0000000..85668ad
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
@@ -0,0 +1,281 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+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.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.
+ * <p>
+ * Listens for ZooKeeper events related to the master address. The node
+ * <code>/master</code> will contain the address of the current master.
+ * This listener is interested in
+ * <code>NodeDeleted</code> and <code>NodeCreated</code> events on
+ * <code>/master</code>.
+ * <p>
+ * Utilizes {@link ZKNodeTracker} for zk interactions.
+ * <p>
+ * You can get the current master via {@link #getMasterAddress()} or via
+ * {@link #getMasterAddress(ZKWatcher)} if you do not have a running
+ * instance of this Tracker in your context.
+ * <p>
+ * This class also includes utility for interacting with the master znode, for
+ * writing and reading the znode content.
+ */
+@InterfaceAudience.Private
+public class MasterAddressTracker extends ZKNodeTracker {
+ /**
+ * Construct a master address listener with the specified
+ * <code>zookeeper</code> reference.
+ * <p>
+ * This constructor does not trigger any actions, you must call methods
+ * explicitly. Normally you will just want to execute {@link #start()} to
+ * begin tracking of the master address.
+ *
+ * @param watcher zk reference and watcher
+ * @param abortable abortable in case of fatal error
+ */
+ public MasterAddressTracker(ZKWatcher watcher, Abortable abortable) {
+ super(watcher, watcher.znodePaths.masterAddressZNode, abortable);
+ }
+
+ /**
+ * Get the address of the current master if one is available. Returns null
+ * if no current master.
+ * @return Server name or null if timed out.
+ */
+ public ServerName getMasterAddress() {
+ return getMasterAddress(false);
+ }
+
+ /**
+ * Get the info port of the current master of one is available.
+ * Return 0 if no current master or zookeeper is unavailable
+ * @return info port or 0 if timed out
+ */
+ public int getMasterInfoPort() {
+ try {
+ final ZooKeeperProtos.Master master = parse(this.getData(false));
+ if (master == null) {
+ return 0;
+ }
+ return master.getInfoPort();
+ } catch (DeserializationException e) {
+ LOG.warn("Failed parse master zk node data", e);
+ return 0;
+ }
+ }
+ /**
+ * Get the info port of the backup master if it is available.
+ * Return 0 if no backup master or zookeeper is unavailable
+ * @param sn server name of backup master
+ * @return info port or 0 if timed out or exceptions
+ */
+ public int getBackupMasterInfoPort(final ServerName sn) {
+ String backupZNode = ZNodePaths.joinZNode(watcher.znodePaths.backupMasterAddressesZNode,
+ sn.toString());
+ try {
+ byte[] data = ZKUtil.getData(watcher, backupZNode);
+ final ZooKeeperProtos.Master backup = parse(data);
+ if (backup == null) {
+ return 0;
+ }
+ return backup.getInfoPort();
+ } catch (Exception e) {
+ LOG.warn("Failed to get backup master: " + sn + "'s info port.", e);
+ return 0;
+ }
+ }
+
+ /**
+ * Get the address of the current master if one is available. Returns null
+ * if no current master. If refresh is set, try to load the data from ZK again,
+ * otherwise, cached data will be used.
+ *
+ * @param refresh whether to refresh the data by calling ZK directly.
+ * @return Server name or null if timed out.
+ */
+ public ServerName getMasterAddress(final boolean refresh) {
+ try {
+ return ProtobufUtil.parseServerNameFrom(super.getData(refresh));
+ } catch (DeserializationException e) {
+ LOG.warn("Failed parse", e);
+ return null;
+ }
+ }
+
+ /**
+ * Get master address.
+ * Use this instead of {@link #getMasterAddress()} if you do not have an
+ * instance of this tracker in your context.
+ * @param zkw ZKWatcher to use
+ * @return ServerName stored in the the master address znode or null if no
+ * znode present.
+ * @throws KeeperException
+ * @throws IOException
+ */
+ public static ServerName getMasterAddress(final ZKWatcher zkw)
+ throws KeeperException, IOException {
+ byte [] data;
+ try {
+ data = ZKUtil.getData(zkw, zkw.znodePaths.masterAddressZNode);
+ } catch (InterruptedException e) {
+ throw new InterruptedIOException();
+ }
+ // TODO javadoc claims we return null in this case. :/
+ if (data == null){
+ throw new IOException("Can't get master address from ZooKeeper; znode data == null");
+ }
+ try {
+ return ProtobufUtil.parseServerNameFrom(data);
+ } catch (DeserializationException e) {
+ KeeperException ke = new KeeperException.DataInconsistencyException();
+ ke.initCause(e);
+ throw ke;
+ }
+ }
+
+ /**
+ * Get master info port.
+ * Use this instead of {@link #getMasterInfoPort()} if you do not have an
+ * instance of this tracker in your context.
+ * @param zkw ZKWatcher to use
+ * @return master info port in the the master address znode or null if no
+ * znode present.
+ * // TODO can't return null for 'int' return type. non-static verison returns 0
+ * @throws KeeperException
+ * @throws IOException
+ */
+ public static int getMasterInfoPort(final ZKWatcher zkw) throws KeeperException,
+ IOException {
+ byte[] data;
+ try {
+ data = ZKUtil.getData(zkw, zkw.znodePaths.masterAddressZNode);
+ } catch (InterruptedException e) {
+ throw new InterruptedIOException();
+ }
+ // TODO javadoc claims we return null in this case. :/
+ if (data == null) {
+ throw new IOException("Can't get master address from ZooKeeper; znode data == null");
+ }
+ try {
+ return parse(data).getInfoPort();
+ } catch (DeserializationException e) {
+ KeeperException ke = new KeeperException.DataInconsistencyException();
+ ke.initCause(e);
+ throw ke;
+ }
+ }
+
+ /**
+ * Set master address into the <code>master</code> znode or into the backup
+ * subdirectory of backup masters; switch off the passed in <code>znode</code>
+ * path.
+ * @param zkw The ZKWatcher to use.
+ * @param znode Where to create the znode; could be at the top level or it
+ * could be under backup masters
+ * @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
+ * @throws KeeperException
+ */
+ public static boolean setMasterAddress(final ZKWatcher zkw,
+ final String znode, final ServerName master, int infoPort)
+ throws KeeperException {
+ return ZKUtil.createEphemeralNodeAndWatch(zkw, znode, toByteArray(master, infoPort));
+ }
+
+ /**
+ * Check if there is a master available.
+ * @return true if there is a master set, false if not.
+ */
+ public boolean hasMaster() {
+ return super.getData(false) != null;
+ }
+
+ /**
+ * @param sn must not be null
+ * @return Content of the master znode as a serialized pb with the pb
+ * magic as prefix.
+ */
+ static byte[] toByteArray(final ServerName sn, int infoPort) {
+ ZooKeeperProtos.Master.Builder mbuilder = ZooKeeperProtos.Master.newBuilder();
+ HBaseProtos.ServerName.Builder snbuilder = HBaseProtos.ServerName.newBuilder();
+ snbuilder.setHostName(sn.getHostname());
+ snbuilder.setPort(sn.getPort());
+ snbuilder.setStartCode(sn.getStartcode());
+ mbuilder.setMaster(snbuilder.build());
+ mbuilder.setRpcVersion(HConstants.RPC_CURRENT_VERSION);
+ mbuilder.setInfoPort(infoPort);
+ return ProtobufUtil.prependPBMagic(mbuilder.build().toByteArray());
+ }
+
+ /**
+ * @param data zookeeper data. may be null
+ * @return pb object of master, null if no active master
+ * @throws DeserializationException
+ */
+ public static ZooKeeperProtos.Master parse(byte[] data) throws DeserializationException {
+ if (data == null) {
+ return null;
+ }
+ int prefixLen = ProtobufUtil.lengthOfPBMagic();
+ try {
+ return ZooKeeperProtos.Master.PARSER.parseFrom(data, prefixLen, data.length - prefixLen);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ }
+ /**
+ * delete the master znode if its content is same as the parameter
+ * @param zkw must not be null
+ * @param content must not be null
+ */
+ public static boolean deleteIfEquals(ZKWatcher zkw, final String content) {
+ if (content == null){
+ throw new IllegalArgumentException("Content must not be null");
+ }
+
+ try {
+ Stat stat = new Stat();
+ byte[] data = ZKUtil.getDataNoWatch(zkw, zkw.znodePaths.masterAddressZNode, stat);
+ ServerName sn = ProtobufUtil.parseServerNameFrom(data);
+ if (sn != null && content.equals(sn.toString())) {
+ return (ZKUtil.deleteNode(zkw, zkw.znodePaths.masterAddressZNode, stat.getVersion()));
+ }
+ } catch (KeeperException e) {
+ LOG.warn("Can't get or delete the master znode", e);
+ } catch (DeserializationException e) {
+ LOG.warn("Can't get or delete the master znode", e);
+ }
+
+ return false;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java
new file mode 100644
index 0000000..952da6f
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java
@@ -0,0 +1,81 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.util.List;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Tracks the master Maintenance Mode via ZK.
+ */
+@InterfaceAudience.Private
+public class MasterMaintenanceModeTracker extends ZKListener {
+ private boolean hasChildren;
+
+ public MasterMaintenanceModeTracker(ZKWatcher watcher) {
+ super(watcher);
+ hasChildren = false;
+ }
+
+ public boolean isInMaintenanceMode() {
+ return hasChildren;
+ }
+
+ private void update(String path) {
+ if (path.startsWith(watcher.znodePaths.masterMaintZNode)) {
+ update();
+ }
+ }
+
+ private void update() {
+ try {
+ List<String> children =
+ ZKUtil.listChildrenAndWatchForNewChildren(watcher, watcher.znodePaths.masterMaintZNode);
+ hasChildren = (children != null && children.size() > 0);
+ } catch (KeeperException e) {
+ // Ignore the ZK keeper exception
+ hasChildren = false;
+ }
+ }
+
+ /**
+ * Starts the tracking of whether master is in Maintenance Mode.
+ */
+ public void start() {
+ watcher.registerListener(this);
+ update();
+ }
+
+ @Override
+ public void nodeCreated(String path) {
+ update(path);
+ }
+
+ @Override
+ public void nodeDeleted(String path) {
+ update(path);
+ }
+
+ @Override
+ public void nodeChildrenChanged(String path) {
+ update(path);
+ }
+}
[13/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper
from hbase-server and hbase-client
Posted by ap...@apache.org.
HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client
- Moved DrainingServerTracker and RegionServerTracker to hbase-server:o.a.h.h.master.
- Moved SplitOrMergeTracker to oahh.master (because it depends on a PB)
- Moving hbase-client:oahh.zookeeper.* to hbase-zookeeper module. After HBASE-19200, hbase-client doesn't need them anymore (except 3 classes).
- Renamed some classes to use a consistent naming for classes - ZK instead of mix of ZK, Zk , ZooKeeper. Couldn't rename following public classes: MiniZooKeeperCluster, ZooKeeperConnectionException. Left RecoverableZooKeeper for lack of better name. (suggestions?)
- Sadly, can't move tests out because they depend on HBaseTestingUtility (which defeats part of the purpose - trimming down hbase-server tests. We need to promote more use of mocks in our tests)
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/330b0d05
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/330b0d05
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/330b0d05
Branch: refs/heads/master
Commit: 330b0d05b99981b4bdc92c81b22ebb5be5ece155
Parents: ca74ec7
Author: Apekshit Sharma <ap...@apache.org>
Authored: Mon Nov 13 11:42:33 2017 -0800
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Fri Nov 17 13:20:18 2017 -0800
----------------------------------------------------------------------
bin/hbase | 14 +-
bin/hbase.cmd | 10 +-
conf/log4j.properties | 2 +-
.../src/main/resources/log4j.properties | 2 +-
.../src/main/resources/log4j.properties | 2 +-
hbase-assembly/pom.xml | 4 +
.../src/main/assembly/hadoop-two-compat.xml | 1 +
.../hadoop/hbase/zookeeper/EmptyWatcher.java | 34 -
.../hadoop/hbase/zookeeper/HQuorumPeer.java | 165 --
.../hadoop/hbase/zookeeper/InstancePending.java | 80 -
.../hbase/zookeeper/MasterAddressTracker.java | 281 ---
.../hbase/zookeeper/MetaTableLocator.java | 629 ------
.../hbase/zookeeper/MetricsZooKeeper.java | 110 -
.../hadoop/hbase/zookeeper/PendingWatcher.java | 53 -
.../hbase/zookeeper/RecoverableZooKeeper.java | 814 -------
.../hadoop/hbase/zookeeper/ZKClusterId.java | 99 -
.../hadoop/hbase/zookeeper/ZKLeaderManager.java | 183 --
.../apache/hadoop/hbase/zookeeper/ZKUtil.java | 2072 ------------------
.../hadoop/hbase/zookeeper/ZkAclReset.java | 116 -
.../hbase/zookeeper/ZooKeeperListener.java | 86 -
.../zookeeper/ZooKeeperMetricsListener.java | 91 -
.../hbase/zookeeper/ZooKeeperNodeTracker.java | 251 ---
.../hbase/zookeeper/ZooKeeperWatcher.java | 638 ------
.../hbase/zookeeper/TestInstancePending.java | 49 -
.../hbase/zookeeper/TestMetricsZooKeeper.java | 77 -
.../hadoop/hbase/zookeeper/TestZKUtil.java | 113 -
.../hbase/zookeeper/TestZooKeeperWatcher.java | 57 -
.../hbase/IntegrationTestMetaReplicas.java | 6 +-
.../test/IntegrationTestZKAndFSPermissions.java | 8 +-
.../apache/hadoop/hbase/mapreduce/Import.java | 6 +-
.../replication/VerifyReplication.java | 6 +-
hbase-replication/pom.xml | 4 +
.../hbase/replication/ReplicationFactory.java | 12 +-
.../replication/ReplicationPeerZKImpl.java | 14 +-
.../replication/ReplicationPeersZKImpl.java | 6 +-
.../replication/ReplicationQueuesArguments.java | 10 +-
.../ReplicationQueuesClientArguments.java | 4 +-
.../ReplicationQueuesClientZKImpl.java | 12 +-
.../replication/ReplicationQueuesZKImpl.java | 10 +-
.../replication/ReplicationStateZKBase.java | 16 +-
.../replication/ReplicationTrackerZKImpl.java | 14 +-
.../TableBasedReplicationQueuesImpl.java | 5 +-
.../hbase/rsgroup/RSGroupInfoManagerImpl.java | 4 +-
.../rsgroup/VerifyingRSGroupAdminClient.java | 6 +-
hbase-server/pom.xml | 4 +
.../java/org/apache/hadoop/hbase/Server.java | 4 +-
.../apache/hadoop/hbase/ZKNamespaceManager.java | 8 +-
.../org/apache/hadoop/hbase/ZNodeClearer.java | 16 +-
.../backup/example/HFileArchiveManager.java | 10 +-
.../example/TableHFileArchiveTracker.java | 14 +-
.../backup/example/ZKTableArchiveClient.java | 4 +-
.../ZKSplitLogManagerCoordination.java | 8 +-
.../coordination/ZkCoordinatedStateManager.java | 6 +-
.../ZkSplitLogWorkerCoordination.java | 14 +-
.../hbase/master/ActiveMasterManager.java | 14 +-
.../hbase/master/DrainingServerTracker.java | 135 ++
.../org/apache/hadoop/hbase/master/HMaster.java | 8 +-
.../hbase/master/MasterMetaBootstrap.java | 4 +-
.../hbase/master/MetricsMasterWrapperImpl.java | 8 +-
.../hbase/master/RegionServerTracker.java | 164 ++
.../hadoop/hbase/master/ServerManager.java | 7 +-
.../hbase/master/SplitOrMergeTracker.java | 153 ++
.../cleaner/ReplicationZKNodeCleaner.java | 8 +-
.../master/replication/ReplicationManager.java | 6 +-
.../hbase/procedure/ZKProcedureCoordinator.java | 6 +-
.../hbase/procedure/ZKProcedureMemberRpcs.java | 6 +-
.../hadoop/hbase/procedure/ZKProcedureUtil.java | 10 +-
.../RegionServerFlushTableProcedureManager.java | 5 +-
.../hbase/regionserver/HRegionServer.java | 12 +-
.../MetricsRegionServerWrapperImpl.java | 4 +-
.../snapshot/RegionServerSnapshotManager.java | 5 +-
.../replication/HBaseReplicationEndpoint.java | 14 +-
.../master/ReplicationHFileCleaner.java | 11 +-
.../master/ReplicationLogCleaner.java | 8 +-
.../replication/master/TableCFsUpdater.java | 6 +-
.../regionserver/DumpReplicationQueues.java | 8 +-
.../regionserver/ReplicationSyncUp.java | 12 +-
.../hbase/security/access/AccessController.java | 4 +-
.../hbase/security/access/TableAuthManager.java | 8 +-
.../security/access/ZKPermissionWatcher.java | 11 +-
.../token/AuthenticationTokenSecretManager.java | 14 +-
.../hadoop/hbase/security/token/TokenUtil.java | 4 +-
.../hbase/security/token/ZKSecretWatcher.java | 12 +-
.../DefaultVisibilityLabelServiceImpl.java | 4 +-
.../visibility/VisibilityLabelsCache.java | 8 +-
.../visibility/ZKVisibilityLabelWatcher.java | 14 +-
.../org/apache/hadoop/hbase/util/HBaseFsck.java | 8 +-
.../apache/hadoop/hbase/util/RegionMover.java | 4 +-
.../hadoop/hbase/util/ZKDataMigrator.java | 10 +-
.../hbase/util/hbck/ReplicationChecker.java | 6 +-
.../hbase/zookeeper/ClusterStatusTracker.java | 99 -
.../hbase/zookeeper/DeletionListener.java | 101 -
.../hbase/zookeeper/DrainingServerTracker.java | 134 --
.../hbase/zookeeper/LoadBalancerTracker.java | 94 -
.../zookeeper/MasterMaintenanceModeTracker.java | 81 -
.../hbase/zookeeper/MiniZooKeeperCluster.java | 472 ----
.../zookeeper/RegionNormalizerTracker.java | 94 -
.../hbase/zookeeper/RegionServerTracker.java | 162 --
.../hbase/zookeeper/SplitOrMergeTracker.java | 149 --
.../hadoop/hbase/zookeeper/ZKServerTool.java | 65 -
.../hadoop/hbase/zookeeper/ZKSplitLog.java | 124 --
.../hbase/zookeeper/ZooKeeperMainServer.java | 126 --
.../main/resources/hbase-webapps/master/zk.jsp | 4 +-
.../hadoop/hbase/HBaseTestingUtility.java | 22 +-
.../hadoop/hbase/MockRegionServerServices.java | 11 +-
.../hbase/TestMetaTableAccessorNoCluster.java | 4 +-
.../hadoop/hbase/TestMetaTableLocator.java | 6 +-
.../org/apache/hadoop/hbase/TestZooKeeper.java | 18 +-
.../TestZooKeeperTableArchiveClient.java | 4 +-
.../hbase/client/TestMetaWithReplicas.java | 10 +-
.../replication/TestReplicationAdmin.java | 8 +-
...TestMasterCoprocessorExceptionWithAbort.java | 10 +-
...estMasterCoprocessorExceptionWithRemove.java | 10 +-
.../hbase/master/MockNoopMasterServices.java | 4 +-
.../hadoop/hbase/master/MockRegionServer.java | 9 +-
.../hbase/master/TestActiveMasterManager.java | 26 +-
.../hbase/master/TestAssignmentListener.java | 6 +-
.../hbase/master/TestClockSkewDetection.java | 6 -
.../master/TestDistributedLogSplitting.java | 22 +-
.../hbase/master/TestHMasterRPCException.java | 4 +-
.../hbase/master/TestMasterNoCluster.java | 4 +-
.../hbase/master/TestMasterStatusServlet.java | 4 +-
.../hbase/master/TestSplitLogManager.java | 12 +-
.../hbase/master/TestTableStateManager.java | 8 +-
.../hbase/master/cleaner/TestHFileCleaner.java | 6 +-
.../master/cleaner/TestHFileLinkCleaner.java | 6 +-
.../hbase/master/cleaner/TestLogsCleaner.java | 10 +-
.../cleaner/TestReplicationHFileCleaner.java | 10 +-
.../cleaner/TestReplicationZKNodeCleaner.java | 6 +-
.../procedure/SimpleRSProcedureManager.java | 4 +-
.../hadoop/hbase/procedure/TestZKProcedure.java | 14 +-
.../procedure/TestZKProcedureControllers.java | 28 +-
.../TestCompactionInDeadRegionServer.java | 5 +-
.../regionserver/TestHeapMemoryManager.java | 4 +-
.../regionserver/TestMasterAddressTracker.java | 10 +-
.../hbase/regionserver/TestRSStatusServlet.java | 4 +-
.../regionserver/TestRegionServerHostname.java | 6 +-
.../regionserver/TestRegionServerNoMaster.java | 4 +-
.../hbase/regionserver/TestSplitLogWorker.java | 12 +-
.../hbase/regionserver/TestWALLockup.java | 4 +-
.../replication/TestMasterReplication.java | 6 +-
.../replication/TestMultiSlaveReplication.java | 19 +-
.../replication/TestPerTableCFReplication.java | 8 +-
.../hbase/replication/TestReplicationBase.java | 12 +-
.../TestReplicationStateHBaseImpl.java | 6 +-
.../replication/TestReplicationStateZKImpl.java | 8 +-
.../replication/TestReplicationTableBase.java | 4 +-
.../TestReplicationTrackerZKImpl.java | 14 +-
.../replication/TestSerialReplication.java | 6 +-
.../replication/master/TestTableCFsUpdater.java | 6 +-
.../regionserver/TestGlobalThrottler.java | 6 +-
.../TestReplicationSourceManager.java | 8 +-
.../security/access/TestAccessController2.java | 4 +-
.../security/access/TestTablePermissions.java | 6 +-
.../access/TestZKPermissionWatcher.java | 6 +-
.../security/token/TestTokenAuthentication.java | 8 +-
.../security/token/TestZKSecretWatcher.java | 26 +-
.../token/TestZKSecretWatcherRefreshKeys.java | 16 +-
...sibilityLabelReplicationWithExpAsString.java | 6 +-
.../TestVisibilityLabelsReplication.java | 10 +-
.../apache/hadoop/hbase/util/MockServer.java | 10 +-
.../hadoop/hbase/util/TestHBaseFsckOneRS.java | 4 +-
.../zookeeper/TestRecoverableZooKeeper.java | 2 +-
.../hbase/zookeeper/TestZKLeaderManager.java | 14 +-
.../hbase/zookeeper/TestZKMainServer.java | 118 +
.../hadoop/hbase/zookeeper/TestZKMulti.java | 4 +-
.../hbase/zookeeper/TestZKNodeTracker.java | 351 +++
.../hbase/zookeeper/TestZooKeeperACL.java | 12 +-
.../zookeeper/TestZooKeeperMainServer.java | 118 -
.../zookeeper/TestZooKeeperNodeTracker.java | 351 ---
hbase-shell/src/main/ruby/hbase/admin.rb | 2 +-
hbase-zookeeper/pom.xml | 412 ++++
.../hbase/zookeeper/ClusterStatusTracker.java | 99 +
.../hbase/zookeeper/DeletionListener.java | 101 +
.../hadoop/hbase/zookeeper/EmptyWatcher.java | 34 +
.../hadoop/hbase/zookeeper/HQuorumPeer.java | 165 ++
.../hadoop/hbase/zookeeper/InstancePending.java | 80 +
.../hbase/zookeeper/LoadBalancerTracker.java | 94 +
.../hbase/zookeeper/MasterAddressTracker.java | 281 +++
.../zookeeper/MasterMaintenanceModeTracker.java | 81 +
.../hbase/zookeeper/MetaTableLocator.java | 629 ++++++
.../hbase/zookeeper/MiniZooKeeperCluster.java | 472 ++++
.../hadoop/hbase/zookeeper/PendingWatcher.java | 53 +
.../hbase/zookeeper/RecoverableZooKeeper.java | 810 +++++++
.../zookeeper/RegionNormalizerTracker.java | 94 +
.../hadoop/hbase/zookeeper/ZKAclReset.java | 116 +
.../hadoop/hbase/zookeeper/ZKClusterId.java | 99 +
.../hadoop/hbase/zookeeper/ZKLeaderManager.java | 183 ++
.../hadoop/hbase/zookeeper/ZKListener.java | 85 +
.../hadoop/hbase/zookeeper/ZKMainServer.java | 126 ++
.../hadoop/hbase/zookeeper/ZKMetrics.java | 108 +
.../hbase/zookeeper/ZKMetricsListener.java | 91 +
.../hadoop/hbase/zookeeper/ZKNodeTracker.java | 251 +++
.../hadoop/hbase/zookeeper/ZKServerTool.java | 65 +
.../hadoop/hbase/zookeeper/ZKSplitLog.java | 122 ++
.../apache/hadoop/hbase/zookeeper/ZKUtil.java | 2072 ++++++++++++++++++
.../hadoop/hbase/zookeeper/ZKWatcher.java | 634 ++++++
.../hbase/zookeeper/TestInstancePending.java | 49 +
.../hadoop/hbase/zookeeper/TestZKMetrics.java | 80 +
.../hadoop/hbase/zookeeper/TestZKUtil.java | 113 +
.../hadoop/hbase/zookeeper/TestZKWatcher.java | 57 +
pom.xml | 6 +
202 files changed, 9129 insertions(+), 8735 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/bin/hbase
----------------------------------------------------------------------
diff --git a/bin/hbase b/bin/hbase
index 998bdbe..d98e7bc 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -17,7 +17,7 @@
# * See the License for the specific language governing permissions and
# * limitations under the License.
# */
-#
+#
# The hbase command script. Based on the hadoop command script putting
# in hbase classes, libs and configurations ahead of hadoop's.
#
@@ -91,10 +91,10 @@ if [ $# = 0 ]; then
echo " hfile Store file analyzer"
echo " zkcli Run the ZooKeeper shell"
echo " master Run an HBase HMaster node"
- echo " regionserver Run an HBase HRegionServer node"
+ echo " regionserver Run an HBase HRegionServer node"
echo " zookeeper Run a ZooKeeper server"
- echo " rest Run an HBase REST server"
- echo " thrift Run the HBase Thrift server"
+ echo " rest Run an HBase REST server"
+ echo " thrift Run the HBase Thrift server"
echo " thrift2 Run the HBase Thrift2 server"
echo " clean Run the HBase clean up script"
echo " classpath Dump hbase CLASSPATH"
@@ -274,7 +274,7 @@ fi
if $cygwin; then
JAVA_LIBRARY_PATH=`cygpath -p "$JAVA_LIBRARY_PATH"`
fi
-
+
# restore ordinary behaviour
unset IFS
@@ -362,7 +362,7 @@ elif [ "$COMMAND" = "wal" ] ; then
elif [ "$COMMAND" = "hfile" ] ; then
CLASS='org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter'
elif [ "$COMMAND" = "zkcli" ] ; then
- CLASS="org.apache.hadoop.hbase.zookeeper.ZooKeeperMainServer"
+ CLASS="org.apache.hadoop.hbase.zookeeper.ZKMainServer"
elif [ "$COMMAND" = "backup" ] ; then
CLASS='org.apache.hadoop.hbase.backup.BackupDriver'
elif [ "$COMMAND" = "restore" ] ; then
@@ -422,7 +422,7 @@ elif [ "$COMMAND" = "zookeeper" ] ; then
fi
elif [ "$COMMAND" = "clean" ] ; then
case $1 in
- --cleanZk|--cleanHdfs|--cleanAll)
+ --cleanZk|--cleanHdfs|--cleanAll)
matches="yes" ;;
*) ;;
esac
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/bin/hbase.cmd
----------------------------------------------------------------------
diff --git a/bin/hbase.cmd b/bin/hbase.cmd
index 070cb61..8e4a59f 100644
--- a/bin/hbase.cmd
+++ b/bin/hbase.cmd
@@ -16,7 +16,7 @@
@rem * See the License for the specific language governing permissions and
@rem * limitations under the License.
@rem */
-@rem
+@rem
@rem The hbase command script. Based on the hadoop command script putting
@rem in hbase classes, libs and configurations ahead of hadoop's.
@rem
@@ -197,7 +197,7 @@ if exist "%HBASE_HOME%\build\native" (
rem This loop would set %hbase-command-arguments%
set _hbasearguments=
:MakeCmdArgsLoop
- if [%1]==[] goto :EndLoop
+ if [%1]==[] goto :EndLoop
if not defined _hbasearguments (
set _hbasearguments=%1
@@ -205,8 +205,8 @@ set _hbasearguments=
set _hbasearguments=!_hbasearguments! %1
)
shift
-goto :MakeCmdArgsLoop
-:EndLoop
+goto :MakeCmdArgsLoop
+:EndLoop
set hbase-command-arguments=%_hbasearguments%
@@ -424,7 +424,7 @@ goto :eof
goto :eof
:zkcli
- set CLASS=org.apache.hadoop.hbase.zookeeper.ZooKeeperMainServer
+ set CLASS=org.apache.hadoop.hbase.zookeeper.ZKMainServer
goto :eof
:mapredcp
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/conf/log4j.properties b/conf/log4j.properties
index 15545ff..6367dbe 100644
--- a/conf/log4j.properties
+++ b/conf/log4j.properties
@@ -98,7 +98,7 @@ log4j.logger.org.apache.hadoop.hbase=INFO
log4j.logger.org.apache.hadoop.hbase.META=INFO
# Make these two classes INFO-level. Make them DEBUG to see more zk debug.
log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=INFO
-log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
+log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKWatcher=INFO
#log4j.logger.org.apache.hadoop.dfs=DEBUG
# Set this class to log INFO only otherwise its OTT
# Enable this to get detailed connection error/retry logging.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-archetypes/hbase-client-project/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-client-project/src/main/resources/log4j.properties b/hbase-archetypes/hbase-client-project/src/main/resources/log4j.properties
index 11f2b75..0b01e57 100644
--- a/hbase-archetypes/hbase-client-project/src/main/resources/log4j.properties
+++ b/hbase-archetypes/hbase-client-project/src/main/resources/log4j.properties
@@ -95,7 +95,7 @@ log4j.logger.org.apache.zookeeper=INFO
log4j.logger.org.apache.hadoop.hbase=INFO
# Make these two classes INFO-level. Make them DEBUG to see more zk debug.
log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=INFO
-log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
+log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKWatcher=INFO
#log4j.logger.org.apache.hadoop.dfs=DEBUG
# Set this class to log INFO only otherwise its OTT
# Enable this to get detailed connection error/retry logging.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-archetypes/hbase-shaded-client-project/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-shaded-client-project/src/main/resources/log4j.properties b/hbase-archetypes/hbase-shaded-client-project/src/main/resources/log4j.properties
index 11f2b75..0b01e57 100644
--- a/hbase-archetypes/hbase-shaded-client-project/src/main/resources/log4j.properties
+++ b/hbase-archetypes/hbase-shaded-client-project/src/main/resources/log4j.properties
@@ -95,7 +95,7 @@ log4j.logger.org.apache.zookeeper=INFO
log4j.logger.org.apache.hadoop.hbase=INFO
# Make these two classes INFO-level. Make them DEBUG to see more zk debug.
log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=INFO
-log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
+log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKWatcher=INFO
#log4j.logger.org.apache.hadoop.dfs=DEBUG
# Set this class to log INFO only otherwise its OTT
# Enable this to get detailed connection error/retry logging.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml
index 71e90f3..75cba4d 100644
--- a/hbase-assembly/pom.xml
+++ b/hbase-assembly/pom.xml
@@ -303,6 +303,10 @@
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-examples</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-zookeeper</artifactId>
+ </dependency>
</dependencies>
<profiles>
<profile>
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
index 03951bd..2aa3e57 100644
--- a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
+++ b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
@@ -59,6 +59,7 @@
<include>org.apache.hbase:hbase-spark-it</include>
<include>org.apache.hbase:hbase-testing-util</include>
<include>org.apache.hbase:hbase-thrift</include>
+ <include>org.apache.hbase:hbase-zookeeper</include>
</includes>
<!-- Binaries for the dependencies also go in the hbase-jars directory -->
<binaries>
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java
deleted file mode 100644
index 191943a..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-
-/**
- * An empty ZooKeeper watcher
- */
-@InterfaceAudience.Private
-public class EmptyWatcher implements Watcher {
- // Used in this package but also by tests so needs to be public
- public static final EmptyWatcher instance = new EmptyWatcher();
- private EmptyWatcher() {}
-
- public void process(WatchedEvent event) {}
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
deleted file mode 100644
index 5f3904a..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT;
-import static org.apache.hadoop.hbase.HConstants.ZK_SESSION_TIMEOUT;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.net.InetAddress;
-import java.net.NetworkInterface;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Enumeration;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.Properties;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-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.Strings;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.zookeeper.server.ServerConfig;
-import org.apache.zookeeper.server.ZooKeeperServerMain;
-import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
-import org.apache.zookeeper.server.quorum.QuorumPeerMain;
-
-/**
- * HBase's version of ZooKeeper's QuorumPeer. When HBase is set to manage
- * ZooKeeper, this class is used to start up QuorumPeer instances. By doing
- * things in here rather than directly calling to ZooKeeper, we have more
- * control over the process. This class uses {@link ZKConfig} to get settings
- * from the hbase-site.xml file.
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
-@InterfaceStability.Evolving
-public class HQuorumPeer {
-
- /**
- * Parse ZooKeeper configuration from HBase XML config and run a QuorumPeer.
- * @param args String[] of command line arguments. Not used.
- */
- public static void main(String[] args) {
- Configuration conf = HBaseConfiguration.create();
- try {
- Properties zkProperties = ZKConfig.makeZKProps(conf);
- writeMyID(zkProperties);
- QuorumPeerConfig zkConfig = new QuorumPeerConfig();
- zkConfig.parseProperties(zkProperties);
-
- // login the zookeeper server principal (if using security)
- ZKUtil.loginServer(conf, HConstants.ZK_SERVER_KEYTAB_FILE,
- HConstants.ZK_SERVER_KERBEROS_PRINCIPAL,
- zkConfig.getClientPortAddress().getHostName());
-
- runZKServer(zkConfig);
- } catch (Exception e) {
- e.printStackTrace();
- System.exit(-1);
- }
- }
-
- private static void runZKServer(QuorumPeerConfig zkConfig) throws UnknownHostException, IOException {
- if (zkConfig.isDistributed()) {
- QuorumPeerMain qp = new QuorumPeerMain();
- qp.runFromConfig(zkConfig);
- } else {
- ZooKeeperServerMain zk = new ZooKeeperServerMain();
- ServerConfig serverConfig = new ServerConfig();
- serverConfig.readFrom(zkConfig);
- zk.runFromConfig(serverConfig);
- }
- }
-
- private static boolean addressIsLocalHost(String address) {
- return address.equals("localhost") || address.equals("127.0.0.1");
- }
-
- static void writeMyID(Properties properties) throws IOException {
- long myId = -1;
-
- Configuration conf = HBaseConfiguration.create();
- String myAddress = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
- conf.get("hbase.zookeeper.dns.interface","default"),
- conf.get("hbase.zookeeper.dns.nameserver","default")));
-
- List<String> ips = new ArrayList<>();
-
- // Add what could be the best (configured) match
- ips.add(myAddress.contains(".") ?
- myAddress :
- StringUtils.simpleHostname(myAddress));
-
- // For all nics get all hostnames and IPs
- Enumeration<?> nics = NetworkInterface.getNetworkInterfaces();
- while(nics.hasMoreElements()) {
- Enumeration<?> rawAdrs =
- ((NetworkInterface)nics.nextElement()).getInetAddresses();
- while(rawAdrs.hasMoreElements()) {
- InetAddress inet = (InetAddress) rawAdrs.nextElement();
- ips.add(StringUtils.simpleHostname(inet.getHostName()));
- ips.add(inet.getHostAddress());
- }
- }
-
- for (Entry<Object, Object> entry : properties.entrySet()) {
- String key = entry.getKey().toString().trim();
- String value = entry.getValue().toString().trim();
- if (key.startsWith("server.")) {
- int dot = key.indexOf('.');
- long id = Long.parseLong(key.substring(dot + 1));
- String[] parts = value.split(":");
- String address = parts[0];
- if (addressIsLocalHost(address) || ips.contains(address)) {
- myId = id;
- break;
- }
- }
- }
-
- // Set the max session timeout from the provided client-side timeout
- properties.setProperty("maxSessionTimeout",
- conf.get(ZK_SESSION_TIMEOUT, Integer.toString(DEFAULT_ZK_SESSION_TIMEOUT)));
-
- if (myId == -1) {
- throw new IOException("Could not find my address: " + myAddress +
- " in list of ZooKeeper quorum servers");
- }
-
- String dataDirStr = properties.get("dataDir").toString().trim();
- File dataDir = new File(dataDirStr);
- if (!dataDir.isDirectory()) {
- if (!dataDir.mkdirs()) {
- throw new IOException("Unable to create data dir " + dataDir);
- }
- }
-
- File myIdFile = new File(dataDir, "myid");
- PrintWriter w = new PrintWriter(myIdFile);
- w.println(myId);
- w.close();
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java
deleted file mode 100644
index e63bfc5..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.util.concurrent.CountDownLatch;
-
-/**
- * Placeholder of an instance which will be accessed by other threads
- * but is not yet created. Thread safe.
- */
-class InstancePending<T> {
- // Based on a subtle part of the Java Language Specification,
- // in order to avoid a slight overhead of synchronization for each access.
-
- private final CountDownLatch pendingLatch = new CountDownLatch(1);
-
- /** Piggybacking on {@code pendingLatch}. */
- private InstanceHolder<T> instanceHolder;
-
- private static class InstanceHolder<T> {
- // The JLS ensures the visibility of a final field and its contents
- // unless they are exposed to another thread while the construction.
- final T instance;
-
- InstanceHolder(T instance) {
- this.instance = instance;
- }
- }
-
- /**
- * Returns the instance given by the method {@link #prepare}.
- * This is an uninterruptible blocking method
- * and the interruption flag will be set just before returning if any.
- */
- T get() {
- InstanceHolder<T> instanceHolder;
- boolean interrupted = false;
-
- while ((instanceHolder = this.instanceHolder) == null) {
- try {
- pendingLatch.await();
- } catch (InterruptedException e) {
- interrupted = true;
- }
- }
-
- if (interrupted) {
- Thread.currentThread().interrupt();
- }
- return instanceHolder.instance;
- }
-
- /**
- * Associates the given instance for the method {@link #get}.
- * This method should be called once, and {@code instance} should be non-null.
- * This method is expected to call as soon as possible
- * because the method {@code get} is uninterruptibly blocked until this method is called.
- */
- void prepare(T instance) {
- assert instance != null;
- instanceHolder = new InstanceHolder<>(instance);
- pendingLatch.countDown();
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
deleted file mode 100644
index fba637f..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
+++ /dev/null
@@ -1,281 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-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.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.
- * <p>
- * Listens for ZooKeeper events related to the master address. The node
- * <code>/master</code> will contain the address of the current master.
- * This listener is interested in
- * <code>NodeDeleted</code> and <code>NodeCreated</code> events on
- * <code>/master</code>.
- * <p>
- * Utilizes {@link ZooKeeperNodeTracker} for zk interactions.
- * <p>
- * You can get the current master via {@link #getMasterAddress()} or via
- * {@link #getMasterAddress(ZooKeeperWatcher)} if you do not have a running
- * instance of this Tracker in your context.
- * <p>
- * This class also includes utility for interacting with the master znode, for
- * writing and reading the znode content.
- */
-@InterfaceAudience.Private
-public class MasterAddressTracker extends ZooKeeperNodeTracker {
- /**
- * Construct a master address listener with the specified
- * <code>zookeeper</code> reference.
- * <p>
- * This constructor does not trigger any actions, you must call methods
- * explicitly. Normally you will just want to execute {@link #start()} to
- * begin tracking of the master address.
- *
- * @param watcher zk reference and watcher
- * @param abortable abortable in case of fatal error
- */
- public MasterAddressTracker(ZooKeeperWatcher watcher, Abortable abortable) {
- super(watcher, watcher.znodePaths.masterAddressZNode, abortable);
- }
-
- /**
- * Get the address of the current master if one is available. Returns null
- * if no current master.
- * @return Server name or null if timed out.
- */
- public ServerName getMasterAddress() {
- return getMasterAddress(false);
- }
-
- /**
- * Get the info port of the current master of one is available.
- * Return 0 if no current master or zookeeper is unavailable
- * @return info port or 0 if timed out
- */
- public int getMasterInfoPort() {
- try {
- final ZooKeeperProtos.Master master = parse(this.getData(false));
- if (master == null) {
- return 0;
- }
- return master.getInfoPort();
- } catch (DeserializationException e) {
- LOG.warn("Failed parse master zk node data", e);
- return 0;
- }
- }
- /**
- * Get the info port of the backup master if it is available.
- * Return 0 if no backup master or zookeeper is unavailable
- * @param sn server name of backup master
- * @return info port or 0 if timed out or exceptions
- */
- public int getBackupMasterInfoPort(final ServerName sn) {
- String backupZNode = ZNodePaths.joinZNode(watcher.znodePaths.backupMasterAddressesZNode,
- sn.toString());
- try {
- byte[] data = ZKUtil.getData(watcher, backupZNode);
- final ZooKeeperProtos.Master backup = parse(data);
- if (backup == null) {
- return 0;
- }
- return backup.getInfoPort();
- } catch (Exception e) {
- LOG.warn("Failed to get backup master: " + sn + "'s info port.", e);
- return 0;
- }
- }
-
- /**
- * Get the address of the current master if one is available. Returns null
- * if no current master. If refresh is set, try to load the data from ZK again,
- * otherwise, cached data will be used.
- *
- * @param refresh whether to refresh the data by calling ZK directly.
- * @return Server name or null if timed out.
- */
- public ServerName getMasterAddress(final boolean refresh) {
- try {
- return ProtobufUtil.parseServerNameFrom(super.getData(refresh));
- } catch (DeserializationException e) {
- LOG.warn("Failed parse", e);
- return null;
- }
- }
-
- /**
- * Get master address.
- * Use this instead of {@link #getMasterAddress()} if you do not have an
- * instance of this tracker in your context.
- * @param zkw ZooKeeperWatcher to use
- * @return ServerName stored in the the master address znode or null if no
- * znode present.
- * @throws KeeperException
- * @throws IOException
- */
- public static ServerName getMasterAddress(final ZooKeeperWatcher zkw)
- throws KeeperException, IOException {
- byte [] data;
- try {
- data = ZKUtil.getData(zkw, zkw.znodePaths.masterAddressZNode);
- } catch (InterruptedException e) {
- throw new InterruptedIOException();
- }
- // TODO javadoc claims we return null in this case. :/
- if (data == null){
- throw new IOException("Can't get master address from ZooKeeper; znode data == null");
- }
- try {
- return ProtobufUtil.parseServerNameFrom(data);
- } catch (DeserializationException e) {
- KeeperException ke = new KeeperException.DataInconsistencyException();
- ke.initCause(e);
- throw ke;
- }
- }
-
- /**
- * Get master info port.
- * Use this instead of {@link #getMasterInfoPort()} if you do not have an
- * instance of this tracker in your context.
- * @param zkw ZooKeeperWatcher to use
- * @return master info port in the the master address znode or null if no
- * znode present.
- * // TODO can't return null for 'int' return type. non-static verison returns 0
- * @throws KeeperException
- * @throws IOException
- */
- public static int getMasterInfoPort(final ZooKeeperWatcher zkw) throws KeeperException,
- IOException {
- byte[] data;
- try {
- data = ZKUtil.getData(zkw, zkw.znodePaths.masterAddressZNode);
- } catch (InterruptedException e) {
- throw new InterruptedIOException();
- }
- // TODO javadoc claims we return null in this case. :/
- if (data == null) {
- throw new IOException("Can't get master address from ZooKeeper; znode data == null");
- }
- try {
- return parse(data).getInfoPort();
- } catch (DeserializationException e) {
- KeeperException ke = new KeeperException.DataInconsistencyException();
- ke.initCause(e);
- throw ke;
- }
- }
-
- /**
- * Set master address into the <code>master</code> znode or into the backup
- * subdirectory of backup masters; switch off the passed in <code>znode</code>
- * path.
- * @param zkw The ZooKeeperWatcher to use.
- * @param znode Where to create the znode; could be at the top level or it
- * could be under backup masters
- * @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
- * @throws KeeperException
- */
- public static boolean setMasterAddress(final ZooKeeperWatcher zkw,
- final String znode, final ServerName master, int infoPort)
- throws KeeperException {
- return ZKUtil.createEphemeralNodeAndWatch(zkw, znode, toByteArray(master, infoPort));
- }
-
- /**
- * Check if there is a master available.
- * @return true if there is a master set, false if not.
- */
- public boolean hasMaster() {
- return super.getData(false) != null;
- }
-
- /**
- * @param sn must not be null
- * @return Content of the master znode as a serialized pb with the pb
- * magic as prefix.
- */
- static byte[] toByteArray(final ServerName sn, int infoPort) {
- ZooKeeperProtos.Master.Builder mbuilder = ZooKeeperProtos.Master.newBuilder();
- HBaseProtos.ServerName.Builder snbuilder = HBaseProtos.ServerName.newBuilder();
- snbuilder.setHostName(sn.getHostname());
- snbuilder.setPort(sn.getPort());
- snbuilder.setStartCode(sn.getStartcode());
- mbuilder.setMaster(snbuilder.build());
- mbuilder.setRpcVersion(HConstants.RPC_CURRENT_VERSION);
- mbuilder.setInfoPort(infoPort);
- return ProtobufUtil.prependPBMagic(mbuilder.build().toByteArray());
- }
-
- /**
- * @param data zookeeper data. may be null
- * @return pb object of master, null if no active master
- * @throws DeserializationException
- */
- public static ZooKeeperProtos.Master parse(byte[] data) throws DeserializationException {
- if (data == null) {
- return null;
- }
- int prefixLen = ProtobufUtil.lengthOfPBMagic();
- try {
- return ZooKeeperProtos.Master.PARSER.parseFrom(data, prefixLen, data.length - prefixLen);
- } catch (InvalidProtocolBufferException e) {
- throw new DeserializationException(e);
- }
- }
- /**
- * delete the master znode if its content is same as the parameter
- * @param zkw must not be null
- * @param content must not be null
- */
- public static boolean deleteIfEquals(ZooKeeperWatcher zkw, final String content) {
- if (content == null){
- throw new IllegalArgumentException("Content must not be null");
- }
-
- try {
- Stat stat = new Stat();
- byte[] data = ZKUtil.getDataNoWatch(zkw, zkw.znodePaths.masterAddressZNode, stat);
- ServerName sn = ProtobufUtil.parseServerNameFrom(data);
- if (sn != null && content.equals(sn.toString())) {
- return (ZKUtil.deleteNode(zkw, zkw.znodePaths.masterAddressZNode, stat.getVersion()));
- }
- } catch (KeeperException e) {
- LOG.warn("Can't get or delete the master znode", e);
- } catch (DeserializationException e) {
- LOG.warn("Can't get or delete the master znode", e);
- }
-
- return false;
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
deleted file mode 100644
index cddde2f..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
+++ /dev/null
@@ -1,629 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.net.ConnectException;
-import java.net.NoRouteToHostException;
-import java.net.SocketException;
-import java.net.SocketTimeoutException;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Locale;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.client.RetriesExhaustedException;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.ipc.FailedServerException;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-
-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.AdminProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-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.MetaRegionServer;
-
-/**
- * Utility class to perform operation (get/wait for/verify/set/delete) on znode in ZooKeeper
- * which keeps hbase:meta region server location.
- *
- * Stateless class with a bunch of static methods. Doesn't manage resources passed in
- * (e.g. Connection, ZooKeeperWatcher etc).
- *
- * Meta region location is set by <code>RegionServerServices</code>.
- * This class doesn't use ZK watchers, rather accesses ZK directly.
- *
- * This class it stateless. The only reason it's not made a non-instantiable util class
- * with a collection of static methods is that it'd be rather hard to mock properly in tests.
- *
- * TODO: rewrite using RPC calls to master to find out about hbase:meta.
- */
-@InterfaceAudience.Private
-public class MetaTableLocator {
- private static final Log LOG = LogFactory.getLog(MetaTableLocator.class);
-
- // only needed to allow non-timeout infinite waits to stop when cluster shuts down
- private volatile boolean stopped = false;
-
- /**
- * Checks if the meta region location is available.
- * @return true if meta region location is available, false if not
- */
- public boolean isLocationAvailable(ZooKeeperWatcher zkw) {
- return getMetaRegionLocation(zkw) != null;
- }
-
- /**
- * @param zkw ZooKeeper watcher to be used
- * @return meta table regions and their locations.
- */
- public List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw) {
- return getMetaRegionsAndLocations(zkw, RegionInfo.DEFAULT_REPLICA_ID);
- }
-
- /**
- *
- * @param zkw
- * @param replicaId
- * @return meta table regions and their locations.
- */
- public List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw,
- int replicaId) {
- ServerName serverName = getMetaRegionLocation(zkw, replicaId);
- List<Pair<RegionInfo, ServerName>> list = new ArrayList<>(1);
- list.add(new Pair<>(RegionReplicaUtil.getRegionInfoForReplica(
- RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), serverName));
- return list;
- }
-
- /**
- * @param zkw ZooKeeper watcher to be used
- * @return List of meta regions
- */
- public List<RegionInfo> getMetaRegions(ZooKeeperWatcher zkw) {
- return getMetaRegions(zkw, RegionInfo.DEFAULT_REPLICA_ID);
- }
-
- /**
- *
- * @param zkw
- * @param replicaId
- * @return List of meta regions
- */
- public List<RegionInfo> getMetaRegions(ZooKeeperWatcher zkw, int replicaId) {
- List<Pair<RegionInfo, ServerName>> result;
- result = getMetaRegionsAndLocations(zkw, replicaId);
- return getListOfRegionInfos(result);
- }
-
- private List<RegionInfo> getListOfRegionInfos(
- final List<Pair<RegionInfo, ServerName>> pairs) {
- if (pairs == null || pairs.isEmpty()) return Collections.EMPTY_LIST;
- List<RegionInfo> result = new ArrayList<>(pairs.size());
- for (Pair<RegionInfo, ServerName> pair: pairs) {
- result.add(pair.getFirst());
- }
- return result;
- }
-
- /**
- * Gets the meta region location, if available. Does not block.
- * @param zkw zookeeper connection to use
- * @return server name or null if we failed to get the data.
- */
- public ServerName getMetaRegionLocation(final ZooKeeperWatcher zkw) {
- try {
- RegionState state = getMetaRegionState(zkw);
- return state.isOpened() ? state.getServerName() : null;
- } catch (KeeperException ke) {
- return null;
- }
- }
-
- /**
- * Gets the meta region location, if available. Does not block.
- * @param zkw
- * @param replicaId
- * @return server name
- */
- public ServerName getMetaRegionLocation(final ZooKeeperWatcher zkw, int replicaId) {
- try {
- RegionState state = getMetaRegionState(zkw, replicaId);
- return state.isOpened() ? state.getServerName() : null;
- } catch (KeeperException ke) {
- return null;
- }
- }
-
- /**
- * Gets the meta region location, if available, and waits for up to the
- * specified timeout if not immediately available.
- * Given the zookeeper notification could be delayed, we will try to
- * get the latest data.
- * @param zkw
- * @param timeout maximum time to wait, in millis
- * @return server name for server hosting meta region formatted as per
- * {@link ServerName}, or null if none available
- * @throws InterruptedException if interrupted while waiting
- * @throws NotAllMetaRegionsOnlineException
- */
- public ServerName waitMetaRegionLocation(ZooKeeperWatcher zkw, long timeout)
- throws InterruptedException, NotAllMetaRegionsOnlineException {
- return waitMetaRegionLocation(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
- }
-
- /**
- * Gets the meta region location, if available, and waits for up to the
- * specified timeout if not immediately available.
- * Given the zookeeper notification could be delayed, we will try to
- * get the latest data.
- * @param zkw
- * @param replicaId
- * @param timeout maximum time to wait, in millis
- * @return server name for server hosting meta region formatted as per
- * {@link ServerName}, or null if none available
- * @throws InterruptedException
- * @throws NotAllMetaRegionsOnlineException
- */
- public ServerName waitMetaRegionLocation(ZooKeeperWatcher zkw, int replicaId, long timeout)
- throws InterruptedException, NotAllMetaRegionsOnlineException {
- try {
- if (ZKUtil.checkExists(zkw, zkw.znodePaths.baseZNode) == -1) {
- String errorMsg = "Check the value configured in 'zookeeper.znode.parent'. "
- + "There could be a mismatch with the one configured in the master.";
- LOG.error(errorMsg);
- throw new IllegalArgumentException(errorMsg);
- }
- } catch (KeeperException e) {
- throw new IllegalStateException("KeeperException while trying to check baseZNode:", e);
- }
- ServerName sn = blockUntilAvailable(zkw, replicaId, timeout);
-
- if (sn == null) {
- throw new NotAllMetaRegionsOnlineException("Timed out; " + timeout + "ms");
- }
-
- return sn;
- }
-
- /**
- * Waits indefinitely for availability of <code>hbase:meta</code>. Used during
- * cluster startup. Does not verify meta, just that something has been
- * set up in zk.
- * @see #waitMetaRegionLocation(org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher, long)
- * @throws InterruptedException if interrupted while waiting
- */
- public void waitMetaRegionLocation(ZooKeeperWatcher zkw) throws InterruptedException {
- long startTime = System.currentTimeMillis();
- while (!stopped) {
- try {
- if (waitMetaRegionLocation(zkw, 100) != null) break;
- long sleepTime = System.currentTimeMillis() - startTime;
- // +1 in case sleepTime=0
- if ((sleepTime + 1) % 10000 == 0) {
- LOG.warn("Have been waiting for meta to be assigned for " + sleepTime + "ms");
- }
- } catch (NotAllMetaRegionsOnlineException e) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("hbase:meta still not available, sleeping and retrying." +
- " Reason: " + e.getMessage());
- }
- }
- }
- }
-
- /**
- * Verify <code>hbase:meta</code> is deployed and accessible.
- * @param hConnection
- * @param zkw
- * @param timeout How long to wait on zk for meta address (passed through to
- * the internal call to {@link #getMetaServerConnection}.
- * @return True if the <code>hbase:meta</code> location is healthy.
- * @throws java.io.IOException
- * @throws InterruptedException
- */
- public boolean verifyMetaRegionLocation(ClusterConnection hConnection,
- ZooKeeperWatcher zkw, final long timeout)
- throws InterruptedException, IOException {
- return verifyMetaRegionLocation(hConnection, zkw, timeout, RegionInfo.DEFAULT_REPLICA_ID);
- }
-
- /**
- * Verify <code>hbase:meta</code> is deployed and accessible.
- * @param connection
- * @param zkw
- * @param timeout How long to wait on zk for meta address (passed through to
- * @param replicaId
- * @return True if the <code>hbase:meta</code> location is healthy.
- * @throws InterruptedException
- * @throws IOException
- */
- public boolean verifyMetaRegionLocation(ClusterConnection connection,
- ZooKeeperWatcher zkw, final long timeout, int replicaId)
- throws InterruptedException, IOException {
- AdminProtos.AdminService.BlockingInterface service = null;
- try {
- service = getMetaServerConnection(connection, zkw, timeout, replicaId);
- } catch (NotAllMetaRegionsOnlineException e) {
- // Pass
- } catch (ServerNotRunningYetException e) {
- // Pass -- remote server is not up so can't be carrying root
- } catch (UnknownHostException e) {
- // Pass -- server name doesn't resolve so it can't be assigned anything.
- } catch (RegionServerStoppedException e) {
- // Pass -- server name sends us to a server that is dying or already dead.
- }
- return (service != null) && verifyRegionLocation(connection, service,
- getMetaRegionLocation(zkw, replicaId), RegionReplicaUtil.getRegionInfoForReplica(
- RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId).getRegionName());
- }
-
- /**
- * Verify we can connect to <code>hostingServer</code> and that its carrying
- * <code>regionName</code>.
- * @param hostingServer Interface to the server hosting <code>regionName</code>
- * @param address The servername that goes with the <code>metaServer</code>
- * Interface. Used logging.
- * @param regionName The regionname we are interested in.
- * @return True if we were able to verify the region located at other side of
- * the Interface.
- * @throws IOException
- */
- // 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
- // HRI is likely a proxy against remote server so the getServerName needs
- // to be fixed to go to a local method or to a cache before we can do this.
- private boolean verifyRegionLocation(final ClusterConnection connection,
- AdminService.BlockingInterface hostingServer, final ServerName address,
- final byte [] regionName)
- throws IOException {
- if (hostingServer == null) {
- LOG.info("Passed hostingServer is null");
- return false;
- }
- Throwable t;
- HBaseRpcController controller = connection.getRpcControllerFactory().newController();
- try {
- // Try and get regioninfo from the hosting server.
- return ProtobufUtil.getRegionInfo(controller, hostingServer, regionName) != null;
- } catch (ConnectException e) {
- t = e;
- } catch (RetriesExhaustedException e) {
- t = e;
- } catch (RemoteException e) {
- IOException ioe = e.unwrapRemoteException();
- t = ioe;
- } catch (IOException e) {
- Throwable cause = e.getCause();
- if (cause != null && cause instanceof EOFException) {
- t = cause;
- } else if (cause != null && cause.getMessage() != null
- && cause.getMessage().contains("Connection reset")) {
- t = cause;
- } else {
- t = e;
- }
- }
- LOG.info("Failed verification of " + Bytes.toStringBinary(regionName) +
- " at address=" + address + ", exception=" + t.getMessage());
- return false;
- }
-
- /**
- * Gets a connection to the server hosting meta, as reported by ZooKeeper,
- * waiting up to the specified timeout for availability.
- * <p>WARNING: Does not retry. Use an {@link org.apache.hadoop.hbase.client.HTable} instead.
- * @param connection
- * @param zkw
- * @param timeout How long to wait on meta location
- * @param replicaId
- * @return connection to server hosting meta
- * @throws InterruptedException
- * @throws NotAllMetaRegionsOnlineException if timed out waiting
- * @throws IOException
- */
- private AdminService.BlockingInterface getMetaServerConnection(ClusterConnection connection,
- ZooKeeperWatcher zkw, long timeout, int replicaId)
- throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
- return getCachedConnection(connection, waitMetaRegionLocation(zkw, replicaId, timeout));
- }
-
- /**
- * @param sn ServerName to get a connection against.
- * @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
- * invocation, or may be null.
- * @throws IOException
- */
- private static AdminService.BlockingInterface getCachedConnection(ClusterConnection connection,
- ServerName sn)
- throws IOException {
- if (sn == null) {
- return null;
- }
- AdminService.BlockingInterface service = null;
- try {
- service = connection.getAdmin(sn);
- } catch (RetriesExhaustedException e) {
- if (e.getCause() != null && e.getCause() instanceof ConnectException) {
- // Catch this; presume it means the cached connection has gone bad.
- } else {
- throw e;
- }
- } catch (SocketTimeoutException e) {
- LOG.debug("Timed out connecting to " + sn);
- } catch (NoRouteToHostException e) {
- LOG.debug("Connecting to " + sn, e);
- } catch (SocketException e) {
- LOG.debug("Exception connecting to " + sn);
- } catch (UnknownHostException e) {
- LOG.debug("Unknown host exception connecting to " + sn);
- } catch (FailedServerException e) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Server " + sn + " is in failed server list.");
- }
- } catch (IOException ioe) {
- Throwable cause = ioe.getCause();
- if (ioe instanceof ConnectException) {
- // Catch. Connect refused.
- } else if (cause != null && cause instanceof EOFException) {
- // Catch. Other end disconnected us.
- } else if (cause != null && cause.getMessage() != null &&
- cause.getMessage().toLowerCase(Locale.ROOT).contains("connection reset")) {
- // Catch. Connection reset.
- } else {
- throw ioe;
- }
-
- }
- return service;
- }
-
- /**
- * Sets the location of <code>hbase:meta</code> in ZooKeeper to the
- * specified server address.
- * @param zookeeper zookeeper reference
- * @param serverName The server hosting <code>hbase:meta</code>
- * @param state The region transition state
- * @throws KeeperException unexpected zookeeper exception
- */
- public static void setMetaLocation(ZooKeeperWatcher zookeeper,
- ServerName serverName, RegionState.State state) throws KeeperException {
- setMetaLocation(zookeeper, serverName, RegionInfo.DEFAULT_REPLICA_ID, state);
- }
-
- /**
- * Sets the location of <code>hbase:meta</code> in ZooKeeper to the
- * specified server address.
- * @param zookeeper
- * @param serverName
- * @param replicaId
- * @param state
- * @throws KeeperException
- */
- public static void setMetaLocation(ZooKeeperWatcher zookeeper,
- ServerName serverName, int replicaId, RegionState.State state) throws KeeperException {
- if (serverName == null) {
- LOG.warn("Tried to set null ServerName in hbase:meta; skipping -- ServerName required");
- return;
- }
- LOG.info("Setting hbase:meta (replicaId=" + replicaId + ") location in ZooKeeper as " +
- serverName);
- // Make the MetaRegionServer pb and then get its bytes and save this as
- // the znode content.
- MetaRegionServer pbrsr = MetaRegionServer.newBuilder()
- .setServer(ProtobufUtil.toServerName(serverName))
- .setRpcVersion(HConstants.RPC_CURRENT_VERSION)
- .setState(state.convert()).build();
- byte[] data = ProtobufUtil.prependPBMagic(pbrsr.toByteArray());
- try {
- ZKUtil.setData(zookeeper,
- zookeeper.znodePaths.getZNodeForReplica(replicaId), data);
- } catch(KeeperException.NoNodeException nne) {
- if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
- LOG.debug("META region location doesn't exist, create it");
- } else {
- LOG.debug("META region location doesn't exist for replicaId=" + replicaId +
- ", create it");
- }
- ZKUtil.createAndWatch(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId), data);
- }
- }
-
- /**
- * Load the meta region state from the meta server ZNode.
- */
- public static RegionState getMetaRegionState(ZooKeeperWatcher zkw) throws KeeperException {
- return getMetaRegionState(zkw, RegionInfo.DEFAULT_REPLICA_ID);
- }
-
- /**
- * Load the meta region state from the meta server ZNode.
- * @param zkw
- * @param replicaId
- * @return regionstate
- * @throws KeeperException
- */
- public static RegionState getMetaRegionState(ZooKeeperWatcher zkw, int replicaId)
- throws KeeperException {
- RegionState.State state = RegionState.State.OPEN;
- ServerName serverName = null;
- try {
- byte[] data = ZKUtil.getData(zkw, zkw.znodePaths.getZNodeForReplica(replicaId));
- if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) {
- try {
- int prefixLen = ProtobufUtil.lengthOfPBMagic();
- ZooKeeperProtos.MetaRegionServer rl =
- ZooKeeperProtos.MetaRegionServer.PARSER.parseFrom
- (data, prefixLen, data.length - prefixLen);
- if (rl.hasState()) {
- state = RegionState.State.convert(rl.getState());
- }
- HBaseProtos.ServerName sn = rl.getServer();
- serverName = ServerName.valueOf(
- sn.getHostName(), sn.getPort(), sn.getStartCode());
- } catch (InvalidProtocolBufferException e) {
- throw new DeserializationException("Unable to parse meta region location");
- }
- } else {
- // old style of meta region location?
- serverName = ProtobufUtil.parseServerNameFrom(data);
- }
- } catch (DeserializationException e) {
- throw ZKUtil.convert(e);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- if (serverName == null) {
- state = RegionState.State.OFFLINE;
- }
- return new RegionState(
- RegionReplicaUtil.getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId),
- state, serverName);
- }
-
- /**
- * Deletes the location of <code>hbase:meta</code> in ZooKeeper.
- * @param zookeeper zookeeper reference
- * @throws KeeperException unexpected zookeeper exception
- */
- public void deleteMetaLocation(ZooKeeperWatcher zookeeper)
- throws KeeperException {
- deleteMetaLocation(zookeeper, RegionInfo.DEFAULT_REPLICA_ID);
- }
-
- public void deleteMetaLocation(ZooKeeperWatcher zookeeper, int replicaId)
- throws KeeperException {
- if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
- LOG.info("Deleting hbase:meta region location in ZooKeeper");
- } else {
- LOG.info("Deleting hbase:meta for " + replicaId + " region location in ZooKeeper");
- }
- try {
- // Just delete the node. Don't need any watches.
- ZKUtil.deleteNode(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId));
- } catch(KeeperException.NoNodeException nne) {
- // Has already been deleted
- }
- }
- /**
- * Wait until the primary meta region is available. Get the secondary
- * locations as well but don't block for those.
- * @param zkw
- * @param timeout
- * @param conf
- * @return ServerName or null if we timed out.
- * @throws InterruptedException
- */
- public List<ServerName> blockUntilAvailable(final ZooKeeperWatcher zkw,
- final long timeout, Configuration conf)
- throws InterruptedException {
- int numReplicasConfigured = 1;
-
- List<ServerName> servers = new ArrayList<>();
- // Make the blocking call first so that we do the wait to know
- // the znodes are all in place or timeout.
- ServerName server = blockUntilAvailable(zkw, timeout);
- if (server == null) return null;
- servers.add(server);
-
- try {
- List<String> metaReplicaNodes = zkw.getMetaReplicaNodes();
- numReplicasConfigured = metaReplicaNodes.size();
- } catch (KeeperException e) {
- LOG.warn("Got ZK exception " + e);
- }
- for (int replicaId = 1; replicaId < numReplicasConfigured; replicaId++) {
- // return all replica locations for the meta
- servers.add(getMetaRegionLocation(zkw, replicaId));
- }
- return servers;
- }
-
- /**
- * Wait until the meta region is available and is not in transition.
- * @param zkw zookeeper connection to use
- * @param timeout maximum time to wait, in millis
- * @return ServerName or null if we timed out.
- * @throws InterruptedException
- */
- public ServerName blockUntilAvailable(final ZooKeeperWatcher zkw,
- final long timeout)
- throws InterruptedException {
- return blockUntilAvailable(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
- }
-
- /**
- * Wait until the meta region is available and is not in transition.
- * @param zkw
- * @param replicaId
- * @param timeout
- * @return ServerName or null if we timed out.
- * @throws InterruptedException
- */
- public ServerName blockUntilAvailable(final ZooKeeperWatcher zkw, int replicaId,
- final long timeout)
- throws InterruptedException {
- if (timeout < 0) throw new IllegalArgumentException();
- if (zkw == null) throw new IllegalArgumentException();
- long startTime = System.currentTimeMillis();
- ServerName sn = null;
- while (true) {
- sn = getMetaRegionLocation(zkw, replicaId);
- if (sn != null || (System.currentTimeMillis() - startTime)
- > timeout - HConstants.SOCKET_RETRY_WAIT_MS) {
- break;
- }
- Thread.sleep(HConstants.SOCKET_RETRY_WAIT_MS);
- }
- return sn;
- }
-
- /**
- * Stop working.
- * Interrupts any ongoing waits.
- */
- public void stop() {
- if (!stopped) {
- LOG.debug("Stopping MetaTableLocator");
- stopped = true;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java
deleted file mode 100644
index 5632031..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.zookeeper;
-
-import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource;
-import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSourceImpl;
-
-/**
- * Class used to push numbers about ZooKeeper into the metrics subsystem. This will take a
- * single function call and turn it into multiple manipulations of the hadoop metrics system.
- */
-@InterfaceAudience.Private
-public class MetricsZooKeeper implements ZooKeeperMetricsListener {
- private final MetricsZooKeeperSource source;
-
- public MetricsZooKeeper() {
- this(CompatibilitySingletonFactory.getInstance(MetricsZooKeeperSource.class));
- }
-
- @VisibleForTesting
- public MetricsZooKeeper(MetricsZooKeeperSource s) {
- this.source = s;
- }
-
- @Override
- public void registerAuthFailedException() {
- source.incrementAuthFailedCount();
- }
-
- @Override
- public void registerConnectionLossException() {
- source.incrementConnectionLossCount();
- }
-
- @Override
- public void registerDataInconsistencyException() {
- source.incrementDataInconsistencyCount();
- }
-
- @Override
- public void registerInvalidACLException() {
- source.incrementInvalidACLCount();
- }
-
- @Override
- public void registerNoAuthException() {
- source.incrementNoAuthCount();
- }
-
- @Override
- public void registerOperationTimeoutException() {
- source.incrementOperationTimeoutCount();
- }
-
- @Override
- public void registerRuntimeInconsistencyException() {
- source.incrementRuntimeInconsistencyCount();
- }
-
- @Override
- public void registerSessionExpiredException() {
- source.incrementSessionExpiredCount();
- }
-
- @Override
- public void registerSystemErrorException() {
- source.incrementSystemErrorCount();
- }
-
- @Override
- public void registerFailedZKCall() {
- source.incrementTotalFailedZKCalls();
- }
-
- @Override
- public void registerReadOperationLatency(long latency) {
- source.recordReadOperationLatency(latency);
- }
-
- @Override
- public void registerWriteOperationLatency(long latency) {
- source.recordWriteOperationLatency(latency);
- }
-
- @Override
- public void registerSyncOperationLatency(long latency) {
- source.recordSyncOperationLatency(latency);
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java
deleted file mode 100644
index da7d176..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.zookeeper;
-
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-
-/**
- * Placeholder of a watcher which might be triggered before the instance is not yet created.
- * <p>
- * {@code ZooKeeper} starts its event thread within its constructor (and that is an anti-pattern),
- * and the watcher passed to the constructor might be called back by the event thread
- * before you get the instance of {@code ZooKeeper} from the constructor.
- * If your watcher calls methods of {@code ZooKeeper},
- * pass this placeholder to the constructor of the {@code ZooKeeper},
- * create your watcher using the instance of {@code ZooKeeper},
- * and then call the method {@code PendingWatcher.prepare}.
- */
-class PendingWatcher implements Watcher {
- private final InstancePending<Watcher> pending = new InstancePending<>();
-
- @Override
- public void process(WatchedEvent event) {
- pending.get().process(event);
- }
-
- /**
- * Associates the substantial watcher of processing events.
- * This method should be called once, and {@code watcher} should be non-null.
- * This method is expected to call as soon as possible
- * because the event processing, being invoked by the ZooKeeper event thread,
- * is uninterruptibly blocked until this method is called.
- */
- void prepare(Watcher watcher) {
- pending.prepare(watcher);
- }
-}
[04/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper
from hbase-server and hbase-client
Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
new file mode 100644
index 0000000..f6c7a2d
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
@@ -0,0 +1,629 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.NoRouteToHostException;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.RetriesExhaustedException;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.ipc.FailedServerException;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+
+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.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+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.MetaRegionServer;
+
+/**
+ * Utility class to perform operation (get/wait for/verify/set/delete) on znode in ZooKeeper
+ * which keeps hbase:meta region server location.
+ *
+ * Stateless class with a bunch of static methods. Doesn't manage resources passed in
+ * (e.g. Connection, ZKWatcher etc).
+ *
+ * Meta region location is set by <code>RegionServerServices</code>.
+ * This class doesn't use ZK watchers, rather accesses ZK directly.
+ *
+ * This class it stateless. The only reason it's not made a non-instantiable util class
+ * with a collection of static methods is that it'd be rather hard to mock properly in tests.
+ *
+ * TODO: rewrite using RPC calls to master to find out about hbase:meta.
+ */
+@InterfaceAudience.Private
+public class MetaTableLocator {
+ private static final Log LOG = LogFactory.getLog(MetaTableLocator.class);
+
+ // only needed to allow non-timeout infinite waits to stop when cluster shuts down
+ private volatile boolean stopped = false;
+
+ /**
+ * Checks if the meta region location is available.
+ * @return true if meta region location is available, false if not
+ */
+ public boolean isLocationAvailable(ZKWatcher zkw) {
+ return getMetaRegionLocation(zkw) != null;
+ }
+
+ /**
+ * @param zkw ZooKeeper watcher to be used
+ * @return meta table regions and their locations.
+ */
+ public List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZKWatcher zkw) {
+ return getMetaRegionsAndLocations(zkw, RegionInfo.DEFAULT_REPLICA_ID);
+ }
+
+ /**
+ *
+ * @param zkw
+ * @param replicaId
+ * @return meta table regions and their locations.
+ */
+ public List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZKWatcher zkw,
+ int replicaId) {
+ ServerName serverName = getMetaRegionLocation(zkw, replicaId);
+ List<Pair<RegionInfo, ServerName>> list = new ArrayList<>(1);
+ list.add(new Pair<>(RegionReplicaUtil.getRegionInfoForReplica(
+ RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), serverName));
+ return list;
+ }
+
+ /**
+ * @param zkw ZooKeeper watcher to be used
+ * @return List of meta regions
+ */
+ public List<RegionInfo> getMetaRegions(ZKWatcher zkw) {
+ return getMetaRegions(zkw, RegionInfo.DEFAULT_REPLICA_ID);
+ }
+
+ /**
+ *
+ * @param zkw
+ * @param replicaId
+ * @return List of meta regions
+ */
+ public List<RegionInfo> getMetaRegions(ZKWatcher zkw, int replicaId) {
+ List<Pair<RegionInfo, ServerName>> result;
+ result = getMetaRegionsAndLocations(zkw, replicaId);
+ return getListOfRegionInfos(result);
+ }
+
+ private List<RegionInfo> getListOfRegionInfos(
+ final List<Pair<RegionInfo, ServerName>> pairs) {
+ if (pairs == null || pairs.isEmpty()) return Collections.EMPTY_LIST;
+ List<RegionInfo> result = new ArrayList<>(pairs.size());
+ for (Pair<RegionInfo, ServerName> pair: pairs) {
+ result.add(pair.getFirst());
+ }
+ return result;
+ }
+
+ /**
+ * Gets the meta region location, if available. Does not block.
+ * @param zkw zookeeper connection to use
+ * @return server name or null if we failed to get the data.
+ */
+ public ServerName getMetaRegionLocation(final ZKWatcher zkw) {
+ try {
+ RegionState state = getMetaRegionState(zkw);
+ return state.isOpened() ? state.getServerName() : null;
+ } catch (KeeperException ke) {
+ return null;
+ }
+ }
+
+ /**
+ * Gets the meta region location, if available. Does not block.
+ * @param zkw
+ * @param replicaId
+ * @return server name
+ */
+ public ServerName getMetaRegionLocation(final ZKWatcher zkw, int replicaId) {
+ try {
+ RegionState state = getMetaRegionState(zkw, replicaId);
+ return state.isOpened() ? state.getServerName() : null;
+ } catch (KeeperException ke) {
+ return null;
+ }
+ }
+
+ /**
+ * Gets the meta region location, if available, and waits for up to the
+ * specified timeout if not immediately available.
+ * Given the zookeeper notification could be delayed, we will try to
+ * get the latest data.
+ * @param zkw
+ * @param timeout maximum time to wait, in millis
+ * @return server name for server hosting meta region formatted as per
+ * {@link ServerName}, or null if none available
+ * @throws InterruptedException if interrupted while waiting
+ * @throws NotAllMetaRegionsOnlineException
+ */
+ public ServerName waitMetaRegionLocation(ZKWatcher zkw, long timeout)
+ throws InterruptedException, NotAllMetaRegionsOnlineException {
+ return waitMetaRegionLocation(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
+ }
+
+ /**
+ * Gets the meta region location, if available, and waits for up to the
+ * specified timeout if not immediately available.
+ * Given the zookeeper notification could be delayed, we will try to
+ * get the latest data.
+ * @param zkw
+ * @param replicaId
+ * @param timeout maximum time to wait, in millis
+ * @return server name for server hosting meta region formatted as per
+ * {@link ServerName}, or null if none available
+ * @throws InterruptedException
+ * @throws NotAllMetaRegionsOnlineException
+ */
+ public ServerName waitMetaRegionLocation(ZKWatcher zkw, int replicaId, long timeout)
+ throws InterruptedException, NotAllMetaRegionsOnlineException {
+ try {
+ if (ZKUtil.checkExists(zkw, zkw.znodePaths.baseZNode) == -1) {
+ String errorMsg = "Check the value configured in 'zookeeper.znode.parent'. "
+ + "There could be a mismatch with the one configured in the master.";
+ LOG.error(errorMsg);
+ throw new IllegalArgumentException(errorMsg);
+ }
+ } catch (KeeperException e) {
+ throw new IllegalStateException("KeeperException while trying to check baseZNode:", e);
+ }
+ ServerName sn = blockUntilAvailable(zkw, replicaId, timeout);
+
+ if (sn == null) {
+ throw new NotAllMetaRegionsOnlineException("Timed out; " + timeout + "ms");
+ }
+
+ return sn;
+ }
+
+ /**
+ * Waits indefinitely for availability of <code>hbase:meta</code>. Used during
+ * cluster startup. Does not verify meta, just that something has been
+ * set up in zk.
+ * @see #waitMetaRegionLocation(ZKWatcher, long)
+ * @throws InterruptedException if interrupted while waiting
+ */
+ public void waitMetaRegionLocation(ZKWatcher zkw) throws InterruptedException {
+ long startTime = System.currentTimeMillis();
+ while (!stopped) {
+ try {
+ if (waitMetaRegionLocation(zkw, 100) != null) break;
+ long sleepTime = System.currentTimeMillis() - startTime;
+ // +1 in case sleepTime=0
+ if ((sleepTime + 1) % 10000 == 0) {
+ LOG.warn("Have been waiting for meta to be assigned for " + sleepTime + "ms");
+ }
+ } catch (NotAllMetaRegionsOnlineException e) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("hbase:meta still not available, sleeping and retrying." +
+ " Reason: " + e.getMessage());
+ }
+ }
+ }
+ }
+
+ /**
+ * Verify <code>hbase:meta</code> is deployed and accessible.
+ * @param hConnection
+ * @param zkw
+ * @param timeout How long to wait on zk for meta address (passed through to
+ * the internal call to {@link #getMetaServerConnection}.
+ * @return True if the <code>hbase:meta</code> location is healthy.
+ * @throws java.io.IOException
+ * @throws InterruptedException
+ */
+ public boolean verifyMetaRegionLocation(ClusterConnection hConnection,
+ ZKWatcher zkw, final long timeout)
+ throws InterruptedException, IOException {
+ return verifyMetaRegionLocation(hConnection, zkw, timeout, RegionInfo.DEFAULT_REPLICA_ID);
+ }
+
+ /**
+ * Verify <code>hbase:meta</code> is deployed and accessible.
+ * @param connection
+ * @param zkw
+ * @param timeout How long to wait on zk for meta address (passed through to
+ * @param replicaId
+ * @return True if the <code>hbase:meta</code> location is healthy.
+ * @throws InterruptedException
+ * @throws IOException
+ */
+ public boolean verifyMetaRegionLocation(ClusterConnection connection,
+ ZKWatcher zkw, final long timeout, int replicaId)
+ throws InterruptedException, IOException {
+ AdminProtos.AdminService.BlockingInterface service = null;
+ try {
+ service = getMetaServerConnection(connection, zkw, timeout, replicaId);
+ } catch (NotAllMetaRegionsOnlineException e) {
+ // Pass
+ } catch (ServerNotRunningYetException e) {
+ // Pass -- remote server is not up so can't be carrying root
+ } catch (UnknownHostException e) {
+ // Pass -- server name doesn't resolve so it can't be assigned anything.
+ } catch (RegionServerStoppedException e) {
+ // Pass -- server name sends us to a server that is dying or already dead.
+ }
+ return (service != null) && verifyRegionLocation(connection, service,
+ getMetaRegionLocation(zkw, replicaId), RegionReplicaUtil.getRegionInfoForReplica(
+ RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId).getRegionName());
+ }
+
+ /**
+ * Verify we can connect to <code>hostingServer</code> and that its carrying
+ * <code>regionName</code>.
+ * @param hostingServer Interface to the server hosting <code>regionName</code>
+ * @param address The servername that goes with the <code>metaServer</code>
+ * Interface. Used logging.
+ * @param regionName The regionname we are interested in.
+ * @return True if we were able to verify the region located at other side of
+ * the Interface.
+ * @throws IOException
+ */
+ // 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
+ // HRI is likely a proxy against remote server so the getServerName needs
+ // to be fixed to go to a local method or to a cache before we can do this.
+ private boolean verifyRegionLocation(final ClusterConnection connection,
+ AdminService.BlockingInterface hostingServer, final ServerName address,
+ final byte [] regionName)
+ throws IOException {
+ if (hostingServer == null) {
+ LOG.info("Passed hostingServer is null");
+ return false;
+ }
+ Throwable t;
+ HBaseRpcController controller = connection.getRpcControllerFactory().newController();
+ try {
+ // Try and get regioninfo from the hosting server.
+ return ProtobufUtil.getRegionInfo(controller, hostingServer, regionName) != null;
+ } catch (ConnectException e) {
+ t = e;
+ } catch (RetriesExhaustedException e) {
+ t = e;
+ } catch (RemoteException e) {
+ IOException ioe = e.unwrapRemoteException();
+ t = ioe;
+ } catch (IOException e) {
+ Throwable cause = e.getCause();
+ if (cause != null && cause instanceof EOFException) {
+ t = cause;
+ } else if (cause != null && cause.getMessage() != null
+ && cause.getMessage().contains("Connection reset")) {
+ t = cause;
+ } else {
+ t = e;
+ }
+ }
+ LOG.info("Failed verification of " + Bytes.toStringBinary(regionName) +
+ " at address=" + address + ", exception=" + t.getMessage());
+ return false;
+ }
+
+ /**
+ * Gets a connection to the server hosting meta, as reported by ZooKeeper,
+ * waiting up to the specified timeout for availability.
+ * <p>WARNING: Does not retry. Use an {@link org.apache.hadoop.hbase.client.HTable} instead.
+ * @param connection
+ * @param zkw
+ * @param timeout How long to wait on meta location
+ * @param replicaId
+ * @return connection to server hosting meta
+ * @throws InterruptedException
+ * @throws NotAllMetaRegionsOnlineException if timed out waiting
+ * @throws IOException
+ */
+ private AdminService.BlockingInterface getMetaServerConnection(ClusterConnection connection,
+ ZKWatcher zkw, long timeout, int replicaId)
+ throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
+ return getCachedConnection(connection, waitMetaRegionLocation(zkw, replicaId, timeout));
+ }
+
+ /**
+ * @param sn ServerName to get a connection against.
+ * @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
+ * invocation, or may be null.
+ * @throws IOException
+ */
+ private static AdminService.BlockingInterface getCachedConnection(ClusterConnection connection,
+ ServerName sn)
+ throws IOException {
+ if (sn == null) {
+ return null;
+ }
+ AdminService.BlockingInterface service = null;
+ try {
+ service = connection.getAdmin(sn);
+ } catch (RetriesExhaustedException e) {
+ if (e.getCause() != null && e.getCause() instanceof ConnectException) {
+ // Catch this; presume it means the cached connection has gone bad.
+ } else {
+ throw e;
+ }
+ } catch (SocketTimeoutException e) {
+ LOG.debug("Timed out connecting to " + sn);
+ } catch (NoRouteToHostException e) {
+ LOG.debug("Connecting to " + sn, e);
+ } catch (SocketException e) {
+ LOG.debug("Exception connecting to " + sn);
+ } catch (UnknownHostException e) {
+ LOG.debug("Unknown host exception connecting to " + sn);
+ } catch (FailedServerException e) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Server " + sn + " is in failed server list.");
+ }
+ } catch (IOException ioe) {
+ Throwable cause = ioe.getCause();
+ if (ioe instanceof ConnectException) {
+ // Catch. Connect refused.
+ } else if (cause != null && cause instanceof EOFException) {
+ // Catch. Other end disconnected us.
+ } else if (cause != null && cause.getMessage() != null &&
+ cause.getMessage().toLowerCase(Locale.ROOT).contains("connection reset")) {
+ // Catch. Connection reset.
+ } else {
+ throw ioe;
+ }
+
+ }
+ return service;
+ }
+
+ /**
+ * Sets the location of <code>hbase:meta</code> in ZooKeeper to the
+ * specified server address.
+ * @param zookeeper zookeeper reference
+ * @param serverName The server hosting <code>hbase:meta</code>
+ * @param state The region transition state
+ * @throws KeeperException unexpected zookeeper exception
+ */
+ public static void setMetaLocation(ZKWatcher zookeeper,
+ ServerName serverName, RegionState.State state) throws KeeperException {
+ setMetaLocation(zookeeper, serverName, RegionInfo.DEFAULT_REPLICA_ID, state);
+ }
+
+ /**
+ * Sets the location of <code>hbase:meta</code> in ZooKeeper to the
+ * specified server address.
+ * @param zookeeper
+ * @param serverName
+ * @param replicaId
+ * @param state
+ * @throws KeeperException
+ */
+ public static void setMetaLocation(ZKWatcher zookeeper,
+ ServerName serverName, int replicaId, RegionState.State state) throws KeeperException {
+ if (serverName == null) {
+ LOG.warn("Tried to set null ServerName in hbase:meta; skipping -- ServerName required");
+ return;
+ }
+ LOG.info("Setting hbase:meta (replicaId=" + replicaId + ") location in ZooKeeper as " +
+ serverName);
+ // Make the MetaRegionServer pb and then get its bytes and save this as
+ // the znode content.
+ MetaRegionServer pbrsr = MetaRegionServer.newBuilder()
+ .setServer(ProtobufUtil.toServerName(serverName))
+ .setRpcVersion(HConstants.RPC_CURRENT_VERSION)
+ .setState(state.convert()).build();
+ byte[] data = ProtobufUtil.prependPBMagic(pbrsr.toByteArray());
+ try {
+ ZKUtil.setData(zookeeper,
+ zookeeper.znodePaths.getZNodeForReplica(replicaId), data);
+ } catch(KeeperException.NoNodeException nne) {
+ if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
+ LOG.debug("META region location doesn't exist, create it");
+ } else {
+ LOG.debug("META region location doesn't exist for replicaId=" + replicaId +
+ ", create it");
+ }
+ ZKUtil.createAndWatch(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId), data);
+ }
+ }
+
+ /**
+ * Load the meta region state from the meta server ZNode.
+ */
+ public static RegionState getMetaRegionState(ZKWatcher zkw) throws KeeperException {
+ return getMetaRegionState(zkw, RegionInfo.DEFAULT_REPLICA_ID);
+ }
+
+ /**
+ * Load the meta region state from the meta server ZNode.
+ * @param zkw
+ * @param replicaId
+ * @return regionstate
+ * @throws KeeperException
+ */
+ public static RegionState getMetaRegionState(ZKWatcher zkw, int replicaId)
+ throws KeeperException {
+ RegionState.State state = RegionState.State.OPEN;
+ ServerName serverName = null;
+ try {
+ byte[] data = ZKUtil.getData(zkw, zkw.znodePaths.getZNodeForReplica(replicaId));
+ if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) {
+ try {
+ int prefixLen = ProtobufUtil.lengthOfPBMagic();
+ ZooKeeperProtos.MetaRegionServer rl =
+ ZooKeeperProtos.MetaRegionServer.PARSER.parseFrom
+ (data, prefixLen, data.length - prefixLen);
+ if (rl.hasState()) {
+ state = RegionState.State.convert(rl.getState());
+ }
+ HBaseProtos.ServerName sn = rl.getServer();
+ serverName = ServerName.valueOf(
+ sn.getHostName(), sn.getPort(), sn.getStartCode());
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException("Unable to parse meta region location");
+ }
+ } else {
+ // old style of meta region location?
+ serverName = ProtobufUtil.parseServerNameFrom(data);
+ }
+ } catch (DeserializationException e) {
+ throw ZKUtil.convert(e);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ if (serverName == null) {
+ state = RegionState.State.OFFLINE;
+ }
+ return new RegionState(
+ RegionReplicaUtil.getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId),
+ state, serverName);
+ }
+
+ /**
+ * Deletes the location of <code>hbase:meta</code> in ZooKeeper.
+ * @param zookeeper zookeeper reference
+ * @throws KeeperException unexpected zookeeper exception
+ */
+ public void deleteMetaLocation(ZKWatcher zookeeper)
+ throws KeeperException {
+ deleteMetaLocation(zookeeper, RegionInfo.DEFAULT_REPLICA_ID);
+ }
+
+ public void deleteMetaLocation(ZKWatcher zookeeper, int replicaId)
+ throws KeeperException {
+ if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
+ LOG.info("Deleting hbase:meta region location in ZooKeeper");
+ } else {
+ LOG.info("Deleting hbase:meta for " + replicaId + " region location in ZooKeeper");
+ }
+ try {
+ // Just delete the node. Don't need any watches.
+ ZKUtil.deleteNode(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId));
+ } catch(KeeperException.NoNodeException nne) {
+ // Has already been deleted
+ }
+ }
+ /**
+ * Wait until the primary meta region is available. Get the secondary
+ * locations as well but don't block for those.
+ * @param zkw
+ * @param timeout
+ * @param conf
+ * @return ServerName or null if we timed out.
+ * @throws InterruptedException
+ */
+ public List<ServerName> blockUntilAvailable(final ZKWatcher zkw,
+ final long timeout, Configuration conf)
+ throws InterruptedException {
+ int numReplicasConfigured = 1;
+
+ List<ServerName> servers = new ArrayList<>();
+ // Make the blocking call first so that we do the wait to know
+ // the znodes are all in place or timeout.
+ ServerName server = blockUntilAvailable(zkw, timeout);
+ if (server == null) return null;
+ servers.add(server);
+
+ try {
+ List<String> metaReplicaNodes = zkw.getMetaReplicaNodes();
+ numReplicasConfigured = metaReplicaNodes.size();
+ } catch (KeeperException e) {
+ LOG.warn("Got ZK exception " + e);
+ }
+ for (int replicaId = 1; replicaId < numReplicasConfigured; replicaId++) {
+ // return all replica locations for the meta
+ servers.add(getMetaRegionLocation(zkw, replicaId));
+ }
+ return servers;
+ }
+
+ /**
+ * Wait until the meta region is available and is not in transition.
+ * @param zkw zookeeper connection to use
+ * @param timeout maximum time to wait, in millis
+ * @return ServerName or null if we timed out.
+ * @throws InterruptedException
+ */
+ public ServerName blockUntilAvailable(final ZKWatcher zkw,
+ final long timeout)
+ throws InterruptedException {
+ return blockUntilAvailable(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
+ }
+
+ /**
+ * Wait until the meta region is available and is not in transition.
+ * @param zkw
+ * @param replicaId
+ * @param timeout
+ * @return ServerName or null if we timed out.
+ * @throws InterruptedException
+ */
+ public ServerName blockUntilAvailable(final ZKWatcher zkw, int replicaId,
+ final long timeout)
+ throws InterruptedException {
+ if (timeout < 0) throw new IllegalArgumentException();
+ if (zkw == null) throw new IllegalArgumentException();
+ long startTime = System.currentTimeMillis();
+ ServerName sn = null;
+ while (true) {
+ sn = getMetaRegionLocation(zkw, replicaId);
+ if (sn != null || (System.currentTimeMillis() - startTime)
+ > timeout - HConstants.SOCKET_RETRY_WAIT_MS) {
+ break;
+ }
+ Thread.sleep(HConstants.SOCKET_RETRY_WAIT_MS);
+ }
+ return sn;
+ }
+
+ /**
+ * Stop working.
+ * Interrupts any ongoing waits.
+ */
+ public void stop() {
+ if (!stopped) {
+ LOG.debug("Stopping MetaTableLocator");
+ stopped = true;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
new file mode 100644
index 0000000..ef643bf
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
@@ -0,0 +1,472 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.InterruptedIOException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Reader;
+import java.net.BindException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.zookeeper.server.NIOServerCnxnFactory;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * TODO: Most of the code in this class is ripped from ZooKeeper tests. Instead
+ * of redoing it, we should contribute updates to their code which let us more
+ * easily access testing helper objects.
+ */
+@InterfaceAudience.Public
+public class MiniZooKeeperCluster {
+ private static final Log LOG = LogFactory.getLog(MiniZooKeeperCluster.class);
+
+ private static final int TICK_TIME = 2000;
+ private static final int DEFAULT_CONNECTION_TIMEOUT = 30000;
+ private int connectionTimeout;
+
+ private boolean started;
+
+ /** The default port. If zero, we use a random port. */
+ private int defaultClientPort = 0;
+
+ private List<NIOServerCnxnFactory> standaloneServerFactoryList;
+ private List<ZooKeeperServer> zooKeeperServers;
+ private List<Integer> clientPortList;
+
+ private int activeZKServerIndex;
+ private int tickTime = 0;
+
+ private Configuration configuration;
+
+ public MiniZooKeeperCluster() {
+ this(new Configuration());
+ }
+
+ public MiniZooKeeperCluster(Configuration configuration) {
+ this.started = false;
+ this.configuration = configuration;
+ activeZKServerIndex = -1;
+ zooKeeperServers = new ArrayList<>();
+ clientPortList = new ArrayList<>();
+ standaloneServerFactoryList = new ArrayList<>();
+ connectionTimeout = configuration.getInt(HConstants.ZK_SESSION_TIMEOUT + ".localHBaseCluster",
+ DEFAULT_CONNECTION_TIMEOUT);
+ }
+
+ /**
+ * Add a client port to the list.
+ *
+ * @param clientPort the specified port
+ */
+ public void addClientPort(int clientPort) {
+ clientPortList.add(clientPort);
+ }
+
+ /**
+ * Get the list of client ports.
+ * @return clientPortList the client port list
+ */
+ @VisibleForTesting
+ public List<Integer> getClientPortList() {
+ return clientPortList;
+ }
+
+ /**
+ * Check whether the client port in a specific position of the client port list is valid.
+ *
+ * @param index the specified position
+ */
+ private boolean hasValidClientPortInList(int index) {
+ return (clientPortList.size() > index && clientPortList.get(index) > 0);
+ }
+
+ public void setDefaultClientPort(int clientPort) {
+ if (clientPort <= 0) {
+ throw new IllegalArgumentException("Invalid default ZK client port: "
+ + clientPort);
+ }
+ this.defaultClientPort = clientPort;
+ }
+
+ /**
+ * Selects a ZK client port.
+ *
+ * @param seedPort the seed port to start with; -1 means first time.
+ * @Returns a valid and unused client port
+ */
+ private int selectClientPort(int seedPort) {
+ int i;
+ int returnClientPort = seedPort + 1;
+ if (returnClientPort == 0) {
+ // If the new port is invalid, find one - starting with the default client port.
+ // If the default client port is not specified, starting with a random port.
+ // The random port is selected from the range between 49152 to 65535. These ports cannot be
+ // registered with IANA and are intended for dynamic allocation (see http://bit.ly/dynports).
+ if (defaultClientPort > 0) {
+ returnClientPort = defaultClientPort;
+ } else {
+ returnClientPort = 0xc000 + new Random().nextInt(0x3f00);
+ }
+ }
+ // Make sure that the port is unused.
+ while (true) {
+ for (i = 0; i < clientPortList.size(); i++) {
+ if (returnClientPort == clientPortList.get(i)) {
+ // Already used. Update the port and retry.
+ returnClientPort++;
+ break;
+ }
+ }
+ if (i == clientPortList.size()) {
+ break; // found a unused port, exit
+ }
+ }
+ return returnClientPort;
+ }
+
+ public void setTickTime(int tickTime) {
+ this.tickTime = tickTime;
+ }
+
+ public int getBackupZooKeeperServerNum() {
+ return zooKeeperServers.size()-1;
+ }
+
+ public int getZooKeeperServerNum() {
+ return zooKeeperServers.size();
+ }
+
+ // / XXX: From o.a.zk.t.ClientBase
+ private static void setupTestEnv() {
+ // during the tests we run with 100K prealloc in the logs.
+ // on windows systems prealloc of 64M was seen to take ~15seconds
+ // resulting in test failure (client timeout on first session).
+ // set env and directly in order to handle static init/gc issues
+ System.setProperty("zookeeper.preAllocSize", "100");
+ FileTxnLog.setPreallocSize(100 * 1024);
+ }
+
+ public int startup(File baseDir) throws IOException, InterruptedException {
+ int numZooKeeperServers = clientPortList.size();
+ if (numZooKeeperServers == 0) {
+ numZooKeeperServers = 1; // need at least 1 ZK server for testing
+ }
+ return startup(baseDir, numZooKeeperServers);
+ }
+
+ /**
+ * @param baseDir
+ * @param numZooKeeperServers
+ * @return ClientPort server bound to, -1 if there was a
+ * binding problem and we couldn't pick another port.
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ public int startup(File baseDir, int numZooKeeperServers) throws IOException,
+ InterruptedException {
+ if (numZooKeeperServers <= 0)
+ return -1;
+
+ setupTestEnv();
+ shutdown();
+
+ int tentativePort = -1; // the seed port
+ int currentClientPort;
+
+ // running all the ZK servers
+ for (int i = 0; i < numZooKeeperServers; i++) {
+ File dir = new File(baseDir, "zookeeper_"+i).getAbsoluteFile();
+ createDir(dir);
+ int tickTimeToUse;
+ if (this.tickTime > 0) {
+ tickTimeToUse = this.tickTime;
+ } else {
+ tickTimeToUse = TICK_TIME;
+ }
+
+ // Set up client port - if we have already had a list of valid ports, use it.
+ if (hasValidClientPortInList(i)) {
+ currentClientPort = clientPortList.get(i);
+ } else {
+ tentativePort = selectClientPort(tentativePort); // update the seed
+ currentClientPort = tentativePort;
+ }
+
+ ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse);
+ // Setting {min,max}SessionTimeout defaults to be the same as in Zookeeper
+ server.setMinSessionTimeout(configuration.getInt("hbase.zookeeper.property.minSessionTimeout", -1));
+ server.setMaxSessionTimeout(configuration.getInt("hbase.zookeeper.property.maxSessionTimeout", -1));
+ NIOServerCnxnFactory standaloneServerFactory;
+ while (true) {
+ try {
+ standaloneServerFactory = new NIOServerCnxnFactory();
+ standaloneServerFactory.configure(
+ new InetSocketAddress(currentClientPort),
+ configuration.getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS, HConstants.DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS));
+ } catch (BindException e) {
+ LOG.debug("Failed binding ZK Server to client port: " +
+ currentClientPort, e);
+ // We're told to use some port but it's occupied, fail
+ if (hasValidClientPortInList(i)) {
+ return -1;
+ }
+ // This port is already in use, try to use another.
+ tentativePort = selectClientPort(tentativePort);
+ currentClientPort = tentativePort;
+ continue;
+ }
+ break;
+ }
+
+ // Start up this ZK server
+ standaloneServerFactory.startup(server);
+ // Runs a 'stat' against the servers.
+ if (!waitForServerUp(currentClientPort, connectionTimeout)) {
+ throw new IOException("Waiting for startup of standalone server");
+ }
+
+ // We have selected a port as a client port. Update clientPortList if necessary.
+ if (clientPortList.size() <= i) { // it is not in the list, add the port
+ clientPortList.add(currentClientPort);
+ }
+ else if (clientPortList.get(i) <= 0) { // the list has invalid port, update with valid port
+ clientPortList.remove(i);
+ clientPortList.add(i, currentClientPort);
+ }
+
+ standaloneServerFactoryList.add(standaloneServerFactory);
+ zooKeeperServers.add(server);
+ }
+
+ // set the first one to be active ZK; Others are backups
+ activeZKServerIndex = 0;
+ started = true;
+ int clientPort = clientPortList.get(activeZKServerIndex);
+ LOG.info("Started MiniZooKeeperCluster and ran successful 'stat' " +
+ "on client port=" + clientPort);
+ return clientPort;
+ }
+
+ private void createDir(File dir) throws IOException {
+ try {
+ if (!dir.exists()) {
+ dir.mkdirs();
+ }
+ } catch (SecurityException e) {
+ throw new IOException("creating dir: " + dir, e);
+ }
+ }
+
+ /**
+ * @throws IOException
+ */
+ public void shutdown() throws IOException {
+ // shut down all the zk servers
+ for (int i = 0; i < standaloneServerFactoryList.size(); i++) {
+ NIOServerCnxnFactory standaloneServerFactory =
+ standaloneServerFactoryList.get(i);
+ int clientPort = clientPortList.get(i);
+
+ standaloneServerFactory.shutdown();
+ if (!waitForServerDown(clientPort, connectionTimeout)) {
+ throw new IOException("Waiting for shutdown of standalone server");
+ }
+ }
+ standaloneServerFactoryList.clear();
+
+ for (ZooKeeperServer zkServer: zooKeeperServers) {
+ //explicitly close ZKDatabase since ZookeeperServer does not close them
+ zkServer.getZKDatabase().close();
+ }
+ zooKeeperServers.clear();
+
+ // clear everything
+ if (started) {
+ started = false;
+ activeZKServerIndex = 0;
+ clientPortList.clear();
+ LOG.info("Shutdown MiniZK cluster with all ZK servers");
+ }
+ }
+
+ /**@return clientPort return clientPort if there is another ZK backup can run
+ * when killing the current active; return -1, if there is no backups.
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ public int killCurrentActiveZooKeeperServer() throws IOException,
+ InterruptedException {
+ if (!started || activeZKServerIndex < 0) {
+ return -1;
+ }
+
+ // Shutdown the current active one
+ NIOServerCnxnFactory standaloneServerFactory =
+ standaloneServerFactoryList.get(activeZKServerIndex);
+ int clientPort = clientPortList.get(activeZKServerIndex);
+
+ standaloneServerFactory.shutdown();
+ if (!waitForServerDown(clientPort, connectionTimeout)) {
+ throw new IOException("Waiting for shutdown of standalone server");
+ }
+
+ zooKeeperServers.get(activeZKServerIndex).getZKDatabase().close();
+
+ // remove the current active zk server
+ standaloneServerFactoryList.remove(activeZKServerIndex);
+ clientPortList.remove(activeZKServerIndex);
+ zooKeeperServers.remove(activeZKServerIndex);
+ LOG.info("Kill the current active ZK servers in the cluster " +
+ "on client port: " + clientPort);
+
+ if (standaloneServerFactoryList.isEmpty()) {
+ // there is no backup servers;
+ return -1;
+ }
+ clientPort = clientPortList.get(activeZKServerIndex);
+ LOG.info("Activate a backup zk server in the cluster " +
+ "on client port: " + clientPort);
+ // return the next back zk server's port
+ return clientPort;
+ }
+
+ /**
+ * Kill one back up ZK servers
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ public void killOneBackupZooKeeperServer() throws IOException,
+ InterruptedException {
+ if (!started || activeZKServerIndex < 0 ||
+ standaloneServerFactoryList.size() <= 1) {
+ return ;
+ }
+
+ int backupZKServerIndex = activeZKServerIndex+1;
+ // Shutdown the current active one
+ NIOServerCnxnFactory standaloneServerFactory =
+ standaloneServerFactoryList.get(backupZKServerIndex);
+ int clientPort = clientPortList.get(backupZKServerIndex);
+
+ standaloneServerFactory.shutdown();
+ if (!waitForServerDown(clientPort, connectionTimeout)) {
+ throw new IOException("Waiting for shutdown of standalone server");
+ }
+
+ zooKeeperServers.get(backupZKServerIndex).getZKDatabase().close();
+
+ // remove this backup zk server
+ standaloneServerFactoryList.remove(backupZKServerIndex);
+ clientPortList.remove(backupZKServerIndex);
+ zooKeeperServers.remove(backupZKServerIndex);
+ LOG.info("Kill one backup ZK servers in the cluster " +
+ "on client port: " + clientPort);
+ }
+
+ // XXX: From o.a.zk.t.ClientBase
+ private static boolean waitForServerDown(int port, long timeout) throws IOException {
+ long start = System.currentTimeMillis();
+ while (true) {
+ try {
+ Socket sock = new Socket("localhost", port);
+ try {
+ OutputStream outstream = sock.getOutputStream();
+ outstream.write("stat".getBytes());
+ outstream.flush();
+ } finally {
+ sock.close();
+ }
+ } catch (IOException e) {
+ return true;
+ }
+
+ if (System.currentTimeMillis() > start + timeout) {
+ break;
+ }
+ try {
+ Thread.sleep(250);
+ } catch (InterruptedException e) {
+ throw (InterruptedIOException)new InterruptedIOException().initCause(e);
+ }
+ }
+ return false;
+ }
+
+ // XXX: From o.a.zk.t.ClientBase
+ private static boolean waitForServerUp(int port, long timeout) throws IOException {
+ long start = System.currentTimeMillis();
+ while (true) {
+ try {
+ Socket sock = new Socket("localhost", port);
+ BufferedReader reader = null;
+ try {
+ OutputStream outstream = sock.getOutputStream();
+ outstream.write("stat".getBytes());
+ outstream.flush();
+
+ Reader isr = new InputStreamReader(sock.getInputStream());
+ reader = new BufferedReader(isr);
+ String line = reader.readLine();
+ if (line != null && line.startsWith("Zookeeper version:")) {
+ return true;
+ }
+ } finally {
+ sock.close();
+ if (reader != null) {
+ reader.close();
+ }
+ }
+ } catch (IOException e) {
+ // ignore as this is expected
+ LOG.info("server localhost:" + port + " not up " + e);
+ }
+
+ if (System.currentTimeMillis() > start + timeout) {
+ break;
+ }
+ try {
+ Thread.sleep(250);
+ } catch (InterruptedException e) {
+ throw (InterruptedIOException)new InterruptedIOException().initCause(e);
+ }
+ }
+ return false;
+ }
+
+ public int getClientPort() {
+ return activeZKServerIndex < 0 || activeZKServerIndex >= clientPortList.size() ? -1
+ : clientPortList.get(activeZKServerIndex);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java
new file mode 100644
index 0000000..da7d176
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+
+/**
+ * Placeholder of a watcher which might be triggered before the instance is not yet created.
+ * <p>
+ * {@code ZooKeeper} starts its event thread within its constructor (and that is an anti-pattern),
+ * and the watcher passed to the constructor might be called back by the event thread
+ * before you get the instance of {@code ZooKeeper} from the constructor.
+ * If your watcher calls methods of {@code ZooKeeper},
+ * pass this placeholder to the constructor of the {@code ZooKeeper},
+ * create your watcher using the instance of {@code ZooKeeper},
+ * and then call the method {@code PendingWatcher.prepare}.
+ */
+class PendingWatcher implements Watcher {
+ private final InstancePending<Watcher> pending = new InstancePending<>();
+
+ @Override
+ public void process(WatchedEvent event) {
+ pending.get().process(event);
+ }
+
+ /**
+ * Associates the substantial watcher of processing events.
+ * This method should be called once, and {@code watcher} should be non-null.
+ * This method is expected to call as soon as possible
+ * because the event processing, being invoked by the ZooKeeper event thread,
+ * is uninterruptibly blocked until this method is called.
+ */
+ void prepare(Watcher watcher) {
+ pending.prepare(watcher);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
new file mode 100644
index 0000000..d6c11af
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
@@ -0,0 +1,810 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.RetryCounter;
+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.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.OpResult;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooKeeper.States;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.proto.CreateRequest;
+import org.apache.zookeeper.proto.SetDataRequest;
+
+/**
+ * A zookeeper that can handle 'recoverable' errors.
+ * To handle recoverable errors, developers need to realize that there are two
+ * classes of requests: idempotent and non-idempotent requests. Read requests
+ * and unconditional sets and deletes are examples of idempotent requests, they
+ * can be reissued with the same results.
+ * (Although, the delete may throw a NoNodeException on reissue its effect on
+ * the ZooKeeper state is the same.) Non-idempotent requests need special
+ * handling, application and library writers need to keep in mind that they may
+ * need to encode information in the data or name of znodes to detect
+ * retries. A simple example is a create that uses a sequence flag.
+ * If a process issues a create("/x-", ..., SEQUENCE) and gets a connection
+ * loss exception, that process will reissue another
+ * create("/x-", ..., SEQUENCE) and get back x-111. When the process does a
+ * getChildren("/"), it sees x-1,x-30,x-109,x-110,x-111, now it could be
+ * that x-109 was the result of the previous create, so the process actually
+ * owns both x-109 and x-111. An easy way around this is to use "x-process id-"
+ * when doing the create. If the process is using an id of 352, before reissuing
+ * the create it will do a getChildren("/") and see "x-222-1", "x-542-30",
+ * "x-352-109", x-333-110". The process will know that the original create
+ * succeeded an the znode it created is "x-352-109".
+ * @see "http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling"
+ */
+@InterfaceAudience.Private
+public class RecoverableZooKeeper {
+ private static final Log LOG = LogFactory.getLog(RecoverableZooKeeper.class);
+ // the actual ZooKeeper client instance
+ private ZooKeeper zk;
+ private final RetryCounterFactory retryCounterFactory;
+ // An identifier of this process in the cluster
+ private final String identifier;
+ private final byte[] id;
+ private Watcher watcher;
+ private int sessionTimeout;
+ private String quorumServers;
+ private final ZKMetricsListener metrics;
+
+ public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
+ Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime)
+ throws IOException {
+ this(quorumServers, sessionTimeout, watcher, maxRetries, retryIntervalMillis, maxSleepTime,
+ null);
+ }
+
+ @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DE_MIGHT_IGNORE",
+ justification="None. Its always been this way.")
+ public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
+ Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime, String identifier)
+ throws IOException {
+ // TODO: Add support for zk 'chroot'; we don't add it to the quorumServers String as we should.
+ this.retryCounterFactory =
+ new RetryCounterFactory(maxRetries+1, retryIntervalMillis, maxSleepTime);
+
+ if (identifier == null || identifier.length() == 0) {
+ // the identifier = processID@hostName
+ identifier = ManagementFactory.getRuntimeMXBean().getName();
+ }
+ LOG.info("Process identifier=" + identifier +
+ " connecting to ZooKeeper ensemble=" + quorumServers);
+ this.identifier = identifier;
+ this.id = Bytes.toBytes(identifier);
+
+ this.watcher = watcher;
+ this.sessionTimeout = sessionTimeout;
+ this.quorumServers = quorumServers;
+ this.metrics = new ZKMetrics();
+ try {checkZk();} catch (Exception x) {/* ignore */}
+ }
+
+ /**
+ * Try to create a ZooKeeper connection. Turns any exception encountered into a
+ * KeeperException.OperationTimeoutException so it can retried.
+ * @return The created ZooKeeper connection object
+ * @throws KeeperException
+ */
+ protected synchronized ZooKeeper checkZk() throws KeeperException {
+ if (this.zk == null) {
+ try {
+ this.zk = new ZooKeeper(quorumServers, sessionTimeout, watcher);
+ } catch (IOException ex) {
+ LOG.warn("Unable to create ZooKeeper Connection", ex);
+ throw new KeeperException.OperationTimeoutException();
+ }
+ }
+ return zk;
+ }
+
+ public synchronized void reconnectAfterExpiration()
+ throws IOException, KeeperException, InterruptedException {
+ if (zk != null) {
+ LOG.info("Closing dead ZooKeeper connection, session" +
+ " was: 0x"+Long.toHexString(zk.getSessionId()));
+ zk.close();
+ // reset the ZooKeeper connection
+ zk = null;
+ }
+ checkZk();
+ LOG.info("Recreated a ZooKeeper, session" +
+ " is: 0x"+Long.toHexString(zk.getSessionId()));
+ }
+
+ /**
+ * delete is an idempotent operation. Retry before throwing exception.
+ * This function will not throw NoNodeException if the path does not
+ * exist.
+ */
+ public void delete(String path, int version) throws InterruptedException, KeeperException {
+ try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.delete")) {
+ RetryCounter retryCounter = retryCounterFactory.create();
+ boolean isRetry = false; // False for first attempt, true for all retries.
+ while (true) {
+ try {
+ long startTime = EnvironmentEdgeManager.currentTime();
+ checkZk().delete(path, version);
+ this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ return;
+ } catch (KeeperException e) {
+ this.metrics.registerFailedZKCall();
+ switch (e.code()) {
+ case NONODE:
+ if (isRetry) {
+ LOG.debug("Node " + path + " already deleted. Assuming a " +
+ "previous attempt succeeded.");
+ return;
+ }
+ LOG.debug("Node " + path + " already deleted, retry=" + isRetry);
+ throw e;
+
+ case CONNECTIONLOSS:
+ this.metrics.registerConnectionLossException();
+ retryOrThrow(retryCounter, e, "delete");
+ break;
+ case OPERATIONTIMEOUT:
+ this.metrics.registerOperationTimeoutException();
+ retryOrThrow(retryCounter, e, "delete");
+ break;
+
+ default:
+ throw e;
+ }
+ }
+ retryCounter.sleepUntilNextRetry();
+ isRetry = true;
+ }
+ }
+ }
+
+ /**
+ * exists is an idempotent operation. Retry before throwing exception
+ * @return A Stat instance
+ */
+ public Stat exists(String path, Watcher watcher) throws KeeperException, InterruptedException {
+ try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.exists")) {
+ RetryCounter retryCounter = retryCounterFactory.create();
+ while (true) {
+ try {
+ long startTime = EnvironmentEdgeManager.currentTime();
+ Stat nodeStat = checkZk().exists(path, watcher);
+ this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ return nodeStat;
+ } catch (KeeperException e) {
+ this.metrics.registerFailedZKCall();
+ switch (e.code()) {
+ case CONNECTIONLOSS:
+ this.metrics.registerConnectionLossException();
+ retryOrThrow(retryCounter, e, "exists");
+ break;
+ case OPERATIONTIMEOUT:
+ this.metrics.registerOperationTimeoutException();
+ retryOrThrow(retryCounter, e, "exists");
+ break;
+
+ default:
+ throw e;
+ }
+ }
+ retryCounter.sleepUntilNextRetry();
+ }
+ }
+ }
+
+ /**
+ * exists is an idempotent operation. Retry before throwing exception
+ * @return A Stat instance
+ */
+ public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException {
+ try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.exists")) {
+ RetryCounter retryCounter = retryCounterFactory.create();
+ while (true) {
+ try {
+ long startTime = EnvironmentEdgeManager.currentTime();
+ Stat nodeStat = checkZk().exists(path, watch);
+ this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ return nodeStat;
+ } catch (KeeperException e) {
+ this.metrics.registerFailedZKCall();
+ switch (e.code()) {
+ case CONNECTIONLOSS:
+ this.metrics.registerConnectionLossException();
+ retryOrThrow(retryCounter, e, "exists");
+ break;
+ case OPERATIONTIMEOUT:
+ this.metrics.registerOperationTimeoutException();
+ retryOrThrow(retryCounter, e, "exists");
+ break;
+
+ default:
+ throw e;
+ }
+ }
+ retryCounter.sleepUntilNextRetry();
+ }
+ }
+ }
+
+ private void retryOrThrow(RetryCounter retryCounter, KeeperException e,
+ String opName) throws KeeperException {
+ if (!retryCounter.shouldRetry()) {
+ LOG.error("ZooKeeper " + opName + " failed after "
+ + retryCounter.getMaxAttempts() + " attempts");
+ throw e;
+ }
+ LOG.debug("Retry, connectivity issue (JVM Pause?); quorum=" + quorumServers + "," +
+ "exception=" + e);
+ }
+
+ /**
+ * getChildren is an idempotent operation. Retry before throwing exception
+ * @return List of children znodes
+ */
+ public List<String> getChildren(String path, Watcher watcher)
+ throws KeeperException, InterruptedException {
+ try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getChildren")) {
+ RetryCounter retryCounter = retryCounterFactory.create();
+ while (true) {
+ try {
+ long startTime = EnvironmentEdgeManager.currentTime();
+ List<String> children = checkZk().getChildren(path, watcher);
+ this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ return children;
+ } catch (KeeperException e) {
+ this.metrics.registerFailedZKCall();
+ switch (e.code()) {
+ case CONNECTIONLOSS:
+ this.metrics.registerConnectionLossException();
+ retryOrThrow(retryCounter, e, "getChildren");
+ break;
+ case OPERATIONTIMEOUT:
+ this.metrics.registerOperationTimeoutException();
+ retryOrThrow(retryCounter, e, "getChildren");
+ break;
+
+ default:
+ throw e;
+ }
+ }
+ retryCounter.sleepUntilNextRetry();
+ }
+ }
+ }
+
+ /**
+ * getChildren is an idempotent operation. Retry before throwing exception
+ * @return List of children znodes
+ */
+ public List<String> getChildren(String path, boolean watch)
+ throws KeeperException, InterruptedException {
+ try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getChildren")) {
+ RetryCounter retryCounter = retryCounterFactory.create();
+ while (true) {
+ try {
+ long startTime = EnvironmentEdgeManager.currentTime();
+ List<String> children = checkZk().getChildren(path, watch);
+ this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ return children;
+ } catch (KeeperException e) {
+ this.metrics.registerFailedZKCall();
+ switch (e.code()) {
+ case CONNECTIONLOSS:
+ this.metrics.registerConnectionLossException();
+ retryOrThrow(retryCounter, e, "getChildren");
+ break;
+ case OPERATIONTIMEOUT:
+ this.metrics.registerOperationTimeoutException();
+ retryOrThrow(retryCounter, e, "getChildren");
+ break;
+
+ default:
+ throw e;
+ }
+ }
+ retryCounter.sleepUntilNextRetry();
+ }
+ }
+ }
+
+ /**
+ * getData is an idempotent operation. Retry before throwing exception
+ * @return Data
+ */
+ public byte[] getData(String path, Watcher watcher, Stat stat)
+ throws KeeperException, InterruptedException {
+ try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getData")) {
+ RetryCounter retryCounter = retryCounterFactory.create();
+ while (true) {
+ try {
+ long startTime = EnvironmentEdgeManager.currentTime();
+ byte[] revData = checkZk().getData(path, watcher, stat);
+ this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ return ZKMetadata.removeMetaData(revData);
+ } catch (KeeperException e) {
+ this.metrics.registerFailedZKCall();
+ switch (e.code()) {
+ case CONNECTIONLOSS:
+ this.metrics.registerConnectionLossException();
+ retryOrThrow(retryCounter, e, "getData");
+ break;
+ case OPERATIONTIMEOUT:
+ this.metrics.registerOperationTimeoutException();
+ retryOrThrow(retryCounter, e, "getData");
+ break;
+
+ default:
+ throw e;
+ }
+ }
+ retryCounter.sleepUntilNextRetry();
+ }
+ }
+ }
+
+ /**
+ * getData is an idempotent operation. Retry before throwing exception
+ * @return Data
+ */
+ public byte[] getData(String path, boolean watch, Stat stat)
+ throws KeeperException, InterruptedException {
+ try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getData")) {
+ RetryCounter retryCounter = retryCounterFactory.create();
+ while (true) {
+ try {
+ long startTime = EnvironmentEdgeManager.currentTime();
+ byte[] revData = checkZk().getData(path, watch, stat);
+ this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ return ZKMetadata.removeMetaData(revData);
+ } catch (KeeperException e) {
+ this.metrics.registerFailedZKCall();
+ switch (e.code()) {
+ case CONNECTIONLOSS:
+ this.metrics.registerConnectionLossException();
+ retryOrThrow(retryCounter, e, "getData");
+ break;
+ case OPERATIONTIMEOUT:
+ this.metrics.registerOperationTimeoutException();
+ retryOrThrow(retryCounter, e, "getData");
+ break;
+
+ default:
+ throw e;
+ }
+ }
+ retryCounter.sleepUntilNextRetry();
+ }
+ }
+ }
+
+ /**
+ * setData is NOT an idempotent operation. Retry may cause BadVersion Exception
+ * Adding an identifier field into the data to check whether
+ * badversion is caused by the result of previous correctly setData
+ * @return Stat instance
+ */
+ public Stat setData(String path, byte[] data, int version)
+ throws KeeperException, InterruptedException {
+ try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.setData")) {
+ RetryCounter retryCounter = retryCounterFactory.create();
+ byte[] newData = ZKMetadata.appendMetaData(id, data);
+ boolean isRetry = false;
+ long startTime;
+ while (true) {
+ try {
+ startTime = EnvironmentEdgeManager.currentTime();
+ Stat nodeStat = checkZk().setData(path, newData, version);
+ this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ return nodeStat;
+ } catch (KeeperException e) {
+ this.metrics.registerFailedZKCall();
+ switch (e.code()) {
+ case CONNECTIONLOSS:
+ this.metrics.registerConnectionLossException();
+ retryOrThrow(retryCounter, e, "setData");
+ break;
+ case OPERATIONTIMEOUT:
+ this.metrics.registerOperationTimeoutException();
+ retryOrThrow(retryCounter, e, "setData");
+ break;
+ case BADVERSION:
+ if (isRetry) {
+ // try to verify whether the previous setData success or not
+ try{
+ Stat stat = new Stat();
+ startTime = EnvironmentEdgeManager.currentTime();
+ byte[] revData = checkZk().getData(path, false, stat);
+ this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ if(Bytes.compareTo(revData, newData) == 0) {
+ // the bad version is caused by previous successful setData
+ return stat;
+ }
+ } catch(KeeperException keeperException){
+ this.metrics.registerFailedZKCall();
+ // the ZK is not reliable at this moment. just throwing exception
+ throw keeperException;
+ }
+ }
+ // throw other exceptions and verified bad version exceptions
+ default:
+ throw e;
+ }
+ }
+ retryCounter.sleepUntilNextRetry();
+ isRetry = true;
+ }
+ }
+ }
+
+ /**
+ * getAcl is an idempotent operation. Retry before throwing exception
+ * @return list of ACLs
+ */
+ public List<ACL> getAcl(String path, Stat stat)
+ throws KeeperException, InterruptedException {
+ try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getAcl")) {
+ RetryCounter retryCounter = retryCounterFactory.create();
+ while (true) {
+ try {
+ long startTime = EnvironmentEdgeManager.currentTime();
+ List<ACL> nodeACL = checkZk().getACL(path, stat);
+ this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ return nodeACL;
+ } catch (KeeperException e) {
+ this.metrics.registerFailedZKCall();
+ switch (e.code()) {
+ case CONNECTIONLOSS:
+ this.metrics.registerConnectionLossException();
+ retryOrThrow(retryCounter, e, "getAcl");
+ break;
+ case OPERATIONTIMEOUT:
+ this.metrics.registerOperationTimeoutException();
+ retryOrThrow(retryCounter, e, "getAcl");
+ break;
+
+ default:
+ throw e;
+ }
+ }
+ retryCounter.sleepUntilNextRetry();
+ }
+ }
+ }
+
+ /**
+ * setAcl is an idempotent operation. Retry before throwing exception
+ * @return list of ACLs
+ */
+ public Stat setAcl(String path, List<ACL> acls, int version)
+ throws KeeperException, InterruptedException {
+ try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.setAcl")) {
+ RetryCounter retryCounter = retryCounterFactory.create();
+ while (true) {
+ try {
+ long startTime = EnvironmentEdgeManager.currentTime();
+ Stat nodeStat = checkZk().setACL(path, acls, version);
+ this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ return nodeStat;
+ } catch (KeeperException e) {
+ this.metrics.registerFailedZKCall();
+ switch (e.code()) {
+ case CONNECTIONLOSS:
+ this.metrics.registerConnectionLossException();
+ retryOrThrow(retryCounter, e, "setAcl");
+ break;
+ case OPERATIONTIMEOUT:
+ this.metrics.registerOperationTimeoutException();
+ retryOrThrow(retryCounter, e, "setAcl");
+ break;
+
+ default:
+ throw e;
+ }
+ }
+ retryCounter.sleepUntilNextRetry();
+ }
+ }
+ }
+
+ /**
+ * <p>
+ * NONSEQUENTIAL create is idempotent operation.
+ * Retry before throwing exceptions.
+ * But this function will not throw the NodeExist exception back to the
+ * application.
+ * </p>
+ * <p>
+ * But SEQUENTIAL is NOT idempotent operation. It is necessary to add
+ * identifier to the path to verify, whether the previous one is successful
+ * or not.
+ * </p>
+ *
+ * @return Path
+ */
+ public String create(String path, byte[] data, List<ACL> acl,
+ CreateMode createMode)
+ throws KeeperException, InterruptedException {
+ try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.create")) {
+ byte[] newData = ZKMetadata.appendMetaData(id, data);
+ switch (createMode) {
+ case EPHEMERAL:
+ case PERSISTENT:
+ return createNonSequential(path, newData, acl, createMode);
+
+ case EPHEMERAL_SEQUENTIAL:
+ case PERSISTENT_SEQUENTIAL:
+ return createSequential(path, newData, acl, createMode);
+
+ default:
+ throw new IllegalArgumentException("Unrecognized CreateMode: " +
+ createMode);
+ }
+ }
+ }
+
+ private String createNonSequential(String path, byte[] data, List<ACL> acl,
+ CreateMode createMode) throws KeeperException, InterruptedException {
+ RetryCounter retryCounter = retryCounterFactory.create();
+ boolean isRetry = false; // False for first attempt, true for all retries.
+ long startTime;
+ while (true) {
+ try {
+ startTime = EnvironmentEdgeManager.currentTime();
+ String nodePath = checkZk().create(path, data, acl, createMode);
+ this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ return nodePath;
+ } catch (KeeperException e) {
+ this.metrics.registerFailedZKCall();
+ switch (e.code()) {
+ case NODEEXISTS:
+ if (isRetry) {
+ // If the connection was lost, there is still a possibility that
+ // we have successfully created the node at our previous attempt,
+ // so we read the node and compare.
+ startTime = EnvironmentEdgeManager.currentTime();
+ byte[] currentData = checkZk().getData(path, false, null);
+ this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ if (currentData != null &&
+ Bytes.compareTo(currentData, data) == 0) {
+ // We successfully created a non-sequential node
+ return path;
+ }
+ LOG.error("Node " + path + " already exists with " +
+ Bytes.toStringBinary(currentData) + ", could not write " +
+ Bytes.toStringBinary(data));
+ throw e;
+ }
+ LOG.debug("Node " + path + " already exists");
+ throw e;
+
+ case CONNECTIONLOSS:
+ this.metrics.registerConnectionLossException();
+ retryOrThrow(retryCounter, e, "create");
+ break;
+ case OPERATIONTIMEOUT:
+ this.metrics.registerOperationTimeoutException();
+ retryOrThrow(retryCounter, e, "create");
+ break;
+
+ default:
+ throw e;
+ }
+ }
+ retryCounter.sleepUntilNextRetry();
+ isRetry = true;
+ }
+ }
+
+ private String createSequential(String path, byte[] data,
+ List<ACL> acl, CreateMode createMode)
+ throws KeeperException, InterruptedException {
+ RetryCounter retryCounter = retryCounterFactory.create();
+ boolean first = true;
+ String newPath = path+this.identifier;
+ while (true) {
+ try {
+ if (!first) {
+ // Check if we succeeded on a previous attempt
+ String previousResult = findPreviousSequentialNode(newPath);
+ if (previousResult != null) {
+ return previousResult;
+ }
+ }
+ first = false;
+ long startTime = EnvironmentEdgeManager.currentTime();
+ String nodePath = checkZk().create(newPath, data, acl, createMode);
+ this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ return nodePath;
+ } catch (KeeperException e) {
+ this.metrics.registerFailedZKCall();
+ switch (e.code()) {
+ case CONNECTIONLOSS:
+ this.metrics.registerConnectionLossException();
+ retryOrThrow(retryCounter, e, "create");
+ break;
+ case OPERATIONTIMEOUT:
+ this.metrics.registerOperationTimeoutException();
+ retryOrThrow(retryCounter, e, "create");
+ break;
+
+ default:
+ throw e;
+ }
+ }
+ retryCounter.sleepUntilNextRetry();
+ }
+ }
+ /**
+ * 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).
+ */
+ private Iterable<Op> prepareZKMulti(Iterable<Op> ops)
+ throws UnsupportedOperationException {
+ if(ops == null) return null;
+
+ List<Op> preparedOps = new LinkedList<>();
+ for (Op op : ops) {
+ if (op.getType() == ZooDefs.OpCode.create) {
+ CreateRequest create = (CreateRequest)op.toRequestRecord();
+ preparedOps.add(Op.create(create.getPath(), ZKMetadata.appendMetaData(id, create.getData()),
+ create.getAcl(), create.getFlags()));
+ } else if (op.getType() == ZooDefs.OpCode.delete) {
+ // no need to appendMetaData for delete
+ preparedOps.add(op);
+ } else if (op.getType() == ZooDefs.OpCode.setData) {
+ SetDataRequest setData = (SetDataRequest)op.toRequestRecord();
+ preparedOps.add(Op.setData(setData.getPath(), ZKMetadata.appendMetaData(id, setData.getData()),
+ setData.getVersion()));
+ } else {
+ throw new UnsupportedOperationException("Unexpected ZKOp type: " + op.getClass().getName());
+ }
+ }
+ return preparedOps;
+ }
+
+ /**
+ * Run multiple operations in a transactional manner. Retry before throwing exception
+ */
+ public List<OpResult> multi(Iterable<Op> ops)
+ throws KeeperException, InterruptedException {
+ try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.multi")) {
+ RetryCounter retryCounter = retryCounterFactory.create();
+ Iterable<Op> multiOps = prepareZKMulti(ops);
+ while (true) {
+ try {
+ long startTime = EnvironmentEdgeManager.currentTime();
+ List<OpResult> opResults = checkZk().multi(multiOps);
+ this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ return opResults;
+ } catch (KeeperException e) {
+ this.metrics.registerFailedZKCall();
+ switch (e.code()) {
+ case CONNECTIONLOSS:
+ this.metrics.registerConnectionLossException();
+ retryOrThrow(retryCounter, e, "multi");
+ break;
+ case OPERATIONTIMEOUT:
+ this.metrics.registerOperationTimeoutException();
+ retryOrThrow(retryCounter, e, "multi");
+ break;
+
+ default:
+ throw e;
+ }
+ }
+ retryCounter.sleepUntilNextRetry();
+ }
+ }
+ }
+
+ private String findPreviousSequentialNode(String path)
+ throws KeeperException, InterruptedException {
+ int lastSlashIdx = path.lastIndexOf('/');
+ assert(lastSlashIdx != -1);
+ String parent = path.substring(0, lastSlashIdx);
+ String nodePrefix = path.substring(lastSlashIdx+1);
+ long startTime = EnvironmentEdgeManager.currentTime();
+ List<String> nodes = checkZk().getChildren(parent, false);
+ this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ List<String> matching = filterByPrefix(nodes, nodePrefix);
+ for (String node : matching) {
+ String nodePath = parent + "/" + node;
+ startTime = EnvironmentEdgeManager.currentTime();
+ Stat stat = checkZk().exists(nodePath, false);
+ this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ if (stat != null) {
+ return nodePath;
+ }
+ }
+ return null;
+ }
+
+ public synchronized long getSessionId() {
+ return zk == null ? -1 : zk.getSessionId();
+ }
+
+ public synchronized void close() throws InterruptedException {
+ if (zk != null) zk.close();
+ }
+
+ public synchronized States getState() {
+ return zk == null ? null : zk.getState();
+ }
+
+ public synchronized ZooKeeper getZooKeeper() {
+ return zk;
+ }
+
+ public synchronized byte[] getSessionPasswd() {
+ return zk == null ? null : zk.getSessionPasswd();
+ }
+
+ public void sync(String path, AsyncCallback.VoidCallback cb, Object ctx) throws KeeperException {
+ long startTime = EnvironmentEdgeManager.currentTime();
+ checkZk().sync(path, cb, null);
+ this.metrics.registerSyncOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+ }
+
+ /**
+ * Filters the given node list by the given prefixes.
+ * This method is all-inclusive--if any element in the node list starts
+ * with any of the given prefixes, then it is included in the result.
+ *
+ * @param nodes the nodes to filter
+ * @param prefixes the prefixes to include in the result
+ * @return list of every element that starts with one of the prefixes
+ */
+ private static List<String> filterByPrefix(List<String> nodes,
+ String... prefixes) {
+ List<String> lockChildren = new ArrayList<>();
+ for (String child : nodes){
+ for (String prefix : prefixes){
+ if (child.startsWith(prefix)){
+ lockChildren.add(child);
+ break;
+ }
+ }
+ }
+ return lockChildren;
+ }
+
+ public String getIdentifier() {
+ return identifier;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java
new file mode 100644
index 0000000..93545ee
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Abortable;
+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.zookeeper.KeeperException;
+
+import java.io.IOException;
+
+/**
+ * Tracks region normalizer state up in ZK
+ */
+public class RegionNormalizerTracker extends ZKNodeTracker {
+ private static final Log LOG = LogFactory.getLog(RegionNormalizerTracker.class);
+
+ public RegionNormalizerTracker(ZKWatcher watcher,
+ Abortable abortable) {
+ super(watcher, watcher.znodePaths.regionNormalizerZNode, abortable);
+ }
+
+ /**
+ * Return true if region normalizer is on, false otherwise
+ */
+ public boolean isNormalizerOn() {
+ byte [] upData = super.getData(false);
+ try {
+ // if data in ZK is null, use default of on.
+ return upData == null || parseFrom(upData).getNormalizerOn();
+ } catch (DeserializationException dex) {
+ LOG.error("ZK state for RegionNormalizer could not be parsed "
+ + Bytes.toStringBinary(upData));
+ // return false to be safe.
+ return false;
+ }
+ }
+
+ /**
+ * Set region normalizer on/off
+ * @param normalizerOn whether normalizer should be on or off
+ * @throws KeeperException
+ */
+ public void setNormalizerOn(boolean normalizerOn) throws KeeperException {
+ byte [] upData = toByteArray(normalizerOn);
+ try {
+ ZKUtil.setData(watcher, watcher.znodePaths.regionNormalizerZNode, upData);
+ } catch(KeeperException.NoNodeException nne) {
+ ZKUtil.createAndWatch(watcher, watcher.znodePaths.regionNormalizerZNode, upData);
+ }
+ super.nodeDataChanged(watcher.znodePaths.regionNormalizerZNode);
+ }
+
+ private byte [] toByteArray(boolean isNormalizerOn) {
+ RegionNormalizerProtos.RegionNormalizerState.Builder builder =
+ RegionNormalizerProtos.RegionNormalizerState.newBuilder();
+ builder.setNormalizerOn(isNormalizerOn);
+ return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
+ }
+
+ private RegionNormalizerProtos.RegionNormalizerState parseFrom(byte [] pbBytes)
+ throws DeserializationException {
+ ProtobufUtil.expectPBMagicPrefix(pbBytes);
+ RegionNormalizerProtos.RegionNormalizerState.Builder builder =
+ RegionNormalizerProtos.RegionNormalizerState.newBuilder();
+ try {
+ int magicLen = ProtobufUtil.lengthOfPBMagic();
+ ProtobufUtil.mergeFrom(builder, pbBytes, magicLen, pbBytes.length - magicLen);
+ } catch (IOException e) {
+ throw new DeserializationException(e);
+ }
+ return builder.build();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAclReset.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAclReset.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAclReset.java
new file mode 100644
index 0000000..4150f54
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAclReset.java
@@ -0,0 +1,116 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+
+/**
+ * You may add the jaas.conf option
+ * -Djava.security.auth.login.config=/PATH/jaas.conf
+ *
+ * You may also specify -D to set options
+ * "hbase.zookeeper.quorum" (it should be in hbase-site.xml)
+ * "zookeeper.znode.parent" (it should be in hbase-site.xml)
+ *
+ * Use -set-acls to set the ACLs, no option to erase ACLs
+ */
+@InterfaceAudience.Private
+public class ZKAclReset extends Configured implements Tool {
+ private static final Log LOG = LogFactory.getLog(ZKAclReset.class);
+
+ private static void resetAcls(final ZKWatcher zkw, final String znode,
+ final boolean eraseAcls) throws Exception {
+ List<String> children = ZKUtil.listChildrenNoWatch(zkw, znode);
+ if (children != null) {
+ for (String child: children) {
+ resetAcls(zkw, ZNodePaths.joinZNode(znode, child), eraseAcls);
+ }
+ }
+
+ ZooKeeper zk = zkw.getRecoverableZooKeeper().getZooKeeper();
+ if (eraseAcls) {
+ LOG.info(" - erase ACLs for " + znode);
+ zk.setACL(znode, ZooDefs.Ids.OPEN_ACL_UNSAFE, -1);
+ } else {
+ LOG.info(" - set ACLs for " + znode);
+ zk.setACL(znode, ZKUtil.createACL(zkw, znode, true), -1);
+ }
+ }
+
+ private static void resetAcls(final Configuration conf, boolean eraseAcls)
+ throws Exception {
+ ZKWatcher zkw = new ZKWatcher(conf, "ZKAclReset", null);
+ try {
+ LOG.info((eraseAcls ? "Erase" : "Set") + " HBase ACLs for " +
+ zkw.getQuorum() + " " + zkw.znodePaths.baseZNode);
+ resetAcls(zkw, zkw.znodePaths.baseZNode, eraseAcls);
+ } finally {
+ zkw.close();
+ }
+ }
+
+ private void printUsageAndExit() {
+ System.err.printf("Usage: hbase %s [options]%n", getClass().getName());
+ System.err.println(" where [options] are:");
+ System.err.println(" -h|-help Show this help and exit.");
+ System.err.println(" -set-acls Setup the hbase znode ACLs for a secure cluster");
+ System.err.println();
+ System.err.println("Examples:");
+ System.err.println(" To reset the ACLs to the unsecure cluster behavior:");
+ System.err.println(" hbase " + getClass().getName());
+ System.err.println();
+ System.err.println(" To reset the ACLs to the secure cluster behavior:");
+ System.err.println(" hbase " + getClass().getName() + " -set-acls");
+ System.exit(1);
+ }
+
+ @Override
+ public int run(String[] args) throws Exception {
+ boolean eraseAcls = true;
+
+ for (int i = 0; i < args.length; ++i) {
+ if (args[i].equals("-help")) {
+ printUsageAndExit();
+ } else if (args[i].equals("-set-acls")) {
+ eraseAcls = false;
+ } else {
+ printUsageAndExit();
+ }
+ }
+
+ resetAcls(getConf(), eraseAcls);
+ return(0);
+ }
+
+ public static void main(String[] args) throws Exception {
+ System.exit(ToolRunner.run(HBaseConfiguration.create(), new ZKAclReset(), args));
+ }
+}
[07/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper
from hbase-server and hbase-client
Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 3b3d568..bb7eca2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -68,6 +68,7 @@ import org.apache.hadoop.hbase.client.ImmutableHRegionInfo;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.trace.TraceUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.BufferedMutator;
@@ -133,7 +134,6 @@ import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -2752,7 +2752,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
}
- public void expireSession(ZooKeeperWatcher nodeZK) throws Exception {
+ public void expireSession(ZKWatcher nodeZK) throws Exception {
expireSession(nodeZK, false);
}
@@ -2767,7 +2767,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param checkStatus - true to check if we can create a Table with the
* current configuration.
*/
- public void expireSession(ZooKeeperWatcher nodeZK, boolean checkStatus)
+ public void expireSession(ZKWatcher nodeZK, boolean checkStatus)
throws Exception {
Configuration c = new Configuration(this.conf);
String quorumServers = ZKConfig.getZKQuorumServersString(c);
@@ -2882,18 +2882,18 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
private HBaseAdmin hbaseAdmin = null;
/**
- * Returns a ZooKeeperWatcher instance.
+ * Returns a ZKWatcher instance.
* This instance is shared between HBaseTestingUtility instance users.
* Don't close it, it will be closed automatically when the
* cluster shutdowns
*
- * @return The ZooKeeperWatcher instance.
+ * @return The ZKWatcher instance.
* @throws IOException
*/
- public synchronized ZooKeeperWatcher getZooKeeperWatcher()
+ public synchronized ZKWatcher getZooKeeperWatcher()
throws IOException {
if (zooKeeperWatcher == null) {
- zooKeeperWatcher = new ZooKeeperWatcher(conf, "testing utility",
+ zooKeeperWatcher = new ZKWatcher(conf, "testing utility",
new Abortable() {
@Override public void abort(String why, Throwable e) {
throw new RuntimeException("Unexpected abort in HBaseTestingUtility:"+why, e);
@@ -2903,7 +2903,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
return zooKeeperWatcher;
}
- private ZooKeeperWatcher zooKeeperWatcher;
+ private ZKWatcher zooKeeperWatcher;
@@ -3508,13 +3508,13 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
/**
- * Gets a ZooKeeperWatcher.
+ * Gets a ZKWatcher.
* @param TEST_UTIL
*/
- public static ZooKeeperWatcher getZooKeeperWatcher(
+ public static ZKWatcher getZooKeeperWatcher(
HBaseTestingUtility TEST_UTIL) throws ZooKeeperConnectionException,
IOException {
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+ ZKWatcher zkw = new ZKWatcher(TEST_UTIL.getConfiguration(),
"unittest", new Abortable() {
boolean aborted = false;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index cac6fd6..53f27d1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -23,7 +23,6 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import java.util.Set;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -54,7 +53,7 @@ import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
@@ -71,18 +70,18 @@ public class MockRegionServerServices implements RegionServerServices {
new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
private HFileSystem hfs = null;
private final Configuration conf;
- private ZooKeeperWatcher zkw = null;
+ private ZKWatcher zkw = null;
private ServerName serverName = null;
private RpcServerInterface rpcServer = null;
private volatile boolean abortRequested;
private volatile boolean stopping = false;
private final AtomicBoolean running = new AtomicBoolean(true);
- MockRegionServerServices(ZooKeeperWatcher zkw) {
+ MockRegionServerServices(ZKWatcher zkw) {
this(zkw, null);
}
- MockRegionServerServices(ZooKeeperWatcher zkw, ServerName serverName) {
+ MockRegionServerServices(ZKWatcher zkw, ServerName serverName) {
this.zkw = zkw;
this.serverName = serverName;
this.conf = (zkw == null ? new Configuration() : zkw.getConfiguration());
@@ -167,7 +166,7 @@ public class MockRegionServerServices implements RegionServerServices {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return zkw;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
index fb1c1e2..961677b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRespon
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -124,7 +124,7 @@ public class TestMetaTableAccessorNoCluster {
public void testRideOverServerNotRunning()
throws IOException, InterruptedException, ServiceException {
// Need a zk watcher.
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(UTIL.getConfiguration(),
+ ZKWatcher zkw = new ZKWatcher(UTIL.getConfiguration(),
this.getClass().getSimpleName(), ABORTABLE, true);
// This is a servername we use in a few places below.
ServerName sn = ServerName.valueOf("example.com", 1234, System.currentTimeMillis());
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
index 6a904a7..c2a4945 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.AfterClass;
@@ -66,7 +66,7 @@ public class TestMetaTableLocator {
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static final ServerName SN =
ServerName.valueOf("example.org", 1234, System.currentTimeMillis());
- private ZooKeeperWatcher watcher;
+ private ZKWatcher watcher;
private Abortable abortable;
@BeforeClass public static void beforeClass() throws Exception {
@@ -91,7 +91,7 @@ public class TestMetaTableLocator {
return false;
}
};
- this.watcher = new ZooKeeperWatcher(UTIL.getConfiguration(),
+ this.watcher = new ZKWatcher(UTIL.getConfiguration(),
this.getClass().getSimpleName(), this.abortable, true);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
index d546d5d..ed93b17 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooDefs;
@@ -184,8 +184,8 @@ public class TestZooKeeper {
*/
@Test
public void testCreateWithParents() throws Exception {
- ZooKeeperWatcher zkw =
- new ZooKeeperWatcher(new Configuration(TEST_UTIL.getConfiguration()),
+ ZKWatcher zkw =
+ new ZKWatcher(new Configuration(TEST_UTIL.getConfiguration()),
TestZooKeeper.class.getName(), null);
byte[] expectedData = new byte[] { 1, 2, 3 };
ZKUtil.createWithParents(zkw, "/l1/l2/l3/l4/testCreateWithParents", expectedData);
@@ -206,7 +206,7 @@ public class TestZooKeeper {
*/
@Test
public void testZNodeDeletes() throws Exception {
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(
+ ZKWatcher zkw = new ZKWatcher(
new Configuration(TEST_UTIL.getConfiguration()),
TestZooKeeper.class.getName(), null);
ZKUtil.createWithParents(zkw, "/l1/l2/l3/l4");
@@ -247,7 +247,7 @@ public class TestZooKeeper {
// Assumes the root of the ZooKeeper space is writable as it creates a node
// wherever the cluster home is defined.
- ZooKeeperWatcher zk2 = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+ ZKWatcher zk2 = new ZKWatcher(TEST_UTIL.getConfiguration(),
"testCreateSilentIsReallySilent", null);
// Save the previous ACL
@@ -330,7 +330,7 @@ public class TestZooKeeper {
@SuppressWarnings("deprecation")
public void testGetChildDataAndWatchForNewChildrenShouldNotThrowNPE()
throws Exception {
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), name.getMethodName(), null);
+ ZKWatcher zkw = new ZKWatcher(TEST_UTIL.getConfiguration(), name.getMethodName(), null);
ZKUtil.getChildDataAndWatchForNewChildren(zkw, "/wrongNode");
}
@@ -345,7 +345,7 @@ public class TestZooKeeper {
cluster.startRegionServer();
cluster.waitForActiveAndReadyMaster(10000);
HMaster m = cluster.getMaster();
- final ZooKeeperWatcher zkw = m.getZooKeeper();
+ final ZKWatcher zkw = m.getZooKeeper();
// now the cluster is up. So assign some regions.
try (Admin admin = TEST_UTIL.getAdmin()) {
byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b"),
@@ -376,14 +376,14 @@ public class TestZooKeeper {
* Count listeners in zkw excluding listeners, that belongs to workers or other
* temporary processes.
*/
- private int countPermanentListeners(ZooKeeperWatcher watcher) {
+ private int countPermanentListeners(ZKWatcher watcher) {
return countListeners(watcher, ZkSplitLogWorkerCoordination.class);
}
/**
* Count listeners in zkw excluding provided classes
*/
- private int countListeners(ZooKeeperWatcher watcher, Class<?>... exclude) {
+ private int countListeners(ZKWatcher watcher, Class<?>... exclude) {
int cnt = 0;
for (Object o : watcher.getListeners()) {
boolean skip = false;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
index aa246c2..1c2279c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.StoppableImplementation;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.AfterClass;
@@ -93,7 +93,7 @@ public class TestZooKeeperTableArchiveClient {
CONNECTION = (ClusterConnection)ConnectionFactory.createConnection(UTIL.getConfiguration());
archivingClient = new ZKTableArchiveClient(UTIL.getConfiguration(), CONNECTION);
// make hfile archiving node so we can archive files
- ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
+ ZKWatcher watcher = UTIL.getZooKeeperWatcher();
String archivingZNode = ZKTableArchiveClient.getArchiveZNode(UTIL.getConfiguration(), watcher);
ZKUtil.createWithParents(watcher, archivingZNode);
rss = mock(RegionServerServices.class);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 6a4ada3..5660a01 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -56,8 +56,8 @@ import org.apache.hadoop.hbase.util.HBaseFsckRepair;
import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.Before;
@@ -132,7 +132,7 @@ public class TestMetaWithReplicas {
@Test
public void testZookeeperNodesForReplicas() throws Exception {
// Checks all the znodes exist when meta's replicas are enabled
- ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
+ ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
Configuration conf = TEST_UTIL.getConfiguration();
String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
@@ -166,7 +166,7 @@ public class TestMetaWithReplicas {
// server holding the primary meta replica. Then it does a put/get into/from
// the test table. The put/get operations would use the replicas to locate the
// location of the test table's region
- ZooKeeperWatcher zkw = util.getZooKeeperWatcher();
+ ZKWatcher zkw = util.getZooKeeperWatcher();
Configuration conf = util.getConfiguration();
conf.setBoolean(HConstants.USE_META_REPLICAS, true);
@@ -368,7 +368,7 @@ public class TestMetaWithReplicas {
false, false);
HBaseFsckRepair.closeRegionSilentlyAndWait(c,
rl.getRegionLocation(2).getServerName(), rl.getRegionLocation(2).getRegionInfo());
- ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
+ ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
ZKUtil.deleteNode(zkw, zkw.znodePaths.getZNodeForReplica(2));
// check that problem exists
HBaseFsck hbck = doFsck(TEST_UTIL.getConfiguration(), false);
@@ -400,7 +400,7 @@ public class TestMetaWithReplicas {
// caches update themselves. Uses the master operations to test
// this
Configuration conf = TEST_UTIL.getConfiguration();
- ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
+ ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
String primaryMetaZnode = ZNodePaths.joinZNode(baseZNode,
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
index 62951ef..c98a02c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -34,17 +34,15 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationPeer;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.replication.ReplicationQueues;
import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Rule;
@@ -172,7 +170,7 @@ public class TestReplicationAdmin {
ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
rpc2.setClusterKey(KEY_SECOND);
Configuration conf = TEST_UTIL.getConfiguration();
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "Test HBaseAdmin", null);
+ ZKWatcher zkw = new ZKWatcher(conf, "Test HBaseAdmin", null);
ReplicationQueues repQueues =
ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, null, zkw));
repQueues.init("server1");
@@ -187,7 +185,7 @@ public class TestReplicationAdmin {
}
repQueues.removeQueue(ID_ONE);
assertEquals(0, repQueues.getAllQueues().size());
-
+
// add recovered queue for ID_ONE
repQueues.addLog(ID_ONE + "-server2", "file1");
try {
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
index 0595a67..2a35365 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
@@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -60,10 +60,10 @@ import static org.junit.Assert.fail;
@Category({CoprocessorTests.class, MediumTests.class})
public class TestMasterCoprocessorExceptionWithAbort {
- public static class MasterTracker extends ZooKeeperNodeTracker {
+ public static class MasterTracker extends ZKNodeTracker {
public boolean masterZKNodeWasDeleted = false;
- public MasterTracker(ZooKeeperWatcher zkw, String masterNode, Abortable abortable) {
+ public MasterTracker(ZKWatcher zkw, String masterNode, Abortable abortable) {
super(zkw, masterNode, abortable);
}
@@ -174,7 +174,7 @@ public class TestMasterCoprocessorExceptionWithAbort {
// set a watch on the zookeeper /hbase/master node. If the master dies,
// the node will be deleted.
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(UTIL.getConfiguration(),
+ ZKWatcher zkw = new ZKWatcher(UTIL.getConfiguration(),
"unittest", new Abortable() {
@Override
public void abort(String why, Throwable e) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
index d4c6e4f..c4defa2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
@@ -38,8 +38,8 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -59,10 +59,10 @@ import static org.junit.Assert.fail;
@Category({CoprocessorTests.class, MediumTests.class})
public class TestMasterCoprocessorExceptionWithRemove {
- public static class MasterTracker extends ZooKeeperNodeTracker {
+ public static class MasterTracker extends ZKNodeTracker {
public boolean masterZKNodeWasDeleted = false;
- public MasterTracker(ZooKeeperWatcher zkw, String masterNode, Abortable abortable) {
+ public MasterTracker(ZKWatcher zkw, String masterNode, Abortable abortable) {
super(zkw, masterNode, abortable);
}
@@ -159,7 +159,7 @@ public class TestMasterCoprocessorExceptionWithRemove {
// we are testing that the default setting of hbase.coprocessor.abortonerror
// =false
// is respected.
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(UTIL.getConfiguration(),
+ ZKWatcher zkw = new ZKWatcher(UTIL.getConfiguration(),
"unittest", new Abortable() {
@Override
public void abort(String why, Throwable e) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 85d2b0b..dadec1f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import com.google.protobuf.Service;
@@ -150,7 +150,7 @@ public class MockNoopMasterServices implements MasterServices, Server {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return null;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 25cfbd9..8358e0a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -25,7 +25,6 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
-import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentSkipListMap;
@@ -65,7 +64,7 @@ import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
@@ -138,7 +137,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuo
class MockRegionServer implements AdminProtos.AdminService.BlockingInterface,
ClientProtos.ClientService.BlockingInterface, RegionServerServices {
private final ServerName sn;
- private final ZooKeeperWatcher zkw;
+ private final ZKWatcher zkw;
private final Configuration conf;
private final Random random = new Random();
@@ -191,7 +190,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
throws ZooKeeperConnectionException, IOException {
this.sn = sn;
this.conf = conf;
- this.zkw = new ZooKeeperWatcher(conf, sn.toString(), this, true);
+ this.zkw = new ZKWatcher(conf, sn.toString(), this, true);
}
/**
@@ -284,7 +283,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return this.zkw;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
index f1feef5..6fcd8ec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
@@ -41,9 +41,9 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@@ -70,7 +70,7 @@ public class TestActiveMasterManager {
}
@Test public void testRestartMaster() throws IOException, KeeperException {
- ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+ ZKWatcher zk = new ZKWatcher(TEST_UTIL.getConfiguration(),
"testActiveMasterManagerFromZK", null, true);
try {
ZKUtil.deleteNode(zk, zk.znodePaths.masterAddressZNode);
@@ -112,7 +112,7 @@ public class TestActiveMasterManager {
*/
@Test
public void testActiveMasterManagerFromZK() throws Exception {
- ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+ ZKWatcher zk = new ZKWatcher(TEST_UTIL.getConfiguration(),
"testActiveMasterManagerFromZK", null, true);
try {
ZKUtil.deleteNode(zk, zk.znodePaths.masterAddressZNode);
@@ -135,7 +135,7 @@ public class TestActiveMasterManager {
ClusterStatusTracker clusterStatusTracker =
ms1.getClusterStatusTracker();
clusterStatusTracker.setClusterUp();
- activeMasterManager.blockUntilBecomingActiveMaster(100,
+ activeMasterManager.blockUntilBecomingActiveMaster(100,
Mockito.mock(MonitoredTask.class));
assertTrue(activeMasterManager.clusterHasActiveMaster.get());
assertMaster(zk, firstMasterAddress);
@@ -193,9 +193,9 @@ public class TestActiveMasterManager {
* @param zk
* @param thisMasterAddress
* @throws KeeperException
- * @throws IOException
+ * @throws IOException
*/
- private void assertMaster(ZooKeeperWatcher zk,
+ private void assertMaster(ZKWatcher zk,
ServerName expectedAddress)
throws KeeperException, IOException {
ServerName readAddress = MasterAddressTracker.getMasterAddress(zk);
@@ -209,7 +209,7 @@ public class TestActiveMasterManager {
DummyMaster dummyMaster;
boolean isActiveMaster;
- public WaitToBeMasterThread(ZooKeeperWatcher zk, ServerName address) {
+ public WaitToBeMasterThread(ZKWatcher zk, ServerName address) {
this.dummyMaster = new DummyMaster(zk,address);
this.manager = this.dummyMaster.getActiveMasterManager();
isActiveMaster = false;
@@ -224,13 +224,13 @@ public class TestActiveMasterManager {
}
}
- public static class NodeDeletionListener extends ZooKeeperListener {
+ public static class NodeDeletionListener extends ZKListener {
private static final Log LOG = LogFactory.getLog(NodeDeletionListener.class);
private Semaphore lock;
private String node;
- public NodeDeletionListener(ZooKeeperWatcher watcher, String node) {
+ public NodeDeletionListener(ZKWatcher watcher, String node) {
super(watcher);
lock = new Semaphore(0);
this.node = node;
@@ -257,7 +257,7 @@ public class TestActiveMasterManager {
private ClusterStatusTracker clusterStatusTracker;
private ActiveMasterManager activeMasterManager;
- public DummyMaster(ZooKeeperWatcher zk, ServerName master) {
+ public DummyMaster(ZKWatcher zk, ServerName master) {
this.clusterStatusTracker =
new ClusterStatusTracker(zk, this);
clusterStatusTracker.start();
@@ -269,7 +269,7 @@ public class TestActiveMasterManager {
@Override
public void abort(final String msg, final Throwable t) {}
-
+
@Override
public boolean isAborted() {
return false;
@@ -281,7 +281,7 @@ public class TestActiveMasterManager {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return null;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
index d8707c3..7f8e9c9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
@@ -47,11 +47,9 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
-import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
@@ -319,7 +317,7 @@ public class TestAssignmentListener {
// are properly added to the ServerManager.drainingServers when they
// register with the ServerManager under these circumstances.
Configuration conf = TEST_UTIL.getConfiguration();
- ZooKeeperWatcher zooKeeper = new ZooKeeperWatcher(conf,
+ ZKWatcher zooKeeper = new ZKWatcher(conf,
"zkWatcher-NewServerDrainTest", abortable, true);
String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
index 852b139..bd7c507 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
@@ -27,19 +27,13 @@ import java.net.InetAddress;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.ClockOutOfSyncException;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Test;
import org.junit.experimental.categories.Category;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
index 3d59639..bc68b28 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
@@ -41,7 +41,6 @@ import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
import org.apache.commons.logging.Log;
@@ -93,8 +92,7 @@ import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@@ -210,7 +208,7 @@ public class TestDistributedLogSplitting {
Path rootdir = FSUtils.getRootDir(conf);
int numRegions = 50;
- Table t = installTable(new ZooKeeperWatcher(conf, "table-creation", null),
+ Table t = installTable(new ZKWatcher(conf, "table-creation", null),
"table", "family", numRegions);
try {
TableName table = t.getName();
@@ -283,7 +281,7 @@ public class TestDistributedLogSplitting {
// they will consume recovered.edits
master.balanceSwitch(false);
- final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+ final ZKWatcher zkw = new ZKWatcher(conf, "table-creation", null);
Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
try {
HRegionServer hrs = findRSToKill(false, "table");
@@ -352,7 +350,7 @@ public class TestDistributedLogSplitting {
final Path logDir = new Path(rootdir,
AbstractFSWALProvider.getWALDirectoryName(hrs.getServerName().toString()));
- Table t = installTable(new ZooKeeperWatcher(conf, "table-creation", null),
+ Table t = installTable(new ZKWatcher(conf, "table-creation", null),
"table", "family", 40);
try {
makeWAL(hrs, ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()),
@@ -407,7 +405,7 @@ public class TestDistributedLogSplitting {
startCluster(NUM_RS); // NUM_RS=6.
- final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
+ final ZKWatcher zkw = new ZKWatcher(conf,
"distributed log splitting test", null);
Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
@@ -520,7 +518,7 @@ public class TestDistributedLogSplitting {
public void testReadWriteSeqIdFiles() throws Exception {
LOG.info("testReadWriteSeqIdFiles");
startCluster(2);
- final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+ final ZKWatcher zkw = new ZKWatcher(conf, "table-creation", null);
Table ht = installTable(zkw, name.getMethodName(), "family", 10);
try {
FileSystem fs = master.getMasterFileSystem().getFileSystem();
@@ -551,12 +549,12 @@ public class TestDistributedLogSplitting {
}
}
- Table installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs) throws Exception {
+ Table installTable(ZKWatcher zkw, String tname, String fname, int nrs) throws Exception {
return installTable(zkw, tname, fname, nrs, 0);
}
- Table installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs,
- int existingRegions) throws Exception {
+ Table installTable(ZKWatcher zkw, String tname, String fname, int nrs,
+ int existingRegions) throws Exception {
// Create a table with regions
TableName table = TableName.valueOf(tname);
byte [] family = Bytes.toBytes(fname);
@@ -737,7 +735,7 @@ public class TestDistributedLogSplitting {
return count;
}
- private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master) throws Exception {
+ private void blockUntilNoRIT(ZKWatcher zkw, HMaster master) throws Exception {
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java
index 64d5a02..4b2c911 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.Before;
@@ -66,7 +66,7 @@ public class TestHMasterRPCException {
conf.setInt(HConstants.ZK_SESSION_TIMEOUT, 2000);
testUtil.startMiniZKCluster();
- ZooKeeperWatcher watcher = testUtil.getZooKeeperWatcher();
+ ZKWatcher watcher = testUtil.getZooKeeperWatcher();
ZKUtil.createWithParents(watcher, watcher.znodePaths.masterAddressZNode, Bytes.toBytes("fake:123"));
master = new HMaster(conf);
rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index 25671fc..20c9fe1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -53,7 +53,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
@@ -105,7 +105,7 @@ public class TestMasterNoCluster {
public void tearDown()
throws KeeperException, ZooKeeperConnectionException, IOException {
// Make sure zk is clean before we run the next test.
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(TESTUTIL.getConfiguration(),
+ ZKWatcher zkw = new ZKWatcher(TESTUTIL.getConfiguration(),
"@Before", new Abortable() {
@Override
public void abort(String why, Throwable e) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
index ea57e15..e2a2312 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
@@ -41,8 +41,8 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.tmpl.master.MasterStatusTmpl;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@@ -95,7 +95,7 @@ public class TestMasterStatusServlet {
Mockito.doReturn(serverManager).when(master).getServerManager();
// Fake ZKW
- ZooKeeperWatcher zkw = Mockito.mock(ZooKeeperWatcher.class);
+ ZKWatcher zkw = Mockito.mock(ZKWatcher.class);
Mockito.doReturn(new ZNodePaths(conf)).when(zkw).getZNodePaths();
Mockito.doReturn("fakequorum").when(zkw).getQuorum();
Mockito.doReturn(zkw).when(master).getZooKeeper();
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
index ca64326..d74b732 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
@@ -61,7 +61,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.apache.zookeeper.CreateMode;
@@ -84,7 +84,7 @@ public class TestSplitLogManager {
Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
}
- private ZooKeeperWatcher zkw;
+ private ZKWatcher zkw;
private DummyMasterServices master;
private SplitLogManager slm;
private Configuration conf;
@@ -93,17 +93,17 @@ public class TestSplitLogManager {
private static HBaseTestingUtility TEST_UTIL;
class DummyMasterServices extends MockNoopMasterServices {
- private ZooKeeperWatcher zkw;
+ private ZKWatcher zkw;
private CoordinatedStateManager cm;
- public DummyMasterServices(ZooKeeperWatcher zkw, Configuration conf) {
+ public DummyMasterServices(ZKWatcher zkw, Configuration conf) {
super(conf);
this.zkw = zkw;
cm = new ZkCoordinatedStateManager(this);
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return zkw;
}
@@ -125,7 +125,7 @@ public class TestSplitLogManager {
conf = TEST_UTIL.getConfiguration();
// Use a different ZK wrapper instance for each tests.
zkw =
- new ZooKeeperWatcher(conf, "split-log-manager-tests" + UUID.randomUUID().toString(), null);
+ new ZKWatcher(conf, "split-log-manager-tests" + UUID.randomUUID().toString(), null);
master = new DummyMasterServices(zkw, conf);
ZKUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.baseZNode);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java
index 6cf259a..1f61ee7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java
@@ -29,8 +29,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.Assert;
@@ -60,7 +60,7 @@ public class TestTableStateManager {
final TableName tableName = TableName.valueOf(name.getMethodName());
TEST_UTIL.startMiniCluster(2, 1);
TEST_UTIL.shutdownMiniHBaseCluster();
- ZooKeeperWatcher watcher = TEST_UTIL.getZooKeeperWatcher();
+ ZKWatcher watcher = TEST_UTIL.getZooKeeperWatcher();
setTableStateInZK(watcher, tableName, ZooKeeperProtos.DeprecatedTableState.State.DISABLED);
TEST_UTIL.restartHBaseCluster(1);
@@ -70,8 +70,8 @@ public class TestTableStateManager {
TableState.State.DISABLED);
}
- private void setTableStateInZK(ZooKeeperWatcher watcher, final TableName tableName,
- final ZooKeeperProtos.DeprecatedTableState.State state)
+ private void setTableStateInZK(ZKWatcher watcher, final TableName tableName,
+ final ZooKeeperProtos.DeprecatedTableState.State state)
throws KeeperException, IOException {
String znode = ZNodePaths.joinZNode(watcher.znodePaths.tableZNode, tableName.getNameAsString());
if (ZKUtil.checkExists(watcher, znode) == -1) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 572816d..97d6189 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.EnvironmentEdge;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
@@ -205,9 +205,9 @@ public class TestHFileCleaner {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
try {
- return new ZooKeeperWatcher(getConfiguration(), "dummy server", this);
+ return new ZKWatcher(getConfiguration(), "dummy server", this);
} catch (IOException e) {
e.printStackTrace();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
index 773d0fc..9820fd0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@@ -146,9 +146,9 @@ public class TestHFileLinkCleaner {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
try {
- return new ZooKeeperWatcher(getConfiguration(), "dummy server", this);
+ return new ZKWatcher(getConfiguration(), "dummy server", this);
} catch (IOException e) {
e.printStackTrace();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index b5ca894..4ccc677 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -57,7 +57,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.junit.AfterClass;
@@ -234,7 +234,7 @@ public class TestLogsCleaner {
// when zk is working both files should be returned
cleaner = new ReplicationLogCleaner();
- try (ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testZooKeeperAbort-normal", null)) {
+ try (ZKWatcher zkw = new ZKWatcher(conf, "testZooKeeperAbort-normal", null)) {
cleaner.setConf(conf, zkw);
cleaner.preClean();
Iterable<FileStatus> filesToDelete = cleaner.getDeletableFiles(dummyFiles);
@@ -255,9 +255,9 @@ public class TestLogsCleaner {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
try {
- return new ZooKeeperWatcher(getConfiguration(), "dummy server", this);
+ return new ZKWatcher(getConfiguration(), "dummy server", this);
} catch (IOException e) {
e.printStackTrace();
}
@@ -321,7 +321,7 @@ public class TestLogsCleaner {
}
}
- static class FaultyZooKeeperWatcher extends ZooKeeperWatcher {
+ static class FaultyZooKeeperWatcher extends ZKWatcher {
private RecoverableZooKeeper zk;
public FaultyZooKeeperWatcher(Configuration conf, String identifier, Abortable abortable)
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
index be7f35e..b2ae3bd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.junit.After;
@@ -240,7 +240,7 @@ public class TestReplicationHFileCleaner {
// when zk is working both files should be returned
cleaner = new ReplicationHFileCleaner();
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testZooKeeperAbort-normal", null);
+ ZKWatcher zkw = new ZKWatcher(conf, "testZooKeeperAbort-normal", null);
try {
cleaner.setConf(conf, zkw);
Iterable<FileStatus> filesToDelete = cleaner.getDeletableFiles(dummyFiles);
@@ -263,9 +263,9 @@ public class TestReplicationHFileCleaner {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
try {
- return new ZooKeeperWatcher(getConfiguration(), "dummy server", this);
+ return new ZKWatcher(getConfiguration(), "dummy server", this);
} catch (IOException e) {
e.printStackTrace();
}
@@ -332,7 +332,7 @@ public class TestReplicationHFileCleaner {
}
}
- static class FaultyZooKeeperWatcher extends ZooKeeperWatcher {
+ static class FaultyZooKeeperWatcher extends ZKWatcher {
private RecoverableZooKeeper zk;
public FaultyZooKeeperWatcher(Configuration conf, String identifier, Abortable abortable)
throws ZooKeeperConnectionException, IOException {
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
index e11143d..6aa59cb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -48,12 +48,12 @@ public class TestReplicationZKNodeCleaner {
private final String SERVER_TWO = "server2";
private final Configuration conf;
- private final ZooKeeperWatcher zkw;
+ private final ZKWatcher zkw;
private final ReplicationQueues repQueues;
public TestReplicationZKNodeCleaner() throws Exception {
conf = TEST_UTIL.getConfiguration();
- zkw = new ZooKeeperWatcher(conf, "TestReplicationZKNodeCleaner", null);
+ zkw = new ZKWatcher(conf, "TestReplicationZKNodeCleaner", null);
repQueues = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, null,
zkw));
assertTrue(repQueues instanceof ReplicationQueuesZKImpl);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
index 58efa87..7d6f80a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.DaemonThreadFactory;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.zookeeper.KeeperException;
@@ -51,7 +51,7 @@ public class SimpleRSProcedureManager extends RegionServerProcedureManager {
@Override
public void initialize(RegionServerServices rss) throws KeeperException {
this.rss = rss;
- ZooKeeperWatcher zkw = rss.getZooKeeper();
+ ZKWatcher zkw = rss.getZooKeeper();
this.memberRpcs = new ZKProcedureMemberRpcs(zkw, getProcedureSignature());
ThreadPoolExecutor pool =
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java
index 14d7eab..36ea086 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.errorhandling.TimeoutException;
import org.apache.hadoop.hbase.procedure.Subprocedure.SubprocedureImpl;
import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -85,8 +85,8 @@ public class TestZKProcedure {
UTIL.shutdownMiniZKCluster();
}
- private static ZooKeeperWatcher newZooKeeperWatcher() throws IOException {
- return new ZooKeeperWatcher(UTIL.getConfiguration(), "testing utility", new Abortable() {
+ private static ZKWatcher newZooKeeperWatcher() throws IOException {
+ return new ZKWatcher(UTIL.getConfiguration(), "testing utility", new Abortable() {
@Override
public void abort(String why, Throwable e) {
throw new RuntimeException(
@@ -123,7 +123,7 @@ public class TestZKProcedure {
List<String> expected = Arrays.asList(members);
// setup the constants
- ZooKeeperWatcher coordZkw = newZooKeeperWatcher();
+ ZKWatcher coordZkw = newZooKeeperWatcher();
String opDescription = "coordination test - " + members.length + " cohort members";
// start running the controller
@@ -144,7 +144,7 @@ public class TestZKProcedure {
List<Pair<ProcedureMember, ZKProcedureMemberRpcs>> procMembers = new ArrayList<>(members.length);
// start each member
for (String member : members) {
- ZooKeeperWatcher watcher = newZooKeeperWatcher();
+ ZKWatcher watcher = newZooKeeperWatcher();
ZKProcedureMemberRpcs comms = new ZKProcedureMemberRpcs(watcher, opDescription);
ThreadPoolExecutor pool2 = ProcedureMember.defaultPool(member, 1, KEEP_ALIVE);
ProcedureMember procMember = new ProcedureMember(comms, pool2, subprocFactory);
@@ -207,7 +207,7 @@ public class TestZKProcedure {
final CountDownLatch coordinatorReceivedErrorLatch = new CountDownLatch(1);
// start running the coordinator and its controller
- ZooKeeperWatcher coordinatorWatcher = newZooKeeperWatcher();
+ ZKWatcher coordinatorWatcher = newZooKeeperWatcher();
ZKProcedureCoordinator coordinatorController = new ZKProcedureCoordinator(
coordinatorWatcher, opDescription, COORDINATOR_NODE_NAME);
ThreadPoolExecutor pool = ProcedureCoordinator.defaultPool(COORDINATOR_NODE_NAME, POOL_SIZE, KEEP_ALIVE);
@@ -217,7 +217,7 @@ public class TestZKProcedure {
SubprocedureFactory subprocFactory = Mockito.mock(SubprocedureFactory.class);
List<Pair<ProcedureMember, ZKProcedureMemberRpcs>> members = new ArrayList<>(expected.size());
for (String member : expected) {
- ZooKeeperWatcher watcher = newZooKeeperWatcher();
+ ZKWatcher watcher = newZooKeeperWatcher();
ZKProcedureMemberRpcs controller = new ZKProcedureMemberRpcs(watcher, opDescription);
ThreadPoolExecutor pool2 = ProcedureMember.defaultPool(member, 1, KEEP_ALIVE);
ProcedureMember mem = new ProcedureMember(controller, pool2, subprocFactory);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
index 4ab9685..5d68074 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -79,7 +79,7 @@ public class TestZKProcedureControllers {
*/
@Test(timeout = 60000)
public void testSimpleZKCohortMemberController() throws Exception {
- ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
+ ZKWatcher watcher = UTIL.getZooKeeperWatcher();
final String operationName = "instanceTest";
final Subprocedure sub = Mockito.mock(Subprocedure.class);
@@ -171,7 +171,7 @@ public class TestZKProcedureControllers {
private void runMockCommitWithOrchestratedControllers(StartControllers controllers,
String operationName, byte[] data, String... cohort) throws Exception {
- ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
+ ZKWatcher watcher = UTIL.getZooKeeperWatcher();
List<String> expected = Lists.newArrayList(cohort);
final Subprocedure sub = Mockito.mock(Subprocedure.class);
@@ -247,7 +247,7 @@ public class TestZKProcedureControllers {
public void runEarlyPrepareNodes(StartControllers controllers, String operationName, byte[] data,
String... cohort) throws Exception {
- ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
+ ZKWatcher watcher = UTIL.getZooKeeperWatcher();
List<String> expected = Lists.newArrayList(cohort);
final Subprocedure sub = Mockito.mock(Subprocedure.class);
@@ -344,7 +344,7 @@ public class TestZKProcedureControllers {
/**
* Verify that the prepare, commit and abort nodes for the operation are removed from zookeeper
*/
- private void verifyZooKeeperClean(String operationName, ZooKeeperWatcher watcher,
+ private void verifyZooKeeperClean(String operationName, ZKWatcher watcher,
ZKProcedureUtil controller) throws Exception {
String prepare = ZKProcedureUtil.getAcquireBarrierNode(controller, operationName);
String commit = ZKProcedureUtil.getReachedBarrierNode(controller, operationName);
@@ -383,18 +383,18 @@ public class TestZKProcedureControllers {
*/
private abstract class StartControllers {
public abstract Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
- ZooKeeperWatcher watcher, String operationName,
- ProcedureCoordinator coordinator, String controllerName,
- ProcedureMember member, List<String> cohortNames) throws Exception;
+ ZKWatcher watcher, String operationName,
+ ProcedureCoordinator coordinator, String controllerName,
+ ProcedureMember member, List<String> cohortNames) throws Exception;
}
private final StartControllers startCoordinatorFirst = new StartControllers() {
@Override
public Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
- ZooKeeperWatcher watcher, String operationName,
- ProcedureCoordinator coordinator, String controllerName,
- ProcedureMember member, List<String> expected) throws Exception {
+ ZKWatcher watcher, String operationName,
+ ProcedureCoordinator coordinator, String controllerName,
+ ProcedureMember member, List<String> expected) throws Exception {
// start the controller
ZKProcedureCoordinator controller = new ZKProcedureCoordinator(
watcher, operationName, CONTROLLER_NODE_NAME);
@@ -420,9 +420,9 @@ public class TestZKProcedureControllers {
@Override
public Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
- ZooKeeperWatcher watcher, String operationName,
- ProcedureCoordinator coordinator, String controllerName,
- ProcedureMember member, List<String> expected) throws Exception {
+ ZKWatcher watcher, String operationName,
+ ProcedureCoordinator coordinator, String controllerName,
+ ProcedureMember member, List<String> expected) throws Exception {
// make a cohort controller for each expected node
List<ZKProcedureMemberRpcs> cohortControllers = new ArrayList<>();
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java
index 5c24c2c..9bd74d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java
@@ -25,7 +25,6 @@ import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
@@ -42,8 +41,8 @@ import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
import org.apache.hadoop.hbase.wal.FSHLogProvider;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALProvider;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -122,7 +121,7 @@ public class TestCompactionInDeadRegionServer {
public void test() throws Exception {
HRegionServer rsToSuspend = UTIL.getRSForFirstRegionInTable(TABLE_NAME);
HRegion region = (HRegion) rsToSuspend.getRegions(TABLE_NAME).get(0);
- ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
+ ZKWatcher watcher = UTIL.getZooKeeperWatcher();
watcher.getRecoverableZooKeeper().delete(
ZNodePaths.joinZNode(watcher.getZNodePaths().rsZNode, rsToSuspend.getServerName().toString()),
-1);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
index b16feca..2a3ce00 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
@@ -49,7 +49,7 @@ import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerResult;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@@ -813,7 +813,7 @@ public class TestHeapMemoryManager {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return null;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java
index 84865cc..f1b4441 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java
@@ -31,9 +31,9 @@ import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -80,7 +80,7 @@ public class TestMasterAddressTracker {
*/
private MasterAddressTracker setupMasterTracker(final ServerName sn, final int infoPort)
throws Exception {
- ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+ ZKWatcher zk = new ZKWatcher(TEST_UTIL.getConfiguration(),
name.getMethodName(), null);
ZKUtil.createAndFailSilent(zk, zk.znodePaths.baseZNode);
@@ -156,13 +156,13 @@ public class TestMasterAddressTracker {
assertEquals("Should receive 0 for backup not found.", 0, addressTracker.getMasterInfoPort());
}
- public static class NodeCreationListener extends ZooKeeperListener {
+ public static class NodeCreationListener extends ZKListener {
private static final Log LOG = LogFactory.getLog(NodeCreationListener.class);
private Semaphore lock;
private String node;
- public NodeCreationListener(ZooKeeperWatcher watcher, String node) {
+ public NodeCreationListener(ZKWatcher watcher, String node) {
super(watcher);
lock = new Semaphore(0);
this.node = node;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
index cfecd9f..c9cae7e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.tmpl.regionserver.RSStatusTmpl;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
@@ -92,7 +92,7 @@ public class TestRSStatusServlet {
Mockito.doReturn(fakeResponse).when(rpcServices).getServerInfo(
(RpcController)Mockito.any(), (GetServerInfoRequest)Mockito.any());
// Fake ZKW
- ZooKeeperWatcher zkw = Mockito.mock(ZooKeeperWatcher.class);
+ ZKWatcher zkw = Mockito.mock(ZKWatcher.class);
Mockito.doReturn("fakequorum").when(zkw).getQuorum();
Mockito.doReturn(zkw).when(rs).getZooKeeper();
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
index 872fec6..abcc497 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -99,7 +99,7 @@ public class TestRegionServerHostname {
TEST_UTIL.getConfiguration().set(HRegionServer.RS_HOSTNAME_KEY, hostName);
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
try {
- ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
+ ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
List<String> servers = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode);
// there would be NUM_RS+1 children - one for the master
assertTrue(servers.size() ==
@@ -160,7 +160,7 @@ public class TestRegionServerHostname {
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
int expectedRS = NUM_RS + (tablesOnMaster? 1: 0);
- try (ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher()) {
+ try (ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher()) {
List<String> servers = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode);
assertEquals(expectedRS, servers.size());
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
index 1f7320e..6e20612 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
@@ -106,7 +106,7 @@ public class TestRegionServerNoMaster {
// so that regions can be assigned during the mocking phase.
HRegionServer hrs = HTU.getHBaseCluster()
.getLiveRegionServerThreads().get(0).getRegionServer();
- ZooKeeperWatcher zkw = hrs.getZooKeeper();
+ ZKWatcher zkw = hrs.getZooKeeper();
MetaTableLocator mtl = new MetaTableLocator();
ServerName sn = mtl.getMetaRegionLocation(zkw);
if (sn != null && !masterAddr.equals(sn)) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
index 0bf9d03..f241f85 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
@@ -52,8 +52,8 @@ import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.apache.zookeeper.CreateMode;
@@ -74,16 +74,16 @@ public class TestSplitLogWorker {
private final static HBaseTestingUtility TEST_UTIL =
new HBaseTestingUtility();
private DummyServer ds;
- private ZooKeeperWatcher zkw;
+ private ZKWatcher zkw;
private SplitLogWorker slw;
private ExecutorService executorService;
class DummyServer implements Server {
- private ZooKeeperWatcher zkw;
+ private ZKWatcher zkw;
private Configuration conf;
private CoordinatedStateManager cm;
- public DummyServer(ZooKeeperWatcher zkw, Configuration conf) {
+ public DummyServer(ZKWatcher zkw, Configuration conf) {
this.zkw = zkw;
this.conf = conf;
cm = new ZkCoordinatedStateManager(this);
@@ -113,7 +113,7 @@ public class TestSplitLogWorker {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return zkw;
}
@@ -193,7 +193,7 @@ public class TestSplitLogWorker {
public void setup() throws Exception {
TEST_UTIL.startMiniZKCluster();
Configuration conf = TEST_UTIL.getConfiguration();
- zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+ zkw = new ZKWatcher(TEST_UTIL.getConfiguration(),
"split-log-worker-tests", null);
ds = new DummyServer(zkw, conf);
ZKUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.baseZNode);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 2e5c552..10a84b1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALProvider.Writer;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
@@ -505,7 +505,7 @@ public class TestWALLockup {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return null;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
index 6b7d36b..58b22c8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
@@ -75,8 +75,8 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.HFileTestUtil;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -203,7 +203,7 @@ public class TestMasterReplication {
Table[] htables = getHTablesOnClusters(tableName);
putAndWait(row, famName, htables[0], htables[0]);
rollWALAndWait(utilities[0], table.getTableName(), row);
- ZooKeeperWatcher zkw = utilities[0].getZooKeeperWatcher();
+ ZKWatcher zkw = utilities[0].getZooKeeperWatcher();
String queuesZnode = ZNodePaths.joinZNode(zkw.getZNodePaths().baseZNode,
ZNodePaths.joinZNode("replication", "rs"));
List<String> listChildrenNoWatch =
@@ -493,7 +493,7 @@ public class TestMasterReplication {
utility.startMiniCluster();
utilities[i] = utility;
configurations[i] = conf;
- new ZooKeeperWatcher(conf, "cluster" + i, null, true);
+ new ZKWatcher(conf, "cluster" + i, null, true);
}
}
[08/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper
from hbase-server and hbase-client
Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
index 814fe07..3347e1c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
@@ -26,6 +26,7 @@ import com.google.protobuf.ByteString;
import com.google.protobuf.ServiceException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
@@ -37,7 +38,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.Job;
@@ -289,7 +289,7 @@ public class TokenUtil {
*/
private static Token<AuthenticationTokenIdentifier> getAuthToken(Configuration conf, User user)
throws IOException, InterruptedException {
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "TokenUtil-getAuthToken", null);
+ ZKWatcher zkw = new ZKWatcher(conf, "TokenUtil-getAuthToken", null);
try {
String clusterId = ZKClusterId.readClusterIdZNode(zkw);
if (clusterId == null) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
index 9f0da78..96502fd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
@@ -25,20 +25,20 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
* Synchronizes token encryption keys across cluster nodes.
*/
@InterfaceAudience.Private
-public class ZKSecretWatcher extends ZooKeeperListener {
+public class ZKSecretWatcher extends ZKListener {
private static final String DEFAULT_ROOT_NODE = "tokenauth";
private static final String DEFAULT_KEYS_PARENT = "keys";
private static final Log LOG = LogFactory.getLog(ZKSecretWatcher.class);
@@ -48,7 +48,7 @@ public class ZKSecretWatcher extends ZooKeeperListener {
private String keysParentZNode;
public ZKSecretWatcher(Configuration conf,
- ZooKeeperWatcher watcher,
+ ZKWatcher watcher,
AuthenticationTokenSecretManager secretManager) {
super(watcher);
this.secretManager = secretManager;
@@ -214,7 +214,7 @@ public class ZKSecretWatcher extends ZooKeeperListener {
watcher.abort("Failed serializing key "+key.getKeyId(), ioe);
}
}
-
+
/**
* refresh keys
*/
@@ -228,7 +228,7 @@ public class ZKSecretWatcher extends ZooKeeperListener {
watcher.abort("Error reading changed keys from zookeeper", ke);
}
}
-
+
/**
* get token keys parent node
* @return token keys parent node
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
index e913b21..8272da6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.TagType;
import org.apache.hadoop.hbase.TagUtil;
import org.apache.hadoop.hbase.coprocessor.HasRegionServerServices;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Mutation;
@@ -67,7 +68,6 @@ import org.apache.hadoop.hbase.security.Superusers;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@InterfaceAudience.Private
public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService {
@@ -120,7 +120,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
// This is a CoreCoprocessor. On creation, we should have gotten an environment that
// implements HasRegionServerServices so we can get at RSS. FIX!!!! Integrate this CP as
// native service.
- ZooKeeperWatcher zk = ((HasRegionServerServices)e).getRegionServerServices().getZooKeeper();
+ ZKWatcher zk = ((HasRegionServerServices)e).getRegionServerServices().getZooKeeper();
try {
labelsCache = VisibilityLabelsCache.createAndGet(zk, this.conf);
} catch (IOException ioe) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java
index 2edf636..85bc0d5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.MultiUs
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations;
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -64,7 +64,7 @@ public class VisibilityLabelsCache implements VisibilityLabelOrdinalProvider {
*/
private ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
- private VisibilityLabelsCache(ZooKeeperWatcher watcher, Configuration conf) throws IOException {
+ private VisibilityLabelsCache(ZKWatcher watcher, Configuration conf) throws IOException {
zkVisibilityWatcher = new ZKVisibilityLabelWatcher(watcher, this, conf);
try {
zkVisibilityWatcher.start();
@@ -81,7 +81,7 @@ public class VisibilityLabelsCache implements VisibilityLabelOrdinalProvider {
* @return Singleton instance of VisibilityLabelsCache
* @throws IOException
*/
- public synchronized static VisibilityLabelsCache createAndGet(ZooKeeperWatcher watcher,
+ public synchronized static VisibilityLabelsCache createAndGet(ZKWatcher watcher,
Configuration conf) throws IOException {
// VisibilityLabelService#init() for different regions (in same RS) passes same instance of
// watcher as all get the instance from RS.
@@ -99,7 +99,7 @@ public class VisibilityLabelsCache implements VisibilityLabelOrdinalProvider {
* @return Singleton instance of VisibilityLabelsCache
* @throws IllegalStateException
* when this is called before calling
- * {@link #createAndGet(ZooKeeperWatcher, Configuration)}
+ * {@link #createAndGet(ZKWatcher, Configuration)}
*/
public static VisibilityLabelsCache get() {
// By the time this method is called, the singleton instance of VisibilityLabelsCache should
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java
index 5a6a414..5cc244c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java
@@ -21,12 +21,12 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -34,12 +34,12 @@ import org.apache.zookeeper.KeeperException;
* /hbase/visibility_labels and will have a serialized form of a set of labels in the system.
*/
@InterfaceAudience.Private
-public class ZKVisibilityLabelWatcher extends ZooKeeperListener {
+public class ZKVisibilityLabelWatcher extends ZKListener {
private static final Log LOG = LogFactory.getLog(ZKVisibilityLabelWatcher.class);
private static final String VISIBILITY_LABEL_ZK_PATH = "zookeeper.znode.visibility.label.parent";
private static final String DEFAULT_VISIBILITY_LABEL_NODE = "visibility/labels";
- private static final String VISIBILITY_USER_AUTHS_ZK_PATH =
+ private static final String VISIBILITY_USER_AUTHS_ZK_PATH =
"zookeeper.znode.visibility.user.auths.parent";
private static final String DEFAULT_VISIBILITY_USER_AUTHS_NODE = "visibility/user_auths";
@@ -47,8 +47,8 @@ public class ZKVisibilityLabelWatcher extends ZooKeeperListener {
private String labelZnode;
private String userAuthsZnode;
- public ZKVisibilityLabelWatcher(ZooKeeperWatcher watcher, VisibilityLabelsCache labelsCache,
- Configuration conf) {
+ public ZKVisibilityLabelWatcher(ZKWatcher watcher, VisibilityLabelsCache labelsCache,
+ Configuration conf) {
super(watcher);
this.labelsCache = labelsCache;
String labelZnodeParent = conf.get(VISIBILITY_LABEL_ZK_PATH, DEFAULT_VISIBILITY_LABEL_NODE);
@@ -132,7 +132,7 @@ public class ZKVisibilityLabelWatcher extends ZooKeeperListener {
/**
* Write a labels mirror or user auths mirror into zookeeper
- *
+ *
* @param data
* @param labelsOrUserAuths true for writing labels and false for user auths.
*/
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index bf4478d..0879a69 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -130,8 +130,8 @@ import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.UserGroupInformation;
@@ -316,7 +316,7 @@ public class HBaseFsck extends Configured implements Closeable {
private Map<TableName, Set<String>> skippedRegions = new HashMap<>();
- private ZooKeeperWatcher zkw = null;
+ private ZKWatcher zkw = null;
private String hbckEphemeralNodePath = null;
private boolean hbckZodeCreated = false;
@@ -1918,8 +1918,8 @@ public class HBaseFsck extends Configured implements Closeable {
return true;
}
- private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
- return new ZooKeeperWatcher(getConf(), "hbase Fsck", new Abortable() {
+ private ZKWatcher createZooKeeperWatcher() throws IOException {
+ return new ZKWatcher(getConf(), "hbase Fsck", new Abortable() {
@Override
public void abort(String why, Throwable e) {
LOG.error(why, e);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
index 0f36a7b..b1c1f07 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
@@ -62,7 +62,7 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
/**
@@ -900,7 +900,7 @@ public class RegionMover extends AbstractHBaseTool {
return null;
}
if (region.isMetaRegion()) {
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(admin.getConfiguration(), "region_mover", null);
+ ZKWatcher zkw = new ZKWatcher(admin.getConfiguration(), "region_mover", null);
MetaTableLocator locator = new MetaTableLocator();
int maxWaitInSeconds =
admin.getConfiguration().getInt(MOVE_WAIT_MAX_KEY, DEFAULT_MOVE_WAIT_MAX);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java
index e07861e..11327e8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java
@@ -25,14 +25,14 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
/**
@@ -52,7 +52,7 @@ public class ZKDataMigrator {
* table descriptor based states.
*/
@Deprecated
- public static Map<TableName, TableState.State> queryForTableStates(ZooKeeperWatcher zkw)
+ public static Map<TableName, TableState.State> queryForTableStates(ZKWatcher zkw)
throws KeeperException, InterruptedException {
Map<TableName, TableState.State> rv = new HashMap<>();
List<String> children = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.tableZNode);
@@ -86,14 +86,14 @@ public class ZKDataMigrator {
/**
* Gets table state from ZK.
- * @param zkw ZooKeeperWatcher instance to use
+ * @param zkw ZKWatcher instance to use
* @param tableName table we're checking
* @return Null or {@link ZooKeeperProtos.DeprecatedTableState.State} found in znode.
* @throws KeeperException
*/
@Deprecated
private static ZooKeeperProtos.DeprecatedTableState.State getTableState(
- final ZooKeeperWatcher zkw, final TableName tableName)
+ final ZKWatcher zkw, final TableName tableName)
throws KeeperException, InterruptedException {
String znode = ZNodePaths.joinZNode(zkw.znodePaths.tableZNode, tableName.getNameAsString());
byte [] data = ZKUtil.getData(zkw, znode);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
index 9fb8459..839b5ad 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
@@ -27,13 +27,13 @@ import java.util.Map.Entry;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleaner;
import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
/*
* Check and fix undeleted replication queues for removed peerId.
@@ -47,8 +47,8 @@ public class ReplicationChecker {
private Set<String> undeletedHFileRefsQueueIds = new HashSet<>();
private final ReplicationZKNodeCleaner cleaner;
- public ReplicationChecker(Configuration conf, ZooKeeperWatcher zkw, ClusterConnection connection,
- ErrorReporter errorReporter) throws IOException {
+ public ReplicationChecker(Configuration conf, ZKWatcher zkw, ClusterConnection connection,
+ ErrorReporter errorReporter) throws IOException {
this.cleaner = new ReplicationZKNodeCleaner(conf, zkw, connection);
this.errorReporter = errorReporter;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
deleted file mode 100644
index fd60765..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Tracker on cluster settings up in zookeeper.
- * This is not related to {@link org.apache.hadoop.hbase.ClusterStatus}. That class
- * is a data structure that holds snapshot of current view on cluster. This class
- * is about tracking cluster attributes up in zookeeper.
- *
- */
-@InterfaceAudience.Private
-public class ClusterStatusTracker extends ZooKeeperNodeTracker {
- private static final Log LOG = LogFactory.getLog(ClusterStatusTracker.class);
-
- /**
- * Creates a cluster status tracker.
- *
- * <p>After construction, use {@link #start} to kick off tracking.
- *
- * @param watcher
- * @param abortable
- */
- public ClusterStatusTracker(ZooKeeperWatcher watcher, Abortable abortable) {
- super(watcher, watcher.znodePaths.clusterStateZNode, abortable);
- }
-
- /**
- * Checks if cluster is up.
- * @return true if the cluster up ('shutdown' is its name up in zk) znode
- * exists with data, false if not
- */
- public boolean isClusterUp() {
- return super.getData(false) != null;
- }
-
- /**
- * Sets the cluster as up.
- * @throws KeeperException unexpected zk exception
- */
- public void setClusterUp()
- throws KeeperException {
- byte [] upData = toByteArray();
- try {
- ZKUtil.createAndWatch(watcher, watcher.znodePaths.clusterStateZNode, upData);
- } catch(KeeperException.NodeExistsException nee) {
- ZKUtil.setData(watcher, watcher.znodePaths.clusterStateZNode, upData);
- }
- }
-
- /**
- * Sets the cluster as down by deleting the znode.
- * @throws KeeperException unexpected zk exception
- */
- public void setClusterDown()
- throws KeeperException {
- try {
- ZKUtil.deleteNode(watcher, watcher.znodePaths.clusterStateZNode);
- } catch(KeeperException.NoNodeException nne) {
- LOG.warn("Attempted to set cluster as down but already down, cluster " +
- "state node (" + watcher.znodePaths.clusterStateZNode + ") not found");
- }
- }
-
- /**
- * @return Content of the clusterup znode as a serialized pb with the pb
- * magic as prefix.
- */
- static byte [] toByteArray() {
- ZooKeeperProtos.ClusterUp.Builder builder =
- ZooKeeperProtos.ClusterUp.newBuilder();
- builder.setStartDate(new java.util.Date().toString());
- return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java
deleted file mode 100644
index c58a840..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/**
- * Copyright The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.util.concurrent.CountDownLatch;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * A ZooKeeper watcher meant to detect deletions of ZNodes.
- */
-@InterfaceAudience.Private
-public class DeletionListener extends ZooKeeperListener {
-
- private static final Log LOG = LogFactory.getLog(DeletionListener.class);
-
- private final String pathToWatch;
- private final CountDownLatch deletedLatch;
-
- private volatile Throwable exception;
-
- /**
- * Create a new instance of the deletion watcher.
- * @param zkWatcher ZookeeperWatcher instance
- * @param pathToWatch (Fully qualified) ZNode path that we are waiting to
- * be deleted.
- * @param deletedLatch Count down on this latch when deletion has occurred.
- */
- public DeletionListener(ZooKeeperWatcher zkWatcher, String pathToWatch,
- CountDownLatch deletedLatch) {
- super(zkWatcher);
- this.pathToWatch = pathToWatch;
- this.deletedLatch = deletedLatch;
- exception = null;
- }
-
- /**
- * Check if an exception has occurred when re-setting the watch.
- * @return True if we were unable to re-set a watch on a ZNode due to
- * an exception.
- */
- public boolean hasException() {
- return exception != null;
- }
-
- /**
- * Get the last exception which has occurred when re-setting the watch.
- * Use hasException() to check whether or not an exception has occurred.
- * @return The last exception observed when re-setting the watch.
- */
- public Throwable getException() {
- return exception;
- }
-
- @Override
- public void nodeDataChanged(String path) {
- if (!path.equals(pathToWatch)) {
- return;
- }
- try {
- if (!(ZKUtil.setWatchIfNodeExists(watcher, pathToWatch))) {
- deletedLatch.countDown();
- }
- } catch (KeeperException ex) {
- exception = ex;
- deletedLatch.countDown();
- LOG.error("Error when re-setting the watch on " + pathToWatch, ex);
- }
- }
-
- @Override
- public void nodeDeleted(String path) {
- if (!path.equals(pathToWatch)) {
- return;
- }
- if (LOG.isDebugEnabled()) {
- LOG.debug("Processing delete on " + pathToWatch);
- }
- deletedLatch.countDown();
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
deleted file mode 100644
index cdc6794..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.NavigableSet;
-import java.util.TreeSet;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.master.ServerListener;
-import org.apache.hadoop.hbase.master.ServerManager;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Tracks the list of draining region servers via ZK.
- *
- * <p>This class is responsible for watching for changes to the draining
- * servers list. It handles adds/deletes in the draining RS list and
- * watches each node.
- *
- * <p>If an RS gets deleted from draining list, we call
- * {@link ServerManager#removeServerFromDrainList(ServerName)}
- *
- * <p>If an RS gets added to the draining list, we add a watcher to it and call
- * {@link ServerManager#addServerToDrainList(ServerName)}
- *
- * <p>This class is deprecated in 2.0 because decommission/draining API goes through
- * master in 2.0. Can remove this class in 3.0.
- *
- */
-@InterfaceAudience.Private
-public class DrainingServerTracker extends ZooKeeperListener {
- private static final Log LOG = LogFactory.getLog(DrainingServerTracker.class);
-
- private ServerManager serverManager;
- private final NavigableSet<ServerName> drainingServers = new TreeSet<>();
- private Abortable abortable;
-
- public DrainingServerTracker(ZooKeeperWatcher watcher,
- Abortable abortable, ServerManager serverManager) {
- super(watcher);
- this.abortable = abortable;
- this.serverManager = serverManager;
- }
-
- /**
- * Starts the tracking of draining RegionServers.
- *
- * <p>All Draining RSs will be tracked after this method is called.
- *
- * @throws KeeperException
- */
- public void start() throws KeeperException, IOException {
- watcher.registerListener(this);
- // Add a ServerListener to check if a server is draining when it's added.
- serverManager.registerListener(new ServerListener() {
- @Override
- public void serverAdded(ServerName sn) {
- if (drainingServers.contains(sn)){
- serverManager.addServerToDrainList(sn);
- }
- }
- });
- List<String> servers =
- ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.drainingZNode);
- add(servers);
- }
-
- private void add(final List<String> servers) throws IOException {
- synchronized(this.drainingServers) {
- this.drainingServers.clear();
- for (String n: servers) {
- final ServerName sn = ServerName.valueOf(ZKUtil.getNodeName(n));
- this.drainingServers.add(sn);
- this.serverManager.addServerToDrainList(sn);
- LOG.info("Draining RS node created, adding to list [" +
- sn + "]");
-
- }
- }
- }
-
- private void remove(final ServerName sn) {
- synchronized(this.drainingServers) {
- this.drainingServers.remove(sn);
- this.serverManager.removeServerFromDrainList(sn);
- }
- }
-
- @Override
- public void nodeDeleted(final String path) {
- if(path.startsWith(watcher.znodePaths.drainingZNode)) {
- final ServerName sn = ServerName.valueOf(ZKUtil.getNodeName(path));
- LOG.info("Draining RS node deleted, removing from list [" +
- sn + "]");
- remove(sn);
- }
- }
-
- @Override
- public void nodeChildrenChanged(final String path) {
- if(path.equals(watcher.znodePaths.drainingZNode)) {
- try {
- final List<String> newNodes =
- ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.drainingZNode);
- add(newNodes);
- } catch (KeeperException e) {
- abortable.abort("Unexpected zk exception getting RS nodes", e);
- } catch (IOException e) {
- abortable.abort("Unexpected zk exception getting RS nodes", e);
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java
deleted file mode 100644
index 527bc17..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-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
- */
-@InterfaceAudience.Private
-public class LoadBalancerTracker extends ZooKeeperNodeTracker {
- private static final Log LOG = LogFactory.getLog(LoadBalancerTracker.class);
-
- public LoadBalancerTracker(ZooKeeperWatcher watcher,
- Abortable abortable) {
- super(watcher, watcher.znodePaths.balancerZNode, abortable);
- }
-
- /**
- * Return true if the balance switch is on, false otherwise
- */
- public boolean isBalancerOn() {
- byte [] upData = super.getData(false);
- try {
- // if data in ZK is null, use default of on.
- return upData == null || parseFrom(upData).getBalancerOn();
- } catch (DeserializationException dex) {
- LOG.error("ZK state for LoadBalancer could not be parsed " + Bytes.toStringBinary(upData));
- // return false to be safe.
- return false;
- }
- }
-
- /**
- * Set the balancer on/off
- * @param balancerOn
- * @throws KeeperException
- */
- public void setBalancerOn(boolean balancerOn) throws KeeperException {
- byte [] upData = toByteArray(balancerOn);
- try {
- ZKUtil.setData(watcher, watcher.znodePaths.balancerZNode, upData);
- } catch(KeeperException.NoNodeException nne) {
- ZKUtil.createAndWatch(watcher, watcher.znodePaths.balancerZNode, upData);
- }
- super.nodeDataChanged(watcher.znodePaths.balancerZNode);
- }
-
- private byte [] toByteArray(boolean isBalancerOn) {
- LoadBalancerProtos.LoadBalancerState.Builder builder =
- LoadBalancerProtos.LoadBalancerState.newBuilder();
- builder.setBalancerOn(isBalancerOn);
- return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
- }
-
- private LoadBalancerProtos.LoadBalancerState parseFrom(byte [] pbBytes)
- throws DeserializationException {
- ProtobufUtil.expectPBMagicPrefix(pbBytes);
- LoadBalancerProtos.LoadBalancerState.Builder builder =
- LoadBalancerProtos.LoadBalancerState.newBuilder();
- try {
- int magicLen = ProtobufUtil.lengthOfPBMagic();
- ProtobufUtil.mergeFrom(builder, pbBytes, magicLen, pbBytes.length - magicLen);
- } catch (IOException e) {
- throw new DeserializationException(e);
- }
- return builder.build();
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java
deleted file mode 100644
index 7d6f9fd..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.util.List;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Tracks the master Maintenance Mode via ZK.
- */
-@InterfaceAudience.Private
-public class MasterMaintenanceModeTracker extends ZooKeeperListener {
- private boolean hasChildren;
-
- public MasterMaintenanceModeTracker(ZooKeeperWatcher watcher) {
- super(watcher);
- hasChildren = false;
- }
-
- public boolean isInMaintenanceMode() {
- return hasChildren;
- }
-
- private void update(String path) {
- if (path.startsWith(watcher.znodePaths.masterMaintZNode)) {
- update();
- }
- }
-
- private void update() {
- try {
- List<String> children =
- ZKUtil.listChildrenAndWatchForNewChildren(watcher, watcher.znodePaths.masterMaintZNode);
- hasChildren = (children != null && children.size() > 0);
- } catch (KeeperException e) {
- // Ignore the ZK keeper exception
- hasChildren = false;
- }
- }
-
- /**
- * Starts the tracking of whether master is in Maintenance Mode.
- */
- public void start() {
- watcher.registerListener(this);
- update();
- }
-
- @Override
- public void nodeCreated(String path) {
- update(path);
- }
-
- @Override
- public void nodeDeleted(String path) {
- update(path);
- }
-
- @Override
- public void nodeChildrenChanged(String path) {
- update(path);
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
deleted file mode 100644
index ef643bf..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
+++ /dev/null
@@ -1,472 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.InterruptedIOException;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.Reader;
-import java.net.BindException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.zookeeper.server.NIOServerCnxnFactory;
-import org.apache.zookeeper.server.ZooKeeperServer;
-import org.apache.zookeeper.server.persistence.FileTxnLog;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
-/**
- * TODO: Most of the code in this class is ripped from ZooKeeper tests. Instead
- * of redoing it, we should contribute updates to their code which let us more
- * easily access testing helper objects.
- */
-@InterfaceAudience.Public
-public class MiniZooKeeperCluster {
- private static final Log LOG = LogFactory.getLog(MiniZooKeeperCluster.class);
-
- private static final int TICK_TIME = 2000;
- private static final int DEFAULT_CONNECTION_TIMEOUT = 30000;
- private int connectionTimeout;
-
- private boolean started;
-
- /** The default port. If zero, we use a random port. */
- private int defaultClientPort = 0;
-
- private List<NIOServerCnxnFactory> standaloneServerFactoryList;
- private List<ZooKeeperServer> zooKeeperServers;
- private List<Integer> clientPortList;
-
- private int activeZKServerIndex;
- private int tickTime = 0;
-
- private Configuration configuration;
-
- public MiniZooKeeperCluster() {
- this(new Configuration());
- }
-
- public MiniZooKeeperCluster(Configuration configuration) {
- this.started = false;
- this.configuration = configuration;
- activeZKServerIndex = -1;
- zooKeeperServers = new ArrayList<>();
- clientPortList = new ArrayList<>();
- standaloneServerFactoryList = new ArrayList<>();
- connectionTimeout = configuration.getInt(HConstants.ZK_SESSION_TIMEOUT + ".localHBaseCluster",
- DEFAULT_CONNECTION_TIMEOUT);
- }
-
- /**
- * Add a client port to the list.
- *
- * @param clientPort the specified port
- */
- public void addClientPort(int clientPort) {
- clientPortList.add(clientPort);
- }
-
- /**
- * Get the list of client ports.
- * @return clientPortList the client port list
- */
- @VisibleForTesting
- public List<Integer> getClientPortList() {
- return clientPortList;
- }
-
- /**
- * Check whether the client port in a specific position of the client port list is valid.
- *
- * @param index the specified position
- */
- private boolean hasValidClientPortInList(int index) {
- return (clientPortList.size() > index && clientPortList.get(index) > 0);
- }
-
- public void setDefaultClientPort(int clientPort) {
- if (clientPort <= 0) {
- throw new IllegalArgumentException("Invalid default ZK client port: "
- + clientPort);
- }
- this.defaultClientPort = clientPort;
- }
-
- /**
- * Selects a ZK client port.
- *
- * @param seedPort the seed port to start with; -1 means first time.
- * @Returns a valid and unused client port
- */
- private int selectClientPort(int seedPort) {
- int i;
- int returnClientPort = seedPort + 1;
- if (returnClientPort == 0) {
- // If the new port is invalid, find one - starting with the default client port.
- // If the default client port is not specified, starting with a random port.
- // The random port is selected from the range between 49152 to 65535. These ports cannot be
- // registered with IANA and are intended for dynamic allocation (see http://bit.ly/dynports).
- if (defaultClientPort > 0) {
- returnClientPort = defaultClientPort;
- } else {
- returnClientPort = 0xc000 + new Random().nextInt(0x3f00);
- }
- }
- // Make sure that the port is unused.
- while (true) {
- for (i = 0; i < clientPortList.size(); i++) {
- if (returnClientPort == clientPortList.get(i)) {
- // Already used. Update the port and retry.
- returnClientPort++;
- break;
- }
- }
- if (i == clientPortList.size()) {
- break; // found a unused port, exit
- }
- }
- return returnClientPort;
- }
-
- public void setTickTime(int tickTime) {
- this.tickTime = tickTime;
- }
-
- public int getBackupZooKeeperServerNum() {
- return zooKeeperServers.size()-1;
- }
-
- public int getZooKeeperServerNum() {
- return zooKeeperServers.size();
- }
-
- // / XXX: From o.a.zk.t.ClientBase
- private static void setupTestEnv() {
- // during the tests we run with 100K prealloc in the logs.
- // on windows systems prealloc of 64M was seen to take ~15seconds
- // resulting in test failure (client timeout on first session).
- // set env and directly in order to handle static init/gc issues
- System.setProperty("zookeeper.preAllocSize", "100");
- FileTxnLog.setPreallocSize(100 * 1024);
- }
-
- public int startup(File baseDir) throws IOException, InterruptedException {
- int numZooKeeperServers = clientPortList.size();
- if (numZooKeeperServers == 0) {
- numZooKeeperServers = 1; // need at least 1 ZK server for testing
- }
- return startup(baseDir, numZooKeeperServers);
- }
-
- /**
- * @param baseDir
- * @param numZooKeeperServers
- * @return ClientPort server bound to, -1 if there was a
- * binding problem and we couldn't pick another port.
- * @throws IOException
- * @throws InterruptedException
- */
- public int startup(File baseDir, int numZooKeeperServers) throws IOException,
- InterruptedException {
- if (numZooKeeperServers <= 0)
- return -1;
-
- setupTestEnv();
- shutdown();
-
- int tentativePort = -1; // the seed port
- int currentClientPort;
-
- // running all the ZK servers
- for (int i = 0; i < numZooKeeperServers; i++) {
- File dir = new File(baseDir, "zookeeper_"+i).getAbsoluteFile();
- createDir(dir);
- int tickTimeToUse;
- if (this.tickTime > 0) {
- tickTimeToUse = this.tickTime;
- } else {
- tickTimeToUse = TICK_TIME;
- }
-
- // Set up client port - if we have already had a list of valid ports, use it.
- if (hasValidClientPortInList(i)) {
- currentClientPort = clientPortList.get(i);
- } else {
- tentativePort = selectClientPort(tentativePort); // update the seed
- currentClientPort = tentativePort;
- }
-
- ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse);
- // Setting {min,max}SessionTimeout defaults to be the same as in Zookeeper
- server.setMinSessionTimeout(configuration.getInt("hbase.zookeeper.property.minSessionTimeout", -1));
- server.setMaxSessionTimeout(configuration.getInt("hbase.zookeeper.property.maxSessionTimeout", -1));
- NIOServerCnxnFactory standaloneServerFactory;
- while (true) {
- try {
- standaloneServerFactory = new NIOServerCnxnFactory();
- standaloneServerFactory.configure(
- new InetSocketAddress(currentClientPort),
- configuration.getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS, HConstants.DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS));
- } catch (BindException e) {
- LOG.debug("Failed binding ZK Server to client port: " +
- currentClientPort, e);
- // We're told to use some port but it's occupied, fail
- if (hasValidClientPortInList(i)) {
- return -1;
- }
- // This port is already in use, try to use another.
- tentativePort = selectClientPort(tentativePort);
- currentClientPort = tentativePort;
- continue;
- }
- break;
- }
-
- // Start up this ZK server
- standaloneServerFactory.startup(server);
- // Runs a 'stat' against the servers.
- if (!waitForServerUp(currentClientPort, connectionTimeout)) {
- throw new IOException("Waiting for startup of standalone server");
- }
-
- // We have selected a port as a client port. Update clientPortList if necessary.
- if (clientPortList.size() <= i) { // it is not in the list, add the port
- clientPortList.add(currentClientPort);
- }
- else if (clientPortList.get(i) <= 0) { // the list has invalid port, update with valid port
- clientPortList.remove(i);
- clientPortList.add(i, currentClientPort);
- }
-
- standaloneServerFactoryList.add(standaloneServerFactory);
- zooKeeperServers.add(server);
- }
-
- // set the first one to be active ZK; Others are backups
- activeZKServerIndex = 0;
- started = true;
- int clientPort = clientPortList.get(activeZKServerIndex);
- LOG.info("Started MiniZooKeeperCluster and ran successful 'stat' " +
- "on client port=" + clientPort);
- return clientPort;
- }
-
- private void createDir(File dir) throws IOException {
- try {
- if (!dir.exists()) {
- dir.mkdirs();
- }
- } catch (SecurityException e) {
- throw new IOException("creating dir: " + dir, e);
- }
- }
-
- /**
- * @throws IOException
- */
- public void shutdown() throws IOException {
- // shut down all the zk servers
- for (int i = 0; i < standaloneServerFactoryList.size(); i++) {
- NIOServerCnxnFactory standaloneServerFactory =
- standaloneServerFactoryList.get(i);
- int clientPort = clientPortList.get(i);
-
- standaloneServerFactory.shutdown();
- if (!waitForServerDown(clientPort, connectionTimeout)) {
- throw new IOException("Waiting for shutdown of standalone server");
- }
- }
- standaloneServerFactoryList.clear();
-
- for (ZooKeeperServer zkServer: zooKeeperServers) {
- //explicitly close ZKDatabase since ZookeeperServer does not close them
- zkServer.getZKDatabase().close();
- }
- zooKeeperServers.clear();
-
- // clear everything
- if (started) {
- started = false;
- activeZKServerIndex = 0;
- clientPortList.clear();
- LOG.info("Shutdown MiniZK cluster with all ZK servers");
- }
- }
-
- /**@return clientPort return clientPort if there is another ZK backup can run
- * when killing the current active; return -1, if there is no backups.
- * @throws IOException
- * @throws InterruptedException
- */
- public int killCurrentActiveZooKeeperServer() throws IOException,
- InterruptedException {
- if (!started || activeZKServerIndex < 0) {
- return -1;
- }
-
- // Shutdown the current active one
- NIOServerCnxnFactory standaloneServerFactory =
- standaloneServerFactoryList.get(activeZKServerIndex);
- int clientPort = clientPortList.get(activeZKServerIndex);
-
- standaloneServerFactory.shutdown();
- if (!waitForServerDown(clientPort, connectionTimeout)) {
- throw new IOException("Waiting for shutdown of standalone server");
- }
-
- zooKeeperServers.get(activeZKServerIndex).getZKDatabase().close();
-
- // remove the current active zk server
- standaloneServerFactoryList.remove(activeZKServerIndex);
- clientPortList.remove(activeZKServerIndex);
- zooKeeperServers.remove(activeZKServerIndex);
- LOG.info("Kill the current active ZK servers in the cluster " +
- "on client port: " + clientPort);
-
- if (standaloneServerFactoryList.isEmpty()) {
- // there is no backup servers;
- return -1;
- }
- clientPort = clientPortList.get(activeZKServerIndex);
- LOG.info("Activate a backup zk server in the cluster " +
- "on client port: " + clientPort);
- // return the next back zk server's port
- return clientPort;
- }
-
- /**
- * Kill one back up ZK servers
- * @throws IOException
- * @throws InterruptedException
- */
- public void killOneBackupZooKeeperServer() throws IOException,
- InterruptedException {
- if (!started || activeZKServerIndex < 0 ||
- standaloneServerFactoryList.size() <= 1) {
- return ;
- }
-
- int backupZKServerIndex = activeZKServerIndex+1;
- // Shutdown the current active one
- NIOServerCnxnFactory standaloneServerFactory =
- standaloneServerFactoryList.get(backupZKServerIndex);
- int clientPort = clientPortList.get(backupZKServerIndex);
-
- standaloneServerFactory.shutdown();
- if (!waitForServerDown(clientPort, connectionTimeout)) {
- throw new IOException("Waiting for shutdown of standalone server");
- }
-
- zooKeeperServers.get(backupZKServerIndex).getZKDatabase().close();
-
- // remove this backup zk server
- standaloneServerFactoryList.remove(backupZKServerIndex);
- clientPortList.remove(backupZKServerIndex);
- zooKeeperServers.remove(backupZKServerIndex);
- LOG.info("Kill one backup ZK servers in the cluster " +
- "on client port: " + clientPort);
- }
-
- // XXX: From o.a.zk.t.ClientBase
- private static boolean waitForServerDown(int port, long timeout) throws IOException {
- long start = System.currentTimeMillis();
- while (true) {
- try {
- Socket sock = new Socket("localhost", port);
- try {
- OutputStream outstream = sock.getOutputStream();
- outstream.write("stat".getBytes());
- outstream.flush();
- } finally {
- sock.close();
- }
- } catch (IOException e) {
- return true;
- }
-
- if (System.currentTimeMillis() > start + timeout) {
- break;
- }
- try {
- Thread.sleep(250);
- } catch (InterruptedException e) {
- throw (InterruptedIOException)new InterruptedIOException().initCause(e);
- }
- }
- return false;
- }
-
- // XXX: From o.a.zk.t.ClientBase
- private static boolean waitForServerUp(int port, long timeout) throws IOException {
- long start = System.currentTimeMillis();
- while (true) {
- try {
- Socket sock = new Socket("localhost", port);
- BufferedReader reader = null;
- try {
- OutputStream outstream = sock.getOutputStream();
- outstream.write("stat".getBytes());
- outstream.flush();
-
- Reader isr = new InputStreamReader(sock.getInputStream());
- reader = new BufferedReader(isr);
- String line = reader.readLine();
- if (line != null && line.startsWith("Zookeeper version:")) {
- return true;
- }
- } finally {
- sock.close();
- if (reader != null) {
- reader.close();
- }
- }
- } catch (IOException e) {
- // ignore as this is expected
- LOG.info("server localhost:" + port + " not up " + e);
- }
-
- if (System.currentTimeMillis() > start + timeout) {
- break;
- }
- try {
- Thread.sleep(250);
- } catch (InterruptedException e) {
- throw (InterruptedIOException)new InterruptedIOException().initCause(e);
- }
- }
- return false;
- }
-
- public int getClientPort() {
- return activeZKServerIndex < 0 || activeZKServerIndex >= clientPortList.size() ? -1
- : clientPortList.get(activeZKServerIndex);
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java
deleted file mode 100644
index a993934..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.zookeeper;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.Abortable;
-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.zookeeper.KeeperException;
-
-import java.io.IOException;
-
-/**
- * Tracks region normalizer state up in ZK
- */
-public class RegionNormalizerTracker extends ZooKeeperNodeTracker {
- private static final Log LOG = LogFactory.getLog(RegionNormalizerTracker.class);
-
- public RegionNormalizerTracker(ZooKeeperWatcher watcher,
- Abortable abortable) {
- super(watcher, watcher.znodePaths.regionNormalizerZNode, abortable);
- }
-
- /**
- * Return true if region normalizer is on, false otherwise
- */
- public boolean isNormalizerOn() {
- byte [] upData = super.getData(false);
- try {
- // if data in ZK is null, use default of on.
- return upData == null || parseFrom(upData).getNormalizerOn();
- } catch (DeserializationException dex) {
- LOG.error("ZK state for RegionNormalizer could not be parsed "
- + Bytes.toStringBinary(upData));
- // return false to be safe.
- return false;
- }
- }
-
- /**
- * Set region normalizer on/off
- * @param normalizerOn whether normalizer should be on or off
- * @throws KeeperException
- */
- public void setNormalizerOn(boolean normalizerOn) throws KeeperException {
- byte [] upData = toByteArray(normalizerOn);
- try {
- ZKUtil.setData(watcher, watcher.znodePaths.regionNormalizerZNode, upData);
- } catch(KeeperException.NoNodeException nne) {
- ZKUtil.createAndWatch(watcher, watcher.znodePaths.regionNormalizerZNode, upData);
- }
- super.nodeDataChanged(watcher.znodePaths.regionNormalizerZNode);
- }
-
- private byte [] toByteArray(boolean isNormalizerOn) {
- RegionNormalizerProtos.RegionNormalizerState.Builder builder =
- RegionNormalizerProtos.RegionNormalizerState.newBuilder();
- builder.setNormalizerOn(isNormalizerOn);
- return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
- }
-
- private RegionNormalizerProtos.RegionNormalizerState parseFrom(byte [] pbBytes)
- throws DeserializationException {
- ProtobufUtil.expectPBMagicPrefix(pbBytes);
- RegionNormalizerProtos.RegionNormalizerState.Builder builder =
- RegionNormalizerProtos.RegionNormalizerState.newBuilder();
- try {
- int magicLen = ProtobufUtil.lengthOfPBMagic();
- ProtobufUtil.mergeFrom(builder, pbBytes, magicLen, pbBytes.length - magicLen);
- } catch (IOException e) {
- throw new DeserializationException(e);
- }
- return builder.build();
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
deleted file mode 100644
index c22876a..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.ServerManager;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Tracks the online region servers via ZK.
- *
- * <p>Handling of new RSs checking in is done via RPC. This class
- * is only responsible for watching for expired nodes. It handles
- * listening for changes in the RS node list and watching each node.
- *
- * <p>If an RS node gets deleted, this automatically handles calling of
- * {@link ServerManager#expireServer(ServerName)}
- */
-@InterfaceAudience.Private
-public class RegionServerTracker extends ZooKeeperListener {
- private static final Log LOG = LogFactory.getLog(RegionServerTracker.class);
- private NavigableMap<ServerName, RegionServerInfo> regionServers = new TreeMap<>();
- private ServerManager serverManager;
- private MasterServices server;
-
- public RegionServerTracker(ZooKeeperWatcher watcher,
- MasterServices server, ServerManager serverManager) {
- super(watcher);
- this.server = server;
- this.serverManager = serverManager;
- }
-
- /**
- * Starts the tracking of online RegionServers.
- *
- * <p>All RSs will be tracked after this method is called.
- *
- * @throws KeeperException
- * @throws IOException
- */
- public void start() throws KeeperException, IOException {
- watcher.registerListener(this);
- List<String> servers =
- ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.rsZNode);
- refresh(servers);
- }
-
- private void refresh(final List<String> servers) throws IOException {
- synchronized(this.regionServers) {
- this.regionServers.clear();
- for (String n: servers) {
- ServerName sn = ServerName.parseServerName(ZKUtil.getNodeName(n));
- if (regionServers.get(sn) == null) {
- RegionServerInfo.Builder rsInfoBuilder = RegionServerInfo.newBuilder();
- try {
- String nodePath = ZNodePaths.joinZNode(watcher.znodePaths.rsZNode, n);
- byte[] data = ZKUtil.getData(watcher, nodePath);
- if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) {
- int magicLen = ProtobufUtil.lengthOfPBMagic();
- ProtobufUtil.mergeFrom(rsInfoBuilder, data, magicLen, data.length - magicLen);
- }
- if (LOG.isTraceEnabled()) {
- LOG.trace("Added tracking of RS " + nodePath);
- }
- } catch (KeeperException e) {
- LOG.warn("Get Rs info port from ephemeral node", e);
- } catch (IOException e) {
- LOG.warn("Illegal data from ephemeral node", e);
- } catch (InterruptedException e) {
- throw new InterruptedIOException();
- }
- this.regionServers.put(sn, rsInfoBuilder.build());
- }
- }
- }
- if (server.isInitialized()) {
- server.checkIfShouldMoveSystemRegionAsync();
- }
- }
-
- private void remove(final ServerName sn) {
- synchronized(this.regionServers) {
- this.regionServers.remove(sn);
- }
- }
-
- @Override
- public void nodeDeleted(String path) {
- if (path.startsWith(watcher.znodePaths.rsZNode)) {
- String serverName = ZKUtil.getNodeName(path);
- LOG.info("RegionServer ephemeral node deleted, processing expiration [" +
- serverName + "]");
- ServerName sn = ServerName.parseServerName(serverName);
- if (!serverManager.isServerOnline(sn)) {
- LOG.warn(serverName.toString() + " is not online or isn't known to the master."+
- "The latter could be caused by a DNS misconfiguration.");
- return;
- }
- remove(sn);
- this.serverManager.expireServer(sn);
- }
- }
-
- @Override
- public void nodeChildrenChanged(String path) {
- if (path.equals(watcher.znodePaths.rsZNode)
- && !server.isAborted() && !server.isStopped()) {
- try {
- List<String> servers =
- ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.rsZNode);
- refresh(servers);
- } catch (IOException e) {
- server.abort("Unexpected zk exception getting RS nodes", e);
- } catch (KeeperException e) {
- server.abort("Unexpected zk exception getting RS nodes", e);
- }
- }
- }
-
- public RegionServerInfo getRegionServerInfo(final ServerName sn) {
- return regionServers.get(sn);
- }
-
- /**
- * Gets the online servers.
- * @return list of online servers
- */
- public List<ServerName> getOnlineServers() {
- synchronized (this.regionServers) {
- return new ArrayList<>(this.regionServers.keySet());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java
deleted file mode 100644
index 2c5ffba..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.MasterSwitchType;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
-
-
-/**
- * Tracks the switch of split and merge states in ZK
- *
- */
-@InterfaceAudience.Private
-public class SplitOrMergeTracker {
-
- private String splitZnode;
- private String mergeZnode;
-
- private SwitchStateTracker splitStateTracker;
- private SwitchStateTracker mergeStateTracker;
-
- public SplitOrMergeTracker(ZooKeeperWatcher watcher, Configuration conf,
- Abortable abortable) {
- try {
- if (ZKUtil.checkExists(watcher, watcher.znodePaths.switchZNode) < 0) {
- ZKUtil.createAndFailSilent(watcher, watcher.znodePaths.switchZNode);
- }
- } catch (KeeperException e) {
- throw new RuntimeException(e);
- }
- splitZnode = ZNodePaths.joinZNode(watcher.znodePaths.switchZNode,
- conf.get("zookeeper.znode.switch.split", "split"));
- mergeZnode = ZNodePaths.joinZNode(watcher.znodePaths.switchZNode,
- conf.get("zookeeper.znode.switch.merge", "merge"));
- splitStateTracker = new SwitchStateTracker(watcher, splitZnode, abortable);
- mergeStateTracker = new SwitchStateTracker(watcher, mergeZnode, abortable);
- }
-
- public void start() {
- splitStateTracker.start();
- mergeStateTracker.start();
- }
-
- public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) {
- switch (switchType) {
- case SPLIT:
- return splitStateTracker.isSwitchEnabled();
- case MERGE:
- return mergeStateTracker.isSwitchEnabled();
- default:
- break;
- }
- return false;
- }
-
- public void setSplitOrMergeEnabled(boolean enabled, MasterSwitchType switchType)
- throws KeeperException {
- switch (switchType) {
- case SPLIT:
- splitStateTracker.setSwitchEnabled(enabled);
- break;
- case MERGE:
- mergeStateTracker.setSwitchEnabled(enabled);
- break;
- default:
- break;
- }
- }
-
- private static class SwitchStateTracker extends ZooKeeperNodeTracker {
-
- public SwitchStateTracker(ZooKeeperWatcher watcher, String node, Abortable abortable) {
- super(watcher, node, abortable);
- }
-
- /**
- * Return true if the switch is on, false otherwise
- */
- public boolean isSwitchEnabled() {
- byte [] upData = super.getData(false);
- try {
- // if data in ZK is null, use default of on.
- return upData == null || parseFrom(upData).getEnabled();
- } catch (DeserializationException dex) {
- LOG.error("ZK state for LoadBalancer could not be parsed " + Bytes.toStringBinary(upData));
- // return false to be safe.
- return false;
- }
- }
-
- /**
- * Set the switch on/off
- * @param enabled switch enabled or not?
- * @throws KeeperException keepException will be thrown out
- */
- public void setSwitchEnabled(boolean enabled) throws KeeperException {
- byte [] upData = toByteArray(enabled);
- try {
- ZKUtil.setData(watcher, node, upData);
- } catch(KeeperException.NoNodeException nne) {
- ZKUtil.createAndWatch(watcher, node, upData);
- }
- super.nodeDataChanged(node);
- }
-
- private byte [] toByteArray(boolean enabled) {
- SwitchState.Builder builder = SwitchState.newBuilder();
- builder.setEnabled(enabled);
- return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
- }
-
- private SwitchState parseFrom(byte [] bytes)
- throws DeserializationException {
- ProtobufUtil.expectPBMagicPrefix(bytes);
- SwitchState.Builder builder = SwitchState.newBuilder();
- try {
- int magicLen = ProtobufUtil.lengthOfPBMagic();
- ProtobufUtil.mergeFrom(builder, bytes, magicLen, bytes.length - magicLen);
- } catch (IOException e) {
- throw new DeserializationException(e);
- }
- return builder.build();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
deleted file mode 100644
index 70e58f8..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.zookeeper;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ServerName;
-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
- * a line-by-line list for use by bash scripts.
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
-public class ZKServerTool {
- public static ServerName[] readZKNodes(Configuration conf) {
- List<ServerName> hosts = new LinkedList<>();
- String quorum = conf.get(HConstants.ZOOKEEPER_QUORUM, HConstants.LOCALHOST);
-
- String[] values = quorum.split(",");
- for (String value : values) {
- String[] parts = value.split(":");
- String host = parts[0];
- int port = HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT;
- if (parts.length > 1) {
- port = Integer.parseInt(parts[1]);
- }
- hosts.add(ServerName.valueOf(host, port, -1));
- }
- return hosts.toArray(new ServerName[hosts.size()]);
- }
-
- /**
- * Run the tool.
- * @param args Command line arguments.
- */
- public static void main(String args[]) {
- for(ServerName server: readZKNodes(HBaseConfiguration.create())) {
- // bin/zookeeper.sh relies on the "ZK host" string for grepping which is case sensitive.
- System.out.println("ZK host: " + server.getHostname());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java
deleted file mode 100644
index 1376ba9..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.net.URLDecoder;
-import java.net.URLEncoder;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Common methods and attributes used by {@link org.apache.hadoop.hbase.master.SplitLogManager}
- * and {@link org.apache.hadoop.hbase.regionserver.SplitLogWorker}
- * running distributed splitting of WAL logs.
- */
-@InterfaceAudience.Private
-public class ZKSplitLog {
- private static final Log LOG = LogFactory.getLog(ZKSplitLog.class);
-
- /**
- * Gets the full path node name for the log file being split.
- * This method will url encode the filename.
- * @param zkw zk reference
- * @param filename log file name (only the basename)
- */
- public static String getEncodedNodeName(ZooKeeperWatcher zkw, String filename) {
- return ZNodePaths.joinZNode(zkw.znodePaths.splitLogZNode, encode(filename));
- }
-
- public static String getFileName(String node) {
- String basename = node.substring(node.lastIndexOf('/') + 1);
- return decode(basename);
- }
-
- static String encode(String s) {
- try {
- return URLEncoder.encode(s, "UTF-8");
- } catch (UnsupportedEncodingException e) {
- throw new RuntimeException("URLENCODER doesn't support UTF-8");
- }
- }
-
- static String decode(String s) {
- try {
- return URLDecoder.decode(s, "UTF-8");
- } catch (UnsupportedEncodingException e) {
- throw new RuntimeException("URLDecoder doesn't support UTF-8");
- }
- }
-
- public static String getRescanNode(ZooKeeperWatcher zkw) {
- return ZNodePaths.joinZNode(zkw.znodePaths.splitLogZNode, "RESCAN");
- }
-
- /**
- * @param name the last part in path
- * @return whether the node name represents a rescan node
- */
- public static boolean isRescanNode(String name) {
- return name.startsWith("RESCAN");
- }
-
- /**
- * @param zkw
- * @param path the absolute path, starts with '/'
- * @return whether the path represents a rescan node
- */
- public static boolean isRescanNode(ZooKeeperWatcher zkw, String path) {
- String prefix = getRescanNode(zkw);
- if (path.length() <= prefix.length()) {
- return false;
- }
- for (int i = 0; i < prefix.length(); i++) {
- if (prefix.charAt(i) != path.charAt(i)) {
- return false;
- }
- }
- return true;
- }
-
- public static Path getSplitLogDir(Path rootdir, String tmpname) {
- return new Path(new Path(rootdir, HConstants.SPLIT_LOGDIR_NAME), tmpname);
- }
-
- public static void markCorrupted(Path rootdir, String logFileName,
- FileSystem fs) {
- Path file = new Path(getSplitLogDir(rootdir, logFileName), "corrupt");
- try {
- fs.createNewFile(file);
- } catch (IOException e) {
- LOG.warn("Could not flag a log file as corrupted. Failed to create " +
- file, e);
- }
- }
-
- public static boolean isCorrupted(Path rootdir, String logFileName,
- FileSystem fs) throws IOException {
- Path file = new Path(getSplitLogDir(rootdir, logFileName), "corrupt");
- boolean isCorrupt;
- isCorrupt = fs.exists(file);
- return isCorrupt;
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java
deleted file mode 100644
index 341fbbd..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.curator.shaded.com.google.common.base.Stopwatch;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooKeeperMain;
-
-/**
- * Tool for running ZookeeperMain from HBase by reading a ZooKeeper server
- * from HBase XML configuration.
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
-public class ZooKeeperMainServer {
- private static final String SERVER_ARG = "-server";
-
- public String parse(final Configuration c) {
- return ZKConfig.getZKQuorumServersString(c);
- }
-
- /**
- * ZooKeeper 3.4.6 broke being able to pass commands on command line.
- * See ZOOKEEPER-1897. This class is a hack to restore this faclity.
- */
- private static class HACK_UNTIL_ZOOKEEPER_1897_ZooKeeperMain extends ZooKeeperMain {
- public HACK_UNTIL_ZOOKEEPER_1897_ZooKeeperMain(String[] args)
- throws IOException, InterruptedException {
- super(args);
- // Make sure we are connected before we proceed. Can take a while on some systems. If we
- // run the command without being connected, we get ConnectionLoss KeeperErrorConnection...
- Stopwatch stopWatch = Stopwatch.createStarted();
- while (!this.zk.getState().isConnected()) {
- Thread.sleep(1);
- if (stopWatch.elapsed(TimeUnit.SECONDS) > 10) {
- throw new InterruptedException("Failed connect after waiting " +
- stopWatch.elapsed(TimeUnit.SECONDS) + "seconds; state=" + this.zk.getState() +
- "; " + this.zk);
- }
- }
- }
-
- /**
- * Run the command-line args passed. Calls System.exit when done.
- * @throws KeeperException
- * @throws IOException
- * @throws InterruptedException
- */
- void runCmdLine() throws KeeperException, IOException, InterruptedException {
- processCmd(this.cl);
- System.exit(0);
- }
- }
-
- /**
- * @param args
- * @return True if argument strings have a '-server' in them.
- */
- private static boolean hasServer(final String args[]) {
- return args.length > 0 && args[0].equals(SERVER_ARG);
- }
-
- /**
- * @param args
- * @return True if command-line arguments were passed.
- */
- private static boolean hasCommandLineArguments(final String args[]) {
- if (hasServer(args)) {
- if (args.length < 2) throw new IllegalStateException("-server param but no value");
- return args.length > 2;
- }
- return args.length > 0;
- }
-
- /**
- * Run the tool.
- * @param args Command line arguments. First arg is path to zookeepers file.
- */
- public static void main(String args[]) throws Exception {
- String [] newArgs = args;
- if (!hasServer(args)) {
- // Add the zk ensemble from configuration if none passed on command-line.
- Configuration conf = HBaseConfiguration.create();
- String hostport = new ZooKeeperMainServer().parse(conf);
- if (hostport != null && hostport.length() > 0) {
- newArgs = new String[args.length + 2];
- System.arraycopy(args, 0, newArgs, 2, args.length);
- newArgs[0] = "-server";
- newArgs[1] = hostport;
- }
- }
- // If command-line arguments, run our hack so they are executed.
- // ZOOKEEPER-1897 was committed to zookeeper-3.4.6 but elsewhere in this class we say
- // 3.4.6 breaks command-processing; TODO.
- if (hasCommandLineArguments(args)) {
- HACK_UNTIL_ZOOKEEPER_1897_ZooKeeperMain zkm =
- new HACK_UNTIL_ZOOKEEPER_1897_ZooKeeperMain(newArgs);
- zkm.runCmdLine();
- } else {
- ZooKeeperMain.main(newArgs);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp b/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp
index 4465162..1951617 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp
@@ -20,11 +20,11 @@
<%@ page contentType="text/html;charset=UTF-8"
import="org.apache.commons.lang3.StringEscapeUtils"
import="org.apache.hadoop.hbase.zookeeper.ZKUtil"
- import="org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher"
+ import="org.apache.hadoop.hbase.zookeeper.ZKWatcher"
import="org.apache.hadoop.hbase.HBaseConfiguration"
import="org.apache.hadoop.hbase.master.HMaster"%><%
HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
- ZooKeeperWatcher watcher = master.getZooKeeper();
+ ZKWatcher watcher = master.getZooKeeper();
%>
<!DOCTYPE html>
<?xml version="1.0" encoding="UTF-8" ?>
[12/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper
from hbase-server and hbase-client
Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
deleted file mode 100644
index 04f709f..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
+++ /dev/null
@@ -1,814 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import static org.apache.hadoop.hbase.zookeeper.ZKMetadata.appendMetaData;
-import static org.apache.hadoop.hbase.zookeeper.ZKMetadata.removeMetaData;
-
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.RetryCounter;
-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.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.OpResult;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.ZooKeeper.States;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.CreateRequest;
-import org.apache.zookeeper.proto.SetDataRequest;
-
-/**
- * A zookeeper that can handle 'recoverable' errors.
- * To handle recoverable errors, developers need to realize that there are two
- * classes of requests: idempotent and non-idempotent requests. Read requests
- * and unconditional sets and deletes are examples of idempotent requests, they
- * can be reissued with the same results.
- * (Although, the delete may throw a NoNodeException on reissue its effect on
- * the ZooKeeper state is the same.) Non-idempotent requests need special
- * handling, application and library writers need to keep in mind that they may
- * need to encode information in the data or name of znodes to detect
- * retries. A simple example is a create that uses a sequence flag.
- * If a process issues a create("/x-", ..., SEQUENCE) and gets a connection
- * loss exception, that process will reissue another
- * create("/x-", ..., SEQUENCE) and get back x-111. When the process does a
- * getChildren("/"), it sees x-1,x-30,x-109,x-110,x-111, now it could be
- * that x-109 was the result of the previous create, so the process actually
- * owns both x-109 and x-111. An easy way around this is to use "x-process id-"
- * when doing the create. If the process is using an id of 352, before reissuing
- * the create it will do a getChildren("/") and see "x-222-1", "x-542-30",
- * "x-352-109", x-333-110". The process will know that the original create
- * succeeded an the znode it created is "x-352-109".
- * @see "http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling"
- */
-@InterfaceAudience.Private
-public class RecoverableZooKeeper {
- private static final Log LOG = LogFactory.getLog(RecoverableZooKeeper.class);
- // the actual ZooKeeper client instance
- private ZooKeeper zk;
- private final RetryCounterFactory retryCounterFactory;
- // An identifier of this process in the cluster
- private final String identifier;
- private final byte[] id;
- private Watcher watcher;
- private int sessionTimeout;
- private String quorumServers;
- private final ZooKeeperMetricsListener metrics;
-
- public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
- Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime)
- throws IOException {
- this(quorumServers, sessionTimeout, watcher, maxRetries, retryIntervalMillis, maxSleepTime,
- null);
- }
-
- @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DE_MIGHT_IGNORE",
- justification="None. Its always been this way.")
- public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
- Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime, String identifier)
- throws IOException {
- // TODO: Add support for zk 'chroot'; we don't add it to the quorumServers String as we should.
- this.retryCounterFactory =
- new RetryCounterFactory(maxRetries+1, retryIntervalMillis, maxSleepTime);
-
- if (identifier == null || identifier.length() == 0) {
- // the identifier = processID@hostName
- identifier = ManagementFactory.getRuntimeMXBean().getName();
- }
- LOG.info("Process identifier=" + identifier +
- " connecting to ZooKeeper ensemble=" + quorumServers);
- this.identifier = identifier;
- this.id = Bytes.toBytes(identifier);
-
- this.watcher = watcher;
- this.sessionTimeout = sessionTimeout;
- this.quorumServers = quorumServers;
- this.metrics = new MetricsZooKeeper();
- try {checkZk();} catch (Exception x) {/* ignore */}
- }
-
- /**
- * Try to create a ZooKeeper connection. Turns any exception encountered into a
- * KeeperException.OperationTimeoutException so it can retried.
- * @return The created ZooKeeper connection object
- * @throws KeeperException
- */
- protected synchronized ZooKeeper checkZk() throws KeeperException {
- if (this.zk == null) {
- try {
- this.zk = new ZooKeeper(quorumServers, sessionTimeout, watcher);
- } catch (IOException ex) {
- LOG.warn("Unable to create ZooKeeper Connection", ex);
- throw new KeeperException.OperationTimeoutException();
- }
- }
- return zk;
- }
-
- public synchronized void reconnectAfterExpiration()
- throws IOException, KeeperException, InterruptedException {
- if (zk != null) {
- LOG.info("Closing dead ZooKeeper connection, session" +
- " was: 0x"+Long.toHexString(zk.getSessionId()));
- zk.close();
- // reset the ZooKeeper connection
- zk = null;
- }
- checkZk();
- LOG.info("Recreated a ZooKeeper, session" +
- " is: 0x"+Long.toHexString(zk.getSessionId()));
- }
-
- /**
- * delete is an idempotent operation. Retry before throwing exception.
- * This function will not throw NoNodeException if the path does not
- * exist.
- */
- public void delete(String path, int version) throws InterruptedException, KeeperException {
- try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.delete")) {
- RetryCounter retryCounter = retryCounterFactory.create();
- boolean isRetry = false; // False for first attempt, true for all retries.
- while (true) {
- try {
- long startTime = EnvironmentEdgeManager.currentTime();
- checkZk().delete(path, version);
- this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- return;
- } catch (KeeperException e) {
- this.metrics.registerFailedZKCall();
- switch (e.code()) {
- case NONODE:
- if (isRetry) {
- LOG.debug("Node " + path + " already deleted. Assuming a " +
- "previous attempt succeeded.");
- return;
- }
- LOG.debug("Node " + path + " already deleted, retry=" + isRetry);
- throw e;
-
- case CONNECTIONLOSS:
- this.metrics.registerConnectionLossException();
- retryOrThrow(retryCounter, e, "delete");
- break;
- case OPERATIONTIMEOUT:
- this.metrics.registerOperationTimeoutException();
- retryOrThrow(retryCounter, e, "delete");
- break;
-
- default:
- throw e;
- }
- }
- retryCounter.sleepUntilNextRetry();
- isRetry = true;
- }
- }
- }
-
- /**
- * exists is an idempotent operation. Retry before throwing exception
- * @return A Stat instance
- */
- public Stat exists(String path, Watcher watcher) throws KeeperException, InterruptedException {
- try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.exists")) {
- RetryCounter retryCounter = retryCounterFactory.create();
- while (true) {
- try {
- long startTime = EnvironmentEdgeManager.currentTime();
- Stat nodeStat = checkZk().exists(path, watcher);
- this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- return nodeStat;
- } catch (KeeperException e) {
- this.metrics.registerFailedZKCall();
- switch (e.code()) {
- case CONNECTIONLOSS:
- this.metrics.registerConnectionLossException();
- retryOrThrow(retryCounter, e, "exists");
- break;
- case OPERATIONTIMEOUT:
- this.metrics.registerOperationTimeoutException();
- retryOrThrow(retryCounter, e, "exists");
- break;
-
- default:
- throw e;
- }
- }
- retryCounter.sleepUntilNextRetry();
- }
- }
- }
-
- /**
- * exists is an idempotent operation. Retry before throwing exception
- * @return A Stat instance
- */
- public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException {
- try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.exists")) {
- RetryCounter retryCounter = retryCounterFactory.create();
- while (true) {
- try {
- long startTime = EnvironmentEdgeManager.currentTime();
- Stat nodeStat = checkZk().exists(path, watch);
- this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- return nodeStat;
- } catch (KeeperException e) {
- this.metrics.registerFailedZKCall();
- switch (e.code()) {
- case CONNECTIONLOSS:
- this.metrics.registerConnectionLossException();
- retryOrThrow(retryCounter, e, "exists");
- break;
- case OPERATIONTIMEOUT:
- this.metrics.registerOperationTimeoutException();
- retryOrThrow(retryCounter, e, "exists");
- break;
-
- default:
- throw e;
- }
- }
- retryCounter.sleepUntilNextRetry();
- }
- }
- }
-
- private void retryOrThrow(RetryCounter retryCounter, KeeperException e,
- String opName) throws KeeperException {
- if (!retryCounter.shouldRetry()) {
- LOG.error("ZooKeeper " + opName + " failed after "
- + retryCounter.getMaxAttempts() + " attempts");
- throw e;
- }
- LOG.debug("Retry, connectivity issue (JVM Pause?); quorum=" + quorumServers + "," +
- "exception=" + e);
- }
-
- /**
- * getChildren is an idempotent operation. Retry before throwing exception
- * @return List of children znodes
- */
- public List<String> getChildren(String path, Watcher watcher)
- throws KeeperException, InterruptedException {
- try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getChildren")) {
- RetryCounter retryCounter = retryCounterFactory.create();
- while (true) {
- try {
- long startTime = EnvironmentEdgeManager.currentTime();
- List<String> children = checkZk().getChildren(path, watcher);
- this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- return children;
- } catch (KeeperException e) {
- this.metrics.registerFailedZKCall();
- switch (e.code()) {
- case CONNECTIONLOSS:
- this.metrics.registerConnectionLossException();
- retryOrThrow(retryCounter, e, "getChildren");
- break;
- case OPERATIONTIMEOUT:
- this.metrics.registerOperationTimeoutException();
- retryOrThrow(retryCounter, e, "getChildren");
- break;
-
- default:
- throw e;
- }
- }
- retryCounter.sleepUntilNextRetry();
- }
- }
- }
-
- /**
- * getChildren is an idempotent operation. Retry before throwing exception
- * @return List of children znodes
- */
- public List<String> getChildren(String path, boolean watch)
- throws KeeperException, InterruptedException {
- try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getChildren")) {
- RetryCounter retryCounter = retryCounterFactory.create();
- while (true) {
- try {
- long startTime = EnvironmentEdgeManager.currentTime();
- List<String> children = checkZk().getChildren(path, watch);
- this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- return children;
- } catch (KeeperException e) {
- this.metrics.registerFailedZKCall();
- switch (e.code()) {
- case CONNECTIONLOSS:
- this.metrics.registerConnectionLossException();
- retryOrThrow(retryCounter, e, "getChildren");
- break;
- case OPERATIONTIMEOUT:
- this.metrics.registerOperationTimeoutException();
- retryOrThrow(retryCounter, e, "getChildren");
- break;
-
- default:
- throw e;
- }
- }
- retryCounter.sleepUntilNextRetry();
- }
- }
- }
-
- /**
- * getData is an idempotent operation. Retry before throwing exception
- * @return Data
- */
- public byte[] getData(String path, Watcher watcher, Stat stat)
- throws KeeperException, InterruptedException {
- try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getData")) {
- RetryCounter retryCounter = retryCounterFactory.create();
- while (true) {
- try {
- long startTime = EnvironmentEdgeManager.currentTime();
- byte[] revData = checkZk().getData(path, watcher, stat);
- this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- return removeMetaData(revData);
- } catch (KeeperException e) {
- this.metrics.registerFailedZKCall();
- switch (e.code()) {
- case CONNECTIONLOSS:
- this.metrics.registerConnectionLossException();
- retryOrThrow(retryCounter, e, "getData");
- break;
- case OPERATIONTIMEOUT:
- this.metrics.registerOperationTimeoutException();
- retryOrThrow(retryCounter, e, "getData");
- break;
-
- default:
- throw e;
- }
- }
- retryCounter.sleepUntilNextRetry();
- }
- }
- }
-
- /**
- * getData is an idempotent operation. Retry before throwing exception
- * @return Data
- */
- public byte[] getData(String path, boolean watch, Stat stat)
- throws KeeperException, InterruptedException {
- try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getData")) {
- RetryCounter retryCounter = retryCounterFactory.create();
- while (true) {
- try {
- long startTime = EnvironmentEdgeManager.currentTime();
- byte[] revData = checkZk().getData(path, watch, stat);
- this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- return removeMetaData(revData);
- } catch (KeeperException e) {
- this.metrics.registerFailedZKCall();
- switch (e.code()) {
- case CONNECTIONLOSS:
- this.metrics.registerConnectionLossException();
- retryOrThrow(retryCounter, e, "getData");
- break;
- case OPERATIONTIMEOUT:
- this.metrics.registerOperationTimeoutException();
- retryOrThrow(retryCounter, e, "getData");
- break;
-
- default:
- throw e;
- }
- }
- retryCounter.sleepUntilNextRetry();
- }
- }
- }
-
- /**
- * setData is NOT an idempotent operation. Retry may cause BadVersion Exception
- * Adding an identifier field into the data to check whether
- * badversion is caused by the result of previous correctly setData
- * @return Stat instance
- */
- public Stat setData(String path, byte[] data, int version)
- throws KeeperException, InterruptedException {
- try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.setData")) {
- RetryCounter retryCounter = retryCounterFactory.create();
- byte[] newData = appendMetaData(id, data);
- boolean isRetry = false;
- long startTime;
- while (true) {
- try {
- startTime = EnvironmentEdgeManager.currentTime();
- Stat nodeStat = checkZk().setData(path, newData, version);
- this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- return nodeStat;
- } catch (KeeperException e) {
- this.metrics.registerFailedZKCall();
- switch (e.code()) {
- case CONNECTIONLOSS:
- this.metrics.registerConnectionLossException();
- retryOrThrow(retryCounter, e, "setData");
- break;
- case OPERATIONTIMEOUT:
- this.metrics.registerOperationTimeoutException();
- retryOrThrow(retryCounter, e, "setData");
- break;
- case BADVERSION:
- if (isRetry) {
- // try to verify whether the previous setData success or not
- try{
- Stat stat = new Stat();
- startTime = EnvironmentEdgeManager.currentTime();
- byte[] revData = checkZk().getData(path, false, stat);
- this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- if(Bytes.compareTo(revData, newData) == 0) {
- // the bad version is caused by previous successful setData
- return stat;
- }
- } catch(KeeperException keeperException){
- this.metrics.registerFailedZKCall();
- // the ZK is not reliable at this moment. just throwing exception
- throw keeperException;
- }
- }
- // throw other exceptions and verified bad version exceptions
- default:
- throw e;
- }
- }
- retryCounter.sleepUntilNextRetry();
- isRetry = true;
- }
- }
- }
-
- /**
- * getAcl is an idempotent operation. Retry before throwing exception
- * @return list of ACLs
- */
- public List<ACL> getAcl(String path, Stat stat)
- throws KeeperException, InterruptedException {
- try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getAcl")) {
- RetryCounter retryCounter = retryCounterFactory.create();
- while (true) {
- try {
- long startTime = EnvironmentEdgeManager.currentTime();
- List<ACL> nodeACL = checkZk().getACL(path, stat);
- this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- return nodeACL;
- } catch (KeeperException e) {
- this.metrics.registerFailedZKCall();
- switch (e.code()) {
- case CONNECTIONLOSS:
- this.metrics.registerConnectionLossException();
- retryOrThrow(retryCounter, e, "getAcl");
- break;
- case OPERATIONTIMEOUT:
- this.metrics.registerOperationTimeoutException();
- retryOrThrow(retryCounter, e, "getAcl");
- break;
-
- default:
- throw e;
- }
- }
- retryCounter.sleepUntilNextRetry();
- }
- }
- }
-
- /**
- * setAcl is an idempotent operation. Retry before throwing exception
- * @return list of ACLs
- */
- public Stat setAcl(String path, List<ACL> acls, int version)
- throws KeeperException, InterruptedException {
- try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.setAcl")) {
- RetryCounter retryCounter = retryCounterFactory.create();
- while (true) {
- try {
- long startTime = EnvironmentEdgeManager.currentTime();
- Stat nodeStat = checkZk().setACL(path, acls, version);
- this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- return nodeStat;
- } catch (KeeperException e) {
- this.metrics.registerFailedZKCall();
- switch (e.code()) {
- case CONNECTIONLOSS:
- this.metrics.registerConnectionLossException();
- retryOrThrow(retryCounter, e, "setAcl");
- break;
- case OPERATIONTIMEOUT:
- this.metrics.registerOperationTimeoutException();
- retryOrThrow(retryCounter, e, "setAcl");
- break;
-
- default:
- throw e;
- }
- }
- retryCounter.sleepUntilNextRetry();
- }
- }
- }
-
- /**
- * <p>
- * NONSEQUENTIAL create is idempotent operation.
- * Retry before throwing exceptions.
- * But this function will not throw the NodeExist exception back to the
- * application.
- * </p>
- * <p>
- * But SEQUENTIAL is NOT idempotent operation. It is necessary to add
- * identifier to the path to verify, whether the previous one is successful
- * or not.
- * </p>
- *
- * @return Path
- */
- public String create(String path, byte[] data, List<ACL> acl,
- CreateMode createMode)
- throws KeeperException, InterruptedException {
- try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.create")) {
- byte[] newData = appendMetaData(id, data);
- switch (createMode) {
- case EPHEMERAL:
- case PERSISTENT:
- return createNonSequential(path, newData, acl, createMode);
-
- case EPHEMERAL_SEQUENTIAL:
- case PERSISTENT_SEQUENTIAL:
- return createSequential(path, newData, acl, createMode);
-
- default:
- throw new IllegalArgumentException("Unrecognized CreateMode: " +
- createMode);
- }
- }
- }
-
- private String createNonSequential(String path, byte[] data, List<ACL> acl,
- CreateMode createMode) throws KeeperException, InterruptedException {
- RetryCounter retryCounter = retryCounterFactory.create();
- boolean isRetry = false; // False for first attempt, true for all retries.
- long startTime;
- while (true) {
- try {
- startTime = EnvironmentEdgeManager.currentTime();
- String nodePath = checkZk().create(path, data, acl, createMode);
- this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- return nodePath;
- } catch (KeeperException e) {
- this.metrics.registerFailedZKCall();
- switch (e.code()) {
- case NODEEXISTS:
- if (isRetry) {
- // If the connection was lost, there is still a possibility that
- // we have successfully created the node at our previous attempt,
- // so we read the node and compare.
- startTime = EnvironmentEdgeManager.currentTime();
- byte[] currentData = checkZk().getData(path, false, null);
- this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- if (currentData != null &&
- Bytes.compareTo(currentData, data) == 0) {
- // We successfully created a non-sequential node
- return path;
- }
- LOG.error("Node " + path + " already exists with " +
- Bytes.toStringBinary(currentData) + ", could not write " +
- Bytes.toStringBinary(data));
- throw e;
- }
- LOG.debug("Node " + path + " already exists");
- throw e;
-
- case CONNECTIONLOSS:
- this.metrics.registerConnectionLossException();
- retryOrThrow(retryCounter, e, "create");
- break;
- case OPERATIONTIMEOUT:
- this.metrics.registerOperationTimeoutException();
- retryOrThrow(retryCounter, e, "create");
- break;
-
- default:
- throw e;
- }
- }
- retryCounter.sleepUntilNextRetry();
- isRetry = true;
- }
- }
-
- private String createSequential(String path, byte[] data,
- List<ACL> acl, CreateMode createMode)
- throws KeeperException, InterruptedException {
- RetryCounter retryCounter = retryCounterFactory.create();
- boolean first = true;
- String newPath = path+this.identifier;
- while (true) {
- try {
- if (!first) {
- // Check if we succeeded on a previous attempt
- String previousResult = findPreviousSequentialNode(newPath);
- if (previousResult != null) {
- return previousResult;
- }
- }
- first = false;
- long startTime = EnvironmentEdgeManager.currentTime();
- String nodePath = checkZk().create(newPath, data, acl, createMode);
- this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- return nodePath;
- } catch (KeeperException e) {
- this.metrics.registerFailedZKCall();
- switch (e.code()) {
- case CONNECTIONLOSS:
- this.metrics.registerConnectionLossException();
- retryOrThrow(retryCounter, e, "create");
- break;
- case OPERATIONTIMEOUT:
- this.metrics.registerOperationTimeoutException();
- retryOrThrow(retryCounter, e, "create");
- break;
-
- default:
- throw e;
- }
- }
- retryCounter.sleepUntilNextRetry();
- }
- }
- /**
- * 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).
- */
- private Iterable<Op> prepareZKMulti(Iterable<Op> ops)
- throws UnsupportedOperationException {
- if(ops == null) return null;
-
- List<Op> preparedOps = new LinkedList<>();
- for (Op op : ops) {
- if (op.getType() == ZooDefs.OpCode.create) {
- CreateRequest create = (CreateRequest)op.toRequestRecord();
- preparedOps.add(Op.create(create.getPath(), appendMetaData(id, create.getData()),
- create.getAcl(), create.getFlags()));
- } else if (op.getType() == ZooDefs.OpCode.delete) {
- // no need to appendMetaData for delete
- preparedOps.add(op);
- } else if (op.getType() == ZooDefs.OpCode.setData) {
- SetDataRequest setData = (SetDataRequest)op.toRequestRecord();
- preparedOps.add(Op.setData(setData.getPath(), appendMetaData(id, setData.getData()),
- setData.getVersion()));
- } else {
- throw new UnsupportedOperationException("Unexpected ZKOp type: " + op.getClass().getName());
- }
- }
- return preparedOps;
- }
-
- /**
- * Run multiple operations in a transactional manner. Retry before throwing exception
- */
- public List<OpResult> multi(Iterable<Op> ops)
- throws KeeperException, InterruptedException {
- try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.multi")) {
- RetryCounter retryCounter = retryCounterFactory.create();
- Iterable<Op> multiOps = prepareZKMulti(ops);
- while (true) {
- try {
- long startTime = EnvironmentEdgeManager.currentTime();
- List<OpResult> opResults = checkZk().multi(multiOps);
- this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- return opResults;
- } catch (KeeperException e) {
- this.metrics.registerFailedZKCall();
- switch (e.code()) {
- case CONNECTIONLOSS:
- this.metrics.registerConnectionLossException();
- retryOrThrow(retryCounter, e, "multi");
- break;
- case OPERATIONTIMEOUT:
- this.metrics.registerOperationTimeoutException();
- retryOrThrow(retryCounter, e, "multi");
- break;
-
- default:
- throw e;
- }
- }
- retryCounter.sleepUntilNextRetry();
- }
- }
- }
-
- private String findPreviousSequentialNode(String path)
- throws KeeperException, InterruptedException {
- int lastSlashIdx = path.lastIndexOf('/');
- assert(lastSlashIdx != -1);
- String parent = path.substring(0, lastSlashIdx);
- String nodePrefix = path.substring(lastSlashIdx+1);
- long startTime = EnvironmentEdgeManager.currentTime();
- List<String> nodes = checkZk().getChildren(parent, false);
- this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- List<String> matching = filterByPrefix(nodes, nodePrefix);
- for (String node : matching) {
- String nodePath = parent + "/" + node;
- startTime = EnvironmentEdgeManager.currentTime();
- Stat stat = checkZk().exists(nodePath, false);
- this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- if (stat != null) {
- return nodePath;
- }
- }
- return null;
- }
-
- public synchronized long getSessionId() {
- return zk == null ? -1 : zk.getSessionId();
- }
-
- public synchronized void close() throws InterruptedException {
- if (zk != null) zk.close();
- }
-
- public synchronized States getState() {
- return zk == null ? null : zk.getState();
- }
-
- public synchronized ZooKeeper getZooKeeper() {
- return zk;
- }
-
- public synchronized byte[] getSessionPasswd() {
- return zk == null ? null : zk.getSessionPasswd();
- }
-
- public void sync(String path, AsyncCallback.VoidCallback cb, Object ctx) throws KeeperException {
- long startTime = EnvironmentEdgeManager.currentTime();
- checkZk().sync(path, cb, null);
- this.metrics.registerSyncOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
- }
-
- /**
- * Filters the given node list by the given prefixes.
- * This method is all-inclusive--if any element in the node list starts
- * with any of the given prefixes, then it is included in the result.
- *
- * @param nodes the nodes to filter
- * @param prefixes the prefixes to include in the result
- * @return list of every element that starts with one of the prefixes
- */
- private static List<String> filterByPrefix(List<String> nodes,
- String... prefixes) {
- List<String> lockChildren = new ArrayList<>();
- for (String child : nodes){
- for (String prefix : prefixes){
- if (child.startsWith(prefix)){
- lockChildren.add(child);
- break;
- }
- }
- }
- return lockChildren;
- }
-
- public String getIdentifier() {
- return identifier;
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java
deleted file mode 100644
index 9ef7691..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.util.UUID;
-
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.ClusterId;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Publishes and synchronizes a unique identifier specific to a given HBase
- * cluster. The stored identifier is read from the file system by the active
- * master on startup, and is subsequently available to all watchers (including
- * clients).
- */
-@InterfaceAudience.Private
-public class ZKClusterId {
- private ZooKeeperWatcher watcher;
- private Abortable abortable;
- private String id;
-
- public ZKClusterId(ZooKeeperWatcher watcher, Abortable abortable) {
- this.watcher = watcher;
- this.abortable = abortable;
- }
-
- public boolean hasId() {
- return getId() != null;
- }
-
- public String getId() {
- try {
- if (id == null) {
- id = readClusterIdZNode(watcher);
- }
- } catch (KeeperException ke) {
- abortable.abort("Unexpected exception from ZooKeeper reading cluster ID",
- ke);
- }
- return id;
- }
-
- public static String readClusterIdZNode(ZooKeeperWatcher watcher)
- throws KeeperException {
- if (ZKUtil.checkExists(watcher, watcher.znodePaths.clusterIdZNode) != -1) {
- byte [] data;
- try {
- data = ZKUtil.getData(watcher, watcher.znodePaths.clusterIdZNode);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- return null;
- }
- if (data != null) {
- try {
- return ClusterId.parseFrom(data).toString();
- } catch (DeserializationException e) {
- throw ZKUtil.convert(e);
- }
- }
- }
- return null;
- }
-
- public static void setClusterId(ZooKeeperWatcher watcher, ClusterId id)
- throws KeeperException {
- ZKUtil.createSetData(watcher, watcher.znodePaths.clusterIdZNode, id.toByteArray());
- }
-
- /**
- * Get the UUID for the provided ZK watcher. Doesn't handle any ZK exceptions
- * @param zkw watcher connected to an ensemble
- * @return the UUID read from zookeeper
- * @throws KeeperException
- */
- public static UUID getUUIDForCluster(ZooKeeperWatcher zkw) throws KeeperException {
- String uuid = readClusterIdZNode(zkw);
- return uuid == null ? null : UUID.fromString(uuid);
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
deleted file mode 100644
index b0610b0..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Handles coordination of a single "leader" instance among many possible
- * candidates. The first {@link ZKLeaderManager} to successfully create
- * the given znode becomes the leader, allowing the instance to continue
- * with whatever processing must be protected. Other {@link ZKLeaderManager}
- * instances will wait to be notified of changes to the leader znode.
- * If the current master instance fails, the ephemeral leader znode will
- * be removed, and all waiting instances will be notified, with the race
- * to claim the leader znode beginning all over again.
- * @deprecated Not used
- */
-@Deprecated
-@InterfaceAudience.Private
-public class ZKLeaderManager extends ZooKeeperListener {
- private static final Log LOG = LogFactory.getLog(ZKLeaderManager.class);
-
- private final AtomicBoolean leaderExists = new AtomicBoolean();
- private String leaderZNode;
- private byte[] nodeId;
- private Stoppable candidate;
-
- public ZKLeaderManager(ZooKeeperWatcher watcher, String leaderZNode,
- byte[] identifier, Stoppable candidate) {
- super(watcher);
- this.leaderZNode = leaderZNode;
- this.nodeId = identifier;
- this.candidate = candidate;
- }
-
- public void start() {
- try {
- watcher.registerListener(this);
- String parent = ZKUtil.getParent(leaderZNode);
- if (ZKUtil.checkExists(watcher, parent) < 0) {
- ZKUtil.createWithParents(watcher, parent);
- }
- } catch (KeeperException ke) {
- watcher.abort("Unhandled zk exception when starting", ke);
- candidate.stop("Unhandled zk exception starting up: "+ke.getMessage());
- }
- }
-
- @Override
- public void nodeCreated(String path) {
- if (leaderZNode.equals(path) && !candidate.isStopped()) {
- handleLeaderChange();
- }
- }
-
- @Override
- public void nodeDeleted(String path) {
- if (leaderZNode.equals(path) && !candidate.isStopped()) {
- handleLeaderChange();
- }
- }
-
- private void handleLeaderChange() {
- try {
- synchronized(leaderExists) {
- if (ZKUtil.watchAndCheckExists(watcher, leaderZNode)) {
- LOG.info("Found new leader for znode: "+leaderZNode);
- leaderExists.set(true);
- } else {
- LOG.info("Leader change, but no new leader found");
- leaderExists.set(false);
- leaderExists.notifyAll();
- }
- }
- } catch (KeeperException ke) {
- watcher.abort("ZooKeeper error checking for leader znode", ke);
- candidate.stop("ZooKeeper error checking for leader: "+ke.getMessage());
- }
- }
-
- /**
- * Blocks until this instance has claimed the leader ZNode in ZooKeeper
- */
- public void waitToBecomeLeader() {
- while (!candidate.isStopped()) {
- try {
- if (ZKUtil.createEphemeralNodeAndWatch(watcher, leaderZNode, nodeId)) {
- // claimed the leader znode
- leaderExists.set(true);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Claimed the leader znode as '"+
- Bytes.toStringBinary(nodeId)+"'");
- }
- return;
- }
-
- // if claiming the node failed, there should be another existing node
- byte[] currentId = ZKUtil.getDataAndWatch(watcher, leaderZNode);
- if (currentId != null && Bytes.equals(currentId, nodeId)) {
- // claimed with our ID, but we didn't grab it, possibly restarted?
- LOG.info("Found existing leader with our ID ("+
- Bytes.toStringBinary(nodeId)+"), removing");
- ZKUtil.deleteNode(watcher, leaderZNode);
- leaderExists.set(false);
- } else {
- LOG.info("Found existing leader with ID: "+Bytes.toStringBinary(nodeId));
- leaderExists.set(true);
- }
- } catch (KeeperException ke) {
- watcher.abort("Unexpected error from ZK, stopping candidate", ke);
- candidate.stop("Unexpected error from ZK: "+ke.getMessage());
- return;
- }
-
- // wait for next chance
- synchronized(leaderExists) {
- while (leaderExists.get() && !candidate.isStopped()) {
- try {
- leaderExists.wait();
- } catch (InterruptedException ie) {
- LOG.debug("Interrupted waiting on leader", ie);
- }
- }
- }
- }
- }
-
- /**
- * Removes the leader znode, if it is currently claimed by this instance.
- */
- public void stepDownAsLeader() {
- try {
- synchronized(leaderExists) {
- if (!leaderExists.get()) {
- return;
- }
- byte[] leaderId = ZKUtil.getData(watcher, leaderZNode);
- if (leaderId != null && Bytes.equals(nodeId, leaderId)) {
- LOG.info("Stepping down as leader");
- ZKUtil.deleteNodeFailSilent(watcher, leaderZNode);
- leaderExists.set(false);
- } else {
- LOG.info("Not current leader, no need to step down");
- }
- }
- } catch (KeeperException ke) {
- watcher.abort("Unhandled zookeeper exception removing leader node", ke);
- candidate.stop("Unhandled zookeeper exception removing leader node: "
- + ke.getMessage());
- } catch (InterruptedException e) {
- watcher.abort("Unhandled zookeeper exception removing leader node", e);
- candidate.stop("Unhandled zookeeper exception removing leader node: "
- + e.getMessage());
- }
- }
-
- public boolean hasLeader() {
- return leaderExists.get();
- }
-}
[09/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper
from hbase-server and hbase-client
Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java
index d261993..389dea7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java
@@ -23,12 +23,12 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -42,12 +42,12 @@ class HFileArchiveManager {
private final String archiveZnode;
private static final Log LOG = LogFactory.getLog(HFileArchiveManager.class);
- private final ZooKeeperWatcher zooKeeper;
+ private final ZKWatcher zooKeeper;
private volatile boolean stopped = false;
public HFileArchiveManager(Connection connection, Configuration conf)
throws ZooKeeperConnectionException, IOException {
- this.zooKeeper = new ZooKeeperWatcher(conf, "hfileArchiveManager-on-" + connection.toString(),
+ this.zooKeeper = new ZKWatcher(conf, "hfileArchiveManager-on-" + connection.toString(),
connection);
this.archiveZnode = ZKTableArchiveClient.getArchiveZNode(this.zooKeeper.getConfiguration(),
this.zooKeeper);
@@ -104,7 +104,7 @@ class HFileArchiveManager {
* @param table table name on which to enable archiving
* @throws KeeperException
*/
- private void enable(ZooKeeperWatcher zooKeeper, byte[] table)
+ private void enable(ZKWatcher zooKeeper, byte[] table)
throws KeeperException {
LOG.debug("Ensuring archiving znode exists");
ZKUtil.createAndFailSilent(zooKeeper, archiveZnode);
@@ -123,7 +123,7 @@ class HFileArchiveManager {
* @param table name of the table to disable
* @throws KeeperException if an unexpected ZK connection issues occurs
*/
- private void disable(ZooKeeperWatcher zooKeeper, byte[] table) throws KeeperException {
+ private void disable(ZKWatcher zooKeeper, byte[] table) throws KeeperException {
// ensure the latest state of the archive node is found
zooKeeper.sync(archiveZnode);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/TableHFileArchiveTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/TableHFileArchiveTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/TableHFileArchiveTracker.java
index 72c292f..73b50a6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/TableHFileArchiveTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/TableHFileArchiveTracker.java
@@ -22,12 +22,12 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
import org.apache.zookeeper.KeeperException;
/**
@@ -38,14 +38,14 @@ import org.apache.zookeeper.KeeperException;
* archive.
*/
@InterfaceAudience.Private
-public class TableHFileArchiveTracker extends ZooKeeperListener {
+public class TableHFileArchiveTracker extends ZKListener {
private static final Log LOG = LogFactory.getLog(TableHFileArchiveTracker.class);
public static final String HFILE_ARCHIVE_ZNODE_PARENT = "hfilearchive";
private HFileArchiveTableMonitor monitor;
private String archiveHFileZNode;
private boolean stopped = false;
- private TableHFileArchiveTracker(ZooKeeperWatcher watcher, HFileArchiveTableMonitor monitor) {
+ private TableHFileArchiveTracker(ZKWatcher watcher, HFileArchiveTableMonitor monitor) {
super(watcher);
watcher.registerListener(this);
this.monitor = monitor;
@@ -235,7 +235,7 @@ public class TableHFileArchiveTracker extends ZooKeeperListener {
*/
public static TableHFileArchiveTracker create(Configuration conf)
throws ZooKeeperConnectionException, IOException {
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "hfileArchiveCleaner", null);
+ ZKWatcher zkw = new ZKWatcher(conf, "hfileArchiveCleaner", null);
return create(zkw, new HFileArchiveTableMonitor());
}
@@ -247,12 +247,12 @@ public class TableHFileArchiveTracker extends ZooKeeperListener {
* @return ZooKeeper tracker to monitor for this server if this server should archive hfiles for a
* given table
*/
- private static TableHFileArchiveTracker create(ZooKeeperWatcher zkw,
+ private static TableHFileArchiveTracker create(ZKWatcher zkw,
HFileArchiveTableMonitor monitor) {
return new TableHFileArchiveTracker(zkw, monitor);
}
- public ZooKeeperWatcher getZooKeeperWatcher() {
+ public ZKWatcher getZooKeeperWatcher() {
return this.watcher;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java
index 9a7903a..39d9543 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java
@@ -19,13 +19,13 @@ package org.apache.hadoop.hbase.backup.example;
import java.io.IOException;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -148,7 +148,7 @@ public class ZKTableArchiveClient extends Configured {
* @param zooKeeper zookeeper to used for building the full path
* @return get the znode for long-term archival of a table for
*/
- public static String getArchiveZNode(Configuration conf, ZooKeeperWatcher zooKeeper) {
+ public static String getArchiveZNode(Configuration conf, ZKWatcher zooKeeper) {
return ZNodePaths.joinZNode(zooKeeper.znodePaths.baseZNode, conf.get(
ZOOKEEPER_ZNODE_HFILE_ARCHIVE_KEY, TableHFileArchiveTracker.HFILE_ARCHIVE_ZNODE_PARENT));
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
index c19c3a5..8a07b4b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
@@ -43,12 +43,12 @@ import org.apache.hadoop.hbase.master.SplitLogManager.Task;
import org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
import org.apache.hadoop.hbase.zookeeper.ZKMetadata;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.AsyncCallback;
@@ -65,7 +65,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTe
* {@link SplitLogManagerCoordination}
*/
@InterfaceAudience.Private
-public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
+public class ZKSplitLogManagerCoordination extends ZKListener implements
SplitLogManagerCoordination {
public static final int DEFAULT_TIMEOUT = 120000;
@@ -85,7 +85,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
public boolean ignoreZKDeleteForTesting = false;
- public ZKSplitLogManagerCoordination(Configuration conf, ZooKeeperWatcher watcher) {
+ public ZKSplitLogManagerCoordination(Configuration conf, ZKWatcher watcher) {
super(watcher);
this.conf = conf;
taskFinisher = new TaskFinisher() {
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
index 10e2642..91b617f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
@@ -19,17 +19,15 @@ package org.apache.hadoop.hbase.coordination;
import java.io.IOException;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator;
import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -37,7 +35,7 @@ import org.apache.zookeeper.KeeperException;
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class ZkCoordinatedStateManager implements CoordinatedStateManager {
- protected ZooKeeperWatcher watcher;
+ protected ZKWatcher watcher;
protected SplitLogWorkerCoordination splitLogWorkerCoordination;
protected SplitLogManagerCoordination splitLogManagerCoordination;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
index ef87498..e64907c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
@@ -42,12 +42,12 @@ import org.apache.hadoop.hbase.regionserver.handler.WALSplitterHandler;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
import org.apache.hadoop.hbase.zookeeper.ZKMetadata;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.AsyncCallback;
@@ -60,7 +60,7 @@ import org.apache.zookeeper.data.Stat;
*
*/
@InterfaceAudience.Private
-public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
+public class ZkSplitLogWorkerCoordination extends ZKListener implements
SplitLogWorkerCoordination {
private static final Log LOG = LogFactory.getLog(ZkSplitLogWorkerCoordination.class);
@@ -85,13 +85,13 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
private final ServerName serverName;
- public ZkSplitLogWorkerCoordination(ServerName serverName, ZooKeeperWatcher watcher) {
+ public ZkSplitLogWorkerCoordination(ServerName serverName, ZKWatcher watcher) {
super(watcher);
this.serverName = serverName;
}
/**
- * Override handler from {@link ZooKeeperListener}
+ * Override handler from {@link ZKListener}
*/
@Override
public void nodeChildrenChanged(String path) {
@@ -107,7 +107,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
}
/**
- * Override handler from {@link ZooKeeperListener}
+ * Override handler from {@link ZKListener}
*/
@Override
public void nodeDataChanged(String path) {
@@ -353,7 +353,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
* @param taskZKVersion version of the task in zk
* @return non-negative integer value when task can be owned by current region server otherwise -1
*/
- protected static int attemptToOwnTask(boolean isFirstTime, ZooKeeperWatcher zkw,
+ protected static int attemptToOwnTask(boolean isFirstTime, ZKWatcher zkw,
ServerName server, String task, int taskZKVersion) {
int latestZKVersion = FAILED_TO_OWN_TASK;
try {
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
index f154347..ce9f290 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
@@ -23,6 +23,10 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
@@ -30,11 +34,7 @@ import org.apache.hadoop.hbase.ZNodeClearer;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
import org.apache.zookeeper.KeeperException;
/**
@@ -51,7 +51,7 @@ import org.apache.zookeeper.KeeperException;
* the active master of the cluster.
*/
@InterfaceAudience.Private
-public class ActiveMasterManager extends ZooKeeperListener {
+public class ActiveMasterManager extends ZKListener {
private static final Log LOG = LogFactory.getLog(ActiveMasterManager.class);
final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
@@ -66,7 +66,7 @@ public class ActiveMasterManager extends ZooKeeperListener {
* @param sn ServerName
* @param master In an instance of a Master.
*/
- ActiveMasterManager(ZooKeeperWatcher watcher, ServerName sn, Server master) {
+ ActiveMasterManager(ZKWatcher watcher, ServerName sn, Server master) {
super(watcher);
watcher.registerListener(this);
this.sn = sn;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DrainingServerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DrainingServerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DrainingServerTracker.java
new file mode 100644
index 0000000..81a8b55
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DrainingServerTracker.java
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Tracks the list of draining region servers via ZK.
+ *
+ * <p>This class is responsible for watching for changes to the draining
+ * servers list. It handles adds/deletes in the draining RS list and
+ * watches each node.
+ *
+ * <p>If an RS gets deleted from draining list, we call
+ * {@link ServerManager#removeServerFromDrainList(ServerName)}
+ *
+ * <p>If an RS gets added to the draining list, we add a watcher to it and call
+ * {@link ServerManager#addServerToDrainList(ServerName)}
+ *
+ * <p>This class is deprecated in 2.0 because decommission/draining API goes through
+ * master in 2.0. Can remove this class in 3.0.
+ *
+ */
+@InterfaceAudience.Private
+public class DrainingServerTracker extends ZKListener {
+ private static final Log LOG = LogFactory.getLog(DrainingServerTracker.class);
+
+ private ServerManager serverManager;
+ private final NavigableSet<ServerName> drainingServers = new TreeSet<>();
+ private Abortable abortable;
+
+ public DrainingServerTracker(ZKWatcher watcher,
+ Abortable abortable, ServerManager serverManager) {
+ super(watcher);
+ this.abortable = abortable;
+ this.serverManager = serverManager;
+ }
+
+ /**
+ * Starts the tracking of draining RegionServers.
+ *
+ * <p>All Draining RSs will be tracked after this method is called.
+ *
+ * @throws KeeperException
+ */
+ public void start() throws KeeperException, IOException {
+ watcher.registerListener(this);
+ // Add a ServerListener to check if a server is draining when it's added.
+ serverManager.registerListener(new ServerListener() {
+ @Override
+ public void serverAdded(ServerName sn) {
+ if (drainingServers.contains(sn)){
+ serverManager.addServerToDrainList(sn);
+ }
+ }
+ });
+ List<String> servers =
+ ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.drainingZNode);
+ add(servers);
+ }
+
+ private void add(final List<String> servers) throws IOException {
+ synchronized(this.drainingServers) {
+ this.drainingServers.clear();
+ for (String n: servers) {
+ final ServerName sn = ServerName.valueOf(ZKUtil.getNodeName(n));
+ this.drainingServers.add(sn);
+ this.serverManager.addServerToDrainList(sn);
+ LOG.info("Draining RS node created, adding to list [" +
+ sn + "]");
+
+ }
+ }
+ }
+
+ private void remove(final ServerName sn) {
+ synchronized(this.drainingServers) {
+ this.drainingServers.remove(sn);
+ this.serverManager.removeServerFromDrainList(sn);
+ }
+ }
+
+ @Override
+ public void nodeDeleted(final String path) {
+ if(path.startsWith(watcher.znodePaths.drainingZNode)) {
+ final ServerName sn = ServerName.valueOf(ZKUtil.getNodeName(path));
+ LOG.info("Draining RS node deleted, removing from list [" +
+ sn + "]");
+ remove(sn);
+ }
+ }
+
+ @Override
+ public void nodeChildrenChanged(final String path) {
+ if(path.equals(watcher.znodePaths.drainingZNode)) {
+ try {
+ final List<String> newNodes =
+ ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.drainingZNode);
+ add(newNodes);
+ } catch (KeeperException e) {
+ abortable.abort("Unexpected zk exception getting RS nodes", e);
+ } catch (IOException e) {
+ abortable.abort("Unexpected zk exception getting RS nodes", e);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index cad77e5..1967296 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -18,7 +18,6 @@
*/
package org.apache.hadoop.hbase.master;
-import com.google.common.base.Enums;
import javax.servlet.ServletException;
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
@@ -175,17 +174,14 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.util.ZKDataMigrator;
-import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.MasterMaintenanceModeTracker;
import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
-import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
-import org.apache.hadoop.hbase.zookeeper.SplitOrMergeTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.eclipse.jetty.server.Server;
@@ -2637,7 +2633,7 @@ public class HMaster extends HRegionServer implements MasterServices {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return zooKeeper;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
index 677a2a0..7abf02c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
@@ -85,7 +85,7 @@ public class MasterMetaBootstrap {
}
private void unassignExcessMetaReplica(int numMetaReplicasConfigured) {
- final ZooKeeperWatcher zooKeeper = master.getZooKeeper();
+ final ZKWatcher zooKeeper = master.getZooKeeper();
// unassign the unneeded replicas (for e.g., if the previous master was configured
// with a replication of 3 and now it is 2, we need to unassign the 1 unneeded replica)
try {
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
index d13fb76..6a9b572 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
/**
* Impl for exposing HMaster Information through JMX
@@ -70,7 +70,7 @@ public class MetricsMasterWrapperImpl implements MetricsMasterWrapper {
@Override
public String getZookeeperQuorum() {
- ZooKeeperWatcher zk = master.getZooKeeper();
+ ZKWatcher zk = master.getZooKeeper();
if (zk == null) {
return "";
}
@@ -100,7 +100,7 @@ public class MetricsMasterWrapperImpl implements MetricsMasterWrapper {
}
return StringUtils.join(serverManager.getOnlineServers().keySet(), ";");
}
-
+
@Override
public int getNumRegionServers() {
ServerManager serverManager = this.master.getServerManager();
@@ -119,7 +119,7 @@ public class MetricsMasterWrapperImpl implements MetricsMasterWrapper {
return StringUtils.join(serverManager.getDeadServers().copyServerNames(), ";");
}
-
+
@Override
public int getNumDeadRegionServers() {
ServerManager serverManager = this.master.getServerManager();
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
new file mode 100644
index 0000000..0e9351d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
@@ -0,0 +1,164 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Tracks the online region servers via ZK.
+ *
+ * <p>Handling of new RSs checking in is done via RPC. This class
+ * is only responsible for watching for expired nodes. It handles
+ * listening for changes in the RS node list and watching each node.
+ *
+ * <p>If an RS node gets deleted, this automatically handles calling of
+ * {@link ServerManager#expireServer(ServerName)}
+ */
+@InterfaceAudience.Private
+public class RegionServerTracker extends ZKListener {
+ private static final Log LOG = LogFactory.getLog(RegionServerTracker.class);
+ private NavigableMap<ServerName, RegionServerInfo> regionServers = new TreeMap<>();
+ private ServerManager serverManager;
+ private MasterServices server;
+
+ public RegionServerTracker(ZKWatcher watcher,
+ MasterServices server, ServerManager serverManager) {
+ super(watcher);
+ this.server = server;
+ this.serverManager = serverManager;
+ }
+
+ /**
+ * Starts the tracking of online RegionServers.
+ *
+ * <p>All RSs will be tracked after this method is called.
+ *
+ * @throws KeeperException
+ * @throws IOException
+ */
+ public void start() throws KeeperException, IOException {
+ watcher.registerListener(this);
+ List<String> servers =
+ ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.rsZNode);
+ refresh(servers);
+ }
+
+ private void refresh(final List<String> servers) throws IOException {
+ synchronized(this.regionServers) {
+ this.regionServers.clear();
+ for (String n: servers) {
+ ServerName sn = ServerName.parseServerName(ZKUtil.getNodeName(n));
+ if (regionServers.get(sn) == null) {
+ RegionServerInfo.Builder rsInfoBuilder = RegionServerInfo.newBuilder();
+ try {
+ String nodePath = ZNodePaths.joinZNode(watcher.znodePaths.rsZNode, n);
+ byte[] data = ZKUtil.getData(watcher, nodePath);
+ if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) {
+ int magicLen = ProtobufUtil.lengthOfPBMagic();
+ ProtobufUtil.mergeFrom(rsInfoBuilder, data, magicLen, data.length - magicLen);
+ }
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Added tracking of RS " + nodePath);
+ }
+ } catch (KeeperException e) {
+ LOG.warn("Get Rs info port from ephemeral node", e);
+ } catch (IOException e) {
+ LOG.warn("Illegal data from ephemeral node", e);
+ } catch (InterruptedException e) {
+ throw new InterruptedIOException();
+ }
+ this.regionServers.put(sn, rsInfoBuilder.build());
+ }
+ }
+ }
+ if (server.isInitialized()) {
+ server.checkIfShouldMoveSystemRegionAsync();
+ }
+ }
+
+ private void remove(final ServerName sn) {
+ synchronized(this.regionServers) {
+ this.regionServers.remove(sn);
+ }
+ }
+
+ @Override
+ public void nodeDeleted(String path) {
+ if (path.startsWith(watcher.znodePaths.rsZNode)) {
+ String serverName = ZKUtil.getNodeName(path);
+ LOG.info("RegionServer ephemeral node deleted, processing expiration [" +
+ serverName + "]");
+ ServerName sn = ServerName.parseServerName(serverName);
+ if (!serverManager.isServerOnline(sn)) {
+ LOG.warn(serverName.toString() + " is not online or isn't known to the master."+
+ "The latter could be caused by a DNS misconfiguration.");
+ return;
+ }
+ remove(sn);
+ this.serverManager.expireServer(sn);
+ }
+ }
+
+ @Override
+ public void nodeChildrenChanged(String path) {
+ if (path.equals(watcher.znodePaths.rsZNode)
+ && !server.isAborted() && !server.isStopped()) {
+ try {
+ List<String> servers =
+ ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.rsZNode);
+ refresh(servers);
+ } catch (IOException e) {
+ server.abort("Unexpected zk exception getting RS nodes", e);
+ } catch (KeeperException e) {
+ server.abort("Unexpected zk exception getting RS nodes", e);
+ }
+ }
+ }
+
+ public RegionServerInfo getRegionServerInfo(final ServerName sn) {
+ return regionServers.get(sn);
+ }
+
+ /**
+ * Gets the online servers.
+ * @return list of online servers
+ */
+ public List<ServerName> getOnlineServers() {
+ synchronized (this.regionServers) {
+ return new ArrayList<>(this.regionServers.keySet());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index f19995f..79ffc8a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -42,7 +42,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClockOutOfSyncException;
-import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionLoad;
@@ -59,7 +58,7 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
@@ -507,7 +506,7 @@ public class ServerManager {
void letRegionServersShutdown() {
long previousLogTime = 0;
ServerName sn = master.getServerName();
- ZooKeeperWatcher zkw = master.getZooKeeper();
+ ZKWatcher zkw = master.getZooKeeper();
int onlineServersCt;
while ((onlineServersCt = onlineServers.size()) > 0){
@@ -554,7 +553,7 @@ public class ServerManager {
}
}
- private List<String> getRegionServersInZK(final ZooKeeperWatcher zkw)
+ private List<String> getRegionServersInZK(final ZKWatcher zkw)
throws KeeperException {
return ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitOrMergeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitOrMergeTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitOrMergeTracker.java
new file mode 100644
index 0000000..55287d2
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitOrMergeTracker.java
@@ -0,0 +1,153 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.client.MasterSwitchType;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+
+
+/**
+ * Tracks the switch of split and merge states in ZK
+ *
+ */
+@InterfaceAudience.Private
+public class SplitOrMergeTracker {
+
+ private String splitZnode;
+ private String mergeZnode;
+
+ private SwitchStateTracker splitStateTracker;
+ private SwitchStateTracker mergeStateTracker;
+
+ public SplitOrMergeTracker(ZKWatcher watcher, Configuration conf,
+ Abortable abortable) {
+ try {
+ if (ZKUtil.checkExists(watcher, watcher.znodePaths.switchZNode) < 0) {
+ ZKUtil.createAndFailSilent(watcher, watcher.znodePaths.switchZNode);
+ }
+ } catch (KeeperException e) {
+ throw new RuntimeException(e);
+ }
+ splitZnode = ZNodePaths.joinZNode(watcher.znodePaths.switchZNode,
+ conf.get("zookeeper.znode.switch.split", "split"));
+ mergeZnode = ZNodePaths.joinZNode(watcher.znodePaths.switchZNode,
+ conf.get("zookeeper.znode.switch.merge", "merge"));
+ splitStateTracker = new SwitchStateTracker(watcher, splitZnode, abortable);
+ mergeStateTracker = new SwitchStateTracker(watcher, mergeZnode, abortable);
+ }
+
+ public void start() {
+ splitStateTracker.start();
+ mergeStateTracker.start();
+ }
+
+ public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) {
+ switch (switchType) {
+ case SPLIT:
+ return splitStateTracker.isSwitchEnabled();
+ case MERGE:
+ return mergeStateTracker.isSwitchEnabled();
+ default:
+ break;
+ }
+ return false;
+ }
+
+ public void setSplitOrMergeEnabled(boolean enabled, MasterSwitchType switchType)
+ throws KeeperException {
+ switch (switchType) {
+ case SPLIT:
+ splitStateTracker.setSwitchEnabled(enabled);
+ break;
+ case MERGE:
+ mergeStateTracker.setSwitchEnabled(enabled);
+ break;
+ default:
+ break;
+ }
+ }
+
+ private static class SwitchStateTracker extends ZKNodeTracker {
+
+ public SwitchStateTracker(ZKWatcher watcher, String node, Abortable abortable) {
+ super(watcher, node, abortable);
+ }
+
+ /**
+ * Return true if the switch is on, false otherwise
+ */
+ public boolean isSwitchEnabled() {
+ byte [] upData = super.getData(false);
+ try {
+ // if data in ZK is null, use default of on.
+ return upData == null || parseFrom(upData).getEnabled();
+ } catch (DeserializationException dex) {
+ LOG.error("ZK state for LoadBalancer could not be parsed " + Bytes.toStringBinary(upData));
+ // return false to be safe.
+ return false;
+ }
+ }
+
+ /**
+ * Set the switch on/off
+ * @param enabled switch enabled or not?
+ * @throws KeeperException keepException will be thrown out
+ */
+ public void setSwitchEnabled(boolean enabled) throws KeeperException {
+ byte [] upData = toByteArray(enabled);
+ try {
+ ZKUtil.setData(watcher, node, upData);
+ } catch(KeeperException.NoNodeException nne) {
+ ZKUtil.createAndWatch(watcher, node, upData);
+ }
+ super.nodeDataChanged(node);
+ }
+
+ private byte [] toByteArray(boolean enabled) {
+ SwitchState.Builder builder = SwitchState.newBuilder();
+ builder.setEnabled(enabled);
+ return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
+ }
+
+ private SwitchState parseFrom(byte [] bytes)
+ throws DeserializationException {
+ ProtobufUtil.expectPBMagicPrefix(bytes);
+ SwitchState.Builder builder = SwitchState.newBuilder();
+ try {
+ int magicLen = ProtobufUtil.lengthOfPBMagic();
+ ProtobufUtil.mergeFrom(builder, bytes, magicLen, bytes.length - magicLen);
+ } catch (IOException e) {
+ throw new DeserializationException(e);
+ }
+ return builder.build();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
index 97d3080..3f7bd74 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
@@ -30,6 +30,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.replication.ReplicationFactory;
import org.apache.hadoop.hbase.replication.ReplicationPeers;
@@ -39,7 +40,6 @@ import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -48,12 +48,12 @@ import org.apache.zookeeper.KeeperException;
@InterfaceAudience.Private
public class ReplicationZKNodeCleaner {
private static final Log LOG = LogFactory.getLog(ReplicationZKNodeCleaner.class);
- private final ZooKeeperWatcher zkw;
+ private final ZKWatcher zkw;
private final ReplicationQueuesClient queuesClient;
private final ReplicationPeers replicationPeers;
private final ReplicationQueueDeletor queueDeletor;
- public ReplicationZKNodeCleaner(Configuration conf, ZooKeeperWatcher zkw, Abortable abortable)
+ public ReplicationZKNodeCleaner(Configuration conf, ZKWatcher zkw, Abortable abortable)
throws IOException {
try {
this.zkw = zkw;
@@ -130,7 +130,7 @@ public class ReplicationZKNodeCleaner {
private class ReplicationQueueDeletor extends ReplicationStateZKBase {
- public ReplicationQueueDeletor(ZooKeeperWatcher zk, Configuration conf, Abortable abortable) {
+ public ReplicationQueueDeletor(ZKWatcher zk, Configuration conf, Abortable abortable) {
super(zk, conf, abortable);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
index df94ffe..7cfaefd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint;
import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -39,7 +40,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.replication.ReplicationPeers;
import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
/**
* Manages and performs all replication admin operations.
@@ -49,11 +49,11 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
public class ReplicationManager {
private final Configuration conf;
- private final ZooKeeperWatcher zkw;
+ private final ZKWatcher zkw;
private final ReplicationQueuesClient replicationQueuesClient;
private final ReplicationPeers replicationPeers;
- public ReplicationManager(Configuration conf, ZooKeeperWatcher zkw, Abortable abortable)
+ public ReplicationManager(Configuration conf, ZKWatcher zkw, Abortable abortable)
throws IOException {
this.conf = conf;
this.zkw = zkw;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
index 6985591..609ce8e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
@@ -24,12 +24,12 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -41,7 +41,7 @@ public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs {
private ZKProcedureUtil zkProc = null;
protected ProcedureCoordinator coordinator = null; // if started this should be non-null
- ZooKeeperWatcher watcher;
+ ZKWatcher watcher;
String procedureType;
String coordName;
@@ -52,7 +52,7 @@ public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs {
* @param coordName name of the node running the coordinator
* @throws KeeperException if an unexpected zk error occurs
*/
- public ZKProcedureCoordinator(ZooKeeperWatcher watcher,
+ public ZKProcedureCoordinator(ZKWatcher watcher,
String procedureClass, String coordName) {
this.watcher = watcher;
this.procedureType = procedureClass;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
index 36f4f44..45e6760 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
@@ -23,13 +23,13 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -61,12 +61,12 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs {
/**
* Must call {@link #start(String, ProcedureMember)} before this can be used.
- * @param watcher {@link ZooKeeperWatcher} to be owned by <tt>this</tt>. Closed via
+ * @param watcher {@link ZKWatcher} to be owned by <tt>this</tt>. Closed via
* {@link #close()}.
* @param procType name of the znode describing the procedure type
* @throws KeeperException if we can't reach zookeeper
*/
- public ZKProcedureMemberRpcs(final ZooKeeperWatcher watcher, final String procType)
+ public ZKProcedureMemberRpcs(final ZKWatcher watcher, final String procType)
throws KeeperException {
this.zkController = new ZKProcedureUtil(watcher, procType) {
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java
index 24693cd..0349290 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java
@@ -23,11 +23,11 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -50,7 +50,7 @@ import org.apache.zookeeper.KeeperException;
*/
@InterfaceAudience.Private
public abstract class ZKProcedureUtil
- extends ZooKeeperListener implements Closeable {
+ extends ZKListener implements Closeable {
private static final Log LOG = LogFactory.getLog(ZKProcedureUtil.class);
@@ -73,7 +73,7 @@ public abstract class ZKProcedureUtil
* @param procDescription name of the znode describing the procedure to run
* @throws KeeperException when the procedure znodes cannot be created
*/
- public ZKProcedureUtil(ZooKeeperWatcher watcher, String procDescription)
+ public ZKProcedureUtil(ZKWatcher watcher, String procDescription)
throws KeeperException {
super(watcher);
// make sure we are listening for events
@@ -157,7 +157,7 @@ public abstract class ZKProcedureUtil
return ZNodePaths.joinZNode(controller.abortZnode, opInstanceName);
}
- public ZooKeeperWatcher getWatcher() {
+ public ZKWatcher getWatcher() {
return watcher;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
index ea34714..d328561 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
@@ -48,9 +48,8 @@ import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -320,7 +319,7 @@ public class RegionServerFlushTableProcedureManager extends RegionServerProcedur
@Override
public void initialize(RegionServerServices rss) throws KeeperException {
this.rss = rss;
- ZooKeeperWatcher zkw = rss.getZooKeeper();
+ ZKWatcher zkw = rss.getZooKeeper();
this.memberRpcs = new ZKProcedureMemberRpcs(zkw,
MasterFlushTableProcedureManager.FLUSH_TABLE_PROCEDURE_SIGNATURE);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 4c34fe0..03657c1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -162,10 +162,10 @@ import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
+import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.util.ReflectionUtils;
@@ -380,7 +380,7 @@ public class HRegionServer extends HasThread implements
final AtomicBoolean online = new AtomicBoolean(false);
// zookeeper connection and watcher
- protected ZooKeeperWatcher zooKeeper;
+ protected ZKWatcher zooKeeper;
// master address tracker
private MasterAddressTracker masterAddressTracker;
@@ -616,7 +616,7 @@ public class HRegionServer extends HasThread implements
// Some unit tests don't need a cluster, so no zookeeper at all
if (!conf.getBoolean("hbase.testing.nocluster", false)) {
// Open connection to zookeeper and set primary watcher
- zooKeeper = new ZooKeeperWatcher(conf, getProcessName() + ":" +
+ zooKeeper = new ZKWatcher(conf, getProcessName() + ":" +
rpcServices.isa.getPort(), this, canCreateBaseZNode());
// If no master in cluster, skip trying to track one or look for a cluster status.
@@ -905,7 +905,7 @@ public class HRegionServer extends HasThread implements
* @throws IOException any IO exception, plus if the RS is stopped
* @throws InterruptedException
*/
- private void blockAndCheckIfStopped(ZooKeeperNodeTracker tracker)
+ private void blockAndCheckIfStopped(ZKNodeTracker tracker)
throws IOException, InterruptedException {
while (tracker.blockUntilAvailable(this.msgInterval, false) == null) {
if (this.stopped) {
@@ -2886,7 +2886,7 @@ public class HRegionServer extends HasThread implements
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return zooKeeper;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
index 7d7833b..75d8e56 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.WALProvider;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hdfs.DFSHedgedReadMetrics;
import org.apache.hadoop.metrics2.MetricsExecutor;
import org.apache.yetus.audience.InterfaceAudience;
@@ -187,7 +187,7 @@ class MetricsRegionServerWrapperImpl
@Override
public String getZookeeperQuorum() {
- ZooKeeperWatcher zk = regionServer.getZooKeeper();
+ ZKWatcher zk = regionServer.getZooKeeper();
if (zk == null) {
return "";
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
index a4b4387..6a7d83e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.DaemonThreadFactory;
import org.apache.hadoop.hbase.DroppedSnapshotException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@@ -53,11 +54,9 @@ import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -394,7 +393,7 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager {
@Override
public void initialize(RegionServerServices rss) throws KeeperException {
this.rss = rss;
- ZooKeeperWatcher zkw = rss.getZooKeeper();
+ ZKWatcher zkw = rss.getZooKeeper();
this.memberRpcs = new ZKProcedureMemberRpcs(zkw,
SnapshotManager.ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
index a34bedd..4985b82 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
@@ -26,13 +26,13 @@ import java.util.UUID;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.AuthFailedException;
import org.apache.zookeeper.KeeperException.ConnectionLossException;
@@ -50,7 +50,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
private static final Log LOG = LogFactory.getLog(HBaseReplicationEndpoint.class);
- private ZooKeeperWatcher zkw = null; // FindBugs: MT_CORRECTNESS
+ private ZKWatcher zkw = null; // FindBugs: MT_CORRECTNESS
private List<ServerName> regionServers = new ArrayList<>(0);
private long lastRegionServerUpdate;
@@ -123,7 +123,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
* Get the ZK connection to this peer
* @return zk connection
*/
- protected ZooKeeperWatcher getZkw() {
+ protected ZKWatcher getZkw() {
return zkw;
}
@@ -133,7 +133,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
*/
void reloadZkWatcher() throws IOException {
if (zkw != null) zkw.close();
- zkw = new ZooKeeperWatcher(ctx.getConfiguration(),
+ zkw = new ZKWatcher(ctx.getConfiguration(),
"connection to cluster: " + ctx.getPeerId(), this);
getZkw().registerListener(new PeerRegionServerListener(this));
}
@@ -155,7 +155,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
* @param zkw zk connection to use
* @return list of region server addresses or an empty list if the slave is unavailable
*/
- protected static List<ServerName> fetchSlavesAddresses(ZooKeeperWatcher zkw)
+ protected static List<ServerName> fetchSlavesAddresses(ZKWatcher zkw)
throws KeeperException {
List<String> children = ZKUtil.listChildrenAndWatchForNewChildren(zkw, zkw.znodePaths.rsZNode);
if (children == null) {
@@ -210,7 +210,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
/**
* Tracks changes to the list of region servers in a peer's cluster.
*/
- public static class PeerRegionServerListener extends ZooKeeperListener {
+ public static class PeerRegionServerListener extends ZKListener {
private final HBaseReplicationEndpoint replicationEndpoint;
private final String regionServerListNode;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
index a14bd01..5972734 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
@@ -28,14 +28,13 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.replication.ReplicationFactory;
import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -45,7 +44,7 @@ import org.apache.zookeeper.KeeperException;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
private static final Log LOG = LogFactory.getLog(ReplicationHFileCleaner.class);
- private ZooKeeperWatcher zkw;
+ private ZKWatcher zkw;
private ReplicationQueuesClient rqc;
private boolean stopped = false;
@@ -130,14 +129,14 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
// I can close myself when time comes.
Configuration conf = new Configuration(config);
try {
- setConf(conf, new ZooKeeperWatcher(conf, "replicationHFileCleaner", null));
+ setConf(conf, new ZKWatcher(conf, "replicationHFileCleaner", null));
} catch (IOException e) {
LOG.error("Error while configuring " + this.getClass().getName(), e);
}
}
@VisibleForTesting
- public void setConf(Configuration conf, ZooKeeperWatcher zk) {
+ public void setConf(Configuration conf, ZKWatcher zk) {
super.setConf(conf);
try {
initReplicationQueuesClient(conf, zk);
@@ -146,7 +145,7 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
}
}
- private void initReplicationQueuesClient(Configuration conf, ZooKeeperWatcher zk)
+ private void initReplicationQueuesClient(Configuration conf, ZKWatcher zk)
throws Exception {
this.zkw = zk;
this.rqc = ReplicationFactory.getReplicationQueuesClient(new ReplicationQueuesClientArguments(
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
index 3dcb332..57ed842 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.replication.master;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@@ -30,7 +31,6 @@ import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
import org.apache.hadoop.hbase.replication.ReplicationFactory;
import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import java.io.IOException;
import java.util.Collections;
@@ -48,7 +48,7 @@ import org.apache.zookeeper.KeeperException;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
private static final Log LOG = LogFactory.getLog(ReplicationLogCleaner.class);
- private ZooKeeperWatcher zkw;
+ private ZKWatcher zkw;
private ReplicationQueuesClient replicationQueues;
private boolean stopped = false;
private Set<String> wals;
@@ -101,14 +101,14 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
// I can close myself when comes time.
Configuration conf = new Configuration(config);
try {
- setConf(conf, new ZooKeeperWatcher(conf, "replicationLogCleaner", null));
+ setConf(conf, new ZKWatcher(conf, "replicationLogCleaner", null));
} catch (IOException e) {
LOG.error("Error while configuring " + this.getClass().getName(), e);
}
}
@VisibleForTesting
- public void setConf(Configuration conf, ZooKeeperWatcher zk) {
+ public void setConf(Configuration conf, ZKWatcher zk) {
super.setConf(conf);
try {
this.zkw = zk;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
index d094d1c..f442495 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.zookeeper.KeeperException;
@@ -47,7 +47,7 @@ public class TableCFsUpdater extends ReplicationStateZKBase {
private static final Log LOG = LogFactory.getLog(TableCFsUpdater.class);
- public TableCFsUpdater(ZooKeeperWatcher zookeeper,
+ public TableCFsUpdater(ZKWatcher zookeeper,
Configuration conf, Abortable abortable) {
super(zookeeper, conf, abortable);
}
@@ -137,7 +137,7 @@ public class TableCFsUpdater extends ReplicationStateZKBase {
printUsageAndExit();
} else if (args[0].equals("update")) {
Configuration conf = HBaseConfiguration.create();
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "TableCFsUpdater", null);
+ ZKWatcher zkw = new ZKWatcher(conf, "TableCFsUpdater", null);
try {
TableCFsUpdater tableCFsUpdater = new TableCFsUpdater(zkw, conf, null);
tableCFsUpdater.update();
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
index 9d38026..fdb5559 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
import org.apache.hadoop.hbase.replication.ReplicationTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.zookeeper.KeeperException;
@@ -211,7 +211,7 @@ public class DumpReplicationQueues extends Configured implements Tool {
ClusterConnection connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
Admin admin = connection.getAdmin();
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "DumpReplicationQueues" + System.currentTimeMillis(),
+ ZKWatcher zkw = new ZKWatcher(conf, "DumpReplicationQueues" + System.currentTimeMillis(),
new WarnOnlyAbortable(), true);
try {
@@ -302,8 +302,8 @@ public class DumpReplicationQueues extends Configured implements Tool {
return sb.toString();
}
- public String dumpQueues(ClusterConnection connection, ZooKeeperWatcher zkw, Set<String> peerIds,
- boolean hdfs) throws Exception {
+ public String dumpQueues(ClusterConnection connection, ZKWatcher zkw, Set<String> peerIds,
+ boolean hdfs) throws Exception {
ReplicationQueuesClient queuesClient;
ReplicationPeers replicationPeers;
ReplicationQueues replicationQueues;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index 280289c..233159e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@@ -77,7 +77,7 @@ public class ReplicationSyncUp extends Configured implements Tool {
ReplicationSourceManager manager;
FileSystem fs;
Path oldLogDir, logDir, walRootDir;
- ZooKeeperWatcher zkw;
+ ZKWatcher zkw;
Abortable abortable = new Abortable() {
@Override
@@ -91,7 +91,7 @@ public class ReplicationSyncUp extends Configured implements Tool {
};
zkw =
- new ZooKeeperWatcher(conf, "syncupReplication" + System.currentTimeMillis(), abortable,
+ new ZKWatcher(conf, "syncupReplication" + System.currentTimeMillis(), abortable,
true);
walRootDir = FSUtils.getWALRootDir(conf);
@@ -123,9 +123,9 @@ public class ReplicationSyncUp extends Configured implements Tool {
static class DummyServer implements Server {
String hostname;
- ZooKeeperWatcher zkw;
+ ZKWatcher zkw;
- DummyServer(ZooKeeperWatcher zkw) {
+ DummyServer(ZKWatcher zkw) {
// an unique name in case the first run fails
hostname = System.currentTimeMillis() + ".SyncUpTool.replication.org";
this.zkw = zkw;
@@ -141,7 +141,7 @@ public class ReplicationSyncUp extends Configured implements Tool {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return zkw;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 6ee9f60..05f9195 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -130,7 +130,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
@@ -953,7 +953,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
+ " accordingly.");
}
- ZooKeeperWatcher zk = null;
+ ZKWatcher zk = null;
if (env instanceof MasterCoprocessorEnvironment) {
// if running on HMaster
MasterCoprocessorEnvironment mEnv = (MasterCoprocessorEnvironment)env;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
index 0db05c2..2ba4ac5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
@@ -39,13 +39,13 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.AuthUtil;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.security.Superusers;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -112,7 +112,7 @@ public class TableAuthManager implements Closeable {
private ZKPermissionWatcher zkperms;
private final AtomicLong mtime = new AtomicLong(0L);
- private TableAuthManager(ZooKeeperWatcher watcher, Configuration conf)
+ private TableAuthManager(ZKWatcher watcher, Configuration conf)
throws IOException {
this.conf = conf;
@@ -734,14 +734,14 @@ public class TableAuthManager implements Closeable {
return mtime.get();
}
- private static Map<ZooKeeperWatcher,TableAuthManager> managerMap = new HashMap<>();
+ private static Map<ZKWatcher,TableAuthManager> managerMap = new HashMap<>();
private static Map<TableAuthManager, Integer> refCount = new HashMap<>();
/** Returns a TableAuthManager from the cache. If not cached, constructs a new one. Returned
* instance should be released back by calling {@link #release(TableAuthManager)}. */
public synchronized static TableAuthManager getOrCreate(
- ZooKeeperWatcher watcher, Configuration conf) throws IOException {
+ ZKWatcher watcher, Configuration conf) throws IOException {
TableAuthManager instance = managerMap.get(watcher);
if (instance == null) {
instance = new TableAuthManager(watcher, conf);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
index 447ba51..d45b5b5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
@@ -20,15 +20,15 @@ package org.apache.hadoop.hbase.security.access;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.DaemonThreadFactory;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import java.io.Closeable;
@@ -41,7 +41,6 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.atomic.AtomicReference;
/**
* Handles synchronization of access control list entries and updates
@@ -53,7 +52,7 @@ import java.util.concurrent.atomic.AtomicReference;
* trigger updates in the {@link TableAuthManager} permission cache.
*/
@InterfaceAudience.Private
-public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable {
+public class ZKPermissionWatcher extends ZKListener implements Closeable {
private static final Log LOG = LogFactory.getLog(ZKPermissionWatcher.class);
// parent node for permissions lists
static final String ACL_NODE = "acl";
@@ -63,7 +62,7 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable
private final ExecutorService executor;
private Future<?> childrenChangedFuture;
- public ZKPermissionWatcher(ZooKeeperWatcher watcher,
+ public ZKPermissionWatcher(ZKWatcher watcher,
TableAuthManager authManager, Configuration conf) {
super(watcher);
this.authManager = authManager;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java
index 27ea509..aa6b1e9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java
@@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Stoppable;
@@ -35,7 +36,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKLeaderManager;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.Token;
@@ -89,8 +89,8 @@ public class AuthenticationTokenSecretManager
* org.apache.hadoop.hbase.ipc.SecureServer so public access is needed.
*/
public AuthenticationTokenSecretManager(Configuration conf,
- ZooKeeperWatcher zk, String serverName,
- long keyUpdateInterval, long tokenMaxLifetime) {
+ ZKWatcher zk, String serverName,
+ long keyUpdateInterval, long tokenMaxLifetime) {
this.zkWatcher = new ZKSecretWatcher(conf, zk, this);
this.keyUpdateInterval = keyUpdateInterval;
this.tokenMaxLifetime = tokenMaxLifetime;
@@ -144,9 +144,9 @@ public class AuthenticationTokenSecretManager
AuthenticationKey masterKey = allKeys.get(identifier.getKeyId());
if(masterKey == null) {
if(zkWatcher.getWatcher().isAborted()) {
- LOG.error("ZooKeeperWatcher is abort");
+ LOG.error("ZKWatcher is abort");
throw new InvalidToken("Token keys could not be sync from zookeeper"
- + " because of ZooKeeperWatcher abort");
+ + " because of ZKWatcher abort");
}
synchronized (this) {
if (!leaderElector.isAlive() || leaderElector.isStopped()) {
@@ -254,7 +254,7 @@ public class AuthenticationTokenSecretManager
}
}
}
-
+
synchronized boolean isCurrentKeyRolled() {
return currentKey != null;
}
@@ -297,7 +297,7 @@ public class AuthenticationTokenSecretManager
private boolean isMaster = false;
private ZKLeaderManager zkLeader;
- public LeaderElector(ZooKeeperWatcher watcher, String serverName) {
+ public LeaderElector(ZKWatcher watcher, String serverName) {
setDaemon(true);
setName("ZKSecretWatcher-leaderElector");
zkLeader = new ZKLeaderManager(watcher,
[10/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper
from hbase-server and hbase-client
Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java
deleted file mode 100644
index a5b084b..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java
+++ /dev/null
@@ -1,251 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Tracks the availability and value of a single ZooKeeper node.
- *
- * <p>Utilizes the {@link ZooKeeperListener} interface to get the necessary
- * ZooKeeper events related to the node.
- *
- * <p>This is the base class used by trackers in both the Master and
- * RegionServers.
- */
-@InterfaceAudience.Private
-public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
- // LOG is being used in subclasses, hence keeping it protected
- protected static final Log LOG = LogFactory.getLog(ZooKeeperNodeTracker.class);
- /** Path of node being tracked */
- protected final String node;
-
- /** Data of the node being tracked */
- private byte [] data;
-
- /** Used to abort if a fatal error occurs */
- protected final Abortable abortable;
-
- private boolean stopped = false;
-
- /**
- * Constructs a new ZK node tracker.
- *
- * <p>After construction, use {@link #start} to kick off tracking.
- *
- * @param watcher
- * @param node
- * @param abortable
- */
- public ZooKeeperNodeTracker(ZooKeeperWatcher watcher, String node,
- Abortable abortable) {
- super(watcher);
- this.node = node;
- this.abortable = abortable;
- this.data = null;
- }
-
- /**
- * Starts the tracking of the node in ZooKeeper.
- *
- * <p>Use {@link #blockUntilAvailable()} to block until the node is available
- * or {@link #getData(boolean)} to get the data of the node if it is available.
- */
- public synchronized void start() {
- this.watcher.registerListener(this);
- try {
- if(ZKUtil.watchAndCheckExists(watcher, node)) {
- byte [] data = ZKUtil.getDataAndWatch(watcher, node);
- if(data != null) {
- this.data = data;
- } else {
- // It existed but now does not, try again to ensure a watch is set
- LOG.debug("Try starting again because there is no data from " + node);
- start();
- }
- }
- } catch (KeeperException e) {
- abortable.abort("Unexpected exception during initialization, aborting", e);
- }
- }
-
- public synchronized void stop() {
- this.stopped = true;
- notifyAll();
- }
-
- /**
- * Gets the data of the node, blocking until the node is available.
- *
- * @return data of the node
- * @throws InterruptedException if the waiting thread is interrupted
- */
- public synchronized byte [] blockUntilAvailable()
- throws InterruptedException {
- return blockUntilAvailable(0, false);
- }
-
- /**
- * Gets the data of the node, blocking until the node is available or the
- * specified timeout has elapsed.
- *
- * @param timeout maximum time to wait for the node data to be available,
- * n milliseconds. Pass 0 for no timeout.
- * @return data of the node
- * @throws InterruptedException if the waiting thread is interrupted
- */
- public synchronized byte [] blockUntilAvailable(long timeout, boolean refresh)
- throws InterruptedException {
- if (timeout < 0) throw new IllegalArgumentException();
- boolean notimeout = timeout == 0;
- long startTime = System.currentTimeMillis();
- long remaining = timeout;
- if (refresh) {
- try {
- // This does not create a watch if the node does not exists
- this.data = ZKUtil.getDataAndWatch(watcher, node);
- } catch(KeeperException e) {
- // We use to abort here, but in some cases the abort is ignored (
- // (empty Abortable), so it's better to log...
- LOG.warn("Unexpected exception handling blockUntilAvailable", e);
- abortable.abort("Unexpected exception handling blockUntilAvailable", e);
- }
- }
- boolean nodeExistsChecked = (!refresh ||data!=null);
- while (!this.stopped && (notimeout || remaining > 0) && this.data == null) {
- if (!nodeExistsChecked) {
- try {
- nodeExistsChecked = (ZKUtil.checkExists(watcher, node) != -1);
- } catch (KeeperException e) {
- LOG.warn(
- "Got exception while trying to check existence in ZooKeeper" +
- " of the node: "+node+", retrying if timeout not reached",e );
- }
-
- // It did not exists, and now it does.
- if (nodeExistsChecked){
- LOG.debug("Node " + node + " now exists, resetting a watcher");
- try {
- // This does not create a watch if the node does not exists
- this.data = ZKUtil.getDataAndWatch(watcher, node);
- } catch (KeeperException e) {
- LOG.warn("Unexpected exception handling blockUntilAvailable", e);
- abortable.abort("Unexpected exception handling blockUntilAvailable", e);
- }
- }
- }
- // We expect a notification; but we wait with a
- // a timeout to lower the impact of a race condition if any
- wait(100);
- remaining = timeout - (System.currentTimeMillis() - startTime);
- }
- return this.data;
- }
-
- /**
- * Gets the data of the node.
- *
- * <p>If the node is currently available, the most up-to-date known version of
- * the data is returned. If the node is not currently available, null is
- * returned.
- * @param refresh whether to refresh the data by calling ZK directly.
- * @return data of the node, null if unavailable
- */
- public synchronized byte [] getData(boolean refresh) {
- if (refresh) {
- try {
- this.data = ZKUtil.getDataAndWatch(watcher, node);
- } catch(KeeperException e) {
- abortable.abort("Unexpected exception handling getData", e);
- }
- }
- return this.data;
- }
-
- public String getNode() {
- return this.node;
- }
-
- @Override
- public synchronized void nodeCreated(String path) {
- if (!path.equals(node)) return;
- try {
- byte [] data = ZKUtil.getDataAndWatch(watcher, node);
- if (data != null) {
- this.data = data;
- notifyAll();
- } else {
- nodeDeleted(path);
- }
- } catch(KeeperException e) {
- abortable.abort("Unexpected exception handling nodeCreated event", e);
- }
- }
-
- @Override
- public synchronized void nodeDeleted(String path) {
- if(path.equals(node)) {
- try {
- if(ZKUtil.watchAndCheckExists(watcher, node)) {
- nodeCreated(path);
- } else {
- this.data = null;
- }
- } catch(KeeperException e) {
- abortable.abort("Unexpected exception handling nodeDeleted event", e);
- }
- }
- }
-
- @Override
- public synchronized void nodeDataChanged(String path) {
- if(path.equals(node)) {
- nodeCreated(path);
- }
- }
-
- /**
- * Checks if the baseznode set as per the property 'zookeeper.znode.parent'
- * exists.
- * @return true if baseznode exists.
- * false if doesnot exists.
- */
- public boolean checkIfBaseNodeAvailable() {
- try {
- if (ZKUtil.checkExists(watcher, watcher.znodePaths.baseZNode) == -1) {
- return false;
- }
- } catch (KeeperException e) {
- abortable.abort("Exception while checking if basenode (" + watcher.znodePaths.baseZNode
- + ") exists in ZooKeeper.",
- e);
- }
- return true;
- }
-
- @Override
- public String toString() {
- return "ZooKeeperNodeTracker{" +
- "node='" + node + ", stopped=" + stopped + '}';
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
deleted file mode 100644
index 5d10cdf..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
+++ /dev/null
@@ -1,638 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.CountDownLatch;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.AuthUtil;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.security.Superusers;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.ZooDefs.Perms;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Id;
-import org.apache.zookeeper.data.Stat;
-
-/**
- * Acts as the single ZooKeeper Watcher. One instance of this is instantiated
- * for each Master, RegionServer, and client process.
- *
- * <p>This is the only class that implements {@link Watcher}. Other internal
- * classes which need to be notified of ZooKeeper events must register with
- * the local instance of this watcher via {@link #registerListener}.
- *
- * <p>This class also holds and manages the connection to ZooKeeper. Code to
- * deal with connection related events and exceptions are handled here.
- */
-@InterfaceAudience.Private
-public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
- private static final Log LOG = LogFactory.getLog(ZooKeeperWatcher.class);
-
- // Identifier for this watcher (for logging only). It is made of the prefix
- // passed on construction and the zookeeper sessionid.
- private String prefix;
- private String identifier;
-
- // zookeeper quorum
- private String quorum;
-
- // zookeeper connection
- private final RecoverableZooKeeper recoverableZooKeeper;
-
- // abortable in case of zk failure
- protected Abortable abortable;
- // Used if abortable is null
- private boolean aborted = false;
-
- public final ZNodePaths znodePaths;
-
- // listeners to be notified
- private final List<ZooKeeperListener> listeners = new CopyOnWriteArrayList<>();
-
- // Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL
- // negotiation to complete
- public CountDownLatch saslLatch = new CountDownLatch(1);
-
-
-
- private final Configuration conf;
-
- /* A pattern that matches a Kerberos name, borrowed from Hadoop's KerberosName */
- private static final Pattern NAME_PATTERN = Pattern.compile("([^/@]*)(/([^/@]*))?@([^/@]*)");
-
- /**
- * Instantiate a ZooKeeper connection and watcher.
- * @param identifier string that is passed to RecoverableZookeeper to be used as
- * identifier for this instance. Use null for default.
- * @throws IOException
- * @throws ZooKeeperConnectionException
- */
- public ZooKeeperWatcher(Configuration conf, String identifier,
- Abortable abortable) throws ZooKeeperConnectionException, IOException {
- this(conf, identifier, abortable, false);
- }
-
- /**
- * Instantiate a ZooKeeper connection and watcher.
- * @param conf
- * @param identifier string that is passed to RecoverableZookeeper to be used as identifier for
- * 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
- * context.
- * @param canCreateBaseZNode
- * @throws IOException
- * @throws ZooKeeperConnectionException
- */
- public ZooKeeperWatcher(Configuration conf, String identifier,
- Abortable abortable, boolean canCreateBaseZNode)
- throws IOException, ZooKeeperConnectionException {
- this.conf = conf;
- this.quorum = ZKConfig.getZKQuorumServersString(conf);
- this.prefix = identifier;
- // Identifier will get the sessionid appended later below down when we
- // handle the syncconnect event.
- this.identifier = identifier + "0x0";
- this.abortable = abortable;
- this.znodePaths = new ZNodePaths(conf);
- PendingWatcher pendingWatcher = new PendingWatcher();
- this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, pendingWatcher, identifier);
- pendingWatcher.prepare(this);
- if (canCreateBaseZNode) {
- try {
- createBaseZNodes();
- } catch (ZooKeeperConnectionException zce) {
- try {
- this.recoverableZooKeeper.close();
- } catch (InterruptedException ie) {
- LOG.debug("Encountered InterruptedException when closing " + this.recoverableZooKeeper);
- Thread.currentThread().interrupt();
- }
- throw zce;
- }
- }
- }
-
- private void createBaseZNodes() throws ZooKeeperConnectionException {
- try {
- // Create all the necessary "directories" of znodes
- ZKUtil.createWithParents(this, znodePaths.baseZNode);
- ZKUtil.createAndFailSilent(this, znodePaths.rsZNode);
- ZKUtil.createAndFailSilent(this, znodePaths.drainingZNode);
- ZKUtil.createAndFailSilent(this, znodePaths.tableZNode);
- ZKUtil.createAndFailSilent(this, znodePaths.splitLogZNode);
- ZKUtil.createAndFailSilent(this, znodePaths.backupMasterAddressesZNode);
- ZKUtil.createAndFailSilent(this, znodePaths.tableLockZNode);
- ZKUtil.createAndFailSilent(this, znodePaths.masterMaintZNode);
- } catch (KeeperException e) {
- throw new ZooKeeperConnectionException(
- prefix("Unexpected KeeperException creating base node"), e);
- }
- }
-
- /** Returns whether the znode is supposed to be readable by the client
- * and DOES NOT contain sensitive information (world readable).*/
- public boolean isClientReadable(String node) {
- // Developer notice: These znodes are world readable. DO NOT add more znodes here UNLESS
- // all clients need to access this data to work. Using zk for sharing data to clients (other
- // than service lookup case is not a recommended design pattern.
- return
- node.equals(znodePaths.baseZNode) ||
- znodePaths.isAnyMetaReplicaZNode(node) ||
- node.equals(znodePaths.masterAddressZNode) ||
- node.equals(znodePaths.clusterIdZNode)||
- node.equals(znodePaths.rsZNode) ||
- // /hbase/table and /hbase/table/foo is allowed, /hbase/table-lock is not
- node.equals(znodePaths.tableZNode) ||
- node.startsWith(znodePaths.tableZNode + "/");
- }
-
- /**
- * On master start, we check the znode ACLs under the root directory and set the ACLs properly
- * if needed. If the cluster goes from an unsecure setup to a secure setup, this step is needed
- * so that the existing znodes created with open permissions are now changed with restrictive
- * perms.
- */
- public void checkAndSetZNodeAcls() {
- if (!ZKUtil.isSecureZooKeeper(getConfiguration())) {
- LOG.info("not a secure deployment, proceeding");
- return;
- }
-
- // Check the base znodes permission first. Only do the recursion if base znode's perms are not
- // correct.
- try {
- List<ACL> actualAcls = recoverableZooKeeper.getAcl(znodePaths.baseZNode, new Stat());
-
- if (!isBaseZnodeAclSetup(actualAcls)) {
- LOG.info("setting znode ACLs");
- setZnodeAclsRecursive(znodePaths.baseZNode);
- }
- } catch(KeeperException.NoNodeException nne) {
- return;
- } catch(InterruptedException ie) {
- interruptedExceptionNoThrow(ie, false);
- } catch (IOException|KeeperException e) {
- LOG.warn("Received exception while checking and setting zookeeper ACLs", e);
- }
- }
-
- /**
- * 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.
- * @param znode
- */
- private void setZnodeAclsRecursive(String znode) throws KeeperException, InterruptedException {
- List<String> children = recoverableZooKeeper.getChildren(znode, false);
-
- for (String child : children) {
- setZnodeAclsRecursive(ZNodePaths.joinZNode(znode, child));
- }
- List<ACL> acls = ZKUtil.createACL(this, znode, true);
- LOG.info("Setting ACLs for znode:" + znode + " , acl:" + acls);
- recoverableZooKeeper.setAcl(znode, acls, -1);
- }
-
- /**
- * Checks whether the ACLs returned from the base znode (/hbase) is set for secure setup.
- * @param acls acls from zookeeper
- * @return whether ACLs are set for the base znode
- * @throws IOException
- */
- private boolean isBaseZnodeAclSetup(List<ACL> acls) throws IOException {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Checking znode ACLs");
- }
- String[] superUsers = conf.getStrings(Superusers.SUPERUSER_CONF_KEY);
- // Check whether ACL set for all superusers
- if (superUsers != null && !checkACLForSuperUsers(superUsers, acls)) {
- return false;
- }
-
- // this assumes that current authenticated user is the same as zookeeper client user
- // configured via JAAS
- String hbaseUser = UserGroupInformation.getCurrentUser().getShortUserName();
-
- if (acls.isEmpty()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("ACL is empty");
- }
- return false;
- }
-
- for (ACL acl : acls) {
- int perms = acl.getPerms();
- Id id = acl.getId();
- // We should only set at most 3 possible ACLs for 3 Ids. One for everyone, one for superuser
- // and one for the hbase user
- if (Ids.ANYONE_ID_UNSAFE.equals(id)) {
- if (perms != Perms.READ) {
- if (LOG.isDebugEnabled()) {
- LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
- id, perms, Perms.READ));
- }
- return false;
- }
- } else if (superUsers != null && isSuperUserId(superUsers, id)) {
- if (perms != Perms.ALL) {
- if (LOG.isDebugEnabled()) {
- LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
- id, perms, Perms.ALL));
- }
- return false;
- }
- } else if ("sasl".equals(id.getScheme())) {
- String name = id.getId();
- // If ZooKeeper recorded the Kerberos full name in the ACL, use only the shortname
- Matcher match = NAME_PATTERN.matcher(name);
- if (match.matches()) {
- name = match.group(1);
- }
- if (name.equals(hbaseUser)) {
- if (perms != Perms.ALL) {
- if (LOG.isDebugEnabled()) {
- LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
- id, perms, Perms.ALL));
- }
- return false;
- }
- } else {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Unexpected shortname in SASL ACL: " + id);
- }
- return false;
- }
- } else {
- if (LOG.isDebugEnabled()) {
- LOG.debug("unexpected ACL id '" + id + "'");
- }
- return false;
- }
- }
- return true;
- }
-
- /*
- * Validate whether ACL set for all superusers.
- */
- private boolean checkACLForSuperUsers(String[] superUsers, List<ACL> acls) {
- for (String user : superUsers) {
- boolean hasAccess = false;
- // TODO: Validate super group members also when ZK supports setting node ACL for groups.
- if (!AuthUtil.isGroupPrincipal(user)) {
- for (ACL acl : acls) {
- if (user.equals(acl.getId().getId())) {
- if (acl.getPerms() == Perms.ALL) {
- hasAccess = true;
- } else {
- if (LOG.isDebugEnabled()) {
- LOG.debug(String.format(
- "superuser '%s' does not have correct permissions: have 0x%x, want 0x%x",
- acl.getId().getId(), acl.getPerms(), Perms.ALL));
- }
- }
- break;
- }
- }
- if (!hasAccess) {
- return false;
- }
- }
- }
- return true;
- }
-
- /*
- * Validate whether ACL ID is superuser.
- */
- public static boolean isSuperUserId(String[] superUsers, Id id) {
- for (String user : superUsers) {
- // TODO: Validate super group members also when ZK supports setting node ACL for groups.
- if (!AuthUtil.isGroupPrincipal(user) && new Id("sasl", user).equals(id)) {
- return true;
- }
- }
- return false;
- }
-
- @Override
- public String toString() {
- return this.identifier + ", quorum=" + quorum + ", baseZNode=" + znodePaths.baseZNode;
- }
-
- /**
- * Adds this instance's identifier as a prefix to the passed <code>str</code>
- * @param str String to amend.
- * @return A new string with this instance's identifier as prefix: e.g.
- * if passed 'hello world', the returned string could be
- */
- public String prefix(final String str) {
- return this.toString() + " " + str;
- }
-
- /**
- * Get the znodes corresponding to the meta replicas from ZK
- * @return list of znodes
- * @throws KeeperException
- */
- public List<String> getMetaReplicaNodes() throws KeeperException {
- List<String> childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, znodePaths.baseZNode);
- List<String> metaReplicaNodes = new ArrayList<>(2);
- if (childrenOfBaseNode != null) {
- String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server");
- for (String child : childrenOfBaseNode) {
- if (child.startsWith(pattern)) metaReplicaNodes.add(child);
- }
- }
- return metaReplicaNodes;
- }
-
- /**
- * Register the specified listener to receive ZooKeeper events.
- * @param listener
- */
- public void registerListener(ZooKeeperListener listener) {
- listeners.add(listener);
- }
-
- /**
- * Register the specified listener to receive ZooKeeper events and add it as
- * the first in the list of current listeners.
- * @param listener
- */
- public void registerListenerFirst(ZooKeeperListener listener) {
- listeners.add(0, listener);
- }
-
- public void unregisterListener(ZooKeeperListener listener) {
- listeners.remove(listener);
- }
-
- /**
- * Clean all existing listeners
- */
- public void unregisterAllListeners() {
- listeners.clear();
- }
-
- /**
- * Get a copy of current registered listeners
- */
- public List<ZooKeeperListener> getListeners() {
- return new ArrayList<>(listeners);
- }
-
- /**
- * @return The number of currently registered listeners
- */
- public int getNumberOfListeners() {
- return listeners.size();
- }
-
- /**
- * Get the connection to ZooKeeper.
- * @return connection reference to zookeeper
- */
- public RecoverableZooKeeper getRecoverableZooKeeper() {
- return recoverableZooKeeper;
- }
-
- public void reconnectAfterExpiration() throws IOException, KeeperException, InterruptedException {
- recoverableZooKeeper.reconnectAfterExpiration();
- }
-
- /**
- * Get the quorum address of this instance.
- * @return quorum string of this zookeeper connection instance
- */
- public String getQuorum() {
- return quorum;
- }
-
- /**
- * Get the znodePaths.
- * <p>
- * Mainly used for mocking as mockito can not mock a field access.
- */
- public ZNodePaths getZNodePaths() {
- return znodePaths;
- }
-
- /**
- * Method called from ZooKeeper for events and connection status.
- * <p>
- * Valid events are passed along to listeners. Connection status changes
- * are dealt with locally.
- */
- @Override
- public void process(WatchedEvent event) {
- LOG.debug(prefix("Received ZooKeeper Event, " +
- "type=" + event.getType() + ", " +
- "state=" + event.getState() + ", " +
- "path=" + event.getPath()));
-
- switch(event.getType()) {
-
- // If event type is NONE, this is a connection status change
- case None: {
- connectionEvent(event);
- break;
- }
-
- // Otherwise pass along to the listeners
-
- case NodeCreated: {
- for(ZooKeeperListener listener : listeners) {
- listener.nodeCreated(event.getPath());
- }
- break;
- }
-
- case NodeDeleted: {
- for(ZooKeeperListener listener : listeners) {
- listener.nodeDeleted(event.getPath());
- }
- break;
- }
-
- case NodeDataChanged: {
- for(ZooKeeperListener listener : listeners) {
- listener.nodeDataChanged(event.getPath());
- }
- break;
- }
-
- case NodeChildrenChanged: {
- for(ZooKeeperListener listener : listeners) {
- listener.nodeChildrenChanged(event.getPath());
- }
- break;
- }
- }
- }
-
- // Connection management
-
- /**
- * Called when there is a connection-related event via the Watcher callback.
- * <p>
- * If Disconnected or Expired, this should shutdown the cluster. But, since
- * we send a KeeperException.SessionExpiredException along with the abort
- * 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.
- * <p>
- * @param event
- */
- private void connectionEvent(WatchedEvent event) {
- switch(event.getState()) {
- case SyncConnected:
- this.identifier = this.prefix + "-0x" +
- Long.toHexString(this.recoverableZooKeeper.getSessionId());
- // Update our identifier. Otherwise ignore.
- LOG.debug(this.identifier + " connected");
- break;
-
- // Abort the server if Disconnected or Expired
- case Disconnected:
- LOG.debug(prefix("Received Disconnected from ZooKeeper, ignoring"));
- break;
-
- case Expired:
- String msg = prefix(this.identifier + " received expired from " +
- "ZooKeeper, aborting");
- // TODO: One thought is to add call to ZooKeeperListener so say,
- // ZooKeeperNodeTracker can zero out its data values.
- if (this.abortable != null) {
- this.abortable.abort(msg, new KeeperException.SessionExpiredException());
- }
- break;
-
- case ConnectedReadOnly:
- case SaslAuthenticated:
- case AuthFailed:
- break;
-
- default:
- throw new IllegalStateException("Received event is not valid: " + event.getState());
- }
- }
-
- /**
- * Forces a synchronization of this ZooKeeper client connection.
- * <p>
- * Executing this method before running other methods will ensure that the
- * subsequent operations are up-to-date and consistent as of the time that
- * the sync is complete.
- * <p>
- * This is used for compareAndSwap type operations where we need to read the
- * data of an existing node and delete or transition that node, utilizing the
- * previously read version and data. We want to ensure that the version read
- * is up-to-date from when we begin the operation.
- */
- public void sync(String path) throws KeeperException {
- this.recoverableZooKeeper.sync(path, null, null);
- }
-
- /**
- * Handles KeeperExceptions in client calls.
- * <p>
- * This may be temporary but for now this gives one place to deal with these.
- * <p>
- * TODO: Currently this method rethrows the exception to let the caller handle
- * <p>
- * @param ke
- * @throws KeeperException
- */
- public void keeperException(KeeperException ke)
- throws KeeperException {
- LOG.error(prefix("Received unexpected KeeperException, re-throwing exception"), ke);
- throw ke;
- }
-
- /**
- * Handles InterruptedExceptions in client calls.
- * @param ie the InterruptedException instance thrown
- * @throws KeeperException the exception to throw, transformed from the InterruptedException
- */
- public void interruptedException(InterruptedException ie) throws KeeperException {
- interruptedExceptionNoThrow(ie, true);
- // Throw a system error exception to let upper level handle it
- throw new KeeperException.SystemErrorException();
- }
-
- /**
- * Log the InterruptedException and interrupt current thread
- * @param ie The IterruptedException to log
- * @param throwLater Whether we will throw the exception latter
- */
- public void interruptedExceptionNoThrow(InterruptedException ie, boolean throwLater) {
- LOG.debug(prefix("Received InterruptedException, will interrupt current thread"
- + (throwLater ? " and rethrow a SystemErrorException" : "")),
- ie);
- // At least preserve interrupt.
- Thread.currentThread().interrupt();
- }
-
- /**
- * Close the connection to ZooKeeper.
- *
- */
- @Override
- public void close() {
- try {
- recoverableZooKeeper.close();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- }
-
- public Configuration getConfiguration() {
- return conf;
- }
-
- @Override
- public void abort(String why, Throwable e) {
- if (this.abortable != null) this.abortable.abort(why, e);
- else this.aborted = true;
- }
-
- @Override
- public boolean isAborted() {
- return this.abortable == null? this.aborted: this.abortable.isAborted();
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java
deleted file mode 100644
index e67c9fd..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(SmallTests.class)
-public class TestInstancePending {
- @Test(timeout = 1000)
- public void test() throws Exception {
- final InstancePending<String> pending = new InstancePending<>();
- final AtomicReference<String> getResultRef = new AtomicReference<>();
-
- new Thread() {
- @Override
- public void run() {
- getResultRef.set(pending.get());
- }
- }.start();
-
- Thread.sleep(100);
- Assert.assertNull(getResultRef.get());
-
- pending.prepare("abc");
- Thread.sleep(100);
- Assert.assertEquals("abc", getResultRef.get());
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java
deleted file mode 100644
index a987bec..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import static org.mockito.Mockito.*;
-
-@Category(SmallTests.class)
-public class TestMetricsZooKeeper {
-
- @Test
- public void testRegisterExceptions() {
- MetricsZooKeeperSource zkSource = mock(MetricsZooKeeperSourceImpl.class);
- MetricsZooKeeper metricsZK = new MetricsZooKeeper(zkSource);
- metricsZK.registerAuthFailedException();
- metricsZK.registerConnectionLossException();
- metricsZK.registerConnectionLossException();
- metricsZK.registerDataInconsistencyException();
- metricsZK.registerInvalidACLException();
- metricsZK.registerNoAuthException();
- metricsZK.registerOperationTimeoutException();
- metricsZK.registerOperationTimeoutException();
- metricsZK.registerRuntimeInconsistencyException();
- metricsZK.registerSessionExpiredException();
- metricsZK.registerSystemErrorException();
- metricsZK.registerSystemErrorException();
- metricsZK.registerFailedZKCall();
-
- verify(zkSource, times(1)).incrementAuthFailedCount();
- // ConnectionLoss Exception was registered twice.
- verify(zkSource, times(2)).incrementConnectionLossCount();
- verify(zkSource, times(1)).incrementDataInconsistencyCount();
- verify(zkSource, times(1)).incrementInvalidACLCount();
- verify(zkSource, times(1)).incrementNoAuthCount();
- // OperationTimeout Exception was registered twice.
- verify(zkSource, times(2)).incrementOperationTimeoutCount();
- verify(zkSource, times(1)).incrementRuntimeInconsistencyCount();
- verify(zkSource, times(1)).incrementSessionExpiredCount();
- // SystemError Exception was registered twice.
- verify(zkSource, times(2)).incrementSystemErrorCount();
- verify(zkSource, times(1)).incrementTotalFailedZKCalls();
- }
-
- @Test
- public void testLatencyHistogramUpdates() {
- MetricsZooKeeperSource zkSource = mock(MetricsZooKeeperSourceImpl.class);
- MetricsZooKeeper metricsZK = new MetricsZooKeeper(zkSource);
- long latency = 100;
-
- metricsZK.registerReadOperationLatency(latency);
- metricsZK.registerReadOperationLatency(latency);
- metricsZK.registerWriteOperationLatency(latency);
- metricsZK.registerSyncOperationLatency(latency);
- // Read Operation Latency update was registered twice.
- verify(zkSource, times(2)).recordReadOperationLatency(latency);
- verify(zkSource, times(1)).recordWriteOperationLatency(latency);
- verify(zkSource, times(1)).recordSyncOperationLatency(latency);
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
deleted file mode 100644
index 6b1e1f0..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.security.Superusers;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.ZooDefs.Perms;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Id;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-
-/**
- *
- */
-@Category({SmallTests.class})
-public class TestZKUtil {
-
- @Test
- public void testUnsecure() throws ZooKeeperConnectionException, IOException {
- Configuration conf = HBaseConfiguration.create();
- conf.set(Superusers.SUPERUSER_CONF_KEY, "user1");
- String node = "/hbase/testUnsecure";
- ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false);
- List<ACL> aclList = ZKUtil.createACL(watcher, node, false);
- Assert.assertEquals(aclList.size(), 1);
- Assert.assertTrue(aclList.contains(Ids.OPEN_ACL_UNSAFE.iterator().next()));
- }
-
- @Test
- public void testSecuritySingleSuperuser() throws ZooKeeperConnectionException, IOException {
- Configuration conf = HBaseConfiguration.create();
- conf.set(Superusers.SUPERUSER_CONF_KEY, "user1");
- String node = "/hbase/testSecuritySingleSuperuser";
- ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false);
- List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
- Assert.assertEquals(aclList.size(), 2); // 1+1, since ACL will be set for the creator by default
- Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user1"))));
- Assert.assertTrue(aclList.contains(Ids.CREATOR_ALL_ACL.iterator().next()));
- }
-
- @Test
- public void testCreateACL() throws ZooKeeperConnectionException, IOException {
- Configuration conf = HBaseConfiguration.create();
- conf.set(Superusers.SUPERUSER_CONF_KEY, "user1,@group1,user2,@group2,user3");
- String node = "/hbase/testCreateACL";
- ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false);
- List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
- Assert.assertEquals(aclList.size(), 4); // 3+1, since ACL will be set for the creator by default
- Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group1"))));
- Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group2"))));
- Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user1"))));
- Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user2"))));
- Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user3"))));
- }
-
- @Test
- public void testCreateACLWithSameUser() throws ZooKeeperConnectionException, IOException {
- Configuration conf = HBaseConfiguration.create();
- conf.set(Superusers.SUPERUSER_CONF_KEY, "user4,@group1,user5,user6");
- UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("user4"));
- String node = "/hbase/testCreateACL";
- ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false);
- List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
- Assert.assertEquals(aclList.size(), 3); // 3, since service user the same as one of superuser
- Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group1"))));
- Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("auth", ""))));
- Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user5"))));
- Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user6"))));
- }
-
- @Test(expected = KeeperException.SystemErrorException.class)
- public void testInterruptedDuringAction()
- throws ZooKeeperConnectionException, IOException, KeeperException, InterruptedException {
- final RecoverableZooKeeper recoverableZk = Mockito.mock(RecoverableZooKeeper.class);
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(HBaseConfiguration.create(), "unittest", null) {
- @Override
- public RecoverableZooKeeper getRecoverableZooKeeper() {
- return recoverableZk;
- }
- };
- Mockito.doThrow(new InterruptedException()).when(recoverableZk)
- .getChildren(zkw.znodePaths.baseZNode, null);
- ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.baseZNode);
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java
deleted file mode 100644
index 3c99175..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ SmallTests.class })
-public class TestZooKeeperWatcher {
-
- @Test
- public void testIsClientReadable() throws IOException {
- ZooKeeperWatcher watcher =
- new ZooKeeperWatcher(HBaseConfiguration.create(), "testIsClientReadable", null, false);
-
- assertTrue(watcher.isClientReadable(watcher.znodePaths.baseZNode));
- assertTrue(watcher.isClientReadable(watcher.znodePaths.getZNodeForReplica(0)));
- assertTrue(watcher.isClientReadable(watcher.znodePaths.masterAddressZNode));
- assertTrue(watcher.isClientReadable(watcher.znodePaths.clusterIdZNode));
- assertTrue(watcher.isClientReadable(watcher.znodePaths.tableZNode));
- assertTrue(
- watcher.isClientReadable(ZNodePaths.joinZNode(watcher.znodePaths.tableZNode, "foo")));
- assertTrue(watcher.isClientReadable(watcher.znodePaths.rsZNode));
-
- assertFalse(watcher.isClientReadable(watcher.znodePaths.tableLockZNode));
- assertFalse(watcher.isClientReadable(watcher.znodePaths.balancerZNode));
- assertFalse(watcher.isClientReadable(watcher.znodePaths.regionNormalizerZNode));
- assertFalse(watcher.isClientReadable(watcher.znodePaths.clusterStateZNode));
- assertFalse(watcher.isClientReadable(watcher.znodePaths.drainingZNode));
- assertFalse(watcher.isClientReadable(watcher.znodePaths.splitLogZNode));
- assertFalse(watcher.isClientReadable(watcher.znodePaths.backupMasterAddressesZNode));
-
- watcher.close();
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java
index eba8518..c44be52 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java
@@ -27,8 +27,8 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -37,7 +37,7 @@ import org.junit.experimental.categories.Category;
/**
* An integration test that starts the cluster with three replicas for the meta
* It then creates a table, flushes the meta, kills the server holding the primary.
- * After that a client issues put/get requests on the created table - the other
+ * After that a client issues put/get requests on the created table - the other
* replicas of the meta would be used to get the location of the region of the created
* table.
*/
@@ -60,7 +60,7 @@ public class IntegrationTestMetaReplicas {
StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 1000);
// Make sure there are three servers.
util.initializeCluster(3);
- ZooKeeperWatcher zkw = util.getZooKeeperWatcher();
+ ZKWatcher zkw = util.getZooKeeperWatcher();
Configuration conf = util.getConfiguration();
String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java
index 92bad7f..09cbda3 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java
@@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.ToolRunner;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.Code;
@@ -139,7 +139,7 @@ public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool {
private void testZNodeACLs() throws IOException, KeeperException, InterruptedException {
- ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, "IntegrationTestZnodeACLs", null);
+ ZKWatcher watcher = new ZKWatcher(conf, "IntegrationTestZnodeACLs", null);
RecoverableZooKeeper zk = ZKUtil.connect(this.conf, watcher);
String baseZNode = watcher.znodePaths.baseZNode;
@@ -155,7 +155,7 @@ public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool {
LOG.info("Checking ZK permissions: SUCCESS");
}
- private void checkZnodePermsRecursive(ZooKeeperWatcher watcher,
+ private void checkZnodePermsRecursive(ZKWatcher watcher,
RecoverableZooKeeper zk, String znode) throws KeeperException, InterruptedException {
boolean expectedWorldReadable = watcher.isClientReadable(znode);
@@ -201,7 +201,7 @@ public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool {
assertTrue(expectedWorldReadable);
// assert that anyone can only read
assertEquals(perms, Perms.READ);
- } else if (superUsers != null && ZooKeeperWatcher.isSuperUserId(superUsers, id)) {
+ } else if (superUsers != null && ZKWatcher.isSuperUserId(superUsers, id)) {
// assert that super user has all the permissions
assertEquals(perms, Perms.ALL);
} else if (new Id("sasl", masterPrincipal).equals(id)) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
index d25d497..bf0081b 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
@@ -64,7 +65,6 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.MapReduceCell;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.io.WritableComparator;
@@ -406,10 +406,10 @@ public class Import extends Configured implements Tool {
LOG.info("setting WAL durability to default.");
}
// TODO: This is kind of ugly doing setup of ZKW just to read the clusterid.
- ZooKeeperWatcher zkw = null;
+ ZKWatcher zkw = null;
Exception ex = null;
try {
- zkw = new ZooKeeperWatcher(conf, context.getTaskAttemptID().toString(), null);
+ zkw = new ZKWatcher(conf, context.getTaskAttemptID().toString(), null);
clusterIds = Collections.singletonList(ZKClusterId.getUUIDForCluster(zkw));
} catch (ZooKeeperConnectionException e) {
ex = e;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index acf6ff8..b8de9ec 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -57,7 +57,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -330,10 +330,10 @@ public class VerifyReplication extends Configured implements Tool {
private static Pair<ReplicationPeerConfig, Configuration> getPeerQuorumConfig(
final Configuration conf, String peerId) throws IOException {
- ZooKeeperWatcher localZKW = null;
+ ZKWatcher localZKW = null;
ReplicationPeerZKImpl peer = null;
try {
- localZKW = new ZooKeeperWatcher(conf, "VerifyReplication",
+ localZKW = new ZKWatcher(conf, "VerifyReplication",
new Abortable() {
@Override public void abort(String why, Throwable e) {}
@Override public boolean isAborted() {return false;}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml
index 942fd8c..039f8ac 100644
--- a/hbase-replication/pom.xml
+++ b/hbase-replication/pom.xml
@@ -130,6 +130,10 @@
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-client</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-zookeeper</artifactId>
+ </dependency>
<!-- General dependencies -->
<dependency>
<groupId>org.apache.commons</groupId>
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
index 9d12211..3ff6914 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
@@ -23,7 +23,7 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
/**
* A factory class for instantiating replication objects that deal with replication state.
@@ -48,17 +48,17 @@ public class ReplicationFactory {
return (ReplicationQueuesClient) ConstructorUtils.invokeConstructor(classToBuild, args);
}
- public static ReplicationPeers getReplicationPeers(final ZooKeeperWatcher zk, Configuration conf,
- Abortable abortable) {
+ public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf,
+ Abortable abortable) {
return getReplicationPeers(zk, conf, null, abortable);
}
- public static ReplicationPeers getReplicationPeers(final ZooKeeperWatcher zk, Configuration conf,
- final ReplicationQueuesClient queuesClient, Abortable abortable) {
+ public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf,
+ final ReplicationQueuesClient queuesClient, Abortable abortable) {
return new ReplicationPeersZKImpl(zk, conf, queuesClient, abortable);
}
- public static ReplicationTracker getReplicationTracker(ZooKeeperWatcher zookeeper,
+ public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper,
final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable,
Stoppable stopper) {
return new ReplicationTrackerZKImpl(zookeeper, replicationPeers, conf, abortable, stopper);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
index 8f09479..214a313 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
@@ -34,9 +34,9 @@ import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
@@ -62,7 +62,7 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
* @param id string representation of this peer's identifier
* @param peerConfig configuration for the replication peer
*/
- public ReplicationPeerZKImpl(ZooKeeperWatcher zkWatcher, Configuration conf,
+ public ReplicationPeerZKImpl(ZKWatcher zkWatcher, Configuration conf,
String id, ReplicationPeerConfig peerConfig,
Abortable abortable)
throws ReplicationException {
@@ -258,9 +258,9 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
/**
* Tracker for state of this peer
*/
- public class PeerStateTracker extends ZooKeeperNodeTracker {
+ public class PeerStateTracker extends ZKNodeTracker {
- public PeerStateTracker(String peerStateZNode, ZooKeeperWatcher watcher,
+ public PeerStateTracker(String peerStateZNode, ZKWatcher watcher,
Abortable abortable) {
super(watcher, peerStateZNode, abortable);
}
@@ -281,11 +281,11 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
/**
* Tracker for PeerConfigNode of this peer
*/
- public class PeerConfigTracker extends ZooKeeperNodeTracker {
+ public class PeerConfigTracker extends ZKNodeTracker {
ReplicationPeerConfigListener listener;
- public PeerConfigTracker(String peerConfigNode, ZooKeeperWatcher watcher,
+ public PeerConfigTracker(String peerConfigNode, ZKWatcher watcher,
Abortable abortable) {
super(watcher, peerConfigNode, abortable);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
index cc84c1d..9c4e3fe 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
@@ -43,8 +43,8 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
@@ -84,8 +84,8 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
private static final Log LOG = LogFactory.getLog(ReplicationPeersZKImpl.class);
- public ReplicationPeersZKImpl(final ZooKeeperWatcher zk, final Configuration conf,
- final ReplicationQueuesClient queuesClient, Abortable abortable) {
+ public ReplicationPeersZKImpl(final ZKWatcher zk, final Configuration conf,
+ final ReplicationQueuesClient queuesClient, Abortable abortable) {
super(zk, conf, abortable);
this.abortable = abortable;
this.peerClusters = new ConcurrentHashMap<>();
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
index c8328bd..c2a5df3 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.replication;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
/**
* Wrapper around common arguments used to construct ReplicationQueues. Used to construct various
@@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@InterfaceAudience.Private
public class ReplicationQueuesArguments {
- private ZooKeeperWatcher zk;
+ private ZKWatcher zk;
private Configuration conf;
private Abortable abort;
@@ -39,16 +39,16 @@ public class ReplicationQueuesArguments {
this.abort = abort;
}
- public ReplicationQueuesArguments(Configuration conf, Abortable abort, ZooKeeperWatcher zk) {
+ public ReplicationQueuesArguments(Configuration conf, Abortable abort, ZKWatcher zk) {
this(conf, abort);
setZk(zk);
}
- public ZooKeeperWatcher getZk() {
+ public ZKWatcher getZk() {
return zk;
}
- public void setZk(ZooKeeperWatcher zk) {
+ public void setZk(ZKWatcher zk) {
this.zk = zk;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
index 67258c7..9b79294 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.replication;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
/**
* Wrapper around common arguments used to construct ReplicationQueuesClient. Used to construct
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@InterfaceAudience.Private
public class ReplicationQueuesClientArguments extends ReplicationQueuesArguments {
public ReplicationQueuesClientArguments(Configuration conf, Abortable abort,
- ZooKeeperWatcher zk) {
+ ZKWatcher zk) {
super(conf, abort, zk);
}
public ReplicationQueuesClientArguments(Configuration conf, Abortable abort) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
index 49e55ef..b998f15 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
@@ -21,16 +21,16 @@ package org.apache.hadoop.hbase.replication;
import java.util.List;
import java.util.Set;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableSet;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
@@ -44,8 +44,8 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
this(args.getZk(), args.getConf(), args.getAbortable());
}
- public ReplicationQueuesClientZKImpl(final ZooKeeperWatcher zk, Configuration conf,
- Abortable abortable) {
+ public ReplicationQueuesClientZKImpl(final ZKWatcher zk, Configuration conf,
+ Abortable abortable) {
super(zk, conf, abortable);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index 7b1d5c2..95fd294 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -25,20 +25,18 @@ import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
/**
@@ -75,8 +73,8 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
this(args.getZk(), args.getConf(), args.getAbortable());
}
- public ReplicationQueuesZKImpl(final ZooKeeperWatcher zk, Configuration conf,
- Abortable abortable) {
+ public ReplicationQueuesZKImpl(final ZKWatcher zk, Configuration conf,
+ Abortable abortable) {
super(zk, conf, abortable);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
index e96401e..ad970c6 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
@@ -22,21 +22,19 @@ import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.List;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
-
/**
* This is a base class for maintaining replication state in zookeeper.
*/
@@ -61,7 +59,7 @@ public abstract class ReplicationStateZKBase {
/** The name of the znode that contains tableCFs */
protected final String tableCFsNodeName;
- protected final ZooKeeperWatcher zookeeper;
+ protected final ZKWatcher zookeeper;
protected final Configuration conf;
protected final Abortable abortable;
@@ -74,8 +72,8 @@ public abstract class ReplicationStateZKBase {
"zookeeper.znode.replication.hfile.refs";
public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
- public ReplicationStateZKBase(ZooKeeperWatcher zookeeper, Configuration conf,
- Abortable abortable) {
+ public ReplicationStateZKBase(ZKWatcher zookeeper, Configuration conf,
+ Abortable abortable) {
this.zookeeper = zookeeper;
this.conf = conf;
this.abortable = abortable;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
index ade1c4d..aa72fc5 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
@@ -24,13 +24,13 @@ import java.util.concurrent.CopyOnWriteArrayList;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -50,7 +50,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
private final ArrayList<String> otherRegionServers = new ArrayList<>();
private final ReplicationPeers replicationPeers;
- public ReplicationTrackerZKImpl(ZooKeeperWatcher zookeeper,
+ public ReplicationTrackerZKImpl(ZKWatcher zookeeper,
final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable,
Stoppable stopper) {
super(zookeeper, conf, abortable);
@@ -88,12 +88,12 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
* Watcher used to be notified of the other region server's death in the local cluster. It
* initiates the process to transfer the queues if it is able to grab the lock.
*/
- public class OtherRegionServerWatcher extends ZooKeeperListener {
+ public class OtherRegionServerWatcher extends ZKListener {
/**
* Construct a ZooKeeper event listener.
*/
- public OtherRegionServerWatcher(ZooKeeperWatcher watcher) {
+ public OtherRegionServerWatcher(ZKWatcher watcher) {
super(watcher);
}
@@ -145,12 +145,12 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
/**
* Watcher used to follow the creation and deletion of peer clusters.
*/
- public class PeersWatcher extends ZooKeeperListener {
+ public class PeersWatcher extends ZKListener {
/**
* Construct a ZooKeeper event listener.
*/
- public PeersWatcher(ZooKeeperWatcher watcher) {
+ public PeersWatcher(ZKWatcher watcher) {
super(watcher);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
index cf1ff20..1f239f8 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
@@ -37,11 +37,10 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import java.io.IOException;
@@ -77,7 +76,7 @@ public class TableBasedReplicationQueuesImpl extends ReplicationTableBase
this(args.getConf(), args.getAbortable(), args.getZk());
}
- public TableBasedReplicationQueuesImpl(Configuration conf, Abortable abort, ZooKeeperWatcher zkw)
+ public TableBasedReplicationQueuesImpl(Configuration conf, Abortable abort, ZKWatcher zkw)
throws IOException {
super(conf, abort);
replicationState = new ReplicationStateZKBase(zkw, conf, abort) {};
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index 3d152bb..5fca659 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -77,8 +77,8 @@ import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
import org.apache.hadoop.hbase.security.access.AccessControlLists;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
@@ -141,7 +141,7 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager {
private final MasterServices masterServices;
private Table rsGroupTable;
private final ClusterConnection conn;
- private final ZooKeeperWatcher watcher;
+ private final ZKWatcher watcher;
private final RSGroupStartupWorker rsGroupStartupWorker = new RSGroupStartupWorker();
// contains list of groups that were last flushed to persistent store
private Set<String> prevRSGroups = new HashSet<>();
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
index 9077f15..ba3534d 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Result;
@@ -32,7 +33,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.Assert;
@@ -45,14 +45,14 @@ import java.util.Set;
@InterfaceAudience.Private
public class VerifyingRSGroupAdminClient implements RSGroupAdmin {
private Table table;
- private ZooKeeperWatcher zkw;
+ private ZKWatcher zkw;
private RSGroupAdmin wrapped;
public VerifyingRSGroupAdminClient(RSGroupAdmin RSGroupAdmin, Configuration conf)
throws IOException {
wrapped = RSGroupAdmin;
table = ConnectionFactory.createConnection(conf).getTable(RSGroupInfoManager.RSGROUP_TABLE_NAME);
- zkw = new ZooKeeperWatcher(conf, this.getClass().getSimpleName(), null);
+ zkw = new ZKWatcher(conf, this.getClass().getSimpleName(), null);
}
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 1e5a1f3..daa34f8 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -390,6 +390,10 @@
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-zookeeper</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
<artifactId>hbase-replication</artifactId>
</dependency>
<dependency>
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
index a6b39f3..0d9eba8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import java.io.IOException;
@@ -43,7 +43,7 @@ public interface Server extends Abortable, Stoppable {
/**
* Gets the ZooKeeper instance for this server.
*/
- ZooKeeperWatcher getZooKeeper();
+ ZKWatcher getZooKeeper();
/**
* Returns a reference to the servers' connection.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
index c4fad44..160d058 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
@@ -27,8 +27,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import java.io.IOException;
@@ -48,12 +48,12 @@ import java.util.concurrent.ConcurrentSkipListMap;
*
*/
@InterfaceAudience.Private
-public class ZKNamespaceManager extends ZooKeeperListener {
+public class ZKNamespaceManager extends ZKListener {
private static final Log LOG = LogFactory.getLog(ZKNamespaceManager.class);
private final String nsZNode;
private volatile NavigableMap<String,NamespaceDescriptor> cache;
- public ZKNamespaceManager(ZooKeeperWatcher zkw) throws IOException {
+ public ZKNamespaceManager(ZKWatcher zkw) throws IOException {
super(zkw);
nsZNode = zkw.znodePaths.namespaceZNode;
cache = new ConcurrentSkipListMap<>();
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java
index d7fdeb7..cda5aff 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java
@@ -32,8 +32,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -132,7 +132,7 @@ public class ZNodeClearer {
* @param rsZnodePath from HBASE_ZNODE_FILE
* @return String representation of ServerName or null if fails
*/
-
+
public static String parseMasterServerName(String rsZnodePath) {
String masterServerName = null;
try {
@@ -141,12 +141,12 @@ public class ZNodeClearer {
} catch (IndexOutOfBoundsException e) {
LOG.warn("String " + rsZnodePath + " has wrong format", e);
}
- return masterServerName;
+ return masterServerName;
}
-
+
/**
- *
- * @return true if cluster is configured with master-rs collocation
+ *
+ * @return true if cluster is configured with master-rs collocation
*/
private static boolean tablesOnMaster(Configuration conf) {
boolean tablesOnMaster = true;
@@ -167,9 +167,9 @@ public class ZNodeClearer {
Configuration tempConf = new Configuration(conf);
tempConf.setInt("zookeeper.recovery.retry", 0);
- ZooKeeperWatcher zkw;
+ ZKWatcher zkw;
try {
- zkw = new ZooKeeperWatcher(tempConf, "clean znode for master",
+ zkw = new ZKWatcher(tempConf, "clean znode for master",
new Abortable() {
@Override public void abort(String why, Throwable e) {}
@Override public boolean isAborted() { return false; }
[03/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper
from hbase-server and hbase-client
Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java
new file mode 100644
index 0000000..2f2b036
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java
@@ -0,0 +1,99 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.util.UUID;
+
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.ClusterId;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Publishes and synchronizes a unique identifier specific to a given HBase
+ * cluster. The stored identifier is read from the file system by the active
+ * master on startup, and is subsequently available to all watchers (including
+ * clients).
+ */
+@InterfaceAudience.Private
+public class ZKClusterId {
+ private ZKWatcher watcher;
+ private Abortable abortable;
+ private String id;
+
+ public ZKClusterId(ZKWatcher watcher, Abortable abortable) {
+ this.watcher = watcher;
+ this.abortable = abortable;
+ }
+
+ public boolean hasId() {
+ return getId() != null;
+ }
+
+ public String getId() {
+ try {
+ if (id == null) {
+ id = readClusterIdZNode(watcher);
+ }
+ } catch (KeeperException ke) {
+ abortable.abort("Unexpected exception from ZooKeeper reading cluster ID",
+ ke);
+ }
+ return id;
+ }
+
+ public static String readClusterIdZNode(ZKWatcher watcher)
+ throws KeeperException {
+ if (ZKUtil.checkExists(watcher, watcher.znodePaths.clusterIdZNode) != -1) {
+ byte [] data;
+ try {
+ data = ZKUtil.getData(watcher, watcher.znodePaths.clusterIdZNode);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ return null;
+ }
+ if (data != null) {
+ try {
+ return ClusterId.parseFrom(data).toString();
+ } catch (DeserializationException e) {
+ throw ZKUtil.convert(e);
+ }
+ }
+ }
+ return null;
+ }
+
+ public static void setClusterId(ZKWatcher watcher, ClusterId id)
+ throws KeeperException {
+ ZKUtil.createSetData(watcher, watcher.znodePaths.clusterIdZNode, id.toByteArray());
+ }
+
+ /**
+ * Get the UUID for the provided ZK watcher. Doesn't handle any ZK exceptions
+ * @param zkw watcher connected to an ensemble
+ * @return the UUID read from zookeeper
+ * @throws KeeperException
+ */
+ public static UUID getUUIDForCluster(ZKWatcher zkw) throws KeeperException {
+ String uuid = readClusterIdZNode(zkw);
+ return uuid == null ? null : UUID.fromString(uuid);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
new file mode 100644
index 0000000..edd2ccd
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Handles coordination of a single "leader" instance among many possible
+ * candidates. The first {@link ZKLeaderManager} to successfully create
+ * the given znode becomes the leader, allowing the instance to continue
+ * with whatever processing must be protected. Other {@link ZKLeaderManager}
+ * instances will wait to be notified of changes to the leader znode.
+ * If the current master instance fails, the ephemeral leader znode will
+ * be removed, and all waiting instances will be notified, with the race
+ * to claim the leader znode beginning all over again.
+ * @deprecated Not used
+ */
+@Deprecated
+@InterfaceAudience.Private
+public class ZKLeaderManager extends ZKListener {
+ private static final Log LOG = LogFactory.getLog(ZKLeaderManager.class);
+
+ private final AtomicBoolean leaderExists = new AtomicBoolean();
+ private String leaderZNode;
+ private byte[] nodeId;
+ private Stoppable candidate;
+
+ public ZKLeaderManager(ZKWatcher watcher, String leaderZNode,
+ byte[] identifier, Stoppable candidate) {
+ super(watcher);
+ this.leaderZNode = leaderZNode;
+ this.nodeId = identifier;
+ this.candidate = candidate;
+ }
+
+ public void start() {
+ try {
+ watcher.registerListener(this);
+ String parent = ZKUtil.getParent(leaderZNode);
+ if (ZKUtil.checkExists(watcher, parent) < 0) {
+ ZKUtil.createWithParents(watcher, parent);
+ }
+ } catch (KeeperException ke) {
+ watcher.abort("Unhandled zk exception when starting", ke);
+ candidate.stop("Unhandled zk exception starting up: "+ke.getMessage());
+ }
+ }
+
+ @Override
+ public void nodeCreated(String path) {
+ if (leaderZNode.equals(path) && !candidate.isStopped()) {
+ handleLeaderChange();
+ }
+ }
+
+ @Override
+ public void nodeDeleted(String path) {
+ if (leaderZNode.equals(path) && !candidate.isStopped()) {
+ handleLeaderChange();
+ }
+ }
+
+ private void handleLeaderChange() {
+ try {
+ synchronized(leaderExists) {
+ if (ZKUtil.watchAndCheckExists(watcher, leaderZNode)) {
+ LOG.info("Found new leader for znode: "+leaderZNode);
+ leaderExists.set(true);
+ } else {
+ LOG.info("Leader change, but no new leader found");
+ leaderExists.set(false);
+ leaderExists.notifyAll();
+ }
+ }
+ } catch (KeeperException ke) {
+ watcher.abort("ZooKeeper error checking for leader znode", ke);
+ candidate.stop("ZooKeeper error checking for leader: "+ke.getMessage());
+ }
+ }
+
+ /**
+ * Blocks until this instance has claimed the leader ZNode in ZooKeeper
+ */
+ public void waitToBecomeLeader() {
+ while (!candidate.isStopped()) {
+ try {
+ if (ZKUtil.createEphemeralNodeAndWatch(watcher, leaderZNode, nodeId)) {
+ // claimed the leader znode
+ leaderExists.set(true);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Claimed the leader znode as '"+
+ Bytes.toStringBinary(nodeId)+"'");
+ }
+ return;
+ }
+
+ // if claiming the node failed, there should be another existing node
+ byte[] currentId = ZKUtil.getDataAndWatch(watcher, leaderZNode);
+ if (currentId != null && Bytes.equals(currentId, nodeId)) {
+ // claimed with our ID, but we didn't grab it, possibly restarted?
+ LOG.info("Found existing leader with our ID ("+
+ Bytes.toStringBinary(nodeId)+"), removing");
+ ZKUtil.deleteNode(watcher, leaderZNode);
+ leaderExists.set(false);
+ } else {
+ LOG.info("Found existing leader with ID: "+Bytes.toStringBinary(nodeId));
+ leaderExists.set(true);
+ }
+ } catch (KeeperException ke) {
+ watcher.abort("Unexpected error from ZK, stopping candidate", ke);
+ candidate.stop("Unexpected error from ZK: "+ke.getMessage());
+ return;
+ }
+
+ // wait for next chance
+ synchronized(leaderExists) {
+ while (leaderExists.get() && !candidate.isStopped()) {
+ try {
+ leaderExists.wait();
+ } catch (InterruptedException ie) {
+ LOG.debug("Interrupted waiting on leader", ie);
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Removes the leader znode, if it is currently claimed by this instance.
+ */
+ public void stepDownAsLeader() {
+ try {
+ synchronized(leaderExists) {
+ if (!leaderExists.get()) {
+ return;
+ }
+ byte[] leaderId = ZKUtil.getData(watcher, leaderZNode);
+ if (leaderId != null && Bytes.equals(nodeId, leaderId)) {
+ LOG.info("Stepping down as leader");
+ ZKUtil.deleteNodeFailSilent(watcher, leaderZNode);
+ leaderExists.set(false);
+ } else {
+ LOG.info("Not current leader, no need to step down");
+ }
+ }
+ } catch (KeeperException ke) {
+ watcher.abort("Unhandled zookeeper exception removing leader node", ke);
+ candidate.stop("Unhandled zookeeper exception removing leader node: "
+ + ke.getMessage());
+ } catch (InterruptedException e) {
+ watcher.abort("Unhandled zookeeper exception removing leader node", e);
+ candidate.stop("Unhandled zookeeper exception removing leader node: "
+ + e.getMessage());
+ }
+ }
+
+ public boolean hasLeader() {
+ return leaderExists.get();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKListener.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKListener.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKListener.java
new file mode 100644
index 0000000..595e713
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKListener.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Base class for internal listeners of ZooKeeper events.
+ *
+ * The {@link ZKWatcher} for a process will execute the appropriate
+ * methods of implementations of this class. In order to receive events from
+ * the watcher, every listener must register itself via {@link ZKWatcher#registerListener}.
+ *
+ * Subclasses need only override those methods in which they are interested.
+ *
+ * Note that the watcher will be blocked when invoking methods in listeners so
+ * they must not be long-running.
+ */
+@InterfaceAudience.Private
+public abstract class ZKListener {
+
+ // Reference to the zk watcher which also contains configuration and constants
+ protected ZKWatcher watcher;
+
+ /**
+ * Construct a ZooKeeper event listener.
+ */
+ public ZKListener(ZKWatcher watcher) {
+ this.watcher = watcher;
+ }
+
+ /**
+ * Called when a new node has been created.
+ * @param path full path of the new node
+ */
+ public void nodeCreated(String path) {
+ // no-op
+ }
+
+ /**
+ * Called when a node has been deleted
+ * @param path full path of the deleted node
+ */
+ public void nodeDeleted(String path) {
+ // no-op
+ }
+
+ /**
+ * Called when an existing node has changed data.
+ * @param path full path of the updated node
+ */
+ public void nodeDataChanged(String path) {
+ // no-op
+ }
+
+ /**
+ * Called when an existing node has a child node added or removed.
+ * @param path full path of the node whose children have changed
+ */
+ public void nodeChildrenChanged(String path) {
+ // no-op
+ }
+
+ /**
+ * @return The watcher associated with this listener
+ */
+ public ZKWatcher getWatcher() {
+ return this.watcher;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMainServer.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMainServer.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMainServer.java
new file mode 100644
index 0000000..9cb0e7d
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMainServer.java
@@ -0,0 +1,126 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.curator.shaded.com.google.common.base.Stopwatch;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeperMain;
+
+/**
+ * Tool for running ZookeeperMain from HBase by reading a ZooKeeper server
+ * from HBase XML configuration.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+public class ZKMainServer {
+ private static final String SERVER_ARG = "-server";
+
+ public String parse(final Configuration c) {
+ return ZKConfig.getZKQuorumServersString(c);
+ }
+
+ /**
+ * ZooKeeper 3.4.6 broke being able to pass commands on command line.
+ * See ZOOKEEPER-1897. This class is a hack to restore this faclity.
+ */
+ private static class HACK_UNTIL_ZOOKEEPER_1897_ZooKeeperMain extends ZooKeeperMain {
+ public HACK_UNTIL_ZOOKEEPER_1897_ZooKeeperMain(String[] args)
+ throws IOException, InterruptedException {
+ super(args);
+ // Make sure we are connected before we proceed. Can take a while on some systems. If we
+ // run the command without being connected, we get ConnectionLoss KeeperErrorConnection...
+ Stopwatch stopWatch = Stopwatch.createStarted();
+ while (!this.zk.getState().isConnected()) {
+ Thread.sleep(1);
+ if (stopWatch.elapsed(TimeUnit.SECONDS) > 10) {
+ throw new InterruptedException("Failed connect after waiting " +
+ stopWatch.elapsed(TimeUnit.SECONDS) + "seconds; state=" + this.zk.getState() +
+ "; " + this.zk);
+ }
+ }
+ }
+
+ /**
+ * Run the command-line args passed. Calls System.exit when done.
+ * @throws KeeperException
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ void runCmdLine() throws KeeperException, IOException, InterruptedException {
+ processCmd(this.cl);
+ System.exit(0);
+ }
+ }
+
+ /**
+ * @param args
+ * @return True if argument strings have a '-server' in them.
+ */
+ private static boolean hasServer(final String args[]) {
+ return args.length > 0 && args[0].equals(SERVER_ARG);
+ }
+
+ /**
+ * @param args
+ * @return True if command-line arguments were passed.
+ */
+ private static boolean hasCommandLineArguments(final String args[]) {
+ if (hasServer(args)) {
+ if (args.length < 2) throw new IllegalStateException("-server param but no value");
+ return args.length > 2;
+ }
+ return args.length > 0;
+ }
+
+ /**
+ * Run the tool.
+ * @param args Command line arguments. First arg is path to zookeepers file.
+ */
+ public static void main(String args[]) throws Exception {
+ String [] newArgs = args;
+ if (!hasServer(args)) {
+ // Add the zk ensemble from configuration if none passed on command-line.
+ Configuration conf = HBaseConfiguration.create();
+ String hostport = new ZKMainServer().parse(conf);
+ if (hostport != null && hostport.length() > 0) {
+ newArgs = new String[args.length + 2];
+ System.arraycopy(args, 0, newArgs, 2, args.length);
+ newArgs[0] = "-server";
+ newArgs[1] = hostport;
+ }
+ }
+ // If command-line arguments, run our hack so they are executed.
+ // ZOOKEEPER-1897 was committed to zookeeper-3.4.6 but elsewhere in this class we say
+ // 3.4.6 breaks command-processing; TODO.
+ if (hasCommandLineArguments(args)) {
+ HACK_UNTIL_ZOOKEEPER_1897_ZooKeeperMain zkm =
+ new HACK_UNTIL_ZOOKEEPER_1897_ZooKeeperMain(newArgs);
+ zkm.runCmdLine();
+ } else {
+ ZooKeeperMain.main(newArgs);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetrics.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetrics.java
new file mode 100644
index 0000000..20d4a55
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetrics.java
@@ -0,0 +1,108 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Class used to push numbers about ZooKeeper into the metrics subsystem. This will take a
+ * single function call and turn it into multiple manipulations of the hadoop metrics system.
+ */
+@InterfaceAudience.Private
+public class ZKMetrics implements ZKMetricsListener {
+ private final MetricsZooKeeperSource source;
+
+ public ZKMetrics() {
+ this(CompatibilitySingletonFactory.getInstance(MetricsZooKeeperSource.class));
+ }
+
+ @VisibleForTesting
+ public ZKMetrics(MetricsZooKeeperSource s) {
+ this.source = s;
+ }
+
+ @Override
+ public void registerAuthFailedException() {
+ source.incrementAuthFailedCount();
+ }
+
+ @Override
+ public void registerConnectionLossException() {
+ source.incrementConnectionLossCount();
+ }
+
+ @Override
+ public void registerDataInconsistencyException() {
+ source.incrementDataInconsistencyCount();
+ }
+
+ @Override
+ public void registerInvalidACLException() {
+ source.incrementInvalidACLCount();
+ }
+
+ @Override
+ public void registerNoAuthException() {
+ source.incrementNoAuthCount();
+ }
+
+ @Override
+ public void registerOperationTimeoutException() {
+ source.incrementOperationTimeoutCount();
+ }
+
+ @Override
+ public void registerRuntimeInconsistencyException() {
+ source.incrementRuntimeInconsistencyCount();
+ }
+
+ @Override
+ public void registerSessionExpiredException() {
+ source.incrementSessionExpiredCount();
+ }
+
+ @Override
+ public void registerSystemErrorException() {
+ source.incrementSystemErrorCount();
+ }
+
+ @Override
+ public void registerFailedZKCall() {
+ source.incrementTotalFailedZKCalls();
+ }
+
+ @Override
+ public void registerReadOperationLatency(long latency) {
+ source.recordReadOperationLatency(latency);
+ }
+
+ @Override
+ public void registerWriteOperationLatency(long latency) {
+ source.recordWriteOperationLatency(latency);
+ }
+
+ @Override
+ public void registerSyncOperationLatency(long latency) {
+ source.recordSyncOperationLatency(latency);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetricsListener.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetricsListener.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetricsListener.java
new file mode 100644
index 0000000..f17925e
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetricsListener.java
@@ -0,0 +1,91 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public interface ZKMetricsListener {
+
+ /**
+ * An AUTHFAILED Exception was seen.
+ */
+ void registerAuthFailedException();
+
+ /**
+ * A CONNECTIONLOSS Exception was seen.
+ */
+ void registerConnectionLossException();
+
+ /**
+ * A DATAINCONSISTENCY Exception was seen.
+ */
+ void registerDataInconsistencyException();
+
+ /**
+ * An INVALIDACL Exception was seen.
+ */
+ void registerInvalidACLException();
+
+ /**
+ * A NOAUTH Exception was seen.
+ */
+ void registerNoAuthException();
+
+ /**
+ * A OPERATIONTIMEOUT Exception was seen.
+ */
+ void registerOperationTimeoutException();
+
+ /**
+ * A RUNTIMEINCONSISTENCY Exception was seen.
+ */
+ void registerRuntimeInconsistencyException();
+
+ /**
+ * A SESSIONEXPIRED Exception was seen.
+ */
+ void registerSessionExpiredException();
+
+ /**
+ * A SYSTEMERROR Exception was seen.
+ */
+ void registerSystemErrorException();
+
+ /**
+ * A ZooKeeper API Call failed.
+ */
+ void registerFailedZKCall();
+
+ /**
+ * Register the latency incurred for read operations.
+ */
+ void registerReadOperationLatency(long latency);
+
+ /**
+ * Register the latency incurred for write operations.
+ */
+ void registerWriteOperationLatency(long latency);
+
+ /**
+ * Register the latency incurred for sync operations.
+ */
+ void registerSyncOperationLatency(long latency);
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKNodeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKNodeTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKNodeTracker.java
new file mode 100644
index 0000000..8ce41e3
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKNodeTracker.java
@@ -0,0 +1,251 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Tracks the availability and value of a single ZooKeeper node.
+ *
+ * <p>Utilizes the {@link ZKListener} interface to get the necessary
+ * ZooKeeper events related to the node.
+ *
+ * <p>This is the base class used by trackers in both the Master and
+ * RegionServers.
+ */
+@InterfaceAudience.Private
+public abstract class ZKNodeTracker extends ZKListener {
+ // LOG is being used in subclasses, hence keeping it protected
+ protected static final Log LOG = LogFactory.getLog(ZKNodeTracker.class);
+ /** Path of node being tracked */
+ protected final String node;
+
+ /** Data of the node being tracked */
+ private byte [] data;
+
+ /** Used to abort if a fatal error occurs */
+ protected final Abortable abortable;
+
+ private boolean stopped = false;
+
+ /**
+ * Constructs a new ZK node tracker.
+ *
+ * <p>After construction, use {@link #start} to kick off tracking.
+ *
+ * @param watcher
+ * @param node
+ * @param abortable
+ */
+ public ZKNodeTracker(ZKWatcher watcher, String node,
+ Abortable abortable) {
+ super(watcher);
+ this.node = node;
+ this.abortable = abortable;
+ this.data = null;
+ }
+
+ /**
+ * Starts the tracking of the node in ZooKeeper.
+ *
+ * <p>Use {@link #blockUntilAvailable()} to block until the node is available
+ * or {@link #getData(boolean)} to get the data of the node if it is available.
+ */
+ public synchronized void start() {
+ this.watcher.registerListener(this);
+ try {
+ if(ZKUtil.watchAndCheckExists(watcher, node)) {
+ byte [] data = ZKUtil.getDataAndWatch(watcher, node);
+ if(data != null) {
+ this.data = data;
+ } else {
+ // It existed but now does not, try again to ensure a watch is set
+ LOG.debug("Try starting again because there is no data from " + node);
+ start();
+ }
+ }
+ } catch (KeeperException e) {
+ abortable.abort("Unexpected exception during initialization, aborting", e);
+ }
+ }
+
+ public synchronized void stop() {
+ this.stopped = true;
+ notifyAll();
+ }
+
+ /**
+ * Gets the data of the node, blocking until the node is available.
+ *
+ * @return data of the node
+ * @throws InterruptedException if the waiting thread is interrupted
+ */
+ public synchronized byte [] blockUntilAvailable()
+ throws InterruptedException {
+ return blockUntilAvailable(0, false);
+ }
+
+ /**
+ * Gets the data of the node, blocking until the node is available or the
+ * specified timeout has elapsed.
+ *
+ * @param timeout maximum time to wait for the node data to be available,
+ * n milliseconds. Pass 0 for no timeout.
+ * @return data of the node
+ * @throws InterruptedException if the waiting thread is interrupted
+ */
+ public synchronized byte [] blockUntilAvailable(long timeout, boolean refresh)
+ throws InterruptedException {
+ if (timeout < 0) throw new IllegalArgumentException();
+ boolean notimeout = timeout == 0;
+ long startTime = System.currentTimeMillis();
+ long remaining = timeout;
+ if (refresh) {
+ try {
+ // This does not create a watch if the node does not exists
+ this.data = ZKUtil.getDataAndWatch(watcher, node);
+ } catch(KeeperException e) {
+ // We use to abort here, but in some cases the abort is ignored (
+ // (empty Abortable), so it's better to log...
+ LOG.warn("Unexpected exception handling blockUntilAvailable", e);
+ abortable.abort("Unexpected exception handling blockUntilAvailable", e);
+ }
+ }
+ boolean nodeExistsChecked = (!refresh ||data!=null);
+ while (!this.stopped && (notimeout || remaining > 0) && this.data == null) {
+ if (!nodeExistsChecked) {
+ try {
+ nodeExistsChecked = (ZKUtil.checkExists(watcher, node) != -1);
+ } catch (KeeperException e) {
+ LOG.warn(
+ "Got exception while trying to check existence in ZooKeeper" +
+ " of the node: "+node+", retrying if timeout not reached",e );
+ }
+
+ // It did not exists, and now it does.
+ if (nodeExistsChecked){
+ LOG.debug("Node " + node + " now exists, resetting a watcher");
+ try {
+ // This does not create a watch if the node does not exists
+ this.data = ZKUtil.getDataAndWatch(watcher, node);
+ } catch (KeeperException e) {
+ LOG.warn("Unexpected exception handling blockUntilAvailable", e);
+ abortable.abort("Unexpected exception handling blockUntilAvailable", e);
+ }
+ }
+ }
+ // We expect a notification; but we wait with a
+ // a timeout to lower the impact of a race condition if any
+ wait(100);
+ remaining = timeout - (System.currentTimeMillis() - startTime);
+ }
+ return this.data;
+ }
+
+ /**
+ * Gets the data of the node.
+ *
+ * <p>If the node is currently available, the most up-to-date known version of
+ * the data is returned. If the node is not currently available, null is
+ * returned.
+ * @param refresh whether to refresh the data by calling ZK directly.
+ * @return data of the node, null if unavailable
+ */
+ public synchronized byte [] getData(boolean refresh) {
+ if (refresh) {
+ try {
+ this.data = ZKUtil.getDataAndWatch(watcher, node);
+ } catch(KeeperException e) {
+ abortable.abort("Unexpected exception handling getData", e);
+ }
+ }
+ return this.data;
+ }
+
+ public String getNode() {
+ return this.node;
+ }
+
+ @Override
+ public synchronized void nodeCreated(String path) {
+ if (!path.equals(node)) return;
+ try {
+ byte [] data = ZKUtil.getDataAndWatch(watcher, node);
+ if (data != null) {
+ this.data = data;
+ notifyAll();
+ } else {
+ nodeDeleted(path);
+ }
+ } catch(KeeperException e) {
+ abortable.abort("Unexpected exception handling nodeCreated event", e);
+ }
+ }
+
+ @Override
+ public synchronized void nodeDeleted(String path) {
+ if(path.equals(node)) {
+ try {
+ if(ZKUtil.watchAndCheckExists(watcher, node)) {
+ nodeCreated(path);
+ } else {
+ this.data = null;
+ }
+ } catch(KeeperException e) {
+ abortable.abort("Unexpected exception handling nodeDeleted event", e);
+ }
+ }
+ }
+
+ @Override
+ public synchronized void nodeDataChanged(String path) {
+ if(path.equals(node)) {
+ nodeCreated(path);
+ }
+ }
+
+ /**
+ * Checks if the baseznode set as per the property 'zookeeper.znode.parent'
+ * exists.
+ * @return true if baseznode exists.
+ * false if doesnot exists.
+ */
+ public boolean checkIfBaseNodeAvailable() {
+ try {
+ if (ZKUtil.checkExists(watcher, watcher.znodePaths.baseZNode) == -1) {
+ return false;
+ }
+ } catch (KeeperException e) {
+ abortable.abort("Exception while checking if basenode (" + watcher.znodePaths.baseZNode
+ + ") exists in ZooKeeper.",
+ e);
+ }
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return "ZKNodeTracker{" +
+ "node='" + node + ", stopped=" + stopped + '}';
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
new file mode 100644
index 0000000..8116c23
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
@@ -0,0 +1,65 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+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
+ * a line-by-line list for use by bash scripts.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+public class ZKServerTool {
+ public static ServerName[] readZKNodes(Configuration conf) {
+ List<ServerName> hosts = new LinkedList<>();
+ String quorum = conf.get(HConstants.ZOOKEEPER_QUORUM, HConstants.LOCALHOST);
+
+ String[] values = quorum.split(",");
+ for (String value : values) {
+ String[] parts = value.split(":");
+ String host = parts[0];
+ int port = HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT;
+ if (parts.length > 1) {
+ port = Integer.parseInt(parts[1]);
+ }
+ hosts.add(ServerName.valueOf(host, port, -1));
+ }
+ return hosts.toArray(new ServerName[hosts.size()]);
+ }
+
+ /**
+ * Run the tool.
+ * @param args Command line arguments.
+ */
+ public static void main(String args[]) {
+ for(ServerName server: readZKNodes(HBaseConfiguration.create())) {
+ // bin/zookeeper.sh relies on the "ZK host" string for grepping which is case sensitive.
+ System.out.println("ZK host: " + server.getHostname());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java
new file mode 100644
index 0000000..33cc43e
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Common methods and attributes used by SplitLogManager and SplitLogWorker running distributed splitting of WAL logs.
+ */
+@InterfaceAudience.Private
+public class ZKSplitLog {
+ private static final Log LOG = LogFactory.getLog(ZKSplitLog.class);
+
+ /**
+ * Gets the full path node name for the log file being split.
+ * This method will url encode the filename.
+ * @param zkw zk reference
+ * @param filename log file name (only the basename)
+ */
+ public static String getEncodedNodeName(ZKWatcher zkw, String filename) {
+ return ZNodePaths.joinZNode(zkw.znodePaths.splitLogZNode, encode(filename));
+ }
+
+ public static String getFileName(String node) {
+ String basename = node.substring(node.lastIndexOf('/') + 1);
+ return decode(basename);
+ }
+
+ static String encode(String s) {
+ try {
+ return URLEncoder.encode(s, "UTF-8");
+ } catch (UnsupportedEncodingException e) {
+ throw new RuntimeException("URLENCODER doesn't support UTF-8");
+ }
+ }
+
+ static String decode(String s) {
+ try {
+ return URLDecoder.decode(s, "UTF-8");
+ } catch (UnsupportedEncodingException e) {
+ throw new RuntimeException("URLDecoder doesn't support UTF-8");
+ }
+ }
+
+ public static String getRescanNode(ZKWatcher zkw) {
+ return ZNodePaths.joinZNode(zkw.znodePaths.splitLogZNode, "RESCAN");
+ }
+
+ /**
+ * @param name the last part in path
+ * @return whether the node name represents a rescan node
+ */
+ public static boolean isRescanNode(String name) {
+ return name.startsWith("RESCAN");
+ }
+
+ /**
+ * @param zkw
+ * @param path the absolute path, starts with '/'
+ * @return whether the path represents a rescan node
+ */
+ public static boolean isRescanNode(ZKWatcher zkw, String path) {
+ String prefix = getRescanNode(zkw);
+ if (path.length() <= prefix.length()) {
+ return false;
+ }
+ for (int i = 0; i < prefix.length(); i++) {
+ if (prefix.charAt(i) != path.charAt(i)) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ public static Path getSplitLogDir(Path rootdir, String tmpname) {
+ return new Path(new Path(rootdir, HConstants.SPLIT_LOGDIR_NAME), tmpname);
+ }
+
+ public static void markCorrupted(Path rootdir, String logFileName,
+ FileSystem fs) {
+ Path file = new Path(getSplitLogDir(rootdir, logFileName), "corrupt");
+ try {
+ fs.createNewFile(file);
+ } catch (IOException e) {
+ LOG.warn("Could not flag a log file as corrupted. Failed to create " +
+ file, e);
+ }
+ }
+
+ public static boolean isCorrupted(Path rootdir, String logFileName,
+ FileSystem fs) throws IOException {
+ Path file = new Path(getSplitLogDir(rootdir, logFileName), "corrupt");
+ boolean isCorrupt;
+ isCorrupt = fs.exists(file);
+ return isCorrupt;
+ }
+}
[06/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper
from hbase-server and hbase-client
Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
index a9896ce..2e555a9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
@@ -28,7 +28,6 @@ import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.Arrays;
-import java.util.List;
import java.util.concurrent.CountDownLatch;
import org.apache.commons.logging.Log;
@@ -39,20 +38,18 @@ import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@@ -104,7 +101,7 @@ public class TestMultiSlaveReplication {
utility1.startMiniZKCluster();
MiniZooKeeperCluster miniZK = utility1.getZkCluster();
utility1.setZkCluster(miniZK);
- new ZooKeeperWatcher(conf1, "cluster1", null, true);
+ new ZKWatcher(conf1, "cluster1", null, true);
conf2 = new Configuration(conf1);
conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
@@ -114,11 +111,11 @@ public class TestMultiSlaveReplication {
utility2 = new HBaseTestingUtility(conf2);
utility2.setZkCluster(miniZK);
- new ZooKeeperWatcher(conf2, "cluster2", null, true);
+ new ZKWatcher(conf2, "cluster2", null, true);
utility3 = new HBaseTestingUtility(conf3);
utility3.setZkCluster(miniZK);
- new ZooKeeperWatcher(conf3, "cluster3", null, true);
+ new ZKWatcher(conf3, "cluster3", null, true);
table = new HTableDescriptor(tableName);
HColumnDescriptor fam = new HColumnDescriptor(famName);
@@ -190,7 +187,7 @@ public class TestMultiSlaveReplication {
// Even if the log was rolled in the middle of the replication
// "row" is still replication.
checkRow(row, 1, htable2);
- // Replication thread of cluster 2 may be sleeping, and since row2 is not there in it,
+ // Replication thread of cluster 2 may be sleeping, and since row2 is not there in it,
// we should wait before checking.
checkWithWait(row, 1, htable3);
@@ -244,7 +241,7 @@ public class TestMultiSlaveReplication {
region.getWAL().unregisterWALActionsListener(listener);
}
-
+
private void checkWithWait(byte[] row, int count, Table table) throws Exception {
Get get = new Get(row);
for (int i = 0; i < NB_RETRIES; i++) {
@@ -267,7 +264,7 @@ public class TestMultiSlaveReplication {
}
}
}
-
+
private void checkRow(byte[] row, int count, Table... tables) throws IOException {
Get get = new Get(row);
for (Table table : tables) {
@@ -299,7 +296,7 @@ public class TestMultiSlaveReplication {
if (removedFromAll) {
break;
} else {
- Thread.sleep(SLEEP_TIME);
+ Thread.sleep(SLEEP_TIME);
}
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
index 6572404..84ce9a3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
@@ -50,7 +50,7 @@ import org.apache.hadoop.hbase.testclassification.FlakeyTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Rule;
@@ -114,7 +114,7 @@ public class TestPerTableCFReplication {
utility1 = new HBaseTestingUtility(conf1);
utility1.startMiniZKCluster();
MiniZooKeeperCluster miniZK = utility1.getZkCluster();
- new ZooKeeperWatcher(conf1, "cluster1", null, true);
+ new ZKWatcher(conf1, "cluster1", null, true);
conf2 = new Configuration(conf1);
conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
@@ -124,11 +124,11 @@ public class TestPerTableCFReplication {
utility2 = new HBaseTestingUtility(conf2);
utility2.setZkCluster(miniZK);
- new ZooKeeperWatcher(conf2, "cluster3", null, true);
+ new ZKWatcher(conf2, "cluster3", null, true);
utility3 = new HBaseTestingUtility(conf3);
utility3.setZkCluster(miniZK);
- new ZooKeeperWatcher(conf3, "cluster3", null, true);
+ new ZKWatcher(conf3, "cluster3", null, true);
table = new HTableDescriptor(tableName);
HColumnDescriptor fam = new HColumnDescriptor(famName);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
index 58b97b9..f7d1009 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.runners.Parameterized.Parameter;
@@ -64,8 +64,8 @@ public class TestReplicationBase {
protected static Configuration conf2;
protected static Configuration CONF_WITH_LOCALFS;
- protected static ZooKeeperWatcher zkw1;
- protected static ZooKeeperWatcher zkw2;
+ protected static ZKWatcher zkw1;
+ protected static ZKWatcher zkw2;
protected static ReplicationAdmin admin;
protected static Admin hbaseAdmin;
@@ -127,8 +127,8 @@ public class TestReplicationBase {
MiniZooKeeperCluster miniZK = utility1.getZkCluster();
// Have to reget conf1 in case zk cluster location different
// than default
- conf1 = utility1.getConfiguration();
- zkw1 = new ZooKeeperWatcher(conf1, "cluster1", null, true);
+ conf1 = utility1.getConfiguration();
+ zkw1 = new ZKWatcher(conf1, "cluster1", null, true);
admin = new ReplicationAdmin(conf1);
LOG.info("Setup first Zk");
@@ -140,7 +140,7 @@ public class TestReplicationBase {
utility2 = new HBaseTestingUtility(conf2);
utility2.setZkCluster(miniZK);
- zkw2 = new ZooKeeperWatcher(conf2, "cluster2", null, true);
+ zkw2 = new ZKWatcher(conf2, "cluster2", null, true);
LOG.info("Setup second Zk");
CONF_WITH_LOCALFS = HBaseConfiguration.create(conf1);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
index cc09751..97daa63 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
@@ -30,8 +30,8 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.AfterClass;
@@ -54,7 +54,7 @@ public class TestReplicationStateHBaseImpl {
private static Configuration conf;
private static HBaseTestingUtility utility;
- private static ZooKeeperWatcher zkw;
+ private static ZKWatcher zkw;
private static String replicationZNode;
private static ReplicationQueues rq1;
@@ -413,7 +413,7 @@ public class TestReplicationStateHBaseImpl {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return null;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
index dde7868..bb9f665 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
@@ -41,8 +41,8 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.AfterClass;
@@ -58,7 +58,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
private static Configuration conf;
private static HBaseTestingUtility utility;
- private static ZooKeeperWatcher zkw;
+ private static ZKWatcher zkw;
private static String replicationZNode;
private ReplicationQueuesZKImpl rqZK;
@@ -80,7 +80,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
// Add a dummy region server and set up the cluster id
Configuration testConf = new Configuration(conf);
testConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, baseZKNode);
- ZooKeeperWatcher zkw1 = new ZooKeeperWatcher(testConf, "test1", null);
+ ZKWatcher zkw1 = new ZKWatcher(testConf, "test1", null);
String fakeRs = ZNodePaths.joinZNode(zkw1.znodePaths.rsZNode, "hostname1.example.org:1234");
ZKUtil.createWithParents(zkw1, fakeRs);
ZKClusterId.setClusterId(zkw1, new ClusterId());
@@ -152,7 +152,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return zkw;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTableBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTableBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTableBase.java
index 83fdad7..665eedb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTableBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTableBase.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@@ -43,7 +43,7 @@ public class TestReplicationTableBase {
private static long TIME_OUT_MILLIS = 3000;
private static Configuration conf;
private static HBaseTestingUtility utility;
- private static ZooKeeperWatcher zkw;
+ private static ZKWatcher zkw;
private static ReplicationTableBase rb;
private static ReplicationQueues rq;
private static ReplicationQueuesClient rqc;
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
index f6e7b39..2db4f76 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
@@ -43,8 +43,8 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
@@ -67,7 +67,7 @@ public class TestReplicationTrackerZKImpl {
private static HBaseTestingUtility utility;
// Each one of the below variables are reinitialized before every test case
- private ZooKeeperWatcher zkw;
+ private ZKWatcher zkw;
private ReplicationPeers rp;
private ReplicationTracker rt;
private AtomicInteger rsRemovedCount;
@@ -82,7 +82,7 @@ public class TestReplicationTrackerZKImpl {
utility = new HBaseTestingUtility();
utility.startMiniZKCluster();
conf = utility.getConfiguration();
- ZooKeeperWatcher zk = HBaseTestingUtility.getZooKeeperWatcher(utility);
+ ZKWatcher zk = HBaseTestingUtility.getZooKeeperWatcher(utility);
ZKUtil.createWithParents(zk, zk.znodePaths.rsZNode);
}
@@ -193,7 +193,7 @@ public class TestReplicationTrackerZKImpl {
int exists = 0;
int hyphen = 0;
rp.registerPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
-
+
try{
rp.registerPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
}catch(IllegalArgumentException e){
@@ -207,11 +207,11 @@ public class TestReplicationTrackerZKImpl {
}
assertEquals(1, exists);
assertEquals(1, hyphen);
-
+
// clean up
rp.unregisterPeer("6");
}
-
+
private class DummyReplicationListener implements ReplicationListener {
@Override
@@ -252,7 +252,7 @@ public class TestReplicationTrackerZKImpl {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return zkw;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
index 7a8e639..1c5aa71 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
@@ -54,7 +54,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Rule;
@@ -102,14 +102,14 @@ public class TestSerialReplication {
utility1 = new HBaseTestingUtility(conf1);
utility1.startMiniZKCluster();
MiniZooKeeperCluster miniZK = utility1.getZkCluster();
- new ZooKeeperWatcher(conf1, "cluster1", null, true);
+ new ZKWatcher(conf1, "cluster1", null, true);
conf2 = new Configuration(conf1);
conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
utility2 = new HBaseTestingUtility(conf2);
utility2.setZkCluster(miniZK);
- new ZooKeeperWatcher(conf2, "cluster2", null, true);
+ new ZKWatcher(conf2, "cluster2", null, true);
utility1.startMiniCluster(1, 10);
utility2.startMiniCluster(1, 1);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
index 1a02317..cb895ca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@@ -55,7 +55,7 @@ public class TestTableCFsUpdater extends TableCFsUpdater {
private static final Log LOG = LogFactory.getLog(TestTableCFsUpdater.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
- private static ZooKeeperWatcher zkw = null;
+ private static ZKWatcher zkw = null;
private static Abortable abortable = null;
@Rule
@@ -80,7 +80,7 @@ public class TestTableCFsUpdater extends TableCFsUpdater {
return false;
}
};
- zkw = new ZooKeeperWatcher(conf, "TableCFs", abortable, true);
+ zkw = new ZKWatcher(conf, "TableCFs", abortable, true);
}
@AfterClass
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalThrottler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalThrottler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalThrottler.java
index 2469c7c..9b1648f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalThrottler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalThrottler.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
@@ -86,14 +86,14 @@ public class TestGlobalThrottler {
utility1 = new HBaseTestingUtility(conf1);
utility1.startMiniZKCluster();
MiniZooKeeperCluster miniZK = utility1.getZkCluster();
- new ZooKeeperWatcher(conf1, "cluster1", null, true);
+ new ZKWatcher(conf1, "cluster1", null, true);
conf2 = new Configuration(conf1);
conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
utility2 = new HBaseTestingUtility(conf2);
utility2.setZkCluster(miniZK);
- new ZooKeeperWatcher(conf2, "cluster2", null, true);
+ new ZKWatcher(conf2, "cluster2", null, true);
ReplicationAdmin admin1 = new ReplicationAdmin(conf1);
ReplicationPeerConfig rpc = new ReplicationPeerConfig();
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index dd1e19a..7730100 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -83,7 +83,7 @@ import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@@ -116,7 +116,7 @@ public abstract class TestReplicationSourceManager {
protected static ReplicationSourceManager manager;
- protected static ZooKeeperWatcher zkw;
+ protected static ZKWatcher zkw;
protected static HTableDescriptor htd;
@@ -149,7 +149,7 @@ public abstract class TestReplicationSourceManager {
protected static void setupZkAndReplication() throws Exception {
// The implementing class should set up the conf
assertNotNull(conf);
- zkw = new ZooKeeperWatcher(conf, "test", null);
+ zkw = new ZKWatcher(conf, "test", null);
ZKUtil.createWithParents(zkw, "/hbase/replication");
ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1");
ZKUtil.setData(zkw, "/hbase/replication/peers/1",
@@ -668,7 +668,7 @@ public abstract class TestReplicationSourceManager {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return zkw;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
index ad8cb14..f9d0b1c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
@@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.TestTableName;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@@ -509,7 +509,7 @@ public class TestAccessController2 extends SecureTestUtil {
// Namespace needs this, as they follow the lazy creation of ACL znode.
grantOnNamespace(TEST_UTIL, TESTGROUP1_USER1.getShortName(), ns, Action.ADMIN);
- ZooKeeperWatcher zkw = TEST_UTIL.getMiniHBaseCluster().getMaster().getZooKeeper();
+ ZKWatcher zkw = TEST_UTIL.getMiniHBaseCluster().getMaster().getZooKeeper();
assertTrue("The acl znode for table should exist", ZKUtil.checkExists(zkw, baseAclZNode +
table.getNameAsString()) != -1);
assertTrue("The acl znode for namespace should exist", ZKUtil.checkExists(zkw, baseAclZNode +
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
index 30e8396..607ea8c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.SecurityTests;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@@ -61,7 +61,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
public class TestTablePermissions {
private static final Log LOG = LogFactory.getLog(TestTablePermissions.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
- private static ZooKeeperWatcher ZKW;
+ private static ZKWatcher ZKW;
private final static Abortable ABORTABLE = new Abortable() {
private final AtomicBoolean abort = new AtomicBoolean(false);
@@ -97,7 +97,7 @@ public class TestTablePermissions {
// Wait for the ACL table to become available
UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);
- ZKW = new ZooKeeperWatcher(UTIL.getConfiguration(),
+ ZKW = new ZKWatcher(UTIL.getConfiguration(),
"TestTablePermissions", ABORTABLE);
UTIL.createTable(TEST_TABLE, TEST_FAMILY);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
index 76de0c6..18fb15f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.SecurityTests;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -75,9 +75,9 @@ public class TestZKPermissionWatcher {
// start minicluster
UTIL.startMiniCluster();
- AUTH_A = TableAuthManager.getOrCreate(new ZooKeeperWatcher(conf,
+ AUTH_A = TableAuthManager.getOrCreate(new ZKWatcher(conf,
"TestZKPermissionsWatcher_1", ABORTABLE), conf);
- AUTH_B = TableAuthManager.getOrCreate(new ZooKeeperWatcher(conf,
+ AUTH_B = TableAuthManager.getOrCreate(new ZKWatcher(conf,
"TestZKPermissionsWatcher_2", ABORTABLE), conf);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
index 5a0b6bd..3e0bddc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
@@ -76,7 +76,7 @@ import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.net.DNS;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.security.authorize.Service;
@@ -127,7 +127,7 @@ public class TestTokenAuthentication {
private HBaseTestingUtility TEST_UTIL;
private RpcServerInterface rpcServer;
private InetSocketAddress isa;
- private ZooKeeperWatcher zookeeper;
+ private ZKWatcher zookeeper;
private Sleeper sleeper;
private boolean started = false;
private boolean aborted = false;
@@ -220,7 +220,7 @@ public class TestTokenAuthentication {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return zookeeper;
}
@@ -261,7 +261,7 @@ public class TestTokenAuthentication {
// ZK configuration must _not_ have hbase.security.authentication or it will require SASL auth
Configuration zkConf = new Configuration(conf);
zkConf.set(User.HBASE_SECURITY_CONF_KEY, "simple");
- this.zookeeper = new ZooKeeperWatcher(zkConf, TokenServer.class.getSimpleName(),
+ this.zookeeper = new ZKWatcher(zkConf, TokenServer.class.getSimpleName(),
this, true);
this.rpcServer.start();
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java
index 857cdd0..ea07338 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.SecurityTests;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -71,13 +71,13 @@ public class TestZKSecretWatcher {
private static class AuthenticationTokenSecretManagerForTest
extends AuthenticationTokenSecretManager {
private CountDownLatch latch = new CountDownLatch(1);
-
+
public AuthenticationTokenSecretManagerForTest(Configuration conf,
- ZooKeeperWatcher zk, String serverName,
- long keyUpdateInterval, long tokenMaxLifetime) {
+ ZKWatcher zk, String serverName,
+ long keyUpdateInterval, long tokenMaxLifetime) {
super(conf, zk, serverName, keyUpdateInterval, tokenMaxLifetime);
}
-
+
@Override
synchronized boolean removeKey(Integer keyId) {
boolean b = super.removeKey(keyId);
@@ -86,19 +86,19 @@ public class TestZKSecretWatcher {
}
return b;
}
-
+
CountDownLatch getLatch() {
return latch;
}
}
-
+
@BeforeClass
public static void setupBeforeClass() throws Exception {
TEST_UTIL = new HBaseTestingUtility();
TEST_UTIL.startMiniZKCluster();
Configuration conf = TEST_UTIL.getConfiguration();
- ZooKeeperWatcher zk = newZK(conf, "server1", new MockAbortable());
+ ZKWatcher zk = newZK(conf, "server1", new MockAbortable());
AuthenticationTokenSecretManagerForTest[] tmp = new AuthenticationTokenSecretManagerForTest[2];
tmp[0] = new AuthenticationTokenSecretManagerForTest(
conf, zk, "server1", 60*60*1000, 60*1000);
@@ -179,7 +179,7 @@ public class TestZKSecretWatcher {
// bring up a new slave
Configuration conf = TEST_UTIL.getConfiguration();
- ZooKeeperWatcher zk = newZK(conf, "server3", new MockAbortable());
+ ZKWatcher zk = newZK(conf, "server3", new MockAbortable());
KEY_SLAVE2 = new AuthenticationTokenSecretManager(
conf, zk, "server3", 60*60*1000, 60*1000);
KEY_SLAVE2.start();
@@ -233,7 +233,7 @@ public class TestZKSecretWatcher {
assertTrue(newCurrent.getKeyId() > current.getKeyId());
// add another slave
- ZooKeeperWatcher zk3 = newZK(conf, "server4", new MockAbortable());
+ ZKWatcher zk3 = newZK(conf, "server4", new MockAbortable());
KEY_SLAVE3 = new AuthenticationTokenSecretManager(
conf, zk3, "server4", 60*60*1000, 60*1000);
KEY_SLAVE3.start();
@@ -275,10 +275,10 @@ public class TestZKSecretWatcher {
assertTrue(newCurrent2.getKeyId() > current2.getKeyId());
}
- private static ZooKeeperWatcher newZK(Configuration conf, String name,
- Abortable abort) throws Exception {
+ private static ZKWatcher newZK(Configuration conf, String name,
+ Abortable abort) throws Exception {
Configuration copy = HBaseConfiguration.create(conf);
- ZooKeeperWatcher zk = new ZooKeeperWatcher(copy, name, abort);
+ ZKWatcher zk = new ZKWatcher(copy, name, abort);
return zk;
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java
index fcadc9d..1757ddd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java
@@ -28,8 +28,8 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
@@ -55,7 +55,7 @@ public class TestZKSecretWatcherRefreshKeys {
return abort;
}
}
-
+
@BeforeClass
public static void setupBeforeClass() throws Exception {
TEST_UTIL = new HBaseTestingUtility();
@@ -67,19 +67,19 @@ public class TestZKSecretWatcherRefreshKeys {
TEST_UTIL.shutdownMiniZKCluster();
}
- private static ZooKeeperWatcher newZK(Configuration conf, String name,
- Abortable abort) throws Exception {
+ private static ZKWatcher newZK(Configuration conf, String name,
+ Abortable abort) throws Exception {
Configuration copy = HBaseConfiguration.create(conf);
- ZooKeeperWatcher zk = new ZooKeeperWatcher(copy, name, abort);
+ ZKWatcher zk = new ZKWatcher(copy, name, abort);
return zk;
}
@Test
public void testRefreshKeys() throws Exception {
Configuration conf = TEST_UTIL.getConfiguration();
- ZooKeeperWatcher zk = newZK(conf, "127.0.0.1", new MockAbortable());
- AuthenticationTokenSecretManager keyManager =
- new AuthenticationTokenSecretManager(conf, zk, "127.0.0.1",
+ ZKWatcher zk = newZK(conf, "127.0.0.1", new MockAbortable());
+ AuthenticationTokenSecretManager keyManager =
+ new AuthenticationTokenSecretManager(conf, zk, "127.0.0.1",
60 * 60 * 1000, 60 * 1000);
ZKSecretWatcher watcher = new ZKSecretWatcher(conf, zk, keyManager);
ZKUtil.deleteChildrenRecursively(zk, watcher.getKeysParentZNode());
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
index e5b1c39..c2da3ec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
@@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.SecurityTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.Before;
import org.junit.experimental.categories.Category;
@@ -109,7 +109,7 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit
TEST_UTIL = new HBaseTestingUtility(conf);
TEST_UTIL.startMiniZKCluster();
MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster();
- zkw1 = new ZooKeeperWatcher(conf, "cluster1", null, true);
+ zkw1 = new ZKWatcher(conf, "cluster1", null, true);
admin = TEST_UTIL.getAdmin();
// Base conf2 on conf1 so it gets the right zk cluster.
@@ -124,7 +124,7 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit
setVisibilityLabelServiceImpl(conf1, ExpAsStringVisibilityLabelServiceImpl.class);
TEST_UTIL1 = new HBaseTestingUtility(conf1);
TEST_UTIL1.setZkCluster(miniZK);
- zkw2 = new ZooKeeperWatcher(conf1, "cluster2", null, true);
+ zkw2 = new ZKWatcher(conf1, "cluster2", null, true);
TEST_UTIL.startMiniCluster(1);
// Wait for the labels table to become available
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
index 513c765..678227b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
@@ -74,7 +74,7 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
@@ -109,8 +109,8 @@ public class TestVisibilityLabelsReplication {
public final static byte[] fam = Bytes.toBytes("info");
public final static byte[] qual = Bytes.toBytes("qual");
public final static byte[] value = Bytes.toBytes("value");
- protected static ZooKeeperWatcher zkw1;
- protected static ZooKeeperWatcher zkw2;
+ protected static ZKWatcher zkw1;
+ protected static ZKWatcher zkw2;
protected static int expected[] = { 4, 6, 4, 0, 3 };
private static final String NON_VISIBILITY = "non-visibility";
protected static String[] expectedVisString = {
@@ -160,7 +160,7 @@ public class TestVisibilityLabelsReplication {
TEST_UTIL = new HBaseTestingUtility(conf);
TEST_UTIL.startMiniZKCluster();
MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster();
- zkw1 = new ZooKeeperWatcher(conf, "cluster1", null, true);
+ zkw1 = new ZKWatcher(conf, "cluster1", null, true);
admin = TEST_UTIL.getAdmin();
// Base conf2 on conf1 so it gets the right zk cluster.
@@ -176,7 +176,7 @@ public class TestVisibilityLabelsReplication {
USER1 = User.createUserForTesting(conf1, "user1", new String[] {});
TEST_UTIL1 = new HBaseTestingUtility(conf1);
TEST_UTIL1.setZkCluster(miniZK);
- zkw2 = new ZooKeeperWatcher(conf1, "cluster2", null, true);
+ zkw2 = new ZKWatcher(conf1, "cluster2", null, true);
TEST_UTIL.startMiniCluster(1);
// Wait for the labels table to become available
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
index db44219..4acec76 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
/**
* Basic mock Server for handler tests.
@@ -39,10 +39,10 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
public class MockServer implements Server {
private static final Log LOG = LogFactory.getLog(MockServer.class);
final static ServerName NAME = ServerName.valueOf("MockServer", 123, -1);
-
+
boolean stopped;
boolean aborted;
- final ZooKeeperWatcher zk;
+ final ZKWatcher zk;
final HBaseTestingUtility htu;
@SuppressWarnings("unused")
@@ -66,7 +66,7 @@ public class MockServer implements Server {
throws ZooKeeperConnectionException, IOException {
this.htu = htu;
this.zk = zkw?
- new ZooKeeperWatcher(htu.getConfiguration(), NAME.toString(), this, true):
+ new ZKWatcher(htu.getConfiguration(), NAME.toString(), this, true):
null;
}
@@ -94,7 +94,7 @@ public class MockServer implements Server {
}
@Override
- public ZooKeeperWatcher getZooKeeper() {
+ public ZKWatcher getZooKeeper() {
return this.zk;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
index 7463da1..ba3475e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
@@ -86,7 +86,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
@@ -1493,7 +1493,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
Assert.assertEquals(1, replicationAdmin.getPeersCount());
// create replicator
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "Test Hbase Fsck", connection);
+ ZKWatcher zkw = new ZKWatcher(conf, "Test Hbase Fsck", connection);
ReplicationQueues repQueues =
ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, connection,
zkw));
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
index e71210d..6fa99fd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
@@ -71,7 +71,7 @@ public class TestRecoverableZooKeeper {
public void testSetDataVersionMismatchInLoop() throws Exception {
String znode = "/hbase/splitWAL/9af7cfc9b15910a0b3d714bf40a3248f";
Configuration conf = TEST_UTIL.getConfiguration();
- ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testSetDataVersionMismatchInLoop",
+ ZKWatcher zkw = new ZKWatcher(conf, "testSetDataVersionMismatchInLoop",
abortable, true);
String ensemble = ZKConfig.getZKQuorumServersString(conf);
RecoverableZooKeeper rzk = ZKUtil.connect(conf, ensemble, zkw);
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java
index 1faf8e5..69915a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java
@@ -61,12 +61,12 @@ public class TestZKLeaderManager {
private static class MockLeader extends Thread implements Stoppable {
private boolean stopped;
- private ZooKeeperWatcher watcher;
+ private ZKWatcher watcher;
private ZKLeaderManager zkLeader;
private AtomicBoolean master = new AtomicBoolean(false);
private int index;
- public MockLeader(ZooKeeperWatcher watcher, int index) {
+ public MockLeader(ZKWatcher watcher, int index) {
setDaemon(true);
setName("TestZKLeaderManager-leader-" + index);
this.index = index;
@@ -83,7 +83,7 @@ public class TestZKLeaderManager {
return index;
}
- public ZooKeeperWatcher getWatcher() {
+ public ZKWatcher getWatcher() {
return watcher;
}
@@ -132,7 +132,7 @@ public class TestZKLeaderManager {
MockAbortable abortable = new MockAbortable();
CANDIDATES = new MockLeader[3];
for (int i = 0; i < 3; i++) {
- ZooKeeperWatcher watcher = newZK(conf, "server"+i, abortable);
+ ZKWatcher watcher = newZK(conf, "server"+i, abortable);
CANDIDATES[i] = new MockLeader(watcher, i);
CANDIDATES[i].start();
}
@@ -225,10 +225,10 @@ public class TestZKLeaderManager {
return currentLeader;
}
- private static ZooKeeperWatcher newZK(Configuration conf, String name,
- Abortable abort) throws Exception {
+ private static ZKWatcher newZK(Configuration conf, String name,
+ Abortable abort) throws Exception {
Configuration copy = HBaseConfiguration.create(conf);
- ZooKeeperWatcher zk = new ZooKeeperWatcher(copy, name, abort);
+ ZKWatcher zk = new ZKWatcher(copy, name, abort);
return zk;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java
new file mode 100644
index 0000000..c99bd05
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java
@@ -0,0 +1,118 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.security.Permission;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MiscTests.class, SmallTests.class})
+public class TestZKMainServer {
+ // ZKMS calls System.exit. Catch the call and prevent exit using trick described up in
+ // http://stackoverflow.com/questions/309396/java-how-to-test-methods-that-call-system-exit
+ protected static class ExitException extends SecurityException {
+ private static final long serialVersionUID = 1L;
+ public final int status;
+ public ExitException(int status) {
+ super("There is no escape!");
+ this.status = status;
+ }
+ }
+
+ private static class NoExitSecurityManager extends SecurityManager {
+ @Override
+ public void checkPermission(Permission perm) {
+ // allow anything.
+ }
+
+ @Override
+ public void checkPermission(Permission perm, Object context) {
+ // allow anything.
+ }
+
+ @Override
+ public void checkExit(int status) {
+ super.checkExit(status);
+ throw new ExitException(status);
+ }
+ }
+
+ /**
+ * We need delete of a znode to work at least.
+ * @throws Exception
+ */
+ @Test
+ public void testCommandLineWorks() throws Exception {
+ System.setSecurityManager(new NoExitSecurityManager());
+ HBaseTestingUtility htu = new HBaseTestingUtility();
+ htu.getConfiguration().setInt(HConstants.ZK_SESSION_TIMEOUT, 1000);
+ htu.startMiniZKCluster();
+ try {
+ ZKWatcher zkw = htu.getZooKeeperWatcher();
+ String znode = "/testCommandLineWorks";
+ ZKUtil.createWithParents(zkw, znode, HConstants.EMPTY_BYTE_ARRAY);
+ ZKUtil.checkExists(zkw, znode);
+ boolean exception = false;
+ try {
+ ZKMainServer.main(new String [] {"-server",
+ "localhost:" + htu.getZkCluster().getClientPort(), "delete", znode});
+ } catch (ExitException ee) {
+ // ZKMS calls System.exit which should trigger this exception.
+ exception = true;
+ }
+ assertTrue(exception);
+ assertEquals(-1, ZKUtil.checkExists(zkw, znode));
+ } finally {
+ htu.shutdownMiniZKCluster();
+ System.setSecurityManager(null); // or save and restore original
+ }
+ }
+
+ @Test
+ public void testHostPortParse() {
+ ZKMainServer parser = new ZKMainServer();
+ Configuration c = HBaseConfiguration.create();
+ assertEquals("localhost:" + c.get(HConstants.ZOOKEEPER_CLIENT_PORT), parser.parse(c));
+ final String port = "1234";
+ c.set(HConstants.ZOOKEEPER_CLIENT_PORT, port);
+ c.set("hbase.zookeeper.quorum", "example.com");
+ assertEquals("example.com:" + port, parser.parse(c));
+ c.set("hbase.zookeeper.quorum", "example1.com,example2.com,example3.com");
+ String ensemble = parser.parse(c);
+ assertTrue(port, ensemble.matches("(example[1-3]\\.com:1234,){2}example[1-3]\\.com:" + port));
+
+ // multiple servers with its own port
+ c.set("hbase.zookeeper.quorum", "example1.com:5678,example2.com:9012,example3.com:3456");
+ ensemble = parser.parse(c);
+ assertEquals(ensemble, "example1.com:5678,example2.com:9012,example3.com:3456");
+
+ // some servers without its own port, which will be assigned the default client port
+ c.set("hbase.zookeeper.quorum", "example1.com:5678,example2.com:9012,example3.com");
+ ensemble = parser.parse(c);
+ assertEquals(ensemble, "example1.com:5678,example2.com:9012,example3.com:" + port);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java
index 7971417..614d4be 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java
@@ -53,7 +53,7 @@ import org.junit.experimental.categories.Category;
public class TestZKMulti {
private static final Log LOG = LogFactory.getLog(TestZKMulti.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
- private static ZooKeeperWatcher zkw = null;
+ private static ZKWatcher zkw = null;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
@@ -70,7 +70,7 @@ public class TestZKMulti {
return false;
}
};
- zkw = new ZooKeeperWatcher(conf,
+ zkw = new ZKWatcher(conf,
"TestZKMulti", abortable, true);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java
new file mode 100644
index 0000000..ac5c11a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java
@@ -0,0 +1,351 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.Semaphore;
+
+import junit.framework.Assert;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.master.TestActiveMasterManager.NodeDeletionListener;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MiscTests.class, MediumTests.class})
+public class TestZKNodeTracker {
+ private static final Log LOG = LogFactory.getLog(TestZKNodeTracker.class);
+ private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+ private final static Random rand = new Random();
+
+ @BeforeClass
+ public static void setUpBeforeClass() throws Exception {
+ TEST_UTIL.startMiniZKCluster();
+ }
+
+ @AfterClass
+ public static void tearDownAfterClass() throws Exception {
+ TEST_UTIL.shutdownMiniZKCluster();
+ }
+
+ /**
+ * Test that we can interrupt a node that is blocked on a wait.
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ @Test public void testInterruptible() throws IOException, InterruptedException {
+ Abortable abortable = new StubAbortable();
+ ZKWatcher zk = new ZKWatcher(TEST_UTIL.getConfiguration(),
+ "testInterruptible", abortable);
+ final TestTracker tracker = new TestTracker(zk, "/xyz", abortable);
+ tracker.start();
+ Thread t = new Thread() {
+ @Override
+ public void run() {
+ try {
+ tracker.blockUntilAvailable();
+ } catch (InterruptedException e) {
+ throw new RuntimeException("Interrupted", e);
+ }
+ }
+ };
+ t.start();
+ while (!t.isAlive()) Threads.sleep(1);
+ tracker.stop();
+ t.join();
+ // If it wasn't interruptible, we'd never get to here.
+ }
+
+ @Test
+ public void testNodeTracker() throws Exception {
+ Abortable abortable = new StubAbortable();
+ ZKWatcher zk = new ZKWatcher(TEST_UTIL.getConfiguration(),
+ "testNodeTracker", abortable);
+ ZKUtil.createAndFailSilent(zk, zk.znodePaths.baseZNode);
+
+ final String node =
+ ZNodePaths.joinZNode(zk.znodePaths.baseZNode, new Long(rand.nextLong()).toString());
+
+ final byte [] dataOne = Bytes.toBytes("dataOne");
+ final byte [] dataTwo = Bytes.toBytes("dataTwo");
+
+ // Start a ZKNT with no node currently available
+ TestTracker localTracker = new TestTracker(zk, node, abortable);
+ localTracker.start();
+ zk.registerListener(localTracker);
+
+ // Make sure we don't have a node
+ assertNull(localTracker.getData(false));
+
+ // Spin up a thread with another ZKNT and have it block
+ WaitToGetDataThread thread = new WaitToGetDataThread(zk, node);
+ thread.start();
+
+ // Verify the thread doesn't have a node
+ assertFalse(thread.hasData);
+
+ // Now, start a new ZKNT with the node already available
+ TestTracker secondTracker = new TestTracker(zk, node, null);
+ secondTracker.start();
+ zk.registerListener(secondTracker);
+
+ // Put up an additional zk listener so we know when zk event is done
+ TestingZKListener zkListener = new TestingZKListener(zk, node);
+ zk.registerListener(zkListener);
+ assertEquals(0, zkListener.createdLock.availablePermits());
+
+ // Create a completely separate zk connection for test triggers and avoid
+ // any weird watcher interactions from the test
+ final ZooKeeper zkconn = new ZooKeeper(
+ ZKConfig.getZKQuorumServersString(TEST_UTIL.getConfiguration()), 60000,
+ new StubWatcher());
+
+ // Add the node with data one
+ zkconn.create(node, dataOne, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+ // Wait for the zk event to be processed
+ zkListener.waitForCreation();
+ thread.join();
+
+ // Both trackers should have the node available with data one
+ assertNotNull(localTracker.getData(false));
+ assertNotNull(localTracker.blockUntilAvailable());
+ assertTrue(Bytes.equals(localTracker.getData(false), dataOne));
+ assertTrue(thread.hasData);
+ assertTrue(Bytes.equals(thread.tracker.getData(false), dataOne));
+ LOG.info("Successfully got data one");
+
+ // Make sure it's available and with the expected data
+ assertNotNull(secondTracker.getData(false));
+ assertNotNull(secondTracker.blockUntilAvailable());
+ assertTrue(Bytes.equals(secondTracker.getData(false), dataOne));
+ LOG.info("Successfully got data one with the second tracker");
+
+ // Drop the node
+ zkconn.delete(node, -1);
+ zkListener.waitForDeletion();
+
+ // Create a new thread but with the existing thread's tracker to wait
+ TestTracker threadTracker = thread.tracker;
+ thread = new WaitToGetDataThread(zk, node, threadTracker);
+ thread.start();
+
+ // Verify other guys don't have data
+ assertFalse(thread.hasData);
+ assertNull(secondTracker.getData(false));
+ assertNull(localTracker.getData(false));
+ LOG.info("Successfully made unavailable");
+
+ // Create with second data
+ zkconn.create(node, dataTwo, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+ // Wait for the zk event to be processed
+ zkListener.waitForCreation();
+ thread.join();
+
+ // All trackers should have the node available with data two
+ assertNotNull(localTracker.getData(false));
+ assertNotNull(localTracker.blockUntilAvailable());
+ assertTrue(Bytes.equals(localTracker.getData(false), dataTwo));
+ assertNotNull(secondTracker.getData(false));
+ assertNotNull(secondTracker.blockUntilAvailable());
+ assertTrue(Bytes.equals(secondTracker.getData(false), dataTwo));
+ assertTrue(thread.hasData);
+ assertTrue(Bytes.equals(thread.tracker.getData(false), dataTwo));
+ LOG.info("Successfully got data two on all trackers and threads");
+
+ // Change the data back to data one
+ zkconn.setData(node, dataOne, -1);
+
+ // Wait for zk event to be processed
+ zkListener.waitForDataChange();
+
+ // All trackers should have the node available with data one
+ assertNotNull(localTracker.getData(false));
+ assertNotNull(localTracker.blockUntilAvailable());
+ assertTrue(Bytes.equals(localTracker.getData(false), dataOne));
+ assertNotNull(secondTracker.getData(false));
+ assertNotNull(secondTracker.blockUntilAvailable());
+ assertTrue(Bytes.equals(secondTracker.getData(false), dataOne));
+ assertTrue(thread.hasData);
+ assertTrue(Bytes.equals(thread.tracker.getData(false), dataOne));
+ LOG.info("Successfully got data one following a data change on all trackers and threads");
+ }
+
+ public static class WaitToGetDataThread extends Thread {
+
+ TestTracker tracker;
+ boolean hasData;
+
+ public WaitToGetDataThread(ZKWatcher zk, String node) {
+ tracker = new TestTracker(zk, node, null);
+ tracker.start();
+ zk.registerListener(tracker);
+ hasData = false;
+ }
+
+ public WaitToGetDataThread(ZKWatcher zk, String node,
+ TestTracker tracker) {
+ this.tracker = tracker;
+ hasData = false;
+ }
+
+ @Override
+ public void run() {
+ LOG.info("Waiting for data to be available in WaitToGetDataThread");
+ try {
+ tracker.blockUntilAvailable();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ LOG.info("Data now available in tracker from WaitToGetDataThread");
+ hasData = true;
+ }
+ }
+
+ public static class TestTracker extends ZKNodeTracker {
+ public TestTracker(ZKWatcher watcher, String node,
+ Abortable abortable) {
+ super(watcher, node, abortable);
+ }
+ }
+
+ public static class TestingZKListener extends ZKListener {
+ private static final Log LOG = LogFactory.getLog(NodeDeletionListener.class);
+
+ private Semaphore deletedLock;
+ private Semaphore createdLock;
+ private Semaphore changedLock;
+ private String node;
+
+ public TestingZKListener(ZKWatcher watcher, String node) {
+ super(watcher);
+ deletedLock = new Semaphore(0);
+ createdLock = new Semaphore(0);
+ changedLock = new Semaphore(0);
+ this.node = node;
+ }
+
+ @Override
+ public void nodeDeleted(String path) {
+ if(path.equals(node)) {
+ LOG.debug("nodeDeleted(" + path + ")");
+ deletedLock.release();
+ }
+ }
+
+ @Override
+ public void nodeCreated(String path) {
+ if(path.equals(node)) {
+ LOG.debug("nodeCreated(" + path + ")");
+ createdLock.release();
+ }
+ }
+
+ @Override
+ public void nodeDataChanged(String path) {
+ if(path.equals(node)) {
+ LOG.debug("nodeDataChanged(" + path + ")");
+ changedLock.release();
+ }
+ }
+
+ public void waitForDeletion() throws InterruptedException {
+ deletedLock.acquire();
+ }
+
+ public void waitForCreation() throws InterruptedException {
+ createdLock.acquire();
+ }
+
+ public void waitForDataChange() throws InterruptedException {
+ changedLock.acquire();
+ }
+ }
+
+ public static class StubAbortable implements Abortable {
+ @Override
+ public void abort(final String msg, final Throwable t) {}
+
+ @Override
+ public boolean isAborted() {
+ return false;
+ }
+
+ }
+
+ public static class StubWatcher implements Watcher {
+ @Override
+ public void process(WatchedEvent event) {}
+ }
+
+ @Test
+ public void testCleanZNode() throws Exception {
+ ZKWatcher zkw = new ZKWatcher(TEST_UTIL.getConfiguration(),
+ "testNodeTracker", new TestZKNodeTracker.StubAbortable());
+
+ final ServerName sn = ServerName.valueOf("127.0.0.1:52", 45L);
+
+ ZKUtil.createAndFailSilent(zkw,
+ TEST_UTIL.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT,
+ HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT));
+
+ final String nodeName = zkw.znodePaths.masterAddressZNode;
+
+ // Check that we manage the case when there is no data
+ ZKUtil.createAndFailSilent(zkw, nodeName);
+ MasterAddressTracker.deleteIfEquals(zkw, sn.toString());
+ Assert.assertFalse(ZKUtil.getData(zkw, nodeName) == null);
+
+ // Check that we don't delete if we're not supposed to
+ ZKUtil.setData(zkw, nodeName, MasterAddressTracker.toByteArray(sn, 0));
+ MasterAddressTracker.deleteIfEquals(zkw, ServerName.valueOf("127.0.0.2:52", 45L).toString());
+ Assert.assertFalse(ZKUtil.getData(zkw, nodeName) == null);
+
+ // Check that we delete when we're supposed to
+ ZKUtil.setData(zkw, nodeName,MasterAddressTracker.toByteArray(sn, 0));
+ MasterAddressTracker.deleteIfEquals(zkw, sn.toString());
+ Assert.assertTrue( ZKUtil.getData(zkw, nodeName)== null );
+
+ // Check that we support the case when the znode does not exist
+ MasterAddressTracker.deleteIfEquals(zkw, sn.toString()); // must not throw an exception
+ }
+
+}
+
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java
index 05ad73e..c5bce00 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java
@@ -49,9 +49,9 @@ public class TestZooKeeperACL {
private final static HBaseTestingUtility TEST_UTIL =
new HBaseTestingUtility();
- private static ZooKeeperWatcher zkw;
+ private static ZKWatcher zkw;
private static boolean secureZKAvailable;
-
+
@BeforeClass
public static void setUpBeforeClass() throws Exception {
File saslConfFile = File.createTempFile("tmp", "jaas.conf");
@@ -76,7 +76,7 @@ public class TestZooKeeperACL {
TEST_UTIL.getConfiguration().setInt("hbase.zookeeper.property.maxClientCnxns", 1000);
// If Hadoop is missing HADOOP-7070 the cluster will fail to start due to
- // the JAAS configuration required by ZK being clobbered by Hadoop
+ // the JAAS configuration required by ZK being clobbered by Hadoop
try {
TEST_UTIL.startMiniCluster();
} catch (IOException e) {
@@ -84,7 +84,7 @@ public class TestZooKeeperACL {
secureZKAvailable = false;
return;
}
- zkw = new ZooKeeperWatcher(
+ zkw = new ZKWatcher(
new Configuration(TEST_UTIL.getConfiguration()),
TestZooKeeper.class.getName(), null);
}
@@ -112,7 +112,7 @@ public class TestZooKeeperACL {
}
/**
- * Create a node and check its ACL. When authentication is enabled on
+ * Create a node and check its ACL. When authentication is enabled on
* ZooKeeper, all nodes (except /hbase/root-region-server, /hbase/master
* and /hbase/hbaseid) should be created so that only the hbase server user
* (master or region server user) that created them can access them, and
@@ -285,7 +285,7 @@ public class TestZooKeeperACL {
assertEquals(testJaasConfig, false);
saslConfFile.delete();
}
-
+
/**
* Check if Programmatic way of setting zookeeper security settings is valid.
*/
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperMainServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperMainServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperMainServer.java
deleted file mode 100644
index 368919c..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperMainServer.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.security.Permission;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({MiscTests.class, SmallTests.class})
-public class TestZooKeeperMainServer {
- // ZKMS calls System.exit. Catch the call and prevent exit using trick described up in
- // http://stackoverflow.com/questions/309396/java-how-to-test-methods-that-call-system-exit
- protected static class ExitException extends SecurityException {
- private static final long serialVersionUID = 1L;
- public final int status;
- public ExitException(int status) {
- super("There is no escape!");
- this.status = status;
- }
- }
-
- private static class NoExitSecurityManager extends SecurityManager {
- @Override
- public void checkPermission(Permission perm) {
- // allow anything.
- }
-
- @Override
- public void checkPermission(Permission perm, Object context) {
- // allow anything.
- }
-
- @Override
- public void checkExit(int status) {
- super.checkExit(status);
- throw new ExitException(status);
- }
- }
-
- /**
- * We need delete of a znode to work at least.
- * @throws Exception
- */
- @Test
- public void testCommandLineWorks() throws Exception {
- System.setSecurityManager(new NoExitSecurityManager());
- HBaseTestingUtility htu = new HBaseTestingUtility();
- htu.getConfiguration().setInt(HConstants.ZK_SESSION_TIMEOUT, 1000);
- htu.startMiniZKCluster();
- try {
- ZooKeeperWatcher zkw = htu.getZooKeeperWatcher();
- String znode = "/testCommandLineWorks";
- ZKUtil.createWithParents(zkw, znode, HConstants.EMPTY_BYTE_ARRAY);
- ZKUtil.checkExists(zkw, znode);
- boolean exception = false;
- try {
- ZooKeeperMainServer.main(new String [] {"-server",
- "localhost:" + htu.getZkCluster().getClientPort(), "delete", znode});
- } catch (ExitException ee) {
- // ZKMS calls System.exit which should trigger this exception.
- exception = true;
- }
- assertTrue(exception);
- assertEquals(-1, ZKUtil.checkExists(zkw, znode));
- } finally {
- htu.shutdownMiniZKCluster();
- System.setSecurityManager(null); // or save and restore original
- }
- }
-
- @Test
- public void testHostPortParse() {
- ZooKeeperMainServer parser = new ZooKeeperMainServer();
- Configuration c = HBaseConfiguration.create();
- assertEquals("localhost:" + c.get(HConstants.ZOOKEEPER_CLIENT_PORT), parser.parse(c));
- final String port = "1234";
- c.set(HConstants.ZOOKEEPER_CLIENT_PORT, port);
- c.set("hbase.zookeeper.quorum", "example.com");
- assertEquals("example.com:" + port, parser.parse(c));
- c.set("hbase.zookeeper.quorum", "example1.com,example2.com,example3.com");
- String ensemble = parser.parse(c);
- assertTrue(port, ensemble.matches("(example[1-3]\\.com:1234,){2}example[1-3]\\.com:" + port));
-
- // multiple servers with its own port
- c.set("hbase.zookeeper.quorum", "example1.com:5678,example2.com:9012,example3.com:3456");
- ensemble = parser.parse(c);
- assertEquals(ensemble, "example1.com:5678,example2.com:9012,example3.com:3456");
-
- // some servers without its own port, which will be assigned the default client port
- c.set("hbase.zookeeper.quorum", "example1.com:5678,example2.com:9012,example3.com");
- ensemble = parser.parse(c);
- assertEquals(ensemble, "example1.com:5678,example2.com:9012,example3.com:" + port);
- }
-}