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:
parent
3330dd517e
commit
e0c4f374b5
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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;
|
||||
|
@ -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);
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;}
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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,
|
||||
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,
|
||||
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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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,7 +84,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
|
|||
|
||||
private static final Log LOG = LogFactory.getLog(ReplicationPeersZKImpl.class);
|
||||
|
||||
public ReplicationPeersZKImpl(final ZooKeeperWatcher zk, final Configuration conf,
|
||||
public ReplicationPeersZKImpl(final ZKWatcher zk, final Configuration conf,
|
||||
final ReplicationQueuesClient queuesClient, Abortable abortable) {
|
||||
super(zk, conf, abortable);
|
||||
this.abortable = abortable;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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,7 +44,7 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
|
|||
this(args.getZk(), args.getConf(), args.getAbortable());
|
||||
}
|
||||
|
||||
public ReplicationQueuesClientZKImpl(final ZooKeeperWatcher zk, Configuration conf,
|
||||
public ReplicationQueuesClientZKImpl(final ZKWatcher zk, Configuration conf,
|
||||
Abortable abortable) {
|
||||
super(zk, conf, abortable);
|
||||
}
|
||||
|
|
|
@ -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,7 +73,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
this(args.getZk(), args.getConf(), args.getAbortable());
|
||||
}
|
||||
|
||||
public ReplicationQueuesZKImpl(final ZooKeeperWatcher zk, Configuration conf,
|
||||
public ReplicationQueuesZKImpl(final ZKWatcher zk, Configuration conf,
|
||||
Abortable abortable) {
|
||||
super(zk, conf, abortable);
|
||||
}
|
||||
|
|
|
@ -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,7 +72,7 @@ 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,
|
||||
public ReplicationStateZKBase(ZKWatcher zookeeper, Configuration conf,
|
||||
Abortable abortable) {
|
||||
this.zookeeper = zookeeper;
|
||||
this.conf = conf;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {};
|
||||
|
|
|
@ -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<>();
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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<>();
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
|
@ -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; }
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 "";
|
||||
}
|
||||
|
|
|
@ -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;
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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 "";
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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,7 +302,7 @@ public class DumpReplicationQueues extends Configured implements Tool {
|
|||
return sb.toString();
|
||||
}
|
||||
|
||||
public String dumpQueues(ClusterConnection connection, ZooKeeperWatcher zkw, Set<String> peerIds,
|
||||
public String dumpQueues(ClusterConnection connection, ZKWatcher zkw, Set<String> peerIds,
|
||||
boolean hdfs) throws Exception {
|
||||
ReplicationQueuesClient queuesClient;
|
||||
ReplicationPeers replicationPeers;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,7 +89,7 @@ public class AuthenticationTokenSecretManager
|
|||
* org.apache.hadoop.hbase.ipc.SecureServer so public access is needed.
|
||||
*/
|
||||
public AuthenticationTokenSecretManager(Configuration conf,
|
||||
ZooKeeperWatcher zk, String serverName,
|
||||
ZKWatcher zk, String serverName,
|
||||
long keyUpdateInterval, long tokenMaxLifetime) {
|
||||
this.zkWatcher = new ZKSecretWatcher(conf, zk, this);
|
||||
this.keyUpdateInterval = keyUpdateInterval;
|
||||
|
@ -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()) {
|
||||
|
@ -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,
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,7 +34,7 @@ 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";
|
||||
|
@ -47,7 +47,7 @@ public class ZKVisibilityLabelWatcher extends ZooKeeperListener {
|
|||
private String labelZnode;
|
||||
private String userAuthsZnode;
|
||||
|
||||
public ZKVisibilityLabelWatcher(ZooKeeperWatcher watcher, VisibilityLabelsCache labelsCache,
|
||||
public ZKVisibilityLabelWatcher(ZKWatcher watcher, VisibilityLabelsCache labelsCache,
|
||||
Configuration conf) {
|
||||
super(watcher);
|
||||
this.labelsCache = labelsCache;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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,7 +47,7 @@ public class ReplicationChecker {
|
|||
private Set<String> undeletedHFileRefsQueueIds = new HashSet<>();
|
||||
private final ReplicationZKNodeCleaner cleaner;
|
||||
|
||||
public ReplicationChecker(Configuration conf, ZooKeeperWatcher zkw, ClusterConnection connection,
|
||||
public ReplicationChecker(Configuration conf, ZKWatcher zkw, ClusterConnection connection,
|
||||
ErrorReporter errorReporter) throws IOException {
|
||||
this.cleaner = new ReplicationZKNodeCleaner(conf, zkw, connection);
|
||||
this.errorReporter = errorReporter;
|
||||
|
|
|
@ -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" ?>
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
@ -195,7 +195,7 @@ public class TestActiveMasterManager {
|
|||
* @throws KeeperException
|
||||
* @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();
|
||||
|
@ -281,7 +281,7 @@ public class TestActiveMasterManager {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ZooKeeperWatcher getZooKeeper() {
|
||||
public ZKWatcher getZooKeeper() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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,11 +549,11 @@ 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,
|
||||
Table installTable(ZKWatcher zkw, String tname, String fname, int nrs,
|
||||
int existingRegions) throws Exception {
|
||||
// Create a table with regions
|
||||
TableName table = TableName.valueOf(tname);
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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,7 +70,7 @@ public class TestTableStateManager {
|
|||
TableState.State.DISABLED);
|
||||
}
|
||||
|
||||
private void setTableStateInZK(ZooKeeperWatcher watcher, final TableName tableName,
|
||||
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());
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 =
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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,7 +383,7 @@ public class TestZKProcedureControllers {
|
|||
*/
|
||||
private abstract class StartControllers {
|
||||
public abstract Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
|
||||
ZooKeeperWatcher watcher, String operationName,
|
||||
ZKWatcher watcher, String operationName,
|
||||
ProcedureCoordinator coordinator, String controllerName,
|
||||
ProcedureMember member, List<String> cohortNames) throws Exception;
|
||||
}
|
||||
|
@ -392,7 +392,7 @@ public class TestZKProcedureControllers {
|
|||
|
||||
@Override
|
||||
public Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
|
||||
ZooKeeperWatcher watcher, String operationName,
|
||||
ZKWatcher watcher, String operationName,
|
||||
ProcedureCoordinator coordinator, String controllerName,
|
||||
ProcedureMember member, List<String> expected) throws Exception {
|
||||
// start the controller
|
||||
|
@ -420,7 +420,7 @@ public class TestZKProcedureControllers {
|
|||
|
||||
@Override
|
||||
public Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
|
||||
ZooKeeperWatcher watcher, String operationName,
|
||||
ZKWatcher watcher, String operationName,
|
||||
ProcedureCoordinator coordinator, String controllerName,
|
||||
ProcedureMember member, List<String> expected) throws Exception {
|
||||
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue