diff --git a/bin/hbase b/bin/hbase index 998bdbeb0cc..d98e7bc0c57 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 diff --git a/bin/hbase.cmd b/bin/hbase.cmd index 070cb616b8e..8e4a59f92de 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 diff --git a/conf/log4j.properties b/conf/log4j.properties index 15545fff80e..6367dbe7af9 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. 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 11f2b75880a..0b01e57e6ea 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. 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 11f2b75880a..0b01e57e6ea 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. diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml index 77e45cbefb8..828d2c6a294 100644 --- a/hbase-assembly/pom.xml +++ b/hbase-assembly/pom.xml @@ -278,6 +278,10 @@ org.apache.hbase hbase-examples + + org.apache.hbase + hbase-zookeeper + diff --git a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml index 8d93ae7f58d..b9bac759430 100644 --- a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml +++ b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml @@ -56,6 +56,7 @@ org.apache.hbase:hbase-shell org.apache.hbase:hbase-testing-util org.apache.hbase:hbase-thrift + org.apache.hbase:hbase-zookeeper 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 eba8518515e..c44be52d59c 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); 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 92bad7fbf77..09cbda3e1b0 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)) { 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 eaa33436053..feebca3db00 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; @@ -608,10 +608,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; 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 acf6ff8f733..b8de9ec088f 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 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;} diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml index a8a75afaa8b..0bffabe9c97 100644 --- a/hbase-replication/pom.xml +++ b/hbase-replication/pom.xml @@ -130,6 +130,10 @@ org.apache.hbase hbase-client + + org.apache.hbase + hbase-zookeeper + org.apache.commons 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 9d12211a96f..3ff6914f825 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); 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 8f09479aa71..214a3136313 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); } 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 cc84c1d89ce..9c4e3fead9b 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<>(); 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 c8328bdec22..c2a5df398cf 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; } 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 67258c7e179..9b79294865d 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) { 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 49e55ef767b..b998f159752 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); } 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 7b1d5c2d24a..95fd29430c6 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); } 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 e96401e2a7d..ad970c67db9 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; 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 ade1c4d42e8..aa72fc5352e 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 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); } 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 cf1ff203b96..1f239f89dea 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) {}; 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 3d152bb1ee2..5fca6595eac 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 prevRSGroups = new HashSet<>(); 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 9077f152304..ba3534de913 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 diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml index 97f8f9dacd0..96e5757653a 100644 --- a/hbase-server/pom.xml +++ b/hbase-server/pom.xml @@ -388,6 +388,10 @@ org.apache.hbase hbase-client + + org.apache.hbase + hbase-zookeeper + org.apache.hbase hbase-replication 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 a6b39f3f7a7..0d9eba89680 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. 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 c4fad44e259..160d058b88e 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 cache; - public ZKNamespaceManager(ZooKeeperWatcher zkw) throws IOException { + public ZKNamespaceManager(ZKWatcher zkw) throws IOException { super(zkw); nsZNode = zkw.znodePaths.namespaceZNode; cache = new ConcurrentSkipListMap<>(); 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 d7fdeb72b02..cda5affac4f 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; } 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 d26199319c3..389dea7fc0d 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); 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 72c292f2444..73b50a652d9 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; } 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 9a7903a6431..39d95439c06 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)); } 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 c19c3a566fd..8a07b4b7c13 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() { 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 10e2642e329..91b617fa11e 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; 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 ef87498d3a8..e64907c2532 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 { 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 f154347410f..ce9f290c56d 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; 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/master/DrainingServerTracker.java similarity index 93% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/master/DrainingServerTracker.java index cdc67942e81..81a8b55385d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DrainingServerTracker.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.zookeeper; +package org.apache.hadoop.hbase.master; import java.io.IOException; import java.util.List; @@ -24,11 +24,12 @@ 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.hadoop.hbase.master.ServerListener; -import org.apache.hadoop.hbase.master.ServerManager; import org.apache.zookeeper.KeeperException; /** @@ -49,14 +50,14 @@ import org.apache.zookeeper.KeeperException; * */ @InterfaceAudience.Private -public class DrainingServerTracker extends ZooKeeperListener { +public class DrainingServerTracker extends ZKListener { private static final Log LOG = LogFactory.getLog(DrainingServerTracker.class); private ServerManager serverManager; private final NavigableSet drainingServers = new TreeSet<>(); private Abortable abortable; - public DrainingServerTracker(ZooKeeperWatcher watcher, + public DrainingServerTracker(ZKWatcher watcher, Abortable abortable, ServerManager serverManager) { super(watcher); this.abortable = abortable; 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 cad77e5d952..19672961d2c 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; } 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 677a2a0b30d..7abf02cf55d 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 { 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 d13fb76ba8d..6a9b572936d 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(); 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/master/RegionServerTracker.java similarity index 94% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java index c22876ae56a..0e9351d226c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.zookeeper; +package org.apache.hadoop.hbase.master; import java.io.IOException; import java.io.InterruptedIOException; @@ -28,11 +28,13 @@ 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.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; /** @@ -46,13 +48,13 @@ import org.apache.zookeeper.KeeperException; * {@link ServerManager#expireServer(ServerName)} */ @InterfaceAudience.Private -public class RegionServerTracker extends ZooKeeperListener { +public class RegionServerTracker extends ZKListener { private static final Log LOG = LogFactory.getLog(RegionServerTracker.class); private NavigableMap regionServers = new TreeMap<>(); private ServerManager serverManager; private MasterServices server; - public RegionServerTracker(ZooKeeperWatcher watcher, + public RegionServerTracker(ZKWatcher watcher, MasterServices server, ServerManager serverManager) { super(watcher); this.server = server; @@ -149,7 +151,7 @@ public class RegionServerTracker extends ZooKeeperListener { public RegionServerInfo getRegionServerInfo(final ServerName sn) { return regionServers.get(sn); } - + /** * Gets the online servers. * @return list of online servers 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 f19995fb8dd..79ffc8a5825 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 getRegionServersInZK(final ZooKeeperWatcher zkw) + private List getRegionServersInZK(final ZKWatcher zkw) throws KeeperException { return ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode); } 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/master/SplitOrMergeTracker.java similarity index 91% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitOrMergeTracker.java index 2c5ffba163b..55287d21d71 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitOrMergeTracker.java @@ -15,18 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.zookeeper; +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.yetus.audience.InterfaceAudience; +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; @@ -43,7 +47,7 @@ public class SplitOrMergeTracker { private SwitchStateTracker splitStateTracker; private SwitchStateTracker mergeStateTracker; - public SplitOrMergeTracker(ZooKeeperWatcher watcher, Configuration conf, + public SplitOrMergeTracker(ZKWatcher watcher, Configuration conf, Abortable abortable) { try { if (ZKUtil.checkExists(watcher, watcher.znodePaths.switchZNode) < 0) { @@ -91,9 +95,9 @@ public class SplitOrMergeTracker { } } - private static class SwitchStateTracker extends ZooKeeperNodeTracker { + private static class SwitchStateTracker extends ZKNodeTracker { - public SwitchStateTracker(ZooKeeperWatcher watcher, String node, Abortable abortable) { + public SwitchStateTracker(ZKWatcher watcher, String node, Abortable abortable) { super(watcher, node, abortable); } 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 97d3080c268..3f7bd744739 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); } 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 df94ffe7a5f..7cfaefdb5b8 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; 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 6985591214f..609ce8ee3b5 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; 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 36f4f44cd3b..45e67609679 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 this. Closed via + * @param watcher {@link ZKWatcher} to be owned by this. 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 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 24693cd2bd1..0349290eabb 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; } 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 ea34714a1ec..d328561ce20 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); 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 4c34fe0e0af..03657c18632 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; } 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 7d7833bcec9..75d8e56fbaa 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 ""; } 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 a4b43879975..6a7d83ebe42 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); 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 a34bedd5ba5..4985b82ae80 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 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 fetchSlavesAddresses(ZooKeeperWatcher zkw) + protected static List fetchSlavesAddresses(ZKWatcher zkw) throws KeeperException { List 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; 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 a14bd0101df..5972734f6dd 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( 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 3dcb332b1f5..57ed8427ac8 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 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; 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 d094d1cd77f..f44249521ff 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(); 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 9d38026592d..fdb555906e5 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 peerIds, - boolean hdfs) throws Exception { + public String dumpQueues(ClusterConnection connection, ZKWatcher zkw, Set peerIds, + boolean hdfs) throws Exception { ReplicationQueuesClient queuesClient; ReplicationPeers replicationPeers; ReplicationQueues replicationQueues; 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 280289c4fab..233159e9b59 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; } 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 6ee9f602ecc..05f919540a1 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; 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 0db05c2d8b8..2ba4ac5549f 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 managerMap = new HashMap<>(); + private static Map managerMap = new HashMap<>(); private static Map 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); 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 447ba51b115..d45b5b5730a 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; 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 27ea5098188..aa6b1e94b76 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, 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 814fe073f61..3347e1caa62 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 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) { 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 9f0da781b9b..96502fd82a1 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 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 e913b21fb12..8272da69486 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) { 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 2edf6365ade..85bc0d51732 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 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 5a6a414fc2c..5cc244cd69c 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. */ 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 bf4478d0a1c..0879a692cf4 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> 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); 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 0f36a7bc419..b1c1f07f03f 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); 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 e07861e8dd9..11327e8baa0 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 queryForTableStates(ZooKeeperWatcher zkw) + public static Map queryForTableStates(ZKWatcher zkw) throws KeeperException, InterruptedException { Map rv = new HashMap<>(); List 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); 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 9fb8459fc11..839b5ade533 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 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; } 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 4465162e951..195161705c3 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(); %> 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 3b3d5683cf0..bb7eca2cafd 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; 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 cac6fd65bc7..53f27d16558 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; } 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 fb1c1e2f0d6..961677b10a2 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()); 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 6a904a7a90e..c2a49451d9b 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); } 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 d546d5dce1b..ed93b1750ac 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; 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 aa246c28743..1c2279cc3c8 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); 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 6a4ada30b93..5660a017ca7 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, 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 62951ef4fa6..c98a02c5e31 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 { 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 0595a677116..2a353658a29 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) { 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 d4c6e4f8b59..c4defa2d74e 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) { 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 85d2b0ba7ca..dadec1fbe4a 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; } 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 25cfbd90b4e..8358e0ad0af 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; } 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 f1feef55d58..6fcd8ecca19 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; } 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 d8707c369b9..7f8e9c9ee12 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); 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 852b139398d..bd7c5073f96 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; 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 3d59639d1cd..bc68b286218 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); } 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 64d5a0292ea..4b2c91150bb 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); 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 25671fcce04..20c9fe1f85e 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) { 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 ea57e1504da..e2a231287c3 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(); 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 ca6432627b0..d74b7320da9 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); 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 6cf259a6207..1f61ee7d341 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) { 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 572816d1632..97d61893fb1 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(); } 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 773d0fcfdeb..9820fd09373 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(); } 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 b5ca894b043..4ccc677a845 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 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) 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 be7f35ec187..b2ae3bd8b21 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 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 { 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 e11143d18c0..6aa59cb33cf 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); 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 58efa87be13..7d6f80a1dfb 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 = 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 14d7eab0605..36ea086dfef 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 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> 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> 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); 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 4ab968515e9..5d680743c5b 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 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 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> start( - ZooKeeperWatcher watcher, String operationName, - ProcedureCoordinator coordinator, String controllerName, - ProcedureMember member, List cohortNames) throws Exception; + ZKWatcher watcher, String operationName, + ProcedureCoordinator coordinator, String controllerName, + ProcedureMember member, List cohortNames) throws Exception; } private final StartControllers startCoordinatorFirst = new StartControllers() { @Override public Pair> start( - ZooKeeperWatcher watcher, String operationName, - ProcedureCoordinator coordinator, String controllerName, - ProcedureMember member, List expected) throws Exception { + ZKWatcher watcher, String operationName, + ProcedureCoordinator coordinator, String controllerName, + ProcedureMember member, List 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> start( - ZooKeeperWatcher watcher, String operationName, - ProcedureCoordinator coordinator, String controllerName, - ProcedureMember member, List expected) throws Exception { + ZKWatcher watcher, String operationName, + ProcedureCoordinator coordinator, String controllerName, + ProcedureMember member, List expected) throws Exception { // make a cohort controller for each expected node List cohortControllers = new ArrayList<>(); 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 5c24c2c174f..9bd74d021a5 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); 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 b16feca1402..2a3ce00ca0b 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; } 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 84865cc84c5..f1b4441a5f7 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; 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 cfecd9fe7b0..c9cae7ee434 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(); 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 872fec6722a..abcc4970573 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 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 servers = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode); assertEquals(expectedRS, servers.size()); } 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 1f7320ec992..6e20612b7e1 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)) { 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 0bf9d0341f4..f241f85538c 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); 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 2e5c5525a97..10a84b13f57 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; } 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 6b7d36b682e..58b22c800fc 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 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); } } 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 a9896ce8c89..2e555a92f9f 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); } } } 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 6572404326b..84ce9a3b99c 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); 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 58b97b9c5b7..f7d1009e6b9 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); 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 cc09751a20a..97daa639e32 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; } 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 dde78680e32..bb9f66594b6 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; } 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 83fdad75168..665eedb1dcb 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; 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 f6e7b39bb48..2db4f76332a 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; } 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 7a8e6390fa2..1c5aa7190ea 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); 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 1a0231748fb..cb895caa193 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 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 2469c7c726e..9b1648f6319 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(); 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 dd1e19a8060..77301005421 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; } 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 ad8cb140675..f9d0b1c219f 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 + 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 30e8396359a..607ea8c5f22 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); 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 76de0c6048c..18fb15fd522 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); } 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 5a0b6bdb3c4..3e0bddcb247 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(); 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 857cdd0ad16..ea0733874d7 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; } } 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 fcadc9dc9d7..1757ddd0b17 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()); 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 e5b1c393917..c2da3ecda69 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 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 513c76511aa..678227b6557 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 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 db442192a5f..4acec769725 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; } 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 7463da101ba..ba3475e6341 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)); 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 e71210d0675..6fa99fd9145 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); 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 1faf8e52e05..69915a5e326 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; } 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/TestZKMainServer.java similarity index 95% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperMainServer.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java index 368919ceb54..c99bd0526ee 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperMainServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java @@ -31,7 +31,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) -public class TestZooKeeperMainServer { +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 { @@ -72,13 +72,13 @@ public class TestZooKeeperMainServer { htu.getConfiguration().setInt(HConstants.ZK_SESSION_TIMEOUT, 1000); htu.startMiniZKCluster(); try { - ZooKeeperWatcher zkw = htu.getZooKeeperWatcher(); + ZKWatcher 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", + ZKMainServer.main(new String [] {"-server", "localhost:" + htu.getZkCluster().getClientPort(), "delete", znode}); } catch (ExitException ee) { // ZKMS calls System.exit which should trigger this exception. @@ -94,7 +94,7 @@ public class TestZooKeeperMainServer { @Test public void testHostPortParse() { - ZooKeeperMainServer parser = new ZooKeeperMainServer(); + ZKMainServer parser = new ZKMainServer(); Configuration c = HBaseConfiguration.create(); assertEquals("localhost:" + c.get(HConstants.ZOOKEEPER_CLIENT_PORT), parser.parse(c)); final String port = "1234"; 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 7971417ca43..614d4be9fa5 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); } 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/TestZKNodeTracker.java similarity index 92% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java index 951c6ba47d0..ac5c11af4ca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java @@ -48,8 +48,8 @@ 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); +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(); @@ -71,7 +71,7 @@ public class TestZooKeeperNodeTracker { */ @Test public void testInterruptible() throws IOException, InterruptedException { Abortable abortable = new StubAbortable(); - ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), + ZKWatcher zk = new ZKWatcher(TEST_UTIL.getConfiguration(), "testInterruptible", abortable); final TestTracker tracker = new TestTracker(zk, "/xyz", abortable); tracker.start(); @@ -95,7 +95,7 @@ public class TestZooKeeperNodeTracker { @Test public void testNodeTracker() throws Exception { Abortable abortable = new StubAbortable(); - ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), + ZKWatcher zk = new ZKWatcher(TEST_UTIL.getConfiguration(), "testNodeTracker", abortable); ZKUtil.createAndFailSilent(zk, zk.znodePaths.baseZNode); @@ -213,15 +213,15 @@ public class TestZooKeeperNodeTracker { TestTracker tracker; boolean hasData; - public WaitToGetDataThread(ZooKeeperWatcher zk, String node) { + public WaitToGetDataThread(ZKWatcher zk, String node) { tracker = new TestTracker(zk, node, null); tracker.start(); zk.registerListener(tracker); hasData = false; } - public WaitToGetDataThread(ZooKeeperWatcher zk, String node, - TestTracker tracker) { + public WaitToGetDataThread(ZKWatcher zk, String node, + TestTracker tracker) { this.tracker = tracker; hasData = false; } @@ -239,14 +239,14 @@ public class TestZooKeeperNodeTracker { } } - public static class TestTracker extends ZooKeeperNodeTracker { - public TestTracker(ZooKeeperWatcher watcher, String node, - Abortable abortable) { + public static class TestTracker extends ZKNodeTracker { + public TestTracker(ZKWatcher watcher, String node, + Abortable abortable) { super(watcher, node, abortable); } } - public static class TestingZKListener extends ZooKeeperListener { + public static class TestingZKListener extends ZKListener { private static final Log LOG = LogFactory.getLog(NodeDeletionListener.class); private Semaphore deletedLock; @@ -254,7 +254,7 @@ public class TestZooKeeperNodeTracker { private Semaphore changedLock; private String node; - public TestingZKListener(ZooKeeperWatcher watcher, String node) { + public TestingZKListener(ZKWatcher watcher, String node) { super(watcher); deletedLock = new Semaphore(0); createdLock = new Semaphore(0); @@ -302,12 +302,12 @@ public class TestZooKeeperNodeTracker { 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 { @@ -317,8 +317,8 @@ public class TestZooKeeperNodeTracker { @Test public void testCleanZNode() throws Exception { - ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), - "testNodeTracker", new TestZooKeeperNodeTracker.StubAbortable()); + ZKWatcher zkw = new ZKWatcher(TEST_UTIL.getConfiguration(), + "testNodeTracker", new TestZKNodeTracker.StubAbortable()); final ServerName sn = ServerName.valueOf("127.0.0.1:52", 45L); 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 05ad73e9745..c5bce009872 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. */ diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb index 0ce38cc5e33..9f8551cd967 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 diff --git a/hbase-zookeeper/pom.xml b/hbase-zookeeper/pom.xml new file mode 100644 index 00000000000..a2544181c33 --- /dev/null +++ b/hbase-zookeeper/pom.xml @@ -0,0 +1,412 @@ + + + + 4.0.0 + + hbase-build-configuration + org.apache.hbase + 2.0.0-beta-1.SNAPSHOT + ../hbase-build-configuration + + hbase-zookeeper + Apache HBase - Zookeeper + Zookeeper Helpers for HBase + + + + + + + src/test/resources/META-INF/ + META-INF/ + + NOTICE + + true + + + + + org.apache.maven.plugins + maven-site-plugin + + true + + + + + + maven-assembly-plugin + + true + + + + + org.apache.maven.plugins + maven-source-plugin + + + package + + jar + test-jar + + + + + + + org.apache.maven.plugins + maven-eclipse-plugin + + + org.jamon.project.jamonnature + + + org.jamon.project.templateBuilder + org.eclipse.jdt.core.javabuilder + org.jamon.project.markerUpdater + + + + .settings/org.jamon.prefs + # now + eclipse.preferences.version=1 + templateSourceDir=src/main/jamon + templateOutputDir=target/generated-jamon + + + + + + + + org.codehaus.mojo + findbugs-maven-plugin + + + + maven-surefire-plugin + + + + listener + org.apache.hadoop.hbase.ResourceCheckerJUnitListener + + + + + + + + + + + org.eclipse.m2e + lifecycle-mapping + + + + + + org.apache.maven.plugins + maven-antrun-plugin + [1.6,) + + run + + + + + false + true + + + + + + org.apache.maven.plugins + maven-dependency-plugin + [2.8,) + + build-classpath + + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + [3.2,) + + compile + + + + + + + + + + + + + + + + org.apache.hbase.thirdparty + hbase-shaded-protobuf + + + org.apache.hbase.thirdparty + hbase-shaded-miscellaneous + + + + org.apache.hbase + hbase-common + + + org.apache.hbase + hbase-common + test-jar + test + + + org.apache.hbase + hbase-client + + + org.apache.hbase + hbase-annotations + test-jar + test + + + org.apache.hbase + hbase-protocol-shaded + + + org.apache.hbase + hbase-hadoop-compat + + + org.apache.hbase + hbase-hadoop2-compat + + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + + + + com.github.stephenc.findbugs + findbugs-annotations + true + + + org.apache.commons + commons-lang3 + + + commons-logging + commons-logging + + + log4j + log4j + + + org.apache.zookeeper + zookeeper + + + + junit + junit + test + + + org.mockito + mockito-core + test + + + + + + apache-release + + + + org.apache.maven.plugins + maven-resources-plugin + + + license-javadocs + prepare-package + + copy-resources + + + ${project.build.directory}/apidocs + + + src/main/javadoc/META-INF/ + META-INF/ + + LICENSE + NOTICE + + true + + + + + + + + + + + + skipZooKeeperTests + + + skipZooKeeperTests + + + + true + true + + + + + + + + hadoop-2.0 + + + + !hadoop.profile + + + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-auth + + + + + + maven-dependency-plugin + + + create-mrapp-generated-classpath + generate-test-resources + + build-classpath + + + + ${project.build.directory}/test-classes/mrapp-generated-classpath + + + + + + + + + + hadoop-3.0 + + + hadoop.profile + 3.0 + + + + ${hadoop-three.version} + + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-auth + + + + + + maven-dependency-plugin + + + create-mrapp-generated-classpath + generate-test-resources + + build-classpath + + + + ${project.build.directory}/test-classes/mrapp-generated-classpath + + + + + + + + + diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java similarity index 94% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java index fd60765a42c..d145d0825ac 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -28,13 +28,13 @@ 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 + * 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 { +public class ClusterStatusTracker extends ZKNodeTracker { private static final Log LOG = LogFactory.getLog(ClusterStatusTracker.class); /** @@ -45,7 +45,7 @@ public class ClusterStatusTracker extends ZooKeeperNodeTracker { * @param watcher * @param abortable */ - public ClusterStatusTracker(ZooKeeperWatcher watcher, Abortable abortable) { + public ClusterStatusTracker(ZKWatcher watcher, Abortable abortable) { super(watcher, watcher.znodePaths.clusterStateZNode, abortable); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java similarity index 94% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java index c58a840f2b7..7c028912066 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java @@ -1,4 +1,4 @@ -/** +/* * Copyright The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one @@ -30,7 +30,7 @@ import org.apache.zookeeper.KeeperException; * A ZooKeeper watcher meant to detect deletions of ZNodes. */ @InterfaceAudience.Private -public class DeletionListener extends ZooKeeperListener { +public class DeletionListener extends ZKListener { private static final Log LOG = LogFactory.getLog(DeletionListener.class); @@ -46,8 +46,8 @@ public class DeletionListener extends ZooKeeperListener { * be deleted. * @param deletedLatch Count down on this latch when deletion has occurred. */ - public DeletionListener(ZooKeeperWatcher zkWatcher, String pathToWatch, - CountDownLatch deletedLatch) { + public DeletionListener(ZKWatcher zkWatcher, String pathToWatch, + CountDownLatch deletedLatch) { super(zkWatcher); this.pathToWatch = pathToWatch; this.deletedLatch = deletedLatch; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java similarity index 99% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java index 191943a46e4..6470faa4e92 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java @@ -1,4 +1,4 @@ -/** +/* * 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 diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java similarity index 98% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java index 5f3904aabca..f07b8416ff1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -142,7 +142,7 @@ public class HQuorumPeer { // 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))); + 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 + diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java similarity index 100% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java similarity index 96% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java index 527bc17c4b8..55dafcb3b20 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -33,10 +33,10 @@ import org.apache.zookeeper.KeeperException; * Tracks the load balancer state up in ZK */ @InterfaceAudience.Private -public class LoadBalancerTracker extends ZooKeeperNodeTracker { +public class LoadBalancerTracker extends ZKNodeTracker { private static final Log LOG = LogFactory.getLog(LoadBalancerTracker.class); - public LoadBalancerTracker(ZooKeeperWatcher watcher, + public LoadBalancerTracker(ZKWatcher watcher, Abortable abortable) { super(watcher, watcher.znodePaths.balancerZNode, abortable); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java similarity index 92% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java index fba637fbe63..85668ad18b7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -41,17 +41,17 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE * NodeDeleted and NodeCreated events on * /master. *

- * Utilizes {@link ZooKeeperNodeTracker} for zk interactions. + * Utilizes {@link ZKNodeTracker} for zk interactions. *

* You can get the current master via {@link #getMasterAddress()} or via - * {@link #getMasterAddress(ZooKeeperWatcher)} if you do not have a running + * {@link #getMasterAddress(ZKWatcher)} if you do not have a running * instance of this Tracker in your context. *

* 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 { +public class MasterAddressTracker extends ZKNodeTracker { /** * Construct a master address listener with the specified * zookeeper reference. @@ -63,7 +63,7 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker { * @param watcher zk reference and watcher * @param abortable abortable in case of fatal error */ - public MasterAddressTracker(ZooKeeperWatcher watcher, Abortable abortable) { + public MasterAddressTracker(ZKWatcher watcher, Abortable abortable) { super(watcher, watcher.znodePaths.masterAddressZNode, abortable); } @@ -136,13 +136,13 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker { * 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 + * @param zkw ZKWatcher to use * @return ServerName stored in the the master address znode or null if no * znode present. - * @throws KeeperException - * @throws IOException + * @throws KeeperException + * @throws IOException */ - public static ServerName getMasterAddress(final ZooKeeperWatcher zkw) + public static ServerName getMasterAddress(final ZKWatcher zkw) throws KeeperException, IOException { byte [] data; try { @@ -167,14 +167,14 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker { * 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 + * @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 ZooKeeperWatcher zkw) throws KeeperException, + public static int getMasterInfoPort(final ZKWatcher zkw) throws KeeperException, IOException { byte[] data; try { @@ -199,14 +199,14 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker { * Set master address into the master znode or into the backup * subdirectory of backup masters; switch off the passed in znode * path. - * @param zkw The ZooKeeperWatcher to use. + * @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 ZooKeeperWatcher zkw, + 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)); @@ -258,7 +258,7 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker { * @param zkw must not be null * @param content must not be null */ - public static boolean deleteIfEquals(ZooKeeperWatcher zkw, final String content) { + public static boolean deleteIfEquals(ZKWatcher zkw, final String content) { if (content == null){ throw new IllegalArgumentException("Content must not be null"); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java similarity index 93% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java index 7d6f9fdd917..952da6f2e64 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -27,10 +27,10 @@ import org.apache.zookeeper.KeeperException; * Tracks the master Maintenance Mode via ZK. */ @InterfaceAudience.Private -public class MasterMaintenanceModeTracker extends ZooKeeperListener { +public class MasterMaintenanceModeTracker extends ZKListener { private boolean hasChildren; - public MasterMaintenanceModeTracker(ZooKeeperWatcher watcher) { + public MasterMaintenanceModeTracker(ZKWatcher watcher) { super(watcher); hasChildren = false; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java similarity index 93% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java index cddde2f521b..f6c7a2d6909 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -65,7 +65,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaReg * 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). + * (e.g. Connection, ZKWatcher etc). * * Meta region location is set by RegionServerServices. * This class doesn't use ZK watchers, rather accesses ZK directly. @@ -86,7 +86,7 @@ public class MetaTableLocator { * Checks if the meta region location is available. * @return true if meta region location is available, false if not */ - public boolean isLocationAvailable(ZooKeeperWatcher zkw) { + public boolean isLocationAvailable(ZKWatcher zkw) { return getMetaRegionLocation(zkw) != null; } @@ -94,7 +94,7 @@ public class MetaTableLocator { * @param zkw ZooKeeper watcher to be used * @return meta table regions and their locations. */ - public List> getMetaRegionsAndLocations(ZooKeeperWatcher zkw) { + public List> getMetaRegionsAndLocations(ZKWatcher zkw) { return getMetaRegionsAndLocations(zkw, RegionInfo.DEFAULT_REPLICA_ID); } @@ -104,7 +104,7 @@ public class MetaTableLocator { * @param replicaId * @return meta table regions and their locations. */ - public List> getMetaRegionsAndLocations(ZooKeeperWatcher zkw, + public List> getMetaRegionsAndLocations(ZKWatcher zkw, int replicaId) { ServerName serverName = getMetaRegionLocation(zkw, replicaId); List> list = new ArrayList<>(1); @@ -117,7 +117,7 @@ public class MetaTableLocator { * @param zkw ZooKeeper watcher to be used * @return List of meta regions */ - public List getMetaRegions(ZooKeeperWatcher zkw) { + public List getMetaRegions(ZKWatcher zkw) { return getMetaRegions(zkw, RegionInfo.DEFAULT_REPLICA_ID); } @@ -127,7 +127,7 @@ public class MetaTableLocator { * @param replicaId * @return List of meta regions */ - public List getMetaRegions(ZooKeeperWatcher zkw, int replicaId) { + public List getMetaRegions(ZKWatcher zkw, int replicaId) { List> result; result = getMetaRegionsAndLocations(zkw, replicaId); return getListOfRegionInfos(result); @@ -148,7 +148,7 @@ public class MetaTableLocator { * @param zkw zookeeper connection to use * @return server name or null if we failed to get the data. */ - public ServerName getMetaRegionLocation(final ZooKeeperWatcher zkw) { + public ServerName getMetaRegionLocation(final ZKWatcher zkw) { try { RegionState state = getMetaRegionState(zkw); return state.isOpened() ? state.getServerName() : null; @@ -163,7 +163,7 @@ public class MetaTableLocator { * @param replicaId * @return server name */ - public ServerName getMetaRegionLocation(final ZooKeeperWatcher zkw, int replicaId) { + public ServerName getMetaRegionLocation(final ZKWatcher zkw, int replicaId) { try { RegionState state = getMetaRegionState(zkw, replicaId); return state.isOpened() ? state.getServerName() : null; @@ -184,7 +184,7 @@ public class MetaTableLocator { * @throws InterruptedException if interrupted while waiting * @throws NotAllMetaRegionsOnlineException */ - public ServerName waitMetaRegionLocation(ZooKeeperWatcher zkw, long timeout) + public ServerName waitMetaRegionLocation(ZKWatcher zkw, long timeout) throws InterruptedException, NotAllMetaRegionsOnlineException { return waitMetaRegionLocation(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout); } @@ -202,7 +202,7 @@ public class MetaTableLocator { * @throws InterruptedException * @throws NotAllMetaRegionsOnlineException */ - public ServerName waitMetaRegionLocation(ZooKeeperWatcher zkw, int replicaId, long timeout) + public ServerName waitMetaRegionLocation(ZKWatcher zkw, int replicaId, long timeout) throws InterruptedException, NotAllMetaRegionsOnlineException { try { if (ZKUtil.checkExists(zkw, zkw.znodePaths.baseZNode) == -1) { @@ -227,10 +227,10 @@ public class MetaTableLocator { * Waits indefinitely for availability of hbase:meta. 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) + * @see #waitMetaRegionLocation(ZKWatcher, long) * @throws InterruptedException if interrupted while waiting */ - public void waitMetaRegionLocation(ZooKeeperWatcher zkw) throws InterruptedException { + public void waitMetaRegionLocation(ZKWatcher zkw) throws InterruptedException { long startTime = System.currentTimeMillis(); while (!stopped) { try { @@ -260,7 +260,7 @@ public class MetaTableLocator { * @throws InterruptedException */ public boolean verifyMetaRegionLocation(ClusterConnection hConnection, - ZooKeeperWatcher zkw, final long timeout) + ZKWatcher zkw, final long timeout) throws InterruptedException, IOException { return verifyMetaRegionLocation(hConnection, zkw, timeout, RegionInfo.DEFAULT_REPLICA_ID); } @@ -276,7 +276,7 @@ public class MetaTableLocator { * @throws IOException */ public boolean verifyMetaRegionLocation(ClusterConnection connection, - ZooKeeperWatcher zkw, final long timeout, int replicaId) + ZKWatcher zkw, final long timeout, int replicaId) throws InterruptedException, IOException { AdminProtos.AdminService.BlockingInterface service = null; try { @@ -360,7 +360,7 @@ public class MetaTableLocator { * @throws IOException */ private AdminService.BlockingInterface getMetaServerConnection(ClusterConnection connection, - ZooKeeperWatcher zkw, long timeout, int replicaId) + ZKWatcher zkw, long timeout, int replicaId) throws InterruptedException, NotAllMetaRegionsOnlineException, IOException { return getCachedConnection(connection, waitMetaRegionLocation(zkw, replicaId, timeout)); } @@ -424,7 +424,7 @@ public class MetaTableLocator { * @param state The region transition state * @throws KeeperException unexpected zookeeper exception */ - public static void setMetaLocation(ZooKeeperWatcher zookeeper, + public static void setMetaLocation(ZKWatcher zookeeper, ServerName serverName, RegionState.State state) throws KeeperException { setMetaLocation(zookeeper, serverName, RegionInfo.DEFAULT_REPLICA_ID, state); } @@ -438,7 +438,7 @@ public class MetaTableLocator { * @param state * @throws KeeperException */ - public static void setMetaLocation(ZooKeeperWatcher zookeeper, + 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"); @@ -470,7 +470,7 @@ public class MetaTableLocator { /** * Load the meta region state from the meta server ZNode. */ - public static RegionState getMetaRegionState(ZooKeeperWatcher zkw) throws KeeperException { + public static RegionState getMetaRegionState(ZKWatcher zkw) throws KeeperException { return getMetaRegionState(zkw, RegionInfo.DEFAULT_REPLICA_ID); } @@ -481,7 +481,7 @@ public class MetaTableLocator { * @return regionstate * @throws KeeperException */ - public static RegionState getMetaRegionState(ZooKeeperWatcher zkw, int replicaId) + public static RegionState getMetaRegionState(ZKWatcher zkw, int replicaId) throws KeeperException { RegionState.State state = RegionState.State.OPEN; ServerName serverName = null; @@ -524,12 +524,12 @@ public class MetaTableLocator { * @param zookeeper zookeeper reference * @throws KeeperException unexpected zookeeper exception */ - public void deleteMetaLocation(ZooKeeperWatcher zookeeper) + public void deleteMetaLocation(ZKWatcher zookeeper) throws KeeperException { deleteMetaLocation(zookeeper, RegionInfo.DEFAULT_REPLICA_ID); } - public void deleteMetaLocation(ZooKeeperWatcher zookeeper, int replicaId) + public void deleteMetaLocation(ZKWatcher zookeeper, int replicaId) throws KeeperException { if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) { LOG.info("Deleting hbase:meta region location in ZooKeeper"); @@ -552,7 +552,7 @@ public class MetaTableLocator { * @return ServerName or null if we timed out. * @throws InterruptedException */ - public List blockUntilAvailable(final ZooKeeperWatcher zkw, + public List blockUntilAvailable(final ZKWatcher zkw, final long timeout, Configuration conf) throws InterruptedException { int numReplicasConfigured = 1; @@ -584,7 +584,7 @@ public class MetaTableLocator { * @return ServerName or null if we timed out. * @throws InterruptedException */ - public ServerName blockUntilAvailable(final ZooKeeperWatcher zkw, + public ServerName blockUntilAvailable(final ZKWatcher zkw, final long timeout) throws InterruptedException { return blockUntilAvailable(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout); @@ -598,8 +598,8 @@ public class MetaTableLocator { * @return ServerName or null if we timed out. * @throws InterruptedException */ - public ServerName blockUntilAvailable(final ZooKeeperWatcher zkw, int replicaId, - final long timeout) + 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(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java similarity index 100% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java similarity index 100% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java similarity index 97% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java index 04f709f92d3..d6c11af7576 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -18,9 +18,6 @@ */ 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; @@ -36,7 +33,6 @@ 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; @@ -86,7 +82,7 @@ public class RecoverableZooKeeper { private Watcher watcher; private int sessionTimeout; private String quorumServers; - private final ZooKeeperMetricsListener metrics; + private final ZKMetricsListener metrics; public RecoverableZooKeeper(String quorumServers, int sessionTimeout, Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime) @@ -116,7 +112,7 @@ public class RecoverableZooKeeper { this.watcher = watcher; this.sessionTimeout = sessionTimeout; this.quorumServers = quorumServers; - this.metrics = new MetricsZooKeeper(); + this.metrics = new ZKMetrics(); try {checkZk();} catch (Exception x) {/* ignore */} } @@ -360,7 +356,7 @@ public class RecoverableZooKeeper { long startTime = EnvironmentEdgeManager.currentTime(); byte[] revData = checkZk().getData(path, watcher, stat); this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); - return removeMetaData(revData); + return ZKMetadata.removeMetaData(revData); } catch (KeeperException e) { this.metrics.registerFailedZKCall(); switch (e.code()) { @@ -395,7 +391,7 @@ public class RecoverableZooKeeper { long startTime = EnvironmentEdgeManager.currentTime(); byte[] revData = checkZk().getData(path, watch, stat); this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1)); - return removeMetaData(revData); + return ZKMetadata.removeMetaData(revData); } catch (KeeperException e) { this.metrics.registerFailedZKCall(); switch (e.code()) { @@ -427,7 +423,7 @@ public class RecoverableZooKeeper { throws KeeperException, InterruptedException { try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.setData")) { RetryCounter retryCounter = retryCounterFactory.create(); - byte[] newData = appendMetaData(id, data); + byte[] newData = ZKMetadata.appendMetaData(id, data); boolean isRetry = false; long startTime; while (true) { @@ -565,7 +561,7 @@ public class RecoverableZooKeeper { CreateMode createMode) throws KeeperException, InterruptedException { try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.create")) { - byte[] newData = appendMetaData(id, data); + byte[] newData = ZKMetadata.appendMetaData(id, data); switch (createMode) { case EPHEMERAL: case PERSISTENT: @@ -686,14 +682,14 @@ public class RecoverableZooKeeper { 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()), + 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(), appendMetaData(id, setData.getData()), + preparedOps.add(Op.setData(setData.getPath(), ZKMetadata.appendMetaData(id, setData.getData()), setData.getVersion())); } else { throw new UnsupportedOperationException("Unexpected ZKOp type: " + op.getClass().getName()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java similarity index 96% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java index a9939347c7a..93545ee74fc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -32,10 +32,10 @@ import java.io.IOException; /** * Tracks region normalizer state up in ZK */ -public class RegionNormalizerTracker extends ZooKeeperNodeTracker { +public class RegionNormalizerTracker extends ZKNodeTracker { private static final Log LOG = LogFactory.getLog(RegionNormalizerTracker.class); - public RegionNormalizerTracker(ZooKeeperWatcher watcher, + public RegionNormalizerTracker(ZKWatcher watcher, Abortable abortable) { super(watcher, watcher.znodePaths.regionNormalizerZNode, abortable); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAclReset.java similarity index 91% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAclReset.java index 045fd973361..4150f5483fa 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAclReset.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -43,11 +43,11 @@ import org.apache.zookeeper.ZooKeeper; * 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); +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 { + private static void resetAcls(final ZKWatcher zkw, final String znode, + final boolean eraseAcls) throws Exception { List children = ZKUtil.listChildrenNoWatch(zkw, znode); if (children != null) { for (String child: children) { @@ -67,7 +67,7 @@ public class ZkAclReset extends Configured implements Tool { private static void resetAcls(final Configuration conf, boolean eraseAcls) throws Exception { - ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "ZkAclReset", null); + ZKWatcher zkw = new ZKWatcher(conf, "ZKAclReset", null); try { LOG.info((eraseAcls ? "Erase" : "Set") + " HBase ACLs for " + zkw.getQuorum() + " " + zkw.znodePaths.baseZNode); @@ -111,6 +111,6 @@ public class ZkAclReset extends Configured implements Tool { } public static void main(String[] args) throws Exception { - System.exit(ToolRunner.run(HBaseConfiguration.create(), new ZkAclReset(), args)); + System.exit(ToolRunner.run(HBaseConfiguration.create(), new ZKAclReset(), args)); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java similarity index 89% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java index 9ef76918066..2f2b0364985 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java @@ -35,11 +35,11 @@ import org.apache.zookeeper.KeeperException; */ @InterfaceAudience.Private public class ZKClusterId { - private ZooKeeperWatcher watcher; + private ZKWatcher watcher; private Abortable abortable; private String id; - public ZKClusterId(ZooKeeperWatcher watcher, Abortable abortable) { + public ZKClusterId(ZKWatcher watcher, Abortable abortable) { this.watcher = watcher; this.abortable = abortable; } @@ -60,7 +60,7 @@ public class ZKClusterId { return id; } - public static String readClusterIdZNode(ZooKeeperWatcher watcher) + public static String readClusterIdZNode(ZKWatcher watcher) throws KeeperException { if (ZKUtil.checkExists(watcher, watcher.znodePaths.clusterIdZNode) != -1) { byte [] data; @@ -81,7 +81,7 @@ public class ZKClusterId { return null; } - public static void setClusterId(ZooKeeperWatcher watcher, ClusterId id) + public static void setClusterId(ZKWatcher watcher, ClusterId id) throws KeeperException { ZKUtil.createSetData(watcher, watcher.znodePaths.clusterIdZNode, id.toByteArray()); } @@ -92,7 +92,7 @@ public class ZKClusterId { * @return the UUID read from zookeeper * @throws KeeperException */ - public static UUID getUUIDForCluster(ZooKeeperWatcher zkw) throws KeeperException { + public static UUID getUUIDForCluster(ZKWatcher zkw) throws KeeperException { String uuid = readClusterIdZNode(zkw); return uuid == null ? null : UUID.fromString(uuid); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java similarity index 97% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java index b0610b01ea9..edd2ccdff3b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java @@ -40,7 +40,7 @@ import org.apache.zookeeper.KeeperException; */ @Deprecated @InterfaceAudience.Private -public class ZKLeaderManager extends ZooKeeperListener { +public class ZKLeaderManager extends ZKListener { private static final Log LOG = LogFactory.getLog(ZKLeaderManager.class); private final AtomicBoolean leaderExists = new AtomicBoolean(); @@ -48,8 +48,8 @@ public class ZKLeaderManager extends ZooKeeperListener { private byte[] nodeId; private Stoppable candidate; - public ZKLeaderManager(ZooKeeperWatcher watcher, String leaderZNode, - byte[] identifier, Stoppable candidate) { + public ZKLeaderManager(ZKWatcher watcher, String leaderZNode, + byte[] identifier, Stoppable candidate) { super(watcher); this.leaderZNode = leaderZNode; this.nodeId = identifier; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKListener.java similarity index 88% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKListener.java index f78f1d8ad45..595e7130426 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKListener.java @@ -1,5 +1,4 @@ -/** - * +/* * 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 @@ -23,9 +22,9 @@ import org.apache.yetus.audience.InterfaceAudience; /** * Base class for internal listeners of ZooKeeper events. * - * The {@link ZooKeeperWatcher} for a process will execute the appropriate + * 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 ZooKeeperWatcher#registerListener}. + * the watcher, every listener must register itself via {@link ZKWatcher#registerListener}. * * Subclasses need only override those methods in which they are interested. * @@ -33,15 +32,15 @@ import org.apache.yetus.audience.InterfaceAudience; * they must not be long-running. */ @InterfaceAudience.Private -public abstract class ZooKeeperListener { +public abstract class ZKListener { // Reference to the zk watcher which also contains configuration and constants - protected ZooKeeperWatcher watcher; + protected ZKWatcher watcher; /** * Construct a ZooKeeper event listener. */ - public ZooKeeperListener(ZooKeeperWatcher watcher) { + public ZKListener(ZKWatcher watcher) { this.watcher = watcher; } @@ -80,7 +79,7 @@ public abstract class ZooKeeperListener { /** * @return The watcher associated with this listener */ - public ZooKeeperWatcher getWatcher() { + public ZKWatcher getWatcher() { return this.watcher; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMainServer.java similarity index 97% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMainServer.java index 341fbbdfa82..9cb0e7d28da 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMainServer.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -35,7 +35,7 @@ import org.apache.zookeeper.ZooKeeperMain; * from HBase XML configuration. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS) -public class ZooKeeperMainServer { +public class ZKMainServer { private static final String SERVER_ARG = "-server"; public String parse(final Configuration c) { @@ -104,7 +104,7 @@ public class ZooKeeperMainServer { 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); + 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); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetrics.java similarity index 90% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetrics.java index 5632031c0d8..20d4a5549c1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetrics.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -23,23 +23,21 @@ 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 { +public class ZKMetrics implements ZKMetricsListener { private final MetricsZooKeeperSource source; - public MetricsZooKeeper() { + public ZKMetrics() { this(CompatibilitySingletonFactory.getInstance(MetricsZooKeeperSource.class)); } @VisibleForTesting - public MetricsZooKeeper(MetricsZooKeeperSource s) { + public ZKMetrics(MetricsZooKeeperSource s) { this.source = s; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetricsListener.java similarity index 98% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetricsListener.java index 12ce2cfca5f..f17925ef281 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetricsListener.java @@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.zookeeper; import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Private -public interface ZooKeeperMetricsListener { +public interface ZKMetricsListener { /** * An AUTHFAILED Exception was seen. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKNodeTracker.java similarity index 95% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKNodeTracker.java index a5b084b0019..8ce41e3a508 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKNodeTracker.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -27,16 +27,16 @@ import org.apache.zookeeper.KeeperException; /** * Tracks the availability and value of a single ZooKeeper node. * - *

Utilizes the {@link ZooKeeperListener} interface to get the necessary + *

Utilizes the {@link ZKListener} interface to get the necessary * ZooKeeper events related to the node. * *

This is the base class used by trackers in both the Master and * RegionServers. */ @InterfaceAudience.Private -public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { +public abstract class ZKNodeTracker extends ZKListener { // LOG is being used in subclasses, hence keeping it protected - protected static final Log LOG = LogFactory.getLog(ZooKeeperNodeTracker.class); + protected static final Log LOG = LogFactory.getLog(ZKNodeTracker.class); /** Path of node being tracked */ protected final String node; @@ -57,8 +57,8 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { * @param node * @param abortable */ - public ZooKeeperNodeTracker(ZooKeeperWatcher watcher, String node, - Abortable abortable) { + public ZKNodeTracker(ZKWatcher watcher, String node, + Abortable abortable) { super(watcher); this.node = node; this.abortable = abortable; @@ -223,7 +223,7 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { nodeCreated(path); } } - + /** * Checks if the baseznode set as per the property 'zookeeper.znode.parent' * exists. @@ -245,7 +245,7 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { @Override public String toString() { - return "ZooKeeperNodeTracker{" + + return "ZKNodeTracker{" + "node='" + node + ", stopped=" + stopped + '}'; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java similarity index 99% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java index 70e58f8712b..8116c2366f2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java similarity index 89% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java index 1376ba9101a..33cc43eab1f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -30,9 +30,7 @@ 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. + * Common methods and attributes used by SplitLogManager and SplitLogWorker running distributed splitting of WAL logs. */ @InterfaceAudience.Private public class ZKSplitLog { @@ -44,7 +42,7 @@ public class ZKSplitLog { * @param zkw zk reference * @param filename log file name (only the basename) */ - public static String getEncodedNodeName(ZooKeeperWatcher zkw, String filename) { + public static String getEncodedNodeName(ZKWatcher zkw, String filename) { return ZNodePaths.joinZNode(zkw.znodePaths.splitLogZNode, encode(filename)); } @@ -69,7 +67,7 @@ public class ZKSplitLog { } } - public static String getRescanNode(ZooKeeperWatcher zkw) { + public static String getRescanNode(ZKWatcher zkw) { return ZNodePaths.joinZNode(zkw.znodePaths.splitLogZNode, "RESCAN"); } @@ -86,7 +84,7 @@ public class ZKSplitLog { * @param path the absolute path, starts with '/' * @return whether the path represents a rescan node */ - public static boolean isRescanNode(ZooKeeperWatcher zkw, String path) { + public static boolean isRescanNode(ZKWatcher zkw, String path) { String prefix = getRescanNode(zkw); if (path.length() <= prefix.length()) { return false; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java similarity index 93% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java index 61a5a4290b1..d8472a9441f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -333,7 +333,7 @@ public class ZKUtil { * @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) + public static boolean watchAndCheckExists(ZKWatcher zkw, String znode) throws KeeperException { try { Stat s = zkw.getRecoverableZooKeeper().exists(znode, zkw); @@ -365,7 +365,7 @@ public class ZKUtil { * @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) + public static boolean setWatchIfNodeExists(ZKWatcher zkw, String znode) throws KeeperException { try { zkw.getRecoverableZooKeeper().getData(znode, true, null); @@ -387,7 +387,7 @@ public class ZKUtil { * @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) + public static int checkExists(ZKWatcher zkw, String znode) throws KeeperException { try { Stat s = zkw.getRecoverableZooKeeper().exists(znode, null); @@ -424,7 +424,7 @@ public class ZKUtil { * @throws KeeperException if unexpected zookeeper exception */ public static List listChildrenAndWatchForNewChildren( - ZooKeeperWatcher zkw, String znode) + ZKWatcher zkw, String znode) throws KeeperException { try { List children = zkw.getRecoverableZooKeeper().getChildren(znode, zkw); @@ -453,7 +453,7 @@ public class ZKUtil { * @return list of znode names, null if the node doesn't exist * @throws KeeperException */ - public static List listChildrenAndWatchThem(ZooKeeperWatcher zkw, + public static List listChildrenAndWatchThem(ZKWatcher zkw, String znode) throws KeeperException { List children = listChildrenAndWatchForNewChildren(zkw, znode); if (children == null) { @@ -479,7 +479,7 @@ public class ZKUtil { * null if parent does not exist * @throws KeeperException if unexpected zookeeper exception */ - public static List listChildrenNoWatch(ZooKeeperWatcher zkw, String znode) + public static List listChildrenNoWatch(ZKWatcher zkw, String znode) throws KeeperException { List children = null; try { @@ -536,7 +536,7 @@ public class ZKUtil { * @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) + public static boolean nodeHasChildren(ZKWatcher zkw, String znode) throws KeeperException { try { return !zkw.getRecoverableZooKeeper().getChildren(znode, null).isEmpty(); @@ -568,7 +568,7 @@ public class ZKUtil { * exist * @throws KeeperException if unexpected zookeeper exception */ - public static int getNumberOfChildren(ZooKeeperWatcher zkw, String znode) + public static int getNumberOfChildren(ZKWatcher zkw, String znode) throws KeeperException { try { Stat stat = zkw.getRecoverableZooKeeper().exists(znode, null); @@ -591,7 +591,7 @@ public class ZKUtil { * @return ZNode data, null if the node does not exist or if there is an * error. */ - public static byte [] getData(ZooKeeperWatcher zkw, String znode) + public static byte [] getData(ZKWatcher zkw, String znode) throws KeeperException, InterruptedException { try { byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, null); @@ -619,7 +619,7 @@ public class ZKUtil { * @return data of the specified znode, or null * @throws KeeperException if unexpected zookeeper exception */ - public static byte [] getDataAndWatch(ZooKeeperWatcher zkw, String znode) + public static byte [] getDataAndWatch(ZKWatcher zkw, String znode) throws KeeperException { return getDataInternal(zkw, znode, null, true); } @@ -636,13 +636,13 @@ public class ZKUtil { * @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 { + public static byte[] getDataAndWatch(ZKWatcher 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) + private static byte[] getDataInternal(ZKWatcher zkw, String znode, Stat stat, + boolean watcherSet) throws KeeperException { try { byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat); @@ -680,8 +680,8 @@ public class ZKUtil { * @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) + public static byte [] getDataNoWatch(ZKWatcher zkw, String znode, + Stat stat) throws KeeperException { try { byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, stat); @@ -721,7 +721,7 @@ public class ZKUtil { */ @Deprecated public static List getChildDataAndWatchForNewChildren( - ZooKeeperWatcher zkw, String baseNode) throws KeeperException { + ZKWatcher zkw, String baseNode) throws KeeperException { List nodes = ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode); if (nodes != null) { @@ -753,8 +753,8 @@ public class ZKUtil { * @deprecated Unused */ @Deprecated - public static void updateExistingNodeData(ZooKeeperWatcher zkw, String znode, - byte [] data, int expectedVersion) + public static void updateExistingNodeData(ZKWatcher zkw, String znode, + byte [] data, int expectedVersion) throws KeeperException { try { zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion); @@ -787,8 +787,8 @@ public class ZKUtil { * @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) + 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; @@ -807,8 +807,8 @@ public class ZKUtil { * @param data data to set for node * @throws KeeperException */ - public static void createSetData(final ZooKeeperWatcher zkw, final String znode, - final byte [] data) + 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); @@ -833,12 +833,12 @@ public class ZKUtil { * @param data data to set for node * @throws KeeperException if unexpected zookeeper exception */ - public static void setData(ZooKeeperWatcher zkw, String znode, byte [] data) + 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(ZooKeeperWatcher zkw, SetData setData) + 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()); @@ -874,12 +874,12 @@ public class ZKUtil { return "kerberos".equalsIgnoreCase(conf.get("hbase.security.authentication")); } - private static ArrayList createACL(ZooKeeperWatcher zkw, String node) { + private static ArrayList createACL(ZKWatcher zkw, String node) { return createACL(zkw, node, isSecureZooKeeper(zkw.getConfiguration())); } - public static ArrayList createACL(ZooKeeperWatcher zkw, String node, - boolean isSecureZooKeeper) { + public static ArrayList createACL(ZKWatcher zkw, String node, + boolean isSecureZooKeeper) { if (!node.startsWith(zkw.znodePaths.baseZNode)) { return Ids.OPEN_ACL_UNSAFE; } @@ -946,7 +946,7 @@ public class ZKUtil { * @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, + public static boolean createEphemeralNodeAndWatch(ZKWatcher zkw, String znode, byte [] data) throws KeeperException { boolean ret = true; @@ -987,7 +987,7 @@ public class ZKUtil { * @throws KeeperException if unexpected zookeeper exception */ public static boolean createNodeIfNotExistsAndWatch( - ZooKeeperWatcher zkw, String znode, byte [] data) + ZKWatcher zkw, String znode, byte [] data) throws KeeperException { boolean ret = true; try { @@ -1022,8 +1022,8 @@ public class ZKUtil { * @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 { + public static String createNodeIfNotExistsNoWatch(ZKWatcher zkw, String znode, + byte[] data, CreateMode createMode) throws KeeperException { String createdZNode = null; try { @@ -1054,7 +1054,7 @@ public class ZKUtil { * @throws KeeperException if unexpected zookeeper exception * @throws KeeperException.NodeExistsException if node already exists */ - public static int createAndWatch(ZooKeeperWatcher zkw, + public static int createAndWatch(ZKWatcher zkw, String znode, byte [] data) throws KeeperException, KeeperException.NodeExistsException { try { @@ -1086,7 +1086,7 @@ public class ZKUtil { * @param cb * @param ctx */ - public static void asyncCreate(ZooKeeperWatcher zkw, + public static void asyncCreate(ZKWatcher zkw, String znode, byte [] data, final AsyncCallback.StringCallback cb, final Object ctx) { zkw.getRecoverableZooKeeper().getZooKeeper().create(znode, data, @@ -1103,7 +1103,7 @@ public class ZKUtil { * @param znode path of node * @throws KeeperException if unexpected zookeeper exception */ - public static void createAndFailSilent(ZooKeeperWatcher zkw, + public static void createAndFailSilent(ZKWatcher zkw, String znode) throws KeeperException { createAndFailSilent(zkw, znode, new byte[0]); } @@ -1119,14 +1119,14 @@ public class ZKUtil { * @param data a byte array data to store in the znode * @throws KeeperException if unexpected zookeeper exception */ - public static void createAndFailSilent(ZooKeeperWatcher zkw, + public static void createAndFailSilent(ZKWatcher zkw, String znode, byte[] data) throws KeeperException { createAndFailSilent(zkw, (CreateAndFailSilent)ZKUtilOp.createAndFailSilent(znode, data)); } - private static void createAndFailSilent(ZooKeeperWatcher zkw, CreateAndFailSilent cafs) + private static void createAndFailSilent(ZKWatcher zkw, CreateAndFailSilent cafs) throws KeeperException { CreateRequest create = (CreateRequest)toZooKeeperOp(zkw, cafs).toRequestRecord(); String znode = create.getPath(); @@ -1161,7 +1161,7 @@ public class ZKUtil { * @param znode path of node * @throws KeeperException if unexpected zookeeper exception */ - public static void createWithParents(ZooKeeperWatcher zkw, String znode) + public static void createWithParents(ZKWatcher zkw, String znode) throws KeeperException { createWithParents(zkw, znode, new byte[0]); } @@ -1179,7 +1179,7 @@ public class ZKUtil { * @param znode path of node * @throws KeeperException if unexpected zookeeper exception */ - public static void createWithParents(ZooKeeperWatcher zkw, String znode, byte[] data) + public static void createWithParents(ZKWatcher zkw, String znode, byte[] data) throws KeeperException { try { if(znode == null) { @@ -1204,7 +1204,7 @@ public class ZKUtil { /** * Delete the specified node. Sets no watches. Throws all exceptions. */ - public static void deleteNode(ZooKeeperWatcher zkw, String node) + public static void deleteNode(ZKWatcher zkw, String node) throws KeeperException { deleteNode(zkw, node, -1); } @@ -1213,8 +1213,8 @@ public class ZKUtil { * Delete the specified node with the specified version. Sets no watches. * Throws all exceptions. */ - public static boolean deleteNode(ZooKeeperWatcher zkw, String node, - int version) + public static boolean deleteNode(ZKWatcher zkw, String node, + int version) throws KeeperException { try { zkw.getRecoverableZooKeeper().delete(node, version); @@ -1233,13 +1233,13 @@ public class ZKUtil { * @param node * @throws KeeperException */ - public static void deleteNodeFailSilent(ZooKeeperWatcher zkw, String node) + public static void deleteNodeFailSilent(ZKWatcher zkw, String node) throws KeeperException { deleteNodeFailSilent(zkw, (DeleteNodeFailSilent)ZKUtilOp.deleteNodeFailSilent(node)); } - private static void deleteNodeFailSilent(ZooKeeperWatcher zkw, + private static void deleteNodeFailSilent(ZKWatcher zkw, DeleteNodeFailSilent dnfs) throws KeeperException { DeleteRequest delete = (DeleteRequest)toZooKeeperOp(zkw, dnfs).toRequestRecord(); try { @@ -1259,7 +1259,7 @@ public class ZKUtil { * Sets no watches. Throws all exceptions besides dealing with deletion of * children. */ - public static void deleteNodeRecursively(ZooKeeperWatcher zkw, String node) + public static void deleteNodeRecursively(ZKWatcher zkw, String node) throws KeeperException { deleteNodeRecursivelyMultiOrSequential(zkw, true, node); } @@ -1272,7 +1272,7 @@ public class ZKUtil { * * @throws KeeperException */ - public static void deleteChildrenRecursively(ZooKeeperWatcher zkw, String node) + public static void deleteChildrenRecursively(ZKWatcher zkw, String node) throws KeeperException { deleteChildrenRecursivelyMultiOrSequential(zkw, true, node); } @@ -1308,8 +1308,8 @@ public class ZKUtil { * if an invalid path is specified */ public static void deleteChildrenRecursivelyMultiOrSequential( - ZooKeeperWatcher zkw, boolean runSequentialOnMultiFailure, - String... pathRoots) throws KeeperException { + ZKWatcher zkw, boolean runSequentialOnMultiFailure, + String... pathRoots) throws KeeperException { if (pathRoots == null || pathRoots.length <= 0) { LOG.warn("Given path is not valid!"); return; @@ -1358,7 +1358,7 @@ public class ZKUtil { * @throws IllegalArgumentException * if an invalid path is specified */ - public static void deleteNodeRecursivelyMultiOrSequential(ZooKeeperWatcher zkw, + 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!"); @@ -1400,7 +1400,7 @@ public class ZKUtil { * @throws KeeperException * if unexpected ZooKeeper exception */ - private static List listChildrenBFSNoWatch(ZooKeeperWatcher zkw, + private static List listChildrenBFSNoWatch(ZKWatcher zkw, final String znode) throws KeeperException { Deque queue = new LinkedList<>(); List tree = new ArrayList<>(); @@ -1436,7 +1436,7 @@ public class ZKUtil { * @throws KeeperException * if unexpected ZooKeeper exception */ - private static List listChildrenBFSAndWatchThem(ZooKeeperWatcher zkw, final String znode) + private static List listChildrenBFSAndWatchThem(ZKWatcher zkw, final String znode) throws KeeperException { Deque queue = new LinkedList<>(); List tree = new ArrayList<>(); @@ -1589,7 +1589,7 @@ public class ZKUtil { /** * Convert from ZKUtilOp to ZKOp */ - private static Op toZooKeeperOp(ZooKeeperWatcher zkw, ZKUtilOp op) + private static Op toZooKeeperOp(ZKWatcher zkw, ZKUtilOp op) throws UnsupportedOperationException { if(op == null) return null; @@ -1628,8 +1628,8 @@ public class ZKUtil { * * @throws KeeperException */ - public static void multiOrSequential(ZooKeeperWatcher zkw, List ops, - boolean runSequentialOnMultiFailure) throws KeeperException { + public static void multiOrSequential(ZKWatcher zkw, List ops, + boolean runSequentialOnMultiFailure) throws KeeperException { if (zkw.getConfiguration().get("hbase.zookeeper.useMulti") != null) { LOG.warn("hbase.zookeeper.useMulti is deprecated. Default to true always."); } @@ -1664,7 +1664,7 @@ public class ZKUtil { } } - private static void processSequentially(ZooKeeperWatcher zkw, List ops) + private static void processSequentially(ZKWatcher zkw, List ops) throws KeeperException, NoNodeException { for (ZKUtilOp op : ops) { if (op instanceof CreateAndFailSilent) { @@ -1685,7 +1685,7 @@ public class ZKUtil { // /** @return String dump of everything in ZooKeeper. */ - public static String dump(ZooKeeperWatcher zkw) { + public static String dump(ZKWatcher zkw) { StringBuilder sb = new StringBuilder(); try { sb.append("HBase is rooted at ").append(zkw.znodePaths.baseZNode); @@ -1749,7 +1749,7 @@ public class ZKUtil { * @param sb * @throws KeeperException */ - private static void getReplicationZnodesDump(ZooKeeperWatcher zkw, StringBuilder sb) + private static void getReplicationZnodesDump(ZKWatcher zkw, StringBuilder sb) throws KeeperException { String replicationZnode = zkw.znodePaths.replicationZNode; if (ZKUtil.checkExists(zkw, replicationZnode) == -1) return; @@ -1768,8 +1768,8 @@ public class ZKUtil { } } - private static void appendHFileRefsZnodes(ZooKeeperWatcher zkw, String hfileRefsZnode, - StringBuilder sb) throws KeeperException { + 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); @@ -1790,13 +1790,13 @@ public class ZKUtil { * @param zkw * @return aq string of replication znodes and log positions */ - public static String getReplicationZnodesDump(ZooKeeperWatcher zkw) throws KeeperException { + public static String getReplicationZnodesDump(ZKWatcher zkw) throws KeeperException { StringBuilder sb = new StringBuilder(); getReplicationZnodesDump(zkw, sb); return sb.toString(); } - private static void appendRSZnodes(ZooKeeperWatcher zkw, String znode, StringBuilder sb) + private static void appendRSZnodes(ZKWatcher zkw, String znode, StringBuilder sb) throws KeeperException { List stack = new LinkedList<>(); stack.add(znode); @@ -1827,8 +1827,8 @@ public class ZKUtil { } while (stack.size() > 0); } - private static void appendPeersZnodes(ZooKeeperWatcher zkw, String peersZnode, - StringBuilder sb) throws KeeperException { + 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)) { @@ -1855,8 +1855,8 @@ public class ZKUtil { } } - private static void appendPeerState(ZooKeeperWatcher zkw, String znodeToProcess, - StringBuilder sb) throws KeeperException, InvalidProtocolBufferException { + 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(); @@ -1922,7 +1922,7 @@ public class ZKUtil { } } - private static void logRetrievedMsg(final ZooKeeperWatcher zkw, + 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) + @@ -2002,7 +2002,7 @@ public class ZKUtil { * 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) { + public static void logZKTree(ZKWatcher zkw, String root) { if (!LOG.isDebugEnabled()) return; LOG.debug("Current zk system:"); String prefix = "|-"; @@ -2016,10 +2016,10 @@ public class ZKUtil { /** * Helper method to print the current state of the ZK tree. - * @see #logZKTree(ZooKeeperWatcher, String) + * @see #logZKTree(ZKWatcher, String) * @throws KeeperException if an unexpected exception occurs */ - protected static void logZKTree(ZooKeeperWatcher zkw, String root, String prefix) + protected static void logZKTree(ZKWatcher zkw, String root, String prefix) throws KeeperException { List children = ZKUtil.listChildrenNoWatch(zkw, root); if (children == null) return; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java similarity index 94% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java index 5d10cdf3351..d0b0081d712 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -21,9 +21,7 @@ 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; @@ -34,8 +32,6 @@ 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; @@ -61,8 +57,8 @@ import org.apache.zookeeper.data.Stat; * 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); +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. @@ -83,7 +79,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { public final ZNodePaths znodePaths; // listeners to be notified - private final List listeners = new CopyOnWriteArrayList<>(); + private final List listeners = new CopyOnWriteArrayList<>(); // Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL // negotiation to complete @@ -103,8 +99,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { * @throws IOException * @throws ZooKeeperConnectionException */ - public ZooKeeperWatcher(Configuration conf, String identifier, - Abortable abortable) throws ZooKeeperConnectionException, IOException { + public ZKWatcher(Configuration conf, String identifier, + Abortable abortable) throws ZooKeeperConnectionException, IOException { this(conf, identifier, abortable, false); } @@ -119,8 +115,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { * @throws IOException * @throws ZooKeeperConnectionException */ - public ZooKeeperWatcher(Configuration conf, String identifier, - Abortable abortable, boolean canCreateBaseZNode) + public ZKWatcher(Configuration conf, String identifier, + Abortable abortable, boolean canCreateBaseZNode) throws IOException, ZooKeeperConnectionException { this.conf = conf; this.quorum = ZKConfig.getZKQuorumServersString(conf); @@ -386,7 +382,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { * Register the specified listener to receive ZooKeeper events. * @param listener */ - public void registerListener(ZooKeeperListener listener) { + public void registerListener(ZKListener listener) { listeners.add(listener); } @@ -395,11 +391,11 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { * the first in the list of current listeners. * @param listener */ - public void registerListenerFirst(ZooKeeperListener listener) { + public void registerListenerFirst(ZKListener listener) { listeners.add(0, listener); } - public void unregisterListener(ZooKeeperListener listener) { + public void unregisterListener(ZKListener listener) { listeners.remove(listener); } @@ -413,7 +409,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { /** * Get a copy of current registered listeners */ - public List getListeners() { + public List getListeners() { return new ArrayList<>(listeners); } @@ -477,28 +473,28 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { // Otherwise pass along to the listeners case NodeCreated: { - for(ZooKeeperListener listener : listeners) { + for(ZKListener listener : listeners) { listener.nodeCreated(event.getPath()); } break; } case NodeDeleted: { - for(ZooKeeperListener listener : listeners) { + for(ZKListener listener : listeners) { listener.nodeDeleted(event.getPath()); } break; } case NodeDataChanged: { - for(ZooKeeperListener listener : listeners) { + for(ZKListener listener : listeners) { listener.nodeDataChanged(event.getPath()); } break; } case NodeChildrenChanged: { - for(ZooKeeperListener listener : listeners) { + for(ZKListener listener : listeners) { listener.nodeChildrenChanged(event.getPath()); } break; @@ -535,8 +531,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { 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. + // 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()); } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java similarity index 100% rename from hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java rename to hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMetrics.java similarity index 92% rename from hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java rename to hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMetrics.java index a987bec6296..2811cc5ee08 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMetrics.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -18,18 +18,21 @@ */ 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; -import static org.mockito.Mockito.*; @Category(SmallTests.class) -public class TestMetricsZooKeeper { +public class TestZKMetrics { @Test public void testRegisterExceptions() { MetricsZooKeeperSource zkSource = mock(MetricsZooKeeperSourceImpl.class); - MetricsZooKeeper metricsZK = new MetricsZooKeeper(zkSource); + ZKMetrics metricsZK = new ZKMetrics(zkSource); metricsZK.registerAuthFailedException(); metricsZK.registerConnectionLossException(); metricsZK.registerConnectionLossException(); @@ -62,7 +65,7 @@ public class TestMetricsZooKeeper { @Test public void testLatencyHistogramUpdates() { MetricsZooKeeperSource zkSource = mock(MetricsZooKeeperSourceImpl.class); - MetricsZooKeeper metricsZK = new MetricsZooKeeper(zkSource); + ZKMetrics metricsZK = new ZKMetrics(zkSource); long latency = 100; metricsZK.registerReadOperationLatency(latency); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java similarity index 92% rename from hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java rename to hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java index 6b1e1f0b93e..7006040bc1f 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java @@ -48,7 +48,7 @@ public class TestZKUtil { Configuration conf = HBaseConfiguration.create(); conf.set(Superusers.SUPERUSER_CONF_KEY, "user1"); String node = "/hbase/testUnsecure"; - ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false); + ZKWatcher watcher = new ZKWatcher(conf, node, null, false); List aclList = ZKUtil.createACL(watcher, node, false); Assert.assertEquals(aclList.size(), 1); Assert.assertTrue(aclList.contains(Ids.OPEN_ACL_UNSAFE.iterator().next())); @@ -59,7 +59,7 @@ public class TestZKUtil { Configuration conf = HBaseConfiguration.create(); conf.set(Superusers.SUPERUSER_CONF_KEY, "user1"); String node = "/hbase/testSecuritySingleSuperuser"; - ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false); + ZKWatcher watcher = new ZKWatcher(conf, node, null, false); List 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")))); @@ -71,7 +71,7 @@ public class TestZKUtil { 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); + ZKWatcher watcher = new ZKWatcher(conf, node, null, false); List 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")))); @@ -87,7 +87,7 @@ public class TestZKUtil { 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); + ZKWatcher watcher = new ZKWatcher(conf, node, null, false); List 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")))); @@ -100,7 +100,7 @@ public class TestZKUtil { public void testInterruptedDuringAction() throws ZooKeeperConnectionException, IOException, KeeperException, InterruptedException { final RecoverableZooKeeper recoverableZk = Mockito.mock(RecoverableZooKeeper.class); - ZooKeeperWatcher zkw = new ZooKeeperWatcher(HBaseConfiguration.create(), "unittest", null) { + ZKWatcher zkw = new ZKWatcher(HBaseConfiguration.create(), "unittest", null) { @Override public RecoverableZooKeeper getRecoverableZooKeeper() { return recoverableZk; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKWatcher.java similarity index 93% rename from hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java rename to hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKWatcher.java index 3c99175ad81..bd4575d6dd6 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKWatcher.java @@ -28,12 +28,12 @@ import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ SmallTests.class }) -public class TestZooKeeperWatcher { +public class TestZKWatcher { @Test public void testIsClientReadable() throws IOException { - ZooKeeperWatcher watcher = - new ZooKeeperWatcher(HBaseConfiguration.create(), "testIsClientReadable", null, false); + ZKWatcher watcher = + new ZKWatcher(HBaseConfiguration.create(), "testIsClientReadable", null, false); assertTrue(watcher.isClientReadable(watcher.znodePaths.baseZNode)); assertTrue(watcher.isClientReadable(watcher.znodePaths.getZNodeForReplica(0))); diff --git a/pom.xml b/pom.xml index 2d3fa8c38d4..0e6b5583ca8 100644 --- a/pom.xml +++ b/pom.xml @@ -90,6 +90,7 @@ hbase-metrics-api hbase-metrics hbase-backup + hbase-zookeeper @@ -1684,6 +1685,11 @@ org.apache.hbase ${project.version} + + hbase-zookeeper + org.apache.hbase + ${project.version} + com.github.stephenc.findbugs