HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

- Moved DrainingServerTracker and RegionServerTracker to hbase-server:o.a.h.h.master.
- Moved SplitOrMergeTracker to oahh.master (because it depends on a PB)
- Moving hbase-client:oahh.zookeeper.*  to hbase-zookeeper module.  After HBASE-19200, hbase-client doesn't need them anymore (except 3 classes).
- Renamed some classes to use a consistent naming for classes - ZK instead of mix of ZK, Zk , ZooKeeper. Couldn't rename following public classes: MiniZooKeeperCluster, ZooKeeperConnectionException. Left RecoverableZooKeeper for lack of better name. (suggestions?)
- Sadly, can't move tests out because they depend on HBaseTestingUtility (which defeats part of the purpose - trimming down hbase-server tests. We need to promote more use of mocks in our tests)
This commit is contained in:
Apekshit Sharma 2017-11-13 11:42:33 -08:00
parent 3330dd517e
commit e0c4f374b5
168 changed files with 1226 additions and 832 deletions

View File

@ -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

View File

@ -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

View File

@ -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.

View File

@ -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.

View File

@ -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.

View File

@ -278,6 +278,10 @@
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-examples</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-zookeeper</artifactId>
</dependency>
</dependencies>
<profiles>
<profile>

View File

@ -56,6 +56,7 @@
<include>org.apache.hbase:hbase-shell</include>
<include>org.apache.hbase:hbase-testing-util</include>
<include>org.apache.hbase:hbase-thrift</include>
<include>org.apache.hbase:hbase-zookeeper</include>
</includes>
<!-- Binaries for the dependencies also go in the hbase-jars directory -->
<binaries>

View File

@ -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);

View File

@ -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)) {

View File

@ -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;

View File

@ -57,7 +57,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.MRJobConfig;
@ -330,10 +330,10 @@ public class VerifyReplication extends Configured implements Tool {
private static Pair<ReplicationPeerConfig, Configuration> getPeerQuorumConfig(
final Configuration conf, String peerId) throws IOException {
ZooKeeperWatcher localZKW = null;
ZKWatcher localZKW = null;
ReplicationPeerZKImpl peer = null;
try {
localZKW = new ZooKeeperWatcher(conf, "VerifyReplication",
localZKW = new ZKWatcher(conf, "VerifyReplication",
new Abortable() {
@Override public void abort(String why, Throwable e) {}
@Override public boolean isAborted() {return false;}

View File

@ -130,6 +130,10 @@
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-client</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-zookeeper</artifactId>
</dependency>
<!-- General dependencies -->
<dependency>
<groupId>org.apache.commons</groupId>

View File

@ -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);

View File

@ -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);
}

View File

@ -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<>();

View File

@ -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;
}

View File

@ -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) {

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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;

View File

@ -24,13 +24,13 @@ import java.util.concurrent.CopyOnWriteArrayList;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.zookeeper.ZKListener;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@ -50,7 +50,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
private final ArrayList<String> otherRegionServers = new ArrayList<>();
private final ReplicationPeers replicationPeers;
public ReplicationTrackerZKImpl(ZooKeeperWatcher zookeeper,
public ReplicationTrackerZKImpl(ZKWatcher zookeeper,
final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable,
Stoppable stopper) {
super(zookeeper, conf, abortable);
@ -88,12 +88,12 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
* Watcher used to be notified of the other region server's death in the local cluster. It
* initiates the process to transfer the queues if it is able to grab the lock.
*/
public class OtherRegionServerWatcher extends ZooKeeperListener {
public class OtherRegionServerWatcher extends ZKListener {
/**
* Construct a ZooKeeper event listener.
*/
public OtherRegionServerWatcher(ZooKeeperWatcher watcher) {
public OtherRegionServerWatcher(ZKWatcher watcher) {
super(watcher);
}
@ -145,12 +145,12 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
/**
* Watcher used to follow the creation and deletion of peer clusters.
*/
public class PeersWatcher extends ZooKeeperListener {
public class PeersWatcher extends ZKListener {
/**
* Construct a ZooKeeper event listener.
*/
public PeersWatcher(ZooKeeperWatcher watcher) {
public PeersWatcher(ZKWatcher watcher) {
super(watcher);
}

View File

@ -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) {};

View File

@ -77,8 +77,8 @@ import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
import org.apache.hadoop.hbase.security.access.AccessControlLists;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
@ -141,7 +141,7 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager {
private final MasterServices masterServices;
private Table rsGroupTable;
private final ClusterConnection conn;
private final ZooKeeperWatcher watcher;
private final ZKWatcher watcher;
private final RSGroupStartupWorker rsGroupStartupWorker = new RSGroupStartupWorker();
// contains list of groups that were last flushed to persistent store
private Set<String> prevRSGroups = new HashSet<>();

View File

@ -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

View File

@ -388,6 +388,10 @@
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-client</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-zookeeper</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-replication</artifactId>

View File

@ -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.

View File

@ -27,8 +27,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKListener;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import java.io.IOException;
@ -48,12 +48,12 @@ import java.util.concurrent.ConcurrentSkipListMap;
*
*/
@InterfaceAudience.Private
public class ZKNamespaceManager extends ZooKeeperListener {
public class ZKNamespaceManager extends ZKListener {
private static final Log LOG = LogFactory.getLog(ZKNamespaceManager.class);
private final String nsZNode;
private volatile NavigableMap<String,NamespaceDescriptor> cache;
public ZKNamespaceManager(ZooKeeperWatcher zkw) throws IOException {
public ZKNamespaceManager(ZKWatcher zkw) throws IOException {
super(zkw);
nsZNode = zkw.znodePaths.namespaceZNode;
cache = new ConcurrentSkipListMap<>();

View File

@ -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; }

View File

@ -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);

View File

@ -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;
}

View File

@ -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));
}

View File

@ -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() {

View File

@ -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;

View File

@ -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 {

View File

@ -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;

View File

@ -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<ServerName> drainingServers = new TreeSet<>();
private Abortable abortable;
public DrainingServerTracker(ZooKeeperWatcher watcher,
public DrainingServerTracker(ZKWatcher watcher,
Abortable abortable, ServerManager serverManager) {
super(watcher);
this.abortable = abortable;

View File

@ -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;
}

View File

@ -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 {

View File

@ -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();

View File

@ -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<ServerName, RegionServerInfo> 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

View File

@ -42,7 +42,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClockOutOfSyncException;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionLoad;
@ -59,7 +58,7 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
@ -507,7 +506,7 @@ public class ServerManager {
void letRegionServersShutdown() {
long previousLogTime = 0;
ServerName sn = master.getServerName();
ZooKeeperWatcher zkw = master.getZooKeeper();
ZKWatcher zkw = master.getZooKeeper();
int onlineServersCt;
while ((onlineServersCt = onlineServers.size()) > 0){
@ -554,7 +553,7 @@ public class ServerManager {
}
}
private List<String> getRegionServersInZK(final ZooKeeperWatcher zkw)
private List<String> getRegionServersInZK(final ZKWatcher zkw)
throws KeeperException {
return ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode);
}

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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;

View File

@ -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;

View File

@ -23,13 +23,13 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@ -61,12 +61,12 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs {
/**
* Must call {@link #start(String, ProcedureMember)} before this can be used.
* @param watcher {@link ZooKeeperWatcher} to be owned by <tt>this</tt>. Closed via
* @param watcher {@link ZKWatcher} to be owned by <tt>this</tt>. Closed via
* {@link #close()}.
* @param procType name of the znode describing the procedure type
* @throws KeeperException if we can't reach zookeeper
*/
public ZKProcedureMemberRpcs(final ZooKeeperWatcher watcher, final String procType)
public ZKProcedureMemberRpcs(final ZKWatcher watcher, final String procType)
throws KeeperException {
this.zkController = new ZKProcedureUtil(watcher, procType) {
@Override

View File

@ -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;
}

View File

@ -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);

View File

@ -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;
}

View File

@ -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 "";
}

View File

@ -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);

View File

@ -26,13 +26,13 @@ import java.util.UUID;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.zookeeper.ZKListener;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.AuthFailedException;
import org.apache.zookeeper.KeeperException.ConnectionLossException;
@ -50,7 +50,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
private static final Log LOG = LogFactory.getLog(HBaseReplicationEndpoint.class);
private ZooKeeperWatcher zkw = null; // FindBugs: MT_CORRECTNESS
private ZKWatcher zkw = null; // FindBugs: MT_CORRECTNESS
private List<ServerName> regionServers = new ArrayList<>(0);
private long lastRegionServerUpdate;
@ -123,7 +123,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
* Get the ZK connection to this peer
* @return zk connection
*/
protected ZooKeeperWatcher getZkw() {
protected ZKWatcher getZkw() {
return zkw;
}
@ -133,7 +133,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
*/
void reloadZkWatcher() throws IOException {
if (zkw != null) zkw.close();
zkw = new ZooKeeperWatcher(ctx.getConfiguration(),
zkw = new ZKWatcher(ctx.getConfiguration(),
"connection to cluster: " + ctx.getPeerId(), this);
getZkw().registerListener(new PeerRegionServerListener(this));
}
@ -155,7 +155,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
* @param zkw zk connection to use
* @return list of region server addresses or an empty list if the slave is unavailable
*/
protected static List<ServerName> fetchSlavesAddresses(ZooKeeperWatcher zkw)
protected static List<ServerName> fetchSlavesAddresses(ZKWatcher zkw)
throws KeeperException {
List<String> children = ZKUtil.listChildrenAndWatchForNewChildren(zkw, zkw.znodePaths.rsZNode);
if (children == null) {
@ -210,7 +210,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
/**
* Tracks changes to the list of region servers in a peer's cluster.
*/
public static class PeerRegionServerListener extends ZooKeeperListener {
public static class PeerRegionServerListener extends ZKListener {
private final HBaseReplicationEndpoint replicationEndpoint;
private final String regionServerListNode;

View File

@ -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(

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.replication.master;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@ -30,7 +31,6 @@ import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
import org.apache.hadoop.hbase.replication.ReplicationFactory;
import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import java.io.IOException;
import java.util.Collections;
@ -48,7 +48,7 @@ import org.apache.zookeeper.KeeperException;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
private static final Log LOG = LogFactory.getLog(ReplicationLogCleaner.class);
private ZooKeeperWatcher zkw;
private ZKWatcher zkw;
private ReplicationQueuesClient replicationQueues;
private boolean stopped = false;
private Set<String> wals;
@ -101,14 +101,14 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
// I can close myself when comes time.
Configuration conf = new Configuration(config);
try {
setConf(conf, new ZooKeeperWatcher(conf, "replicationLogCleaner", null));
setConf(conf, new ZKWatcher(conf, "replicationLogCleaner", null));
} catch (IOException e) {
LOG.error("Error while configuring " + this.getClass().getName(), e);
}
}
@VisibleForTesting
public void setConf(Configuration conf, ZooKeeperWatcher zk) {
public void setConf(Configuration conf, ZKWatcher zk) {
super.setConf(conf);
try {
this.zkw = zk;

View File

@ -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();

View File

@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
import org.apache.hadoop.hbase.replication.ReplicationTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.zookeeper.KeeperException;
@ -211,7 +211,7 @@ public class DumpReplicationQueues extends Configured implements Tool {
ClusterConnection connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
Admin admin = connection.getAdmin();
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "DumpReplicationQueues" + System.currentTimeMillis(),
ZKWatcher zkw = new ZKWatcher(conf, "DumpReplicationQueues" + System.currentTimeMillis(),
new WarnOnlyAbortable(), true);
try {
@ -302,8 +302,8 @@ public class DumpReplicationQueues extends Configured implements Tool {
return sb.toString();
}
public String dumpQueues(ClusterConnection connection, ZooKeeperWatcher zkw, Set<String> peerIds,
boolean hdfs) throws Exception {
public String dumpQueues(ClusterConnection connection, ZKWatcher zkw, Set<String> peerIds,
boolean hdfs) throws Exception {
ReplicationQueuesClient queuesClient;
ReplicationPeers replicationPeers;
ReplicationQueues replicationQueues;

View File

@ -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;
}

View File

@ -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;

View File

@ -39,13 +39,13 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.AuthUtil;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.security.Superusers;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@ -112,7 +112,7 @@ public class TableAuthManager implements Closeable {
private ZKPermissionWatcher zkperms;
private final AtomicLong mtime = new AtomicLong(0L);
private TableAuthManager(ZooKeeperWatcher watcher, Configuration conf)
private TableAuthManager(ZKWatcher watcher, Configuration conf)
throws IOException {
this.conf = conf;
@ -734,14 +734,14 @@ public class TableAuthManager implements Closeable {
return mtime.get();
}
private static Map<ZooKeeperWatcher,TableAuthManager> managerMap = new HashMap<>();
private static Map<ZKWatcher,TableAuthManager> managerMap = new HashMap<>();
private static Map<TableAuthManager, Integer> refCount = new HashMap<>();
/** Returns a TableAuthManager from the cache. If not cached, constructs a new one. Returned
* instance should be released back by calling {@link #release(TableAuthManager)}. */
public synchronized static TableAuthManager getOrCreate(
ZooKeeperWatcher watcher, Configuration conf) throws IOException {
ZKWatcher watcher, Configuration conf) throws IOException {
TableAuthManager instance = managerMap.get(watcher);
if (instance == null) {
instance = new TableAuthManager(watcher, conf);

View File

@ -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;

View File

@ -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,

View File

@ -26,6 +26,7 @@ import com.google.protobuf.ByteString;
import com.google.protobuf.ServiceException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
@ -37,7 +38,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.Job;
@ -289,7 +289,7 @@ public class TokenUtil {
*/
private static Token<AuthenticationTokenIdentifier> getAuthToken(Configuration conf, User user)
throws IOException, InterruptedException {
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "TokenUtil-getAuthToken", null);
ZKWatcher zkw = new ZKWatcher(conf, "TokenUtil-getAuthToken", null);
try {
String clusterId = ZKClusterId.readClusterIdZNode(zkw);
if (clusterId == null) {

View File

@ -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

View File

@ -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) {

View File

@ -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

View File

@ -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.
*/

View File

@ -130,8 +130,8 @@ import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.UserGroupInformation;
@ -316,7 +316,7 @@ public class HBaseFsck extends Configured implements Closeable {
private Map<TableName, Set<String>> skippedRegions = new HashMap<>();
private ZooKeeperWatcher zkw = null;
private ZKWatcher zkw = null;
private String hbckEphemeralNodePath = null;
private boolean hbckZodeCreated = false;
@ -1918,8 +1918,8 @@ public class HBaseFsck extends Configured implements Closeable {
return true;
}
private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
return new ZooKeeperWatcher(getConf(), "hbase Fsck", new Abortable() {
private ZKWatcher createZooKeeperWatcher() throws IOException {
return new ZKWatcher(getConf(), "hbase Fsck", new Abortable() {
@Override
public void abort(String why, Throwable e) {
LOG.error(why, e);

View File

@ -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);

View File

@ -25,14 +25,14 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
/**
@ -52,7 +52,7 @@ public class ZKDataMigrator {
* table descriptor based states.
*/
@Deprecated
public static Map<TableName, TableState.State> queryForTableStates(ZooKeeperWatcher zkw)
public static Map<TableName, TableState.State> queryForTableStates(ZKWatcher zkw)
throws KeeperException, InterruptedException {
Map<TableName, TableState.State> rv = new HashMap<>();
List<String> children = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.tableZNode);
@ -86,14 +86,14 @@ public class ZKDataMigrator {
/**
* Gets table state from ZK.
* @param zkw ZooKeeperWatcher instance to use
* @param zkw ZKWatcher instance to use
* @param tableName table we're checking
* @return Null or {@link ZooKeeperProtos.DeprecatedTableState.State} found in znode.
* @throws KeeperException
*/
@Deprecated
private static ZooKeeperProtos.DeprecatedTableState.State getTableState(
final ZooKeeperWatcher zkw, final TableName tableName)
final ZKWatcher zkw, final TableName tableName)
throws KeeperException, InterruptedException {
String znode = ZNodePaths.joinZNode(zkw.znodePaths.tableZNode, tableName.getNameAsString());
byte [] data = ZKUtil.getData(zkw, znode);

View File

@ -27,13 +27,13 @@ import java.util.Map.Entry;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleaner;
import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
/*
* Check and fix undeleted replication queues for removed peerId.
@ -47,8 +47,8 @@ public class ReplicationChecker {
private Set<String> undeletedHFileRefsQueueIds = new HashSet<>();
private final ReplicationZKNodeCleaner cleaner;
public ReplicationChecker(Configuration conf, ZooKeeperWatcher zkw, ClusterConnection connection,
ErrorReporter errorReporter) throws IOException {
public ReplicationChecker(Configuration conf, ZKWatcher zkw, ClusterConnection connection,
ErrorReporter errorReporter) throws IOException {
this.cleaner = new ReplicationZKNodeCleaner(conf, zkw, connection);
this.errorReporter = errorReporter;
}

View File

@ -20,11 +20,11 @@
<%@ page contentType="text/html;charset=UTF-8"
import="org.apache.commons.lang3.StringEscapeUtils"
import="org.apache.hadoop.hbase.zookeeper.ZKUtil"
import="org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher"
import="org.apache.hadoop.hbase.zookeeper.ZKWatcher"
import="org.apache.hadoop.hbase.HBaseConfiguration"
import="org.apache.hadoop.hbase.master.HMaster"%><%
HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
ZooKeeperWatcher watcher = master.getZooKeeper();
ZKWatcher watcher = master.getZooKeeper();
%>
<!DOCTYPE html>
<?xml version="1.0" encoding="UTF-8" ?>

View File

@ -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;

View File

@ -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;
}

View File

@ -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());

View File

@ -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);
}

View File

@ -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;

View File

@ -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);

View File

@ -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,

View File

@ -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 {

View File

@ -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) {

View File

@ -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) {

View File

@ -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;
}

View File

@ -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;
}

View File

@ -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;
}

View File

@ -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);

View File

@ -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;

View File

@ -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);
}

View File

@ -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);

View File

@ -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) {

View File

@ -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();

View File

@ -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);

View File

@ -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) {

View File

@ -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();
}

View File

@ -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();
}

View File

@ -57,7 +57,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.junit.AfterClass;
@ -234,7 +234,7 @@ public class TestLogsCleaner {
// when zk is working both files should be returned
cleaner = new ReplicationLogCleaner();
try (ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testZooKeeperAbort-normal", null)) {
try (ZKWatcher zkw = new ZKWatcher(conf, "testZooKeeperAbort-normal", null)) {
cleaner.setConf(conf, zkw);
cleaner.preClean();
Iterable<FileStatus> filesToDelete = cleaner.getDeletableFiles(dummyFiles);
@ -255,9 +255,9 @@ public class TestLogsCleaner {
}
@Override
public ZooKeeperWatcher getZooKeeper() {
public ZKWatcher getZooKeeper() {
try {
return new ZooKeeperWatcher(getConfiguration(), "dummy server", this);
return new ZKWatcher(getConfiguration(), "dummy server", this);
} catch (IOException e) {
e.printStackTrace();
}
@ -321,7 +321,7 @@ public class TestLogsCleaner {
}
}
static class FaultyZooKeeperWatcher extends ZooKeeperWatcher {
static class FaultyZooKeeperWatcher extends ZKWatcher {
private RecoverableZooKeeper zk;
public FaultyZooKeeperWatcher(Configuration conf, String identifier, Abortable abortable)

View File

@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.junit.After;
@ -240,7 +240,7 @@ public class TestReplicationHFileCleaner {
// when zk is working both files should be returned
cleaner = new ReplicationHFileCleaner();
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testZooKeeperAbort-normal", null);
ZKWatcher zkw = new ZKWatcher(conf, "testZooKeeperAbort-normal", null);
try {
cleaner.setConf(conf, zkw);
Iterable<FileStatus> filesToDelete = cleaner.getDeletableFiles(dummyFiles);
@ -263,9 +263,9 @@ public class TestReplicationHFileCleaner {
}
@Override
public ZooKeeperWatcher getZooKeeper() {
public ZKWatcher getZooKeeper() {
try {
return new ZooKeeperWatcher(getConfiguration(), "dummy server", this);
return new ZKWatcher(getConfiguration(), "dummy server", this);
} catch (IOException e) {
e.printStackTrace();
}
@ -332,7 +332,7 @@ public class TestReplicationHFileCleaner {
}
}
static class FaultyZooKeeperWatcher extends ZooKeeperWatcher {
static class FaultyZooKeeperWatcher extends ZKWatcher {
private RecoverableZooKeeper zk;
public FaultyZooKeeperWatcher(Configuration conf, String identifier, Abortable abortable)
throws ZooKeeperConnectionException, IOException {

View File

@ -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);

View File

@ -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 =

View File

@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.errorhandling.TimeoutException;
import org.apache.hadoop.hbase.procedure.Subprocedure.SubprocedureImpl;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -85,8 +85,8 @@ public class TestZKProcedure {
UTIL.shutdownMiniZKCluster();
}
private static ZooKeeperWatcher newZooKeeperWatcher() throws IOException {
return new ZooKeeperWatcher(UTIL.getConfiguration(), "testing utility", new Abortable() {
private static ZKWatcher newZooKeeperWatcher() throws IOException {
return new ZKWatcher(UTIL.getConfiguration(), "testing utility", new Abortable() {
@Override
public void abort(String why, Throwable e) {
throw new RuntimeException(
@ -123,7 +123,7 @@ public class TestZKProcedure {
List<String> expected = Arrays.asList(members);
// setup the constants
ZooKeeperWatcher coordZkw = newZooKeeperWatcher();
ZKWatcher coordZkw = newZooKeeperWatcher();
String opDescription = "coordination test - " + members.length + " cohort members";
// start running the controller
@ -144,7 +144,7 @@ public class TestZKProcedure {
List<Pair<ProcedureMember, ZKProcedureMemberRpcs>> procMembers = new ArrayList<>(members.length);
// start each member
for (String member : members) {
ZooKeeperWatcher watcher = newZooKeeperWatcher();
ZKWatcher watcher = newZooKeeperWatcher();
ZKProcedureMemberRpcs comms = new ZKProcedureMemberRpcs(watcher, opDescription);
ThreadPoolExecutor pool2 = ProcedureMember.defaultPool(member, 1, KEEP_ALIVE);
ProcedureMember procMember = new ProcedureMember(comms, pool2, subprocFactory);
@ -207,7 +207,7 @@ public class TestZKProcedure {
final CountDownLatch coordinatorReceivedErrorLatch = new CountDownLatch(1);
// start running the coordinator and its controller
ZooKeeperWatcher coordinatorWatcher = newZooKeeperWatcher();
ZKWatcher coordinatorWatcher = newZooKeeperWatcher();
ZKProcedureCoordinator coordinatorController = new ZKProcedureCoordinator(
coordinatorWatcher, opDescription, COORDINATOR_NODE_NAME);
ThreadPoolExecutor pool = ProcedureCoordinator.defaultPool(COORDINATOR_NODE_NAME, POOL_SIZE, KEEP_ALIVE);
@ -217,7 +217,7 @@ public class TestZKProcedure {
SubprocedureFactory subprocFactory = Mockito.mock(SubprocedureFactory.class);
List<Pair<ProcedureMember, ZKProcedureMemberRpcs>> members = new ArrayList<>(expected.size());
for (String member : expected) {
ZooKeeperWatcher watcher = newZooKeeperWatcher();
ZKWatcher watcher = newZooKeeperWatcher();
ZKProcedureMemberRpcs controller = new ZKProcedureMemberRpcs(watcher, opDescription);
ThreadPoolExecutor pool2 = ProcedureMember.defaultPool(member, 1, KEEP_ALIVE);
ProcedureMember mem = new ProcedureMember(controller, pool2, subprocFactory);

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -79,7 +79,7 @@ public class TestZKProcedureControllers {
*/
@Test(timeout = 60000)
public void testSimpleZKCohortMemberController() throws Exception {
ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
ZKWatcher watcher = UTIL.getZooKeeperWatcher();
final String operationName = "instanceTest";
final Subprocedure sub = Mockito.mock(Subprocedure.class);
@ -171,7 +171,7 @@ public class TestZKProcedureControllers {
private void runMockCommitWithOrchestratedControllers(StartControllers controllers,
String operationName, byte[] data, String... cohort) throws Exception {
ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
ZKWatcher watcher = UTIL.getZooKeeperWatcher();
List<String> expected = Lists.newArrayList(cohort);
final Subprocedure sub = Mockito.mock(Subprocedure.class);
@ -247,7 +247,7 @@ public class TestZKProcedureControllers {
public void runEarlyPrepareNodes(StartControllers controllers, String operationName, byte[] data,
String... cohort) throws Exception {
ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
ZKWatcher watcher = UTIL.getZooKeeperWatcher();
List<String> expected = Lists.newArrayList(cohort);
final Subprocedure sub = Mockito.mock(Subprocedure.class);
@ -344,7 +344,7 @@ public class TestZKProcedureControllers {
/**
* Verify that the prepare, commit and abort nodes for the operation are removed from zookeeper
*/
private void verifyZooKeeperClean(String operationName, ZooKeeperWatcher watcher,
private void verifyZooKeeperClean(String operationName, ZKWatcher watcher,
ZKProcedureUtil controller) throws Exception {
String prepare = ZKProcedureUtil.getAcquireBarrierNode(controller, operationName);
String commit = ZKProcedureUtil.getReachedBarrierNode(controller, operationName);
@ -383,18 +383,18 @@ public class TestZKProcedureControllers {
*/
private abstract class StartControllers {
public abstract Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
ZooKeeperWatcher watcher, String operationName,
ProcedureCoordinator coordinator, String controllerName,
ProcedureMember member, List<String> cohortNames) throws Exception;
ZKWatcher watcher, String operationName,
ProcedureCoordinator coordinator, String controllerName,
ProcedureMember member, List<String> cohortNames) throws Exception;
}
private final StartControllers startCoordinatorFirst = new StartControllers() {
@Override
public Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
ZooKeeperWatcher watcher, String operationName,
ProcedureCoordinator coordinator, String controllerName,
ProcedureMember member, List<String> expected) throws Exception {
ZKWatcher watcher, String operationName,
ProcedureCoordinator coordinator, String controllerName,
ProcedureMember member, List<String> expected) throws Exception {
// start the controller
ZKProcedureCoordinator controller = new ZKProcedureCoordinator(
watcher, operationName, CONTROLLER_NODE_NAME);
@ -420,9 +420,9 @@ public class TestZKProcedureControllers {
@Override
public Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
ZooKeeperWatcher watcher, String operationName,
ProcedureCoordinator coordinator, String controllerName,
ProcedureMember member, List<String> expected) throws Exception {
ZKWatcher watcher, String operationName,
ProcedureCoordinator coordinator, String controllerName,
ProcedureMember member, List<String> expected) throws Exception {
// make a cohort controller for each expected node
List<ZKProcedureMemberRpcs> cohortControllers = new ArrayList<>();

Some files were not shown because too many files have changed in this diff Show More