HBASE-19200 Make hbase-client only depend on ZKAsyncRegistry and ZNodePaths

- Removes zookeeper connection from ClusterConnection
- Deletes class ZooKeeperKeepAliveConnection
- Removes Registry, ZooKeeperRegistry, and RegistryFactory
This commit is contained in:
zhangduo 2017-11-10 22:16:45 +08:00 committed by Apekshit Sharma
parent 31234eb862
commit 72270866cb
66 changed files with 686 additions and 1043 deletions

View File

@ -226,6 +226,10 @@ public interface ClusterConnection extends Connection {
*/
MasterService.BlockingInterface getMaster() throws IOException;
/**
* Get the admin service for master.
*/
AdminService.BlockingInterface getAdminForMaster() throws IOException;
/**
* Establishes a connection to the region server at the specified address.

View File

@ -25,16 +25,23 @@ import static org.apache.hadoop.hbase.client.MetricsConnection.CLIENT_SIDE_METRI
import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsentEx;
import com.google.common.base.Throwables;
import edu.umd.cs.findbugs.annotations.Nullable;
import java.io.Closeable;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.reflect.UndeclaredThrowableException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Date;
import java.util.List;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
@ -47,7 +54,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CallQueueTooBigException;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
@ -72,9 +78,6 @@ import org.apache.hadoop.hbase.util.ExceptionUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
@ -124,8 +127,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Remov
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
import edu.umd.cs.findbugs.annotations.Nullable;
/**
* Main implementation of {@link Connection} and {@link ClusterConnection} interfaces.
* Encapsulates connection to zookeeper and regionservers.
@ -168,12 +169,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
private final Object metaRegionLock = new Object();
// We have a single lock for master & zk to prevent deadlocks. Having
// one lock for ZK and one lock for master is not possible:
// When creating a connection to master, we need a connection to ZK to get
// its address. But another thread could have taken the ZK lock, and could
// be waiting for the master lock => deadlock.
private final Object masterAndZKLock = new Object();
private final Object masterLock = new Object();
// thread executor shared by all Table instances created
// by this connection
@ -206,7 +202,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
/**
* Cluster registry of basic info such as clusterid and meta region location.
*/
Registry registry;
private final AsyncRegistry registry;
private final ClientBackoffPolicy backoffPolicy;
@ -284,7 +280,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
this.conf.get(BufferedMutator.CLASSNAME_KEY);
try {
this.registry = setupRegistry();
this.registry = AsyncRegistryFactory.getRegistry(conf);
retrieveClusterId();
this.rpcClient = RpcClientFactory.createClient(this.conf, this.clusterId, this.metrics);
@ -494,13 +490,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
}
/**
* @return The cluster registry implementation to use.
*/
private Registry setupRegistry() throws IOException {
return RegistryFactory.getRegistry(this);
}
/**
* For tests only.
*/
@ -523,7 +512,11 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
if (clusterId != null) {
return;
}
this.clusterId = this.registry.getClusterId();
try {
this.clusterId = this.registry.getClusterId().get();
} catch (InterruptedException | ExecutionException e) {
LOG.warn("Retrieve cluster id failed", e);
}
if (clusterId == null) {
clusterId = HConstants.CLUSTER_ID_DEFAULT;
LOG.debug("clusterid came back null, using default " + clusterId);
@ -535,25 +528,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
return this.conf;
}
private void checkIfBaseNodeAvailable(ZooKeeperWatcher zkw)
throws MasterNotRunningException {
String errorMsg;
try {
if (ZKUtil.checkExists(zkw, zkw.znodePaths.baseZNode) == -1) {
errorMsg = "The node " + zkw.znodePaths.baseZNode+" is not in ZooKeeper. "
+ "It should have been written by the master. "
+ "Check the value configured in 'zookeeper.znode.parent'. "
+ "There could be a mismatch with the one configured in the master.";
LOG.error(errorMsg);
throw new MasterNotRunningException(errorMsg);
}
} catch (KeeperException e) {
errorMsg = "Can't get connection to ZooKeeper: " + e.getMessage();
LOG.error(errorMsg);
throw new MasterNotRunningException(errorMsg, e);
}
}
/**
* @return true if the master is running, throws an exception otherwise
* @throws org.apache.hadoop.hbase.MasterNotRunningException - if the master is not running
@ -652,8 +626,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
@Override
public HRegionLocation locateRegion(final byte[] regionName) throws IOException {
RegionLocations locations = locateRegion(HRegionInfo.getTable(regionName),
HRegionInfo.getStartKey(regionName), false, true);
RegionLocations locations = locateRegion(RegionInfo.getTable(regionName),
RegionInfo.getStartKey(regionName), false, true);
return locations == null ? null : locations.getRegionLocation();
}
@ -667,17 +641,20 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
@Override
public List<HRegionLocation> locateRegions(final TableName tableName)
throws IOException {
public List<HRegionLocation> locateRegions(TableName tableName) throws IOException {
return locateRegions(tableName, false, true);
}
@Override
public List<HRegionLocation> locateRegions(final TableName tableName,
final boolean useCache, final boolean offlined) throws IOException {
List<RegionInfo> regions = MetaTableAccessor
.getTableRegions(this, tableName, !offlined);
final List<HRegionLocation> locations = new ArrayList<>();
public List<HRegionLocation> locateRegions(TableName tableName, boolean useCache,
boolean offlined) throws IOException {
List<RegionInfo> regions;
if (TableName.isMetaTableName(tableName)) {
regions = Collections.singletonList(RegionInfoBuilder.FIRST_META_REGIONINFO);
} else {
regions = MetaTableAccessor.getTableRegions(this, tableName, !offlined);
}
List<HRegionLocation> locations = new ArrayList<>();
for (RegionInfo regionInfo : regions) {
RegionLocations list = locateRegion(tableName, regionInfo.getStartKey(), useCache, true);
if (list != null) {
@ -772,7 +749,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
// Look up from zookeeper
locations = this.registry.getMetaRegionLocation();
locations = get(this.registry.getMetaRegionLocation());
if (locations != null) {
cacheLocation(tableName, locations);
}
@ -799,7 +776,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
// build the key of the meta region we should be looking for.
// the extra 9's on the end are necessary to allow "exact" matches
// without knowing the precise region names.
byte[] metaKey = HRegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
byte[] metaKey = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
Scan s = new Scan();
s.setReversed(true);
@ -851,7 +828,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
throw new IOException("HRegionInfo was null in " +
tableName + ", row=" + regionInfoRow);
}
HRegionInfo regionInfo = locations.getRegionLocation(replicaId).getRegionInfo();
RegionInfo regionInfo = locations.getRegionLocation(replicaId).getRegion();
if (regionInfo == null) {
throw new IOException("HRegionInfo was null or empty in " +
TableName.META_TABLE_NAME + ", row=" + regionInfoRow);
@ -1124,37 +1101,25 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
*/
private MasterProtos.MasterService.BlockingInterface makeStubNoRetries()
throws IOException, KeeperException {
ZooKeeperKeepAliveConnection zkw;
try {
zkw = getKeepAliveZooKeeperWatcher();
} catch (IOException e) {
ExceptionUtil.rethrowIfInterrupt(e);
throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e);
ServerName sn = get(registry.getMasterAddress());
if (sn == null) {
String msg = "ZooKeeper available but no active master location found";
LOG.info(msg);
throw new MasterNotRunningException(msg);
}
try {
checkIfBaseNodeAvailable(zkw);
ServerName sn = MasterAddressTracker.getMasterAddress(zkw);
if (sn == null) {
String msg = "ZooKeeper available but no active master location found";
LOG.info(msg);
throw new MasterNotRunningException(msg);
}
if (isDeadServer(sn)) {
throw new MasterNotRunningException(sn + " is dead.");
}
// Use the security info interface name as our stub key
String key = getStubKey(MasterProtos.MasterService.getDescriptor().getName(), sn,
hostnamesCanChange);
MasterProtos.MasterService.BlockingInterface stub =
(MasterProtos.MasterService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> {
BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(sn, user, rpcTimeout);
return MasterProtos.MasterService.newBlockingStub(channel);
});
isMasterRunning(stub);
return stub;
} finally {
zkw.close();
if (isDeadServer(sn)) {
throw new MasterNotRunningException(sn + " is dead.");
}
// Use the security info interface name as our stub key
String key =
getStubKey(MasterProtos.MasterService.getDescriptor().getName(), sn, hostnamesCanChange);
MasterProtos.MasterService.BlockingInterface stub =
(MasterProtos.MasterService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> {
BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(sn, user, rpcTimeout);
return MasterProtos.MasterService.newBlockingStub(channel);
});
isMasterRunning(stub);
return stub;
}
/**
@ -1165,7 +1130,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
MasterProtos.MasterService.BlockingInterface makeStub() throws IOException {
// The lock must be at the beginning to prevent multiple master creations
// (and leaks) in a multithread context
synchronized (masterAndZKLock) {
synchronized (masterLock) {
Exception exceptionCaught = null;
if (!closed) {
try {
@ -1183,6 +1148,11 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
}
@Override
public AdminProtos.AdminService.BlockingInterface getAdminForMaster() throws IOException {
return getAdmin(get(registry.getMasterAddress()));
}
@Override
public AdminProtos.AdminService.BlockingInterface getAdmin(ServerName serverName)
throws IOException {
@ -1212,48 +1182,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
});
}
private ZooKeeperKeepAliveConnection keepAliveZookeeper;
private AtomicInteger keepAliveZookeeperUserCount = new AtomicInteger(0);
/**
* Retrieve a shared ZooKeeperWatcher. You must close it it once you've have finished with it.
* @return The shared instance. Never returns null.
*/
ZooKeeperKeepAliveConnection getKeepAliveZooKeeperWatcher()
throws IOException {
synchronized (masterAndZKLock) {
if (keepAliveZookeeper == null) {
if (this.closed) {
throw new IOException(toString() + " closed");
}
// We don't check that our link to ZooKeeper is still valid
// But there is a retry mechanism in the ZooKeeperWatcher itself
keepAliveZookeeper = new ZooKeeperKeepAliveConnection(conf, this.toString(), this);
}
keepAliveZookeeperUserCount.addAndGet(1);
return keepAliveZookeeper;
}
}
void releaseZooKeeperWatcher(final ZooKeeperWatcher zkw) {
if (zkw == null){
return;
}
}
private void closeZooKeeperWatcher() {
synchronized (masterAndZKLock) {
if (keepAliveZookeeper != null) {
LOG.info("Closing zookeeper sessionid=0x" +
Long.toHexString(
keepAliveZookeeper.getRecoverableZooKeeper().getSessionId()));
keepAliveZookeeper.internalClose();
keepAliveZookeeper = null;
}
keepAliveZookeeperUserCount.set(0);
}
}
final MasterServiceState masterServiceState = new MasterServiceState(this);
@Override
@ -1268,7 +1196,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
@Override
public MasterKeepAliveConnection getKeepAliveMasterService()
throws MasterNotRunningException {
synchronized (masterAndZKLock) {
synchronized (masterLock) {
if (!isKeepAliveMasterConnectedAndRunning(this.masterServiceState)) {
MasterServiceStubMaker stubMaker = new MasterServiceStubMaker();
try {
@ -1812,7 +1740,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
if (mss.getStub() == null) {
return;
}
synchronized (masterAndZKLock) {
synchronized (masterLock) {
--mss.userCount;
}
}
@ -1830,13 +1758,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
* connection itself.
*/
private void closeMaster() {
synchronized (masterAndZKLock) {
synchronized (masterLock) {
closeMasterService(masterServiceState);
}
}
void updateCachedLocation(HRegionInfo hri, ServerName source,
ServerName serverName, long seqNum) {
void updateCachedLocation(RegionInfo hri, ServerName source, ServerName serverName, long seqNum) {
HRegionLocation newHrl = new HRegionLocation(hri, serverName, seqNum);
cacheLocation(hri.getTable(), source, newHrl);
}
@ -1889,7 +1816,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
return;
}
HRegionInfo regionInfo = oldLocation.getRegionInfo();
RegionInfo regionInfo = oldLocation.getRegion();
Throwable cause = ClientExceptionsUtil.findException(exception);
if (cause != null) {
if (!ClientExceptionsUtil.isMetaClearingException(cause)) {
@ -1902,7 +1829,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
if (LOG.isTraceEnabled()) {
LOG.trace("Region " + regionInfo.getRegionNameAsString() + " moved to " +
rme.getHostname() + ":" + rme.getPort() +
" according to " + source.getHostAndPort());
" according to " + source.getAddress());
}
// We know that the region is not anymore on this region server, but we know
// the new location.
@ -1947,26 +1874,14 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
@Override
public void abort(final String msg, Throwable t) {
if (t instanceof KeeperException.SessionExpiredException
&& keepAliveZookeeper != null) {
synchronized (masterAndZKLock) {
if (keepAliveZookeeper != null) {
LOG.warn("This client just lost it's session with ZooKeeper," +
" closing it." +
" It will be recreated next time someone needs it", t);
closeZooKeeperWatcher();
}
}
if (t != null) {
LOG.fatal(msg, t);
} else {
if (t != null) {
LOG.fatal(msg, t);
} else {
LOG.fatal(msg);
}
this.aborted = true;
close();
this.closed = true;
LOG.fatal(msg);
}
this.aborted = true;
close();
this.closed = true;
}
@Override
@ -1981,7 +1896,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
@Override
public int getCurrentNrHRS() throws IOException {
return this.registry.getCurrentNrHRS();
return get(this.registry.getCurrentNrHRS());
}
@Override
@ -1995,7 +1910,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
this.metrics.shutdown();
}
this.closed = true;
closeZooKeeperWatcher();
registry.close();
this.stubs.clear();
if (clusterStatusListener != null) {
clusterStatusListener.close();
@ -2061,4 +1976,17 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
public RpcControllerFactory getRpcControllerFactory() {
return this.rpcControllerFactory;
}
private static <T> T get(CompletableFuture<T> future) throws IOException {
try {
return future.get();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw (IOException) new InterruptedIOException().initCause(e);
} catch (ExecutionException e) {
Throwable cause = e.getCause();
Throwables.propagateIfPossible(cause, IOException.class);
throw new IOException(cause);
}
}
}

View File

@ -18,6 +18,10 @@
*/
package org.apache.hadoop.hbase.client;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Message;
import com.google.protobuf.RpcController;
import java.io.Closeable;
import java.io.IOException;
import java.io.InterruptedIOException;
@ -97,14 +101,10 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
@ -206,10 +206,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Message;
import com.google.protobuf.RpcController;
/**
* HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that
* this is an HBase-internal class as defined in
@ -399,18 +395,11 @@ public class HBaseAdmin implements Admin {
}
@Override
public List<RegionInfo> getRegions(final TableName tableName) throws IOException {
ZooKeeperWatcher zookeeper =
new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
try {
if (TableName.META_TABLE_NAME.equals(tableName)) {
return new MetaTableLocator().getMetaRegions(zookeeper);
} else {
return MetaTableAccessor.getTableRegions(connection, tableName, true);
}
} finally {
zookeeper.close();
public List<RegionInfo> getRegions(TableName tableName) throws IOException {
if (TableName.isMetaTableName(tableName)) {
return Arrays.asList(RegionInfoBuilder.FIRST_META_REGIONINFO);
} else {
return MetaTableAccessor.getTableRegions(connection, tableName, true);
}
}
@ -1248,9 +1237,9 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void compactRegionServer(final ServerName sn, boolean major)
throws IOException, InterruptedException {
for (HRegionInfo region : getOnlineRegions(sn)) {
compact(sn, region, major, null);
throws IOException, InterruptedException {
for (RegionInfo region : getRegions(sn)) {
compact(this.connection.getAdmin(sn), region, major, null);
}
}
@ -1295,41 +1284,28 @@ public class HBaseAdmin implements Admin {
CompactType compactType) throws IOException {
switch (compactType) {
case MOB:
ServerName master = getMasterAddress();
compact(master, getMobRegionInfo(tableName), major, columnFamily);
compact(this.connection.getAdminForMaster(), getMobRegionInfo(tableName), major,
columnFamily);
break;
case NORMAL:
default:
ZooKeeperWatcher zookeeper = null;
try {
checkTableExists(tableName);
zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
List<Pair<RegionInfo, ServerName>> pairs;
if (TableName.META_TABLE_NAME.equals(tableName)) {
pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
} else {
pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
checkTableExists(tableName);
for (HRegionLocation loc :connection.locateRegions(tableName, false, false)) {
ServerName sn = loc.getServerName();
if (sn == null) {
continue;
}
for (Pair<RegionInfo, ServerName> pair: pairs) {
if (pair.getFirst().isOffline()) continue;
if (pair.getSecond() == null) continue;
try {
compact(pair.getSecond(), pair.getFirst(), major, columnFamily);
} catch (NotServingRegionException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to" + (major ? " major" : "") + " compact " +
pair.getFirst() + ": " +
StringUtils.stringifyException(e));
}
try {
compact(this.connection.getAdmin(sn), loc.getRegion(), major, columnFamily);
} catch (NotServingRegionException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to" + (major ? " major" : "") + " compact " + loc.getRegion() +
": " + StringUtils.stringifyException(e));
}
}
} finally {
if (zookeeper != null) {
zookeeper.close();
}
}
break;
default:
throw new IllegalArgumentException("Unknown compactType: " + compactType);
}
}
@ -1343,8 +1319,8 @@ public class HBaseAdmin implements Admin {
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
private void compactRegion(final byte[] regionName, final byte[] columnFamily,final boolean major)
throws IOException {
private void compactRegion(final byte[] regionName, final byte[] columnFamily,
final boolean major) throws IOException {
Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
if (regionServerPair == null) {
throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
@ -1352,13 +1328,12 @@ public class HBaseAdmin implements Admin {
if (regionServerPair.getSecond() == null) {
throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
}
compact(regionServerPair.getSecond(), regionServerPair.getFirst(), major, columnFamily);
compact(this.connection.getAdmin(regionServerPair.getSecond()), regionServerPair.getFirst(),
major, columnFamily);
}
private void compact(final ServerName sn, final RegionInfo hri,
final boolean major, final byte [] family)
throws IOException {
final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
private void compact(AdminService.BlockingInterface admin, RegionInfo hri, boolean major,
byte[] family) throws IOException {
Callable<Void> callable = new Callable<Void>() {
@Override
public Void call() throws Exception {
@ -1863,37 +1838,25 @@ public class HBaseAdmin implements Admin {
}
@Override
public void split(final TableName tableName, final byte [] splitPoint) throws IOException {
ZooKeeperWatcher zookeeper = null;
try {
checkTableExists(tableName);
zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
List<Pair<RegionInfo, ServerName>> pairs;
if (TableName.META_TABLE_NAME.equals(tableName)) {
pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
} else {
pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
public void split(final TableName tableName, final byte[] splitPoint) throws IOException {
checkTableExists(tableName);
for (HRegionLocation loc : connection.locateRegions(tableName, false, false)) {
ServerName sn = loc.getServerName();
if (sn == null) {
continue;
}
if (splitPoint == null) {
LOG.info("SplitPoint is null, will find bestSplitPoint from Region");
RegionInfo r = loc.getRegion();
// check for parents
if (r.isSplitParent()) {
continue;
}
for (Pair<RegionInfo, ServerName> pair: pairs) {
// May not be a server for a particular row
if (pair.getSecond() == null) continue;
RegionInfo r = pair.getFirst();
// check for parents
if (r.isSplitParent()) continue;
// if a split point given, only split that particular region
if (r.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID ||
(splitPoint != null && !r.containsRow(splitPoint))) continue;
// call out to master to do split now
splitRegionAsync(pair.getFirst(), splitPoint);
}
} finally {
if (zookeeper != null) {
zookeeper.close();
// if a split point given, only split that particular region
if (r.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID ||
(splitPoint != null && !r.containsRow(splitPoint))) {
continue;
}
// call out to master to do split now
splitRegionAsync(r, splitPoint);
}
}
@ -2345,33 +2308,15 @@ public class HBaseAdmin implements Admin {
});
}
/**
* Check to see if HBase is running. Throw an exception if not.
* @param conf system configuration
* @throws MasterNotRunningException if the master is not running
* @throws ZooKeeperConnectionException if unable to connect to zookeeper
* @deprecated since hbase-2.0.0 because throws a ServiceException. We don't want to have
* protobuf as part of our public API. Use {@link #available(Configuration)}
*/
// Used by tests and by the Merge tool. Merge tool uses it to figure if HBase is up or not.
// MOB uses it too.
// NOTE: hbase-2.0.0 removes ServiceException from the throw.
@Deprecated
public static void checkHBaseAvailable(Configuration conf)
throws MasterNotRunningException, ZooKeeperConnectionException, IOException,
com.google.protobuf.ServiceException {
available(conf);
}
/**
* Is HBase available? Throw an exception if not.
* @param conf system configuration
* @throws MasterNotRunningException if the master is not running.
* @throws ZooKeeperConnectionException if unable to connect to zookeeper.
* // TODO do not expose ZKConnectionException.
* @throws ZooKeeperConnectionException if unable to connect to zookeeper. // TODO do not expose
* ZKConnectionException.
*/
public static void available(final Configuration conf)
throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
Configuration copyOfConf = HBaseConfiguration.create(conf);
// We set it to make it fail as soon as possible if HBase is not available
copyOfConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
@ -2381,26 +2326,6 @@ public class HBaseAdmin implements Admin {
// If the connection exists, we may have a connection to ZK that does not work anymore
try (ClusterConnection connection =
(ClusterConnection) ConnectionFactory.createConnection(copyOfConf)) {
// Check ZK first.
// If the connection exists, we may have a connection to ZK that does not work anymore
ZooKeeperKeepAliveConnection zkw = null;
try {
// This is NASTY. FIX!!!! Dependent on internal implementation! TODO
zkw = ((ConnectionImplementation) connection)
.getKeepAliveZooKeeperWatcher();
zkw.getRecoverableZooKeeper().getZooKeeper().exists(zkw.znodePaths.baseZNode, false);
} catch (IOException e) {
throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e);
} catch (InterruptedException e) {
throw (InterruptedIOException)
new InterruptedIOException("Can't connect to ZooKeeper").initCause(e);
} catch (KeeperException e){
throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e);
} finally {
if (zkw != null) {
zkw.close();
}
}
// can throw MasterNotRunningException
connection.isMasterRunning();
}
@ -3232,17 +3157,6 @@ public class HBaseAdmin implements Admin {
}
}
private ServerName getMasterAddress() throws IOException {
// TODO: Fix! Reaching into internal implementation!!!!
ConnectionImplementation connection = (ConnectionImplementation)this.connection;
ZooKeeperKeepAliveConnection zkw = connection.getKeepAliveZooKeeperWatcher();
try {
return MasterAddressTracker.getMasterAddress(zkw);
} catch (KeeperException e) {
throw new IOException("Failed to get master server name from MasterAddressTracker", e);
}
}
@Override
public long getLastMajorCompactionTimestamp(final TableName tableName) throws IOException {
return executeCallable(new MasterCallable<Long>(getConnection(), getRpcControllerFactory()) {
@ -3311,102 +3225,88 @@ public class HBaseAdmin implements Admin {
* {@inheritDoc}
*/
@Override
public CompactionState getCompactionState(final TableName tableName,
CompactType compactType) throws IOException {
public CompactionState getCompactionState(final TableName tableName, CompactType compactType)
throws IOException {
AdminProtos.GetRegionInfoResponse.CompactionState state =
AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
checkTableExists(tableName);
// TODO: There is no timeout on this controller. Set one!
final HBaseRpcController rpcController = rpcControllerFactory.newController();
HBaseRpcController rpcController = rpcControllerFactory.newController();
switch (compactType) {
case MOB:
final AdminProtos.AdminService.BlockingInterface masterAdmin =
this.connection.getAdmin(getMasterAddress());
this.connection.getAdminForMaster();
Callable<AdminProtos.GetRegionInfoResponse.CompactionState> callable =
new Callable<AdminProtos.GetRegionInfoResponse.CompactionState>() {
@Override
public AdminProtos.GetRegionInfoResponse.CompactionState call() throws Exception {
HRegionInfo info = getMobRegionInfo(tableName);
GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
info.getRegionName(), true);
GetRegionInfoResponse response = masterAdmin.getRegionInfo(rpcController, request);
return response.getCompactionState();
}
};
new Callable<AdminProtos.GetRegionInfoResponse.CompactionState>() {
@Override
public AdminProtos.GetRegionInfoResponse.CompactionState call() throws Exception {
RegionInfo info = getMobRegionInfo(tableName);
GetRegionInfoRequest request =
RequestConverter.buildGetRegionInfoRequest(info.getRegionName(), true);
GetRegionInfoResponse response = masterAdmin.getRegionInfo(rpcController, request);
return response.getCompactionState();
}
};
state = ProtobufUtil.call(callable);
break;
case NORMAL:
default:
ZooKeeperWatcher zookeeper = null;
try {
List<Pair<RegionInfo, ServerName>> pairs;
if (TableName.META_TABLE_NAME.equals(tableName)) {
zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
} else {
pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
for (HRegionLocation loc : connection.locateRegions(tableName, false, false)) {
ServerName sn = loc.getServerName();
if (sn == null) {
continue;
}
for (Pair<RegionInfo, ServerName> pair: pairs) {
if (pair.getFirst().isOffline()) continue;
if (pair.getSecond() == null) continue;
final ServerName sn = pair.getSecond();
final byte [] regionName = pair.getFirst().getRegionName();
final AdminService.BlockingInterface snAdmin = this.connection.getAdmin(sn);
try {
Callable<GetRegionInfoResponse> regionInfoCallable =
new Callable<GetRegionInfoResponse>() {
byte[] regionName = loc.getRegion().getRegionName();
AdminService.BlockingInterface snAdmin = this.connection.getAdmin(sn);
try {
Callable<GetRegionInfoResponse> regionInfoCallable =
new Callable<GetRegionInfoResponse>() {
@Override
public GetRegionInfoResponse call() throws Exception {
GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
regionName, true);
GetRegionInfoRequest request =
RequestConverter.buildGetRegionInfoRequest(regionName, true);
return snAdmin.getRegionInfo(rpcController, request);
}
};
GetRegionInfoResponse response = ProtobufUtil.call(regionInfoCallable);
switch (response.getCompactionState()) {
case MAJOR_AND_MINOR:
GetRegionInfoResponse response = ProtobufUtil.call(regionInfoCallable);
switch (response.getCompactionState()) {
case MAJOR_AND_MINOR:
return CompactionState.MAJOR_AND_MINOR;
case MAJOR:
if (state == AdminProtos.GetRegionInfoResponse.CompactionState.MINOR) {
return CompactionState.MAJOR_AND_MINOR;
case MAJOR:
if (state == AdminProtos.GetRegionInfoResponse.CompactionState.MINOR) {
return CompactionState.MAJOR_AND_MINOR;
}
state = AdminProtos.GetRegionInfoResponse.CompactionState.MAJOR;
break;
case MINOR:
if (state == AdminProtos.GetRegionInfoResponse.CompactionState.MAJOR) {
return CompactionState.MAJOR_AND_MINOR;
}
state = AdminProtos.GetRegionInfoResponse.CompactionState.MINOR;
break;
case NONE:
default: // nothing, continue
}
} catch (NotServingRegionException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to get compaction state of " +
pair.getFirst() + ": " +
StringUtils.stringifyException(e));
}
} catch (RemoteException e) {
if (e.getMessage().indexOf(NotServingRegionException.class.getName()) >= 0) {
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to get compaction state of " + pair.getFirst() + ": "
+ StringUtils.stringifyException(e));
}
} else {
throw e;
}
state = AdminProtos.GetRegionInfoResponse.CompactionState.MAJOR;
break;
case MINOR:
if (state == AdminProtos.GetRegionInfoResponse.CompactionState.MAJOR) {
return CompactionState.MAJOR_AND_MINOR;
}
state = AdminProtos.GetRegionInfoResponse.CompactionState.MINOR;
break;
case NONE:
default: // nothing, continue
}
} catch (NotServingRegionException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to get compaction state of " + loc.getRegion() + ": " +
StringUtils.stringifyException(e));
}
} catch (RemoteException e) {
if (e.getMessage().indexOf(NotServingRegionException.class.getName()) >= 0) {
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to get compaction state of " + loc.getRegion() + ": " +
StringUtils.stringifyException(e));
}
} else {
throw e;
}
}
} finally {
if (zookeeper != null) {
zookeeper.close();
}
}
break;
default:
throw new IllegalArgumentException("Unknowne compactType: " + compactType);
}
if(state != null) {
if (state != null) {
return ProtobufUtil.createCompactionState(state);
}
return null;
@ -3927,9 +3827,9 @@ public class HBaseAdmin implements Admin {
});
}
private HRegionInfo getMobRegionInfo(TableName tableName) {
return new HRegionInfo(tableName, Bytes.toBytes(".mob"),
HConstants.EMPTY_END_ROW, false, 0);
private RegionInfo getMobRegionInfo(TableName tableName) {
return RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes(".mob")).setRegionId(0)
.build();
}
private RpcControllerFactory getRpcControllerFactory() {

View File

@ -30,14 +30,13 @@ import java.util.concurrent.CopyOnWriteArraySet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.types.CopyOnWriteArrayMap;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A cache implementation for region locations from meta.
@ -88,7 +87,7 @@ public class MetaCache {
// this one. the exception case is when the endkey is
// HConstants.EMPTY_END_ROW, signifying that the region we're
// checking is actually the last region in the table.
byte[] endKey = possibleRegion.getRegionLocation().getRegionInfo().getEndKey();
byte[] endKey = possibleRegion.getRegionLocation().getRegion().getEndKey();
// Here we do direct Bytes.compareTo and not doing CellComparator/MetaCellComparator path.
// MetaCellComparator is for comparing against data in META table which need special handling.
// Not doing that is ok for this case because
@ -117,7 +116,7 @@ public class MetaCache {
public void cacheLocation(final TableName tableName, final ServerName source,
final HRegionLocation location) {
assert source != null;
byte [] startKey = location.getRegionInfo().getStartKey();
byte [] startKey = location.getRegion().getStartKey();
ConcurrentMap<byte[], RegionLocations> tableLocations = getTableLocations(tableName);
RegionLocations locations = new RegionLocations(new HRegionLocation[] {location}) ;
RegionLocations oldLocations = tableLocations.putIfAbsent(startKey, locations);
@ -132,7 +131,7 @@ public class MetaCache {
// If the server in cache sends us a redirect, assume it's always valid.
HRegionLocation oldLocation = oldLocations.getRegionLocation(
location.getRegionInfo().getReplicaId());
location.getRegion().getReplicaId());
boolean force = oldLocation != null && oldLocation.getServerName() != null
&& oldLocation.getServerName().equals(source);
@ -157,7 +156,7 @@ public class MetaCache {
* @param locations the new locations
*/
public void cacheLocation(final TableName tableName, final RegionLocations locations) {
byte [] startKey = locations.getRegionLocation().getRegionInfo().getStartKey();
byte [] startKey = locations.getRegionLocation().getRegion().getStartKey();
ConcurrentMap<byte[], RegionLocations> tableLocations = getTableLocations(tableName);
RegionLocations oldLocation = tableLocations.putIfAbsent(startKey, locations);
boolean isNewCacheEntry = (oldLocation == null);
@ -299,7 +298,7 @@ public class MetaCache {
RegionLocations regionLocations = getCachedLocation(tableName, row);
if (regionLocations != null) {
byte[] startKey = regionLocations.getRegionLocation().getRegionInfo().getStartKey();
byte[] startKey = regionLocations.getRegionLocation().getRegion().getStartKey();
boolean removed = tableLocations.remove(startKey, regionLocations);
if (removed) {
if (metrics != null) {
@ -326,7 +325,7 @@ public class MetaCache {
HRegionLocation toBeRemoved = regionLocations.getRegionLocation(replicaId);
if (toBeRemoved != null) {
RegionLocations updatedLocations = regionLocations.remove(replicaId);
byte[] startKey = regionLocations.getRegionLocation().getRegionInfo().getStartKey();
byte[] startKey = regionLocations.getRegionLocation().getRegion().getStartKey();
boolean removed;
if (updatedLocations.isEmpty()) {
removed = tableLocations.remove(startKey, regionLocations);
@ -356,7 +355,7 @@ public class MetaCache {
if (regionLocations != null) {
RegionLocations updatedLocations = regionLocations.removeByServer(serverName);
if (updatedLocations != regionLocations) {
byte[] startKey = regionLocations.getRegionLocation().getRegionInfo().getStartKey();
byte[] startKey = regionLocations.getRegionLocation().getRegion().getStartKey();
boolean removed = false;
if (updatedLocations.isEmpty()) {
removed = tableLocations.remove(startKey, regionLocations);
@ -380,7 +379,7 @@ public class MetaCache {
* Deletes the cached location of the region if necessary, based on some error from source.
* @param hri The region in question.
*/
public void clearCache(HRegionInfo hri) {
public void clearCache(RegionInfo hri) {
ConcurrentMap<byte[], RegionLocations> tableLocations = getTableLocations(hri.getTable());
RegionLocations regionLocations = tableLocations.get(hri.getStartKey());
if (regionLocations != null) {
@ -410,17 +409,17 @@ public class MetaCache {
if (location == null) {
return;
}
TableName tableName = location.getRegionInfo().getTable();
TableName tableName = location.getRegion().getTable();
ConcurrentMap<byte[], RegionLocations> tableLocations = getTableLocations(tableName);
RegionLocations regionLocations = tableLocations.get(location.getRegionInfo().getStartKey());
RegionLocations regionLocations = tableLocations.get(location.getRegion().getStartKey());
if (regionLocations != null) {
RegionLocations updatedLocations = regionLocations.remove(location);
boolean removed;
if (updatedLocations != regionLocations) {
if (updatedLocations.isEmpty()) {
removed = tableLocations.remove(location.getRegionInfo().getStartKey(), regionLocations);
removed = tableLocations.remove(location.getRegion().getStartKey(), regionLocations);
} else {
removed = tableLocations.replace(location.getRegionInfo().getStartKey(), regionLocations,
removed = tableLocations.replace(location.getRegion().getStartKey(), regionLocations,
updatedLocations);
}
if (removed) {

View File

@ -1,53 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import java.io.IOException;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Cluster registry.
* Implementations hold cluster information such as this cluster's id, location of hbase:meta, etc.
* Internal use only.
*/
@InterfaceAudience.Private
interface Registry {
/**
* @param connection
*/
void init(Connection connection);
/**
* @return Meta region location
* @throws IOException
*/
RegionLocations getMetaRegionLocation() throws IOException;
/**
* @return Cluster id.
*/
String getClusterId();
/**
* @return Count of 'running' regionservers
* @throws IOException
*/
int getCurrentNrHRS() throws IOException;
}

View File

@ -1,50 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import java.io.IOException;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Get instance of configured Registry.
*/
@InterfaceAudience.Private
final class RegistryFactory {
static final String REGISTRY_IMPL_CONF_KEY = "hbase.client.registry.impl";
private RegistryFactory() {}
/**
* @return The cluster registry implementation to use.
* @throws IOException
*/
static Registry getRegistry(final Connection connection)
throws IOException {
String registryClass = connection.getConfiguration().get(REGISTRY_IMPL_CONF_KEY,
ZooKeeperRegistry.class.getName());
Registry registry = null;
try {
registry = (Registry)Class.forName(registryClass).newInstance();
} catch (Throwable t) {
throw new IOException(t);
}
registry.init(connection);
return registry;
}
}

View File

@ -19,12 +19,12 @@ package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT;
import static org.apache.hadoop.hbase.HConstants.ZK_SESSION_TIMEOUT;
import static org.apache.hadoop.hbase.HRegionInfo.DEFAULT_REPLICA_ID;
import static org.apache.hadoop.hbase.HRegionInfo.FIRST_META_REGIONINFO;
import static org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID;
import static org.apache.hadoop.hbase.client.RegionInfoBuilder.FIRST_META_REGIONINFO;
import static org.apache.hadoop.hbase.client.RegionReplicaUtil.getRegionInfoForDefaultReplica;
import static org.apache.hadoop.hbase.client.RegionReplicaUtil.getRegionInfoForReplica;
import static org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.lengthOfPBMagic;
import static org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.removeMetaData;
import static org.apache.hadoop.hbase.zookeeper.ZKMetadata.removeMetaData;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
@ -42,17 +42,18 @@ import org.apache.hadoop.hbase.ClusterId;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.data.Stat;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
/**
* Fetch the registry data from zookeeper.
*/

View File

@ -1,56 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
/**
* We inherit the current ZooKeeperWatcher implementation to change the semantic
* of the close: the new close won't immediately close the connection but
* will have a keep alive. See {@link ConnectionImplementation}.
* This allows to make it available with a consistent interface. The whole
* ZooKeeperWatcher use in ConnectionImplementation will be then changed to remove the
* watcher part.
*
* This class is intended to be used internally by HBase classes; but not by
* final user code. Hence it's package protected.
*/
class ZooKeeperKeepAliveConnection extends ZooKeeperWatcher{
ZooKeeperKeepAliveConnection(
Configuration conf, String descriptor,
ConnectionImplementation conn) throws IOException {
super(conf, descriptor, conn);
}
@Override
public void close() {
if (this.abortable != null) {
((ConnectionImplementation)abortable).releaseZooKeeperWatcher(this);
}
}
void internalClose(){
super.close();
}
}

View File

@ -1,129 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
/**
* A cluster registry that stores to zookeeper.
*/
@InterfaceAudience.Private
class ZooKeeperRegistry implements Registry {
private static final Log LOG = LogFactory.getLog(ZooKeeperRegistry.class);
// Needs an instance of hci to function. Set after construct this instance.
ConnectionImplementation hci;
@Override
public void init(Connection connection) {
if (!(connection instanceof ConnectionImplementation)) {
throw new RuntimeException("This registry depends on ConnectionImplementation");
}
this.hci = (ConnectionImplementation)connection;
}
@Override
public RegionLocations getMetaRegionLocation() throws IOException {
ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher();
try {
if (LOG.isTraceEnabled()) {
LOG.trace("Looking up meta region location in ZK," + " connection=" + this);
}
List<ServerName> servers = new MetaTableLocator().blockUntilAvailable(zkw, hci.rpcTimeout,
hci.getConfiguration());
if (LOG.isTraceEnabled()) {
if (servers == null) {
LOG.trace("Looked up meta region location, connection=" + this +
"; servers = null");
} else {
StringBuilder str = new StringBuilder();
for (ServerName s : servers) {
str.append(s.toString());
str.append(" ");
}
LOG.trace("Looked up meta region location, connection=" + this +
"; servers = " + str.toString());
}
}
if (servers == null) return null;
HRegionLocation[] locs = new HRegionLocation[servers.size()];
int i = 0;
for (ServerName server : servers) {
RegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
RegionInfoBuilder.FIRST_META_REGIONINFO, i);
if (server == null) locs[i++] = null;
else locs[i++] = new HRegionLocation(h, server, 0);
}
return new RegionLocations(locs);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return null;
} finally {
zkw.close();
}
}
private String clusterId = null;
@Override
public String getClusterId() {
if (this.clusterId != null) return this.clusterId;
// No synchronized here, worse case we will retrieve it twice, that's
// not an issue.
ZooKeeperKeepAliveConnection zkw = null;
try {
zkw = hci.getKeepAliveZooKeeperWatcher();
this.clusterId = ZKClusterId.readClusterIdZNode(zkw);
if (this.clusterId == null) {
LOG.info("ClusterId read in ZooKeeper is null");
}
} catch (KeeperException e) {
LOG.warn("Can't retrieve clusterId from ZooKeeper", e);
} catch (IOException e) {
LOG.warn("Can't retrieve clusterId from ZooKeeper", e);
} finally {
if (zkw != null) zkw.close();
}
return this.clusterId;
}
@Override
public int getCurrentNrHRS() throws IOException {
ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher();
try {
// We go to zk rather than to master to get count of regions to avoid
// HTable having a Master dependency. See HBase-2828
return ZKUtil.getNumberOfChildren(zkw, zkw.znodePaths.rsZNode);
} catch (KeeperException ke) {
throw new IOException("Unexpected ZooKeeper exception", ke);
} finally {
zkw.close();
}
}
}

View File

@ -100,7 +100,7 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker {
* @return info port or 0 if timed out or exceptions
*/
public int getBackupMasterInfoPort(final ServerName sn) {
String backupZNode = ZKUtil.joinZNode(watcher.znodePaths.backupMasterAddressesZNode,
String backupZNode = ZNodePaths.joinZNode(watcher.znodePaths.backupMasterAddressesZNode,
sn.toString());
try {
byte[] data = ZKUtil.getData(watcher, backupZNode);

View File

@ -18,20 +18,24 @@
*/
package org.apache.hadoop.hbase.zookeeper;
import static org.apache.hadoop.hbase.zookeeper.ZKMetadata.appendMetaData;
import static org.apache.hadoop.hbase.zookeeper.ZKMetadata.removeMetaData;
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.RetryCounterFactory;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@ -45,8 +49,6 @@ import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.proto.CreateRequest;
import org.apache.zookeeper.proto.SetDataRequest;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
/**
* A zookeeper that can handle 'recoverable' errors.
@ -83,22 +85,8 @@ public class RecoverableZooKeeper {
private Watcher watcher;
private int sessionTimeout;
private String quorumServers;
private final Random salter;
private final ZooKeeperMetricsListener metrics;
// The metadata attached to each piece of data has the
// format:
// <magic> 1-byte constant
// <id length> 4-byte big-endian integer (length of next field)
// <id> identifier corresponding uniquely to this process
// It is prepended to the data supplied by the user.
// the magic number is to be backward compatible
private static final byte MAGIC =(byte) 0XFF;
private static final int MAGIC_SIZE = Bytes.SIZEOF_BYTE;
private static final int ID_LENGTH_OFFSET = MAGIC_SIZE;
private static final int ID_LENGTH_SIZE = Bytes.SIZEOF_INT;
public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime)
throws IOException {
@ -129,7 +117,6 @@ public class RecoverableZooKeeper {
this.quorumServers = quorumServers;
this.metrics = new MetricsZooKeeper();
try {checkZk();} catch (Exception x) {/* ignore */}
salter = new Random();
}
/**
@ -472,7 +459,7 @@ public class RecoverableZooKeeper {
try {
traceScope = Trace.startSpan("RecoverableZookeeper.setData");
RetryCounter retryCounter = retryCounterFactory.create();
byte[] newData = appendMetaData(data);
byte[] newData = appendMetaData(id, data);
boolean isRetry = false;
long startTime;
while (true) {
@ -622,7 +609,7 @@ public class RecoverableZooKeeper {
TraceScope traceScope = null;
try {
traceScope = Trace.startSpan("RecoverableZookeeper.create");
byte[] newData = appendMetaData(data);
byte[] newData = appendMetaData(id, data);
switch (createMode) {
case EPHEMERAL:
case PERSISTENT:
@ -745,14 +732,14 @@ public class RecoverableZooKeeper {
for (Op op : ops) {
if (op.getType() == ZooDefs.OpCode.create) {
CreateRequest create = (CreateRequest)op.toRequestRecord();
preparedOps.add(Op.create(create.getPath(), appendMetaData(create.getData()),
preparedOps.add(Op.create(create.getPath(), appendMetaData(id, create.getData()),
create.getAcl(), create.getFlags()));
} else if (op.getType() == ZooDefs.OpCode.delete) {
// no need to appendMetaData for delete
preparedOps.add(op);
} else if (op.getType() == ZooDefs.OpCode.setData) {
SetDataRequest setData = (SetDataRequest)op.toRequestRecord();
preparedOps.add(Op.setData(setData.getPath(), appendMetaData(setData.getData()),
preparedOps.add(Op.setData(setData.getPath(), appendMetaData(id, setData.getData()),
setData.getVersion()));
} else {
throw new UnsupportedOperationException("Unexpected ZKOp type: " + op.getClass().getName());
@ -822,41 +809,6 @@ public class RecoverableZooKeeper {
return null;
}
public static byte[] removeMetaData(byte[] data) {
if(data == null || data.length == 0) {
return data;
}
// check the magic data; to be backward compatible
byte magic = data[0];
if(magic != MAGIC) {
return data;
}
int idLength = Bytes.toInt(data, ID_LENGTH_OFFSET);
int dataLength = data.length-MAGIC_SIZE-ID_LENGTH_SIZE-idLength;
int dataOffset = MAGIC_SIZE+ID_LENGTH_SIZE+idLength;
byte[] newData = new byte[dataLength];
System.arraycopy(data, dataOffset, newData, 0, dataLength);
return newData;
}
private byte[] appendMetaData(byte[] data) {
if(data == null || data.length == 0){
return data;
}
byte[] salt = Bytes.toBytes(salter.nextLong());
int idLength = id.length + salt.length;
byte[] newData = new byte[MAGIC_SIZE+ID_LENGTH_SIZE+idLength+data.length];
int pos = 0;
pos = Bytes.putByte(newData, pos, MAGIC);
pos = Bytes.putInt(newData, pos, idLength);
pos = Bytes.putBytes(newData, pos, id, 0, id.length);
pos = Bytes.putBytes(newData, pos, salt, 0, salt.length);
pos = Bytes.putBytes(newData, pos, data, 0, data.length);
return newData;
}
public synchronized long getSessionId() {
return zk == null ? -1 : zk.getSessionId();
}

View File

@ -0,0 +1,80 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.zookeeper;
import java.util.concurrent.ThreadLocalRandom;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The metadata append to the start of data on zookeeper.
*/
@InterfaceAudience.Private
public class ZKMetadata {
private ZKMetadata() {
}
// The metadata attached to each piece of data has the format:
// <magic> 1-byte constant
// <id length> 4-byte big-endian integer (length of next field)
// <id> identifier corresponding uniquely to this process
// It is prepended to the data supplied by the user.
// the magic number is to be backward compatible
private static final byte MAGIC = (byte) 0XFF;
private static final int MAGIC_SIZE = Bytes.SIZEOF_BYTE;
private static final int ID_LENGTH_OFFSET = MAGIC_SIZE;
private static final int ID_LENGTH_SIZE = Bytes.SIZEOF_INT;
public static byte[] appendMetaData(byte[] id, byte[] data) {
if (data == null || data.length == 0) {
return data;
}
byte[] salt = Bytes.toBytes(ThreadLocalRandom.current().nextLong());
int idLength = id.length + salt.length;
byte[] newData = new byte[MAGIC_SIZE + ID_LENGTH_SIZE + idLength + data.length];
int pos = 0;
pos = Bytes.putByte(newData, pos, MAGIC);
pos = Bytes.putInt(newData, pos, idLength);
pos = Bytes.putBytes(newData, pos, id, 0, id.length);
pos = Bytes.putBytes(newData, pos, salt, 0, salt.length);
pos = Bytes.putBytes(newData, pos, data, 0, data.length);
return newData;
}
public static byte[] removeMetaData(byte[] data) {
if (data == null || data.length == 0) {
return data;
}
// check the magic data; to be backward compatible
byte magic = data[0];
if (magic != MAGIC) {
return data;
}
int idLength = Bytes.toInt(data, ID_LENGTH_OFFSET);
int dataLength = data.length - MAGIC_SIZE - ID_LENGTH_SIZE - idLength;
int dataOffset = MAGIC_SIZE + ID_LENGTH_SIZE + idLength;
byte[] newData = new byte[dataLength];
System.arraycopy(data, dataOffset, newData, 0, dataLength);
return newData;
}
}

View File

@ -87,8 +87,6 @@ import org.apache.zookeeper.server.ZooKeeperSaslServer;
public class ZKUtil {
private static final Log LOG = LogFactory.getLog(ZKUtil.class);
// TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved.
public static final char ZNODE_PATH_SEPARATOR = '/';
private static int zkDumpConnectionTimeOut;
/**
@ -302,28 +300,13 @@ public class ZKUtil {
//
// Helper methods
//
/**
* Join the prefix znode name with the suffix znode name to generate a proper
* full znode name.
*
* Assumes prefix does not end with slash and suffix does not begin with it.
*
* @param prefix beginning of znode name
* @param suffix ending of znode name
* @return result of properly joining prefix with suffix
*/
public static String joinZNode(String prefix, String suffix) {
return prefix + ZNODE_PATH_SEPARATOR + suffix;
}
/**
* Returns the full path of the immediate parent of the specified node.
* @param node path to get parent of
* @return parent of path, null if passed the root node or an invalid node
*/
public static String getParent(String node) {
int idx = node.lastIndexOf(ZNODE_PATH_SEPARATOR);
int idx = node.lastIndexOf(ZNodePaths.ZNODE_PATH_SEPARATOR);
return idx <= 0 ? null : node.substring(0, idx);
}
@ -477,7 +460,7 @@ public class ZKUtil {
return null;
}
for (String child : children) {
watchAndCheckExists(zkw, joinZNode(znode, child));
watchAndCheckExists(zkw, ZNodePaths.joinZNode(znode, child));
}
return children;
}
@ -744,7 +727,7 @@ public class ZKUtil {
if (nodes != null) {
List<NodeAndData> newNodes = new ArrayList<>();
for (String node : nodes) {
String nodePath = ZKUtil.joinZNode(baseNode, node);
String nodePath = ZNodePaths.joinZNode(baseNode, node);
byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath);
newNodes.add(new NodeAndData(nodePath, data));
}
@ -1774,7 +1757,7 @@ public class ZKUtil {
sb.append("\n").append(replicationZnode).append(": ");
List<String> children = ZKUtil.listChildrenNoWatch(zkw, replicationZnode);
for (String child : children) {
String znode = joinZNode(replicationZnode, child);
String znode = ZNodePaths.joinZNode(replicationZnode, child);
if (znode.equals(zkw.znodePaths.peersZNode)) {
appendPeersZnodes(zkw, znode, sb);
} else if (znode.equals(zkw.znodePaths.queuesZNode)) {
@ -1789,7 +1772,7 @@ public class ZKUtil {
StringBuilder sb) throws KeeperException {
sb.append("\n").append(hfileRefsZnode).append(": ");
for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, hfileRefsZnode)) {
String znodeToProcess = ZKUtil.joinZNode(hfileRefsZnode, peerIdZnode);
String znodeToProcess = ZNodePaths.joinZNode(hfileRefsZnode, peerIdZnode);
sb.append("\n").append(znodeToProcess).append(": ");
List<String> peerHFileRefsZnodes = ZKUtil.listChildrenNoWatch(zkw, znodeToProcess);
int size = peerHFileRefsZnodes.size();
@ -1839,7 +1822,7 @@ public class ZKUtil {
}
}
for (String zNodeChild : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
stack.add(ZKUtil.joinZNode(znodeToProcess, zNodeChild));
stack.add(ZNodePaths.joinZNode(znodeToProcess, zNodeChild));
}
} while (stack.size() > 0);
}
@ -1849,7 +1832,7 @@ public class ZKUtil {
int pblen = ProtobufUtil.lengthOfPBMagic();
sb.append("\n").append(peersZnode).append(": ");
for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, peersZnode)) {
String znodeToProcess = ZKUtil.joinZNode(peersZnode, peerIdZnode);
String znodeToProcess = ZNodePaths.joinZNode(peersZnode, peerIdZnode);
byte[] data;
try {
data = ZKUtil.getData(zkw, znodeToProcess);
@ -1879,7 +1862,7 @@ public class ZKUtil {
int pblen = ProtobufUtil.lengthOfPBMagic();
for (String child : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
if (!child.equals(peerState)) continue;
String peerStateZnode = ZKUtil.joinZNode(znodeToProcess, child);
String peerStateZnode = ZNodePaths.joinZNode(znodeToProcess, child);
sb.append("\n").append(peerStateZnode).append(": ");
byte[] peerStateData;
try {
@ -2042,7 +2025,7 @@ public class ZKUtil {
if (children == null) return;
for (String child : children) {
LOG.debug(prefix + child);
String node = ZKUtil.joinZNode(root.equals("/") ? "" : root, child);
String node = ZNodePaths.joinZNode(root.equals("/") ? "" : root, child);
logZKTree(zkw, node, prefix + "---");
}
}

View File

@ -22,22 +22,24 @@ import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT;
import static org.apache.hadoop.hbase.HConstants.META_REPLICAS_NUM;
import static org.apache.hadoop.hbase.HConstants.SPLIT_LOGDIR_NAME;
import static org.apache.hadoop.hbase.HConstants.ZOOKEEPER_ZNODE_PARENT;
import static org.apache.hadoop.hbase.HRegionInfo.DEFAULT_REPLICA_ID;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableMap;
import static org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID;
import java.util.Optional;
import java.util.stream.IntStream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableMap;
/**
* Class that hold all the paths of znode for HBase.
*/
@InterfaceAudience.Private
public class ZNodePaths {
// TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved.
public static final char ZNODE_PATH_SEPARATOR = '/';
public final static String META_ZNODE_PREFIX = "meta-region-server";
@ -90,43 +92,35 @@ public class ZNodePaths {
baseZNode = conf.get(ZOOKEEPER_ZNODE_PARENT, DEFAULT_ZOOKEEPER_ZNODE_PARENT);
ImmutableMap.Builder<Integer, String> builder = ImmutableMap.builder();
metaZNodePrefix = conf.get("zookeeper.znode.metaserver", META_ZNODE_PREFIX);
String defaultMetaReplicaZNode = ZKUtil.joinZNode(baseZNode, metaZNodePrefix);
String defaultMetaReplicaZNode = ZNodePaths.joinZNode(baseZNode, metaZNodePrefix);
builder.put(DEFAULT_REPLICA_ID, defaultMetaReplicaZNode);
int numMetaReplicas = conf.getInt(META_REPLICAS_NUM, DEFAULT_META_REPLICA_NUM);
IntStream.range(1, numMetaReplicas)
.forEachOrdered(i -> builder.put(i, defaultMetaReplicaZNode + "-" + i));
metaReplicaZNodes = builder.build();
rsZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.rs", "rs"));
drainingZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.draining.rs", "draining"));
masterAddressZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.master", "master"));
backupMasterAddressesZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.backup.masters", "backup-masters"));
clusterStateZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.state", "running"));
tableZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.tableEnableDisable", "table"));
clusterIdZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.clusterId", "hbaseid"));
splitLogZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.splitlog", SPLIT_LOGDIR_NAME));
balancerZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.balancer", "balancer"));
regionNormalizerZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.regionNormalizer", "normalizer"));
switchZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.switch", "switch"));
tableLockZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.tableLock", "table-lock"));
namespaceZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.namespace", "namespace"));
masterMaintZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.masterMaintenance", "master-maintenance"));
replicationZNode =
ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.replication", "replication"));
rsZNode = joinZNode(baseZNode, conf.get("zookeeper.znode.rs", "rs"));
drainingZNode = joinZNode(baseZNode, conf.get("zookeeper.znode.draining.rs", "draining"));
masterAddressZNode = joinZNode(baseZNode, conf.get("zookeeper.znode.master", "master"));
backupMasterAddressesZNode =
joinZNode(baseZNode, conf.get("zookeeper.znode.backup.masters", "backup-masters"));
clusterStateZNode = joinZNode(baseZNode, conf.get("zookeeper.znode.state", "running"));
tableZNode = joinZNode(baseZNode, conf.get("zookeeper.znode.tableEnableDisable", "table"));
clusterIdZNode = joinZNode(baseZNode, conf.get("zookeeper.znode.clusterId", "hbaseid"));
splitLogZNode = joinZNode(baseZNode, conf.get("zookeeper.znode.splitlog", SPLIT_LOGDIR_NAME));
balancerZNode = joinZNode(baseZNode, conf.get("zookeeper.znode.balancer", "balancer"));
regionNormalizerZNode =
joinZNode(baseZNode, conf.get("zookeeper.znode.regionNormalizer", "normalizer"));
switchZNode = joinZNode(baseZNode, conf.get("zookeeper.znode.switch", "switch"));
tableLockZNode = joinZNode(baseZNode, conf.get("zookeeper.znode.tableLock", "table-lock"));
namespaceZNode = joinZNode(baseZNode, conf.get("zookeeper.znode.namespace", "namespace"));
masterMaintZNode =
joinZNode(baseZNode, conf.get("zookeeper.znode.masterMaintenance", "master-maintenance"));
replicationZNode = joinZNode(baseZNode, conf.get("zookeeper.znode.replication", "replication"));
peersZNode =
ZKUtil.joinZNode(replicationZNode, conf.get("zookeeper.znode.replication.peers", "peers"));
queuesZNode =
ZKUtil.joinZNode(replicationZNode, conf.get("zookeeper.znode.replication.rs", "rs"));
hfileRefsZNode =
ZKUtil.joinZNode(replicationZNode,
conf.get("zookeeper.znode.replication.hfile.refs", "hfile-refs"));
joinZNode(replicationZNode, conf.get("zookeeper.znode.replication.peers", "peers"));
queuesZNode = joinZNode(replicationZNode, conf.get("zookeeper.znode.replication.rs", "rs"));
hfileRefsZNode = joinZNode(replicationZNode,
conf.get("zookeeper.znode.replication.hfile.refs", "hfile-refs"));
}
@Override
@ -175,7 +169,7 @@ public class ZNodePaths {
*/
public int getMetaReplicaIdFromZnode(String znode) {
if (znode.equals(metaZNodePrefix)) {
return HRegionInfo.DEFAULT_REPLICA_ID;
return RegionInfo.DEFAULT_REPLICA_ID;
}
return Integer.parseInt(znode.substring(metaZNodePrefix.length() + 1));
}
@ -188,4 +182,18 @@ public class ZNodePaths {
public boolean isDefaultMetaReplicaZnode(String znode) {
return metaReplicaZNodes.get(DEFAULT_REPLICA_ID).equals(znode);
}
/**
* Join the prefix znode name with the suffix znode name to generate a proper
* full znode name.
*
* Assumes prefix does not end with slash and suffix does not begin with it.
*
* @param prefix beginning of znode name
* @param suffix ending of znode name
* @return result of properly joining prefix with suffix
*/
public static String joinZNode(String prefix, String suffix) {
return prefix + ZNodePaths.ZNODE_PATH_SEPARATOR + suffix;
}
}

View File

@ -51,7 +51,7 @@ public class ZkAclReset extends Configured implements Tool {
List<String> children = ZKUtil.listChildrenNoWatch(zkw, znode);
if (children != null) {
for (String child: children) {
resetAcls(zkw, ZKUtil.joinZNode(znode, child), eraseAcls);
resetAcls(zkw, ZNodePaths.joinZNode(znode, child), eraseAcls);
}
}

View File

@ -221,7 +221,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
List<String> children = recoverableZooKeeper.getChildren(znode, false);
for (String child : children) {
setZnodeAclsRecursive(ZKUtil.joinZNode(znode, child));
setZnodeAclsRecursive(ZNodePaths.joinZNode(znode, child));
}
List<ACL> acls = ZKUtil.createACL(this, znode, true);
LOG.info("Setting ACLs for znode:" + znode + " , acl:" + acls);

View File

@ -0,0 +1,64 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import java.util.concurrent.CompletableFuture;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Registry that does nothing. Otherwise, default Registry wants zookeeper up and running.
*/
@InterfaceAudience.Private
class DoNothingAsyncRegistry implements AsyncRegistry {
public DoNothingAsyncRegistry(Configuration conf) {
}
@Override
public CompletableFuture<RegionLocations> getMetaRegionLocation() {
return CompletableFuture.completedFuture(null);
}
@Override
public CompletableFuture<String> getClusterId() {
return CompletableFuture.completedFuture(null);
}
@Override
public CompletableFuture<Integer> getCurrentNrHRS() {
return CompletableFuture.completedFuture(0);
}
@Override
public CompletableFuture<ServerName> getMasterAddress() {
return CompletableFuture.completedFuture(null);
}
@Override
public CompletableFuture<Integer> getMasterInfoPort() {
return CompletableFuture.completedFuture(0);
}
@Override
public void close() {
}
}

View File

@ -37,6 +37,7 @@ import java.util.Random;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
@ -456,23 +457,20 @@ public class TestAsyncProcess {
* Returns our async process.
*/
static class MyConnectionImpl extends ConnectionImplementation {
public static class TestRegistry implements Registry {
@Override
public void init(Connection connection) {}
public static class TestRegistry extends DoNothingAsyncRegistry {
@Override
public RegionLocations getMetaRegionLocation() throws IOException {
return null;
public TestRegistry(Configuration conf) {
super(conf);
}
@Override
public String getClusterId() {
return "testClusterId";
public CompletableFuture<String> getClusterId() {
return CompletableFuture.completedFuture("testClusterId");
}
@Override
public int getCurrentNrHRS() throws IOException {
return 1;
public CompletableFuture<Integer> getCurrentNrHRS() {
return CompletableFuture.completedFuture(1);
}
}
@ -483,7 +481,8 @@ public class TestAsyncProcess {
}
private static Configuration setupConf(Configuration conf) {
conf.setClass(RegistryFactory.REGISTRY_IMPL_CONF_KEY, TestRegistry.class, Registry.class);
conf.setClass(AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY, TestRegistry.class,
AsyncRegistry.class);
return conf;
}

View File

@ -17,14 +17,12 @@
*/
package org.apache.hadoop.hbase.client;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.testclassification.ClientTests;
@ -39,35 +37,6 @@ public class TestBufferedMutator {
@Rule
public TestName name = new TestName();
/**
* Registry that does nothing.
* Otherwise, default Registry wants zookeeper up and running.
*/
public static class DoNothingRegistry implements Registry {
@Override
public void init(Connection connection) {
// TODO Auto-generated method stub
}
@Override
public RegionLocations getMetaRegionLocation() throws IOException {
// TODO Auto-generated method stub
return null;
}
@Override
public String getClusterId() {
// TODO Auto-generated method stub
return null;
}
@Override
public int getCurrentNrHRS() throws IOException {
// TODO Auto-generated method stub
return 0;
}
}
/**
* My BufferedMutator.
* Just to prove that I can insert a BM other than default.
@ -83,7 +52,7 @@ public class TestBufferedMutator {
public void testAlternateBufferedMutatorImpl() throws IOException {
BufferedMutatorParams params = new BufferedMutatorParams(TableName.valueOf(name.getMethodName()));
Configuration conf = HBaseConfiguration.create();
conf.set(RegistryFactory.REGISTRY_IMPL_CONF_KEY, DoNothingRegistry.class.getName());
conf.set(AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY, DoNothingAsyncRegistry.class.getName());
try (Connection connection = ConnectionFactory.createConnection(conf)) {
BufferedMutator bm = connection.getBufferedMutator(params);
// Assert we get default BM if nothing specified.

View File

@ -27,18 +27,20 @@ import java.util.HashMap;
import java.util.Map;
import java.util.Random;
import java.util.SortedMap;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.commons.lang3.NotImplementedException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
@ -49,6 +51,26 @@ import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.RegionTooBusyException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Stopwatch;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
@ -73,26 +95,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrEx
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Stopwatch;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
/**
* Test client behavior w/o setting up a cluster.
@ -117,27 +119,27 @@ public class TestClientNoCluster extends Configured implements Tool {
/**
* Simple cluster registry inserted in place of our usual zookeeper based one.
*/
static class SimpleRegistry implements Registry {
static class SimpleRegistry extends DoNothingAsyncRegistry {
final ServerName META_HOST = META_SERVERNAME;
@Override
public void init(Connection connection) {
public SimpleRegistry(Configuration conf) {
super(conf);
}
@Override
public RegionLocations getMetaRegionLocation() throws IOException {
return new RegionLocations(
new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, META_HOST));
public CompletableFuture<RegionLocations> getMetaRegionLocation() {
return CompletableFuture.completedFuture(new RegionLocations(
new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO, META_HOST)));
}
@Override
public String getClusterId() {
return HConstants.CLUSTER_ID_DEFAULT;
public CompletableFuture<String> getClusterId() {
return CompletableFuture.completedFuture(HConstants.CLUSTER_ID_DEFAULT);
}
@Override
public int getCurrentNrHRS() throws IOException {
return 1;
public CompletableFuture<Integer> getCurrentNrHRS() {
return CompletableFuture.completedFuture(1);
}
}

View File

@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.zookeeper;
import static org.junit.Assert.assertFalse;
@ -33,15 +32,16 @@ public class TestZooKeeperWatcher {
@Test
public void testIsClientReadable() throws IOException {
ZooKeeperWatcher watcher = new ZooKeeperWatcher(HBaseConfiguration.create(),
"testIsClientReadable", null, false);
ZooKeeperWatcher watcher =
new ZooKeeperWatcher(HBaseConfiguration.create(), "testIsClientReadable", null, false);
assertTrue(watcher.isClientReadable(watcher.znodePaths.baseZNode));
assertTrue(watcher.isClientReadable(watcher.znodePaths.getZNodeForReplica(0)));
assertTrue(watcher.isClientReadable(watcher.znodePaths.masterAddressZNode));
assertTrue(watcher.isClientReadable(watcher.znodePaths.clusterIdZNode));
assertTrue(watcher.isClientReadable(watcher.znodePaths.tableZNode));
assertTrue(watcher.isClientReadable(ZKUtil.joinZNode(watcher.znodePaths.tableZNode, "foo")));
assertTrue(
watcher.isClientReadable(ZNodePaths.joinZNode(watcher.znodePaths.tableZNode, "foo")));
assertTrue(watcher.isClientReadable(watcher.znodePaths.rsZNode));
assertFalse(watcher.isClientReadable(watcher.znodePaths.tableLockZNode));
@ -54,5 +54,4 @@ public class TestZooKeeperWatcher {
watcher.close();
}
}

View File

@ -27,6 +27,7 @@ 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.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -63,7 +64,7 @@ public class IntegrationTestMetaReplicas {
Configuration conf = util.getConfiguration();
String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
String primaryMetaZnode = ZKUtil.joinZNode(baseZNode,
String primaryMetaZnode = ZNodePaths.joinZNode(baseZNode,
conf.get("zookeeper.znode.metaserver", "meta-region-server"));
// check that the data in the znode is parseable (this would also mean the znode exists)
byte[] data = ZKUtil.getData(zkw, primaryMetaZnode);

View File

@ -39,6 +39,7 @@ 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.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.ToolRunner;
import org.apache.zookeeper.KeeperException;
@ -165,7 +166,7 @@ public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool {
List<String> children = zk.getChildren(znode, false);
for (String child : children) {
checkZnodePermsRecursive(watcher, zk, ZKUtil.joinZNode(znode, child));
checkZnodePermsRecursive(watcher, zk, ZNodePaths.joinZNode(znode, child));
}
} catch (KeeperException ke) {
// if we are not authenticated for listChildren, it is fine.

View File

@ -44,6 +44,7 @@ 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.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -152,7 +153,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
throw new IllegalArgumentException("Cannot remove peer with id=" + id
+ " because that id does not exist.");
}
ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id));
ZKUtil.deleteNodeRecursively(this.zookeeper, ZNodePaths.joinZNode(this.peersZNode, id));
} catch (KeeperException e) {
throw new ReplicationException("Could not remove peer with id=" + id, e);
}

View File

@ -29,6 +29,7 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
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.apache.zookeeper.data.Stat;
@ -61,8 +62,8 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
@Override
public List<String> getLogsInQueue(String serverName, String queueId) throws KeeperException {
String znode = ZKUtil.joinZNode(this.queuesZNode, serverName);
znode = ZKUtil.joinZNode(znode, queueId);
String znode = ZNodePaths.joinZNode(this.queuesZNode, serverName);
znode = ZNodePaths.joinZNode(znode, queueId);
List<String> result = null;
try {
result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
@ -76,7 +77,7 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
@Override
public List<String> getAllQueues(String serverName) throws KeeperException {
String znode = ZKUtil.joinZNode(this.queuesZNode, serverName);
String znode = ZNodePaths.joinZNode(this.queuesZNode, serverName);
List<String> result = null;
try {
result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
@ -162,7 +163,7 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
@Override
public List<String> getReplicableHFiles(String peerId) throws KeeperException {
String znode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
String znode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
List<String> result = null;
try {
result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);

View File

@ -37,6 +37,7 @@ 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.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -81,7 +82,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
@Override
public void init(String serverName) throws ReplicationException {
this.myQueuesZnode = ZKUtil.joinZNode(this.queuesZNode, serverName);
this.myQueuesZnode = ZNodePaths.joinZNode(this.queuesZNode, serverName);
try {
if (ZKUtil.checkExists(this.zookeeper, this.myQueuesZnode) < 0) {
ZKUtil.createWithParents(this.zookeeper, this.myQueuesZnode);
@ -105,7 +106,8 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
@Override
public void removeQueue(String queueId) {
try {
ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.myQueuesZnode, queueId));
ZKUtil.deleteNodeRecursively(this.zookeeper,
ZNodePaths.joinZNode(this.myQueuesZnode, queueId));
} catch (KeeperException e) {
this.abortable.abort("Failed to delete queue (queueId=" + queueId + ")", e);
}
@ -113,8 +115,8 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
@Override
public void addLog(String queueId, String filename) throws ReplicationException {
String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
znode = ZKUtil.joinZNode(znode, filename);
String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
znode = ZNodePaths.joinZNode(znode, filename);
try {
ZKUtil.createWithParents(this.zookeeper, znode);
} catch (KeeperException e) {
@ -127,8 +129,8 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
@Override
public void removeLog(String queueId, String filename) {
try {
String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
znode = ZKUtil.joinZNode(znode, filename);
String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
znode = ZNodePaths.joinZNode(znode, filename);
ZKUtil.deleteNode(this.zookeeper, znode);
} catch (KeeperException e) {
this.abortable.abort("Failed to remove wal from queue (queueId=" + queueId + ", filename="
@ -139,8 +141,8 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
@Override
public void setLogPosition(String queueId, String filename, long position) {
try {
String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
znode = ZKUtil.joinZNode(znode, filename);
String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
znode = ZNodePaths.joinZNode(znode, filename);
// Why serialize String of Long and not Long as bytes?
ZKUtil.setData(this.zookeeper, znode, ZKUtil.positionToByteArray(position));
} catch (KeeperException e) {
@ -151,8 +153,8 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
@Override
public long getLogPosition(String queueId, String filename) throws ReplicationException {
String clusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
String znode = ZKUtil.joinZNode(clusterZnode, filename);
String clusterZnode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
String znode = ZNodePaths.joinZNode(clusterZnode, filename);
byte[] bytes = null;
try {
bytes = ZKUtil.getData(this.zookeeper, znode);
@ -176,7 +178,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
@Override
public boolean isThisOurRegionServer(String regionserver) {
return ZKUtil.joinZNode(this.queuesZNode, regionserver).equals(this.myQueuesZnode);
return ZNodePaths.joinZNode(this.queuesZNode, regionserver).equals(this.myQueuesZnode);
}
@Override
@ -184,7 +186,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
if (isThisOurRegionServer(regionserver)) {
return null;
}
String rsZnodePath = ZKUtil.joinZNode(this.queuesZNode, regionserver);
String rsZnodePath = ZNodePaths.joinZNode(this.queuesZNode, regionserver);
List<String> queues = null;
try {
queues = ZKUtil.listChildrenNoWatch(this.zookeeper, rsZnodePath);
@ -202,7 +204,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
@Override
public void removeReplicatorIfQueueIsEmpty(String regionserver) {
String rsPath = ZKUtil.joinZNode(this.queuesZNode, regionserver);
String rsPath = ZNodePaths.joinZNode(this.queuesZNode, regionserver);
try {
List<String> list = ZKUtil.listChildrenNoWatch(this.zookeeper, rsPath);
if (list != null && list.isEmpty()){
@ -229,7 +231,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
@Override
public List<String> getLogsInQueue(String queueId) {
String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
List<String> result = null;
try {
result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
@ -260,21 +262,21 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
private Pair<String, SortedSet<String>> moveQueueUsingMulti(String znode, String peerId) {
try {
// hbase/replication/rs/deadrs
String deadRSZnodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
String deadRSZnodePath = ZNodePaths.joinZNode(this.queuesZNode, znode);
List<ZKUtilOp> listOfOps = new ArrayList<>();
ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
String newPeerId = peerId + "-" + znode;
String newPeerZnode = ZKUtil.joinZNode(this.myQueuesZnode, newPeerId);
String newPeerZnode = ZNodePaths.joinZNode(this.myQueuesZnode, newPeerId);
// check the logs queue for the old peer cluster
String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId);
String oldClusterZnode = ZNodePaths.joinZNode(deadRSZnodePath, peerId);
List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
if (!peerExists(replicationQueueInfo.getPeerId())) {
LOG.warn("Peer " + replicationQueueInfo.getPeerId() +
" didn't exist, will move its queue to avoid the failure of multi op");
for (String wal : wals) {
String oldWalZnode = ZKUtil.joinZNode(oldClusterZnode, wal);
String oldWalZnode = ZNodePaths.joinZNode(oldClusterZnode, wal);
listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode));
}
listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
@ -291,10 +293,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
listOfOps.add(op);
// get the offset of the logs and set it to new znodes
for (String wal : wals) {
String oldWalZnode = ZKUtil.joinZNode(oldClusterZnode, wal);
String oldWalZnode = ZNodePaths.joinZNode(oldClusterZnode, wal);
byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalZnode);
LOG.debug("Creating " + wal + " with data " + Bytes.toString(logOffset));
String newLogZnode = ZKUtil.joinZNode(newPeerZnode, wal);
String newLogZnode = ZNodePaths.joinZNode(newPeerZnode, wal);
listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset));
listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode));
logQueue.add(wal);
@ -322,7 +324,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
@Override
public void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs)
throws ReplicationException {
String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
boolean debugEnabled = LOG.isDebugEnabled();
if (debugEnabled) {
LOG.debug("Adding hfile references " + pairs + " in queue " + peerZnode);
@ -333,7 +335,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
for (int i = 0; i < size; i++) {
listOfOps.add(ZKUtilOp.createAndFailSilent(
ZKUtil.joinZNode(peerZnode, pairs.get(i).getSecond().getName()),
ZNodePaths.joinZNode(peerZnode, pairs.get(i).getSecond().getName()),
HConstants.EMPTY_BYTE_ARRAY));
}
if (debugEnabled) {
@ -349,7 +351,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
@Override
public void removeHFileRefs(String peerId, List<String> files) {
String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
boolean debugEnabled = LOG.isDebugEnabled();
if (debugEnabled) {
LOG.debug("Removing hfile references " + files + " from queue " + peerZnode);
@ -359,7 +361,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
List<ZKUtilOp> listOfOps = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
listOfOps.add(ZKUtilOp.deleteNodeFailSilent(ZKUtil.joinZNode(peerZnode, files.get(i))));
listOfOps.add(ZKUtilOp.deleteNodeFailSilent(ZNodePaths.joinZNode(peerZnode, files.get(i))));
}
if (debugEnabled) {
LOG.debug(" The multi list size for removing hfile references in zk for node " + peerZnode
@ -374,7 +376,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
@Override
public void addPeerToHFileRefs(String peerId) throws ReplicationException {
String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
try {
if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) {
LOG.info("Adding peer " + peerId + " to hfile reference queue.");
@ -388,7 +390,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
@Override
public void removePeerFromHFileRefs(String peerId) {
final String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
final String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
try {
if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) {
if (LOG.isDebugEnabled()) {

View File

@ -32,6 +32,7 @@ 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.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -87,11 +88,11 @@ public abstract class ReplicationStateZKBase {
this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs");
this.ourClusterKey = ZKConfig.getZooKeeperClusterKey(this.conf);
this.replicationZNode = ZKUtil.joinZNode(this.zookeeper.znodePaths.baseZNode,
this.replicationZNode = ZNodePaths.joinZNode(this.zookeeper.znodePaths.baseZNode,
replicationZNodeName);
this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
this.queuesZNode = ZKUtil.joinZNode(replicationZNode, queuesZNodeName);
this.hfileRefsZNode = ZKUtil.joinZNode(replicationZNode, hfileRefsZNodeName);
this.peersZNode = ZNodePaths.joinZNode(replicationZNode, peersZNodeName);
this.queuesZNode = ZNodePaths.joinZNode(replicationZNode, queuesZNodeName);
this.hfileRefsZNode = ZNodePaths.joinZNode(replicationZNode, hfileRefsZNodeName);
}
public List<String> getListOfReplicators() {
@ -127,7 +128,7 @@ public abstract class ReplicationStateZKBase {
}
protected boolean peerExists(String id) throws KeeperException {
return ZKUtil.checkExists(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id)) >= 0;
return ZKUtil.checkExists(this.zookeeper, ZNodePaths.joinZNode(this.peersZNode, id)) >= 0;
}
/**
@ -141,15 +142,15 @@ public abstract class ReplicationStateZKBase {
@VisibleForTesting
protected String getTableCFsNode(String id) {
return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.tableCFsNodeName));
return ZNodePaths.joinZNode(this.peersZNode, ZNodePaths.joinZNode(id, this.tableCFsNodeName));
}
@VisibleForTesting
protected String getPeerStateNode(String id) {
return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.peerStateNodeName));
return ZNodePaths.joinZNode(this.peersZNode, ZNodePaths.joinZNode(id, this.peerStateNodeName));
}
@VisibleForTesting
protected String getPeerNode(String id) {
return ZKUtil.joinZNode(this.peersZNode, id);
return ZNodePaths.joinZNode(this.peersZNode, id);
}
}

View File

@ -77,6 +77,7 @@ 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.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
@ -319,13 +320,13 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager {
}
List<RSGroupInfo> retrieveGroupListFromZookeeper() throws IOException {
String groupBasePath = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, rsGroupZNode);
String groupBasePath = ZNodePaths.joinZNode(watcher.znodePaths.baseZNode, rsGroupZNode);
List<RSGroupInfo> RSGroupInfoList = Lists.newArrayList();
//Overwrite any info stored by table, this takes precedence
try {
if(ZKUtil.checkExists(watcher, groupBasePath) != -1) {
for(String znode: ZKUtil.listChildrenAndWatchForNewChildren(watcher, groupBasePath)) {
byte[] data = ZKUtil.getData(watcher, ZKUtil.joinZNode(groupBasePath, znode));
byte[] data = ZKUtil.getData(watcher, ZNodePaths.joinZNode(groupBasePath, znode));
if(data.length > 0) {
ProtobufUtil.expectPBMagicPrefix(data);
ByteArrayInputStream bis = new ByteArrayInputStream(
@ -469,20 +470,20 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager {
resetRSGroupAndTableMaps(newGroupMap, newTableMap);
try {
String groupBasePath = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, rsGroupZNode);
String groupBasePath = ZNodePaths.joinZNode(watcher.znodePaths.baseZNode, rsGroupZNode);
ZKUtil.createAndFailSilent(watcher, groupBasePath, ProtobufMagic.PB_MAGIC);
List<ZKUtil.ZKUtilOp> zkOps = new ArrayList<>(newGroupMap.size());
for(String groupName : prevRSGroups) {
if(!newGroupMap.containsKey(groupName)) {
String znode = ZKUtil.joinZNode(groupBasePath, groupName);
String znode = ZNodePaths.joinZNode(groupBasePath, groupName);
zkOps.add(ZKUtil.ZKUtilOp.deleteNodeFailSilent(znode));
}
}
for (RSGroupInfo RSGroupInfo : newGroupMap.values()) {
String znode = ZKUtil.joinZNode(groupBasePath, RSGroupInfo.getName());
String znode = ZNodePaths.joinZNode(groupBasePath, RSGroupInfo.getName());
RSGroupProtos.RSGroupInfo proto = RSGroupProtobufUtil.toProtoGroupInfo(RSGroupInfo);
LOG.debug("Updating znode: "+znode);
ZKUtil.createAndFailSilent(watcher, znode);

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.net.Address;
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;
@ -124,9 +125,9 @@ public class VerifyingRSGroupAdminClient implements RSGroupAdmin {
Assert.assertEquals(Sets.newHashSet(groupMap.values()),
Sets.newHashSet(wrapped.listRSGroups()));
try {
String groupBasePath = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "rsgroup");
String groupBasePath = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "rsgroup");
for(String znode: ZKUtil.listChildrenNoWatch(zkw, groupBasePath)) {
byte[] data = ZKUtil.getData(zkw, ZKUtil.joinZNode(groupBasePath, znode));
byte[] data = ZKUtil.getData(zkw, ZNodePaths.joinZNode(groupBasePath, znode));
if(data.length > 0) {
ProtobufUtil.expectPBMagicPrefix(data);
ByteArrayInputStream bis = new ByteArrayInputStream(

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
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.zookeeper.KeeperException;
@ -165,7 +166,7 @@ public class ZKNamespaceManager extends ZooKeeperListener {
}
private void deleteNamespace(String name) throws IOException {
String zNode = ZKUtil.joinZNode(nsZNode, name);
String zNode = ZNodePaths.joinZNode(nsZNode, name);
try {
ZKUtil.deleteNode(watcher, zNode);
} catch (KeeperException e) {
@ -180,7 +181,7 @@ public class ZKNamespaceManager extends ZooKeeperListener {
}
private void writeNamespace(NamespaceDescriptor ns) throws IOException {
String zNode = ZKUtil.joinZNode(nsZNode, ns.getName());
String zNode = ZNodePaths.joinZNode(nsZNode, ns.getName());
try {
ZKUtil.createWithParents(watcher, zNode);
ZKUtil.updateExistingNodeData(watcher, zNode,

View File

@ -32,6 +32,7 @@ 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.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -181,11 +182,12 @@ public class ZNodeClearer {
String znodeFileContent;
try {
znodeFileContent = ZNodeClearer.readMyEphemeralNodeOnDisk();
if(ZNodeClearer.tablesOnMaster(conf)) {
//In case of master crash also remove rsZnode since master is also regionserver
ZKUtil.deleteNodeFailSilent(zkw, ZKUtil.joinZNode(zkw.znodePaths.rsZNode,znodeFileContent));
return MasterAddressTracker.deleteIfEquals(zkw,
ZNodeClearer.parseMasterServerName(znodeFileContent));
if (ZNodeClearer.tablesOnMaster(conf)) {
// In case of master crash also remove rsZnode since master is also regionserver
ZKUtil.deleteNodeFailSilent(zkw,
ZNodePaths.joinZNode(zkw.znodePaths.rsZNode, znodeFileContent));
return MasterAddressTracker.deleteIfEquals(zkw,
ZNodeClearer.parseMasterServerName(znodeFileContent));
} else {
return MasterAddressTracker.deleteIfEquals(zkw, znodeFileContent);
}

View File

@ -27,6 +27,7 @@ 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;
@ -164,6 +165,6 @@ class HFileArchiveManager {
* @return znode for the table's archive status
*/
private String getTableNode(byte[] table) {
return ZKUtil.joinZNode(archiveZnode, Bytes.toString(table));
return ZNodePaths.joinZNode(archiveZnode, Bytes.toString(table));
}
}

View File

@ -24,7 +24,7 @@ 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.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -149,7 +149,7 @@ public class ZKTableArchiveClient extends Configured {
* @return get the znode for long-term archival of a table for
*/
public static String getArchiveZNode(Configuration conf, ZooKeeperWatcher zooKeeper) {
return ZKUtil.joinZNode(zooKeeper.znodePaths.baseZNode, conf.get(
return ZNodePaths.joinZNode(zooKeeper.znodePaths.baseZNode, conf.get(
ZOOKEEPER_ZNODE_HFILE_ARCHIVE_KEY, TableHFileArchiveTracker.HFILE_ARCHIVE_ZNODE_PARENT));
}
}

View File

@ -41,12 +41,12 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective;
import org.apache.hadoop.hbase.master.SplitLogManager.Task;
import org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
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.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.StringUtils;
@ -58,6 +58,8 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.Stat;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* ZooKeeper based implementation of
* {@link SplitLogManagerCoordination}
@ -355,7 +357,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
setDone(path, FAILURE);
return;
}
data = RecoverableZooKeeper.removeMetaData(data);
data = ZKMetadata.removeMetaData(data);
SplitLogTask slt = SplitLogTask.parseFrom(data);
if (slt.isUnassigned()) {
LOG.debug("task not yet acquired " + path + " ver = " + version);
@ -478,7 +480,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
int listSize = orphans.size();
for (int i = 0; i < listSize; i++) {
String path = orphans.get(i);
String nodepath = ZKUtil.joinZNode(watcher.znodePaths.splitLogZNode, path);
String nodepath = ZNodePaths.joinZNode(watcher.znodePaths.splitLogZNode, path);
if (ZKSplitLog.isRescanNode(watcher, nodepath)) {
rescan_nodes++;
LOG.debug("found orphan rescan node " + path);

View File

@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SplitLogCounters;
import org.apache.hadoop.hbase.SplitLogTask;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
@ -43,12 +42,14 @@ 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.RecoverableZooKeeper;
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.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;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
@ -419,7 +420,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
// don't call ZKSplitLog.getNodeName() because that will lead to
// double encoding of the path name
if (this.calculateAvailableSplitters(numTasks) > 0) {
grabTask(ZKUtil.joinZNode(watcher.znodePaths.splitLogZNode, paths.get(idx)));
grabTask(ZNodePaths.joinZNode(watcher.znodePaths.splitLogZNode, paths.get(idx)));
} else {
LOG.debug("Current region server " + server.getServerName() + " has "
+ this.tasksInProgress.get() + " tasks in progress and can't take more.");
@ -524,7 +525,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
getDataSetWatchFailure(path);
return;
}
data = RecoverableZooKeeper.removeMetaData(data);
data = ZKMetadata.removeMetaData(data);
getDataSetWatchSuccess(path, data);
}
}

View File

@ -32,6 +32,7 @@ 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.zookeeper.KeeperException;
@ -155,7 +156,7 @@ public class ActiveMasterManager extends ZooKeeperListener {
*/
boolean blockUntilBecomingActiveMaster(
int checkInterval, MonitoredTask startupStatus) {
String backupZNode = ZKUtil.joinZNode(
String backupZNode = ZNodePaths.joinZNode(
this.watcher.znodePaths.backupMasterAddressesZNode, this.sn.toString());
while (!(master.isAborted() || master.isStopped())) {
startupStatus.setStatus("Trying to register in ZK as active master");

View File

@ -183,6 +183,7 @@ 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.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
@ -1980,7 +1981,7 @@ public class HMaster extends HRegionServer implements MasterServices {
}
private void startActiveMasterManager(int infoPort) throws KeeperException {
String backupZNode = ZKUtil.joinZNode(
String backupZNode = ZNodePaths.joinZNode(
zooKeeper.znodePaths.backupMasterAddressesZNode, serverName.toString());
/*
* Add a ZNode for ourselves in the backup master directory since we
@ -2511,7 +2512,7 @@ public class HMaster extends HRegionServer implements MasterServices {
try {
byte [] bytes;
try {
bytes = ZKUtil.getData(this.zooKeeper, ZKUtil.joinZNode(
bytes = ZKUtil.getData(this.zooKeeper, ZNodePaths.joinZNode(
this.zooKeeper.znodePaths.backupMasterAddressesZNode, s));
} catch (InterruptedException e) {
throw new InterruptedIOException();
@ -3429,7 +3430,7 @@ public class HMaster extends HRegionServer implements MasterServices {
String parentZnode = getZooKeeper().znodePaths.drainingZNode;
for (ServerName server : servers) {
try {
String node = ZKUtil.joinZNode(parentZnode, server.getServerName());
String node = ZNodePaths.joinZNode(parentZnode, server.getServerName());
ZKUtil.createAndFailSilent(getZooKeeper(), node);
} catch (KeeperException ke) {
throw new HBaseIOException(
@ -3476,7 +3477,7 @@ public class HMaster extends HRegionServer implements MasterServices {
final List<byte[]> encodedRegionNames) throws HBaseIOException {
// Remove the server from decommissioned (draining) server list.
String parentZnode = getZooKeeper().znodePaths.drainingZNode;
String node = ZKUtil.joinZNode(parentZnode, server.getServerName());
String node = ZNodePaths.joinZNode(parentZnode, server.getServerName());
try {
ZKUtil.deleteNodeFailSilent(getZooKeeper(), node);
} catch (KeeperException ke) {

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
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;
@ -139,8 +140,8 @@ public class ReplicationZKNodeCleaner {
* @throws IOException
*/
public void removeQueue(final String replicator, final String queueId) throws IOException {
String queueZnodePath = ZKUtil.joinZNode(ZKUtil.joinZNode(this.queuesZNode, replicator),
queueId);
String queueZnodePath =
ZNodePaths.joinZNode(ZNodePaths.joinZNode(this.queuesZNode, replicator), queueId);
try {
ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
if (!replicationPeers.getAllPeerIds().contains(queueInfo.getPeerId())) {
@ -159,7 +160,7 @@ public class ReplicationZKNodeCleaner {
* @throws IOException
*/
public void removeHFileRefsQueue(final String hfileRefsQueueId) throws IOException {
String node = ZKUtil.joinZNode(this.hfileRefsZNode, hfileRefsQueueId);
String node = ZNodePaths.joinZNode(this.hfileRefsZNode, hfileRefsQueueId);
try {
if (!replicationPeers.getAllPeerIds().contains(hfileRefsQueueId)) {
ZKUtil.deleteNodeRecursively(this.zookeeper, node);

View File

@ -28,6 +28,7 @@ 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;
@ -96,7 +97,7 @@ public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs {
ZKUtil.createWithParents(zkProc.getWatcher(), acquire, data);
// loop through all the children of the acquire phase and watch for them
for (String node : nodeNames) {
String znode = ZKUtil.joinZNode(acquire, node);
String znode = ZNodePaths.joinZNode(acquire, node);
LOG.debug("Watching for acquire node:" + znode);
if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), znode)) {
coordinator.memberAcquiredBarrier(procName, node);
@ -119,7 +120,7 @@ public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs {
ZKUtil.createWithParents(zkProc.getWatcher(), reachedNode);
// loop through all the children of the acquire phase and watch for them
for (String node : nodeNames) {
String znode = ZKUtil.joinZNode(reachedNode, node);
String znode = ZNodePaths.joinZNode(reachedNode, node);
if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), znode)) {
byte[] dataFromMember = ZKUtil.getData(zkProc.getWatcher(), znode);
// ProtobufUtil.isPBMagicPrefix will check null

View File

@ -28,6 +28,7 @@ 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;
@ -136,7 +137,7 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs {
// this is the list of the currently aborted procedues
for (String node : ZKUtil.listChildrenAndWatchForNewChildren(zkController.getWatcher(),
zkController.getAbortZnode())) {
String abortNode = ZKUtil.joinZNode(zkController.getAbortZnode(), node);
String abortNode = ZNodePaths.joinZNode(zkController.getAbortZnode(), node);
abort(abortNode);
}
} catch (KeeperException e) {
@ -166,7 +167,7 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs {
}
for (String procName : runningProcedures) {
// then read in the procedure information
String path = ZKUtil.joinZNode(zkController.getAcquiredBarrier(), procName);
String path = ZNodePaths.joinZNode(zkController.getAcquiredBarrier(), procName);
startNewSubprocedure(path);
}
}
@ -238,7 +239,7 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs {
try {
LOG.debug("Member: '" + memberName + "' joining acquired barrier for procedure (" + procName
+ ") in zk");
String acquiredZNode = ZKUtil.joinZNode(ZKProcedureUtil.getAcquireBarrierNode(
String acquiredZNode = ZNodePaths.joinZNode(ZKProcedureUtil.getAcquireBarrierNode(
zkController, procName), memberName);
ZKUtil.createAndFailSilent(zkController.getWatcher(), acquiredZNode);
@ -262,7 +263,8 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs {
String procName = sub.getName();
LOG.debug("Marking procedure '" + procName + "' completed for member '" + memberName
+ "' in zk");
String joinPath = ZKUtil.joinZNode(zkController.getReachedBarrierNode(procName), memberName);
String joinPath =
ZNodePaths.joinZNode(zkController.getReachedBarrierNode(procName), memberName);
// ProtobufUtil.prependPBMagic does not take care of null
if (data == null) {
data = new byte[0];

View File

@ -25,6 +25,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
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;
@ -78,10 +79,10 @@ public abstract class ZKProcedureUtil
// make sure we are listening for events
watcher.registerListener(this);
// setup paths for the zknodes used in procedures
this.baseZNode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, procDescription);
acquiredZnode = ZKUtil.joinZNode(baseZNode, ACQUIRED_BARRIER_ZNODE_DEFAULT);
reachedZnode = ZKUtil.joinZNode(baseZNode, REACHED_BARRIER_ZNODE_DEFAULT);
abortZnode = ZKUtil.joinZNode(baseZNode, ABORT_ZNODE_DEFAULT);
this.baseZNode = ZNodePaths.joinZNode(watcher.znodePaths.baseZNode, procDescription);
acquiredZnode = ZNodePaths.joinZNode(baseZNode, ACQUIRED_BARRIER_ZNODE_DEFAULT);
reachedZnode = ZNodePaths.joinZNode(baseZNode, REACHED_BARRIER_ZNODE_DEFAULT);
abortZnode = ZNodePaths.joinZNode(baseZNode, ABORT_ZNODE_DEFAULT);
// first make sure all the ZK nodes exist
// make sure all the parents exist (sometimes not the case in tests)
@ -130,7 +131,7 @@ public abstract class ZKProcedureUtil
*/
public static String getAcquireBarrierNode(ZKProcedureUtil controller,
String opInstanceName) {
return ZKUtil.joinZNode(controller.acquiredZnode, opInstanceName);
return ZNodePaths.joinZNode(controller.acquiredZnode, opInstanceName);
}
/**
@ -142,7 +143,7 @@ public abstract class ZKProcedureUtil
*/
public static String getReachedBarrierNode(ZKProcedureUtil controller,
String opInstanceName) {
return ZKUtil.joinZNode(controller.reachedZnode, opInstanceName);
return ZNodePaths.joinZNode(controller.reachedZnode, opInstanceName);
}
/**
@ -153,7 +154,7 @@ public abstract class ZKProcedureUtil
* @return full znode path to the abort znode
*/
public static String getAbortNode(ZKProcedureUtil controller, String opInstanceName) {
return ZKUtil.joinZNode(controller.abortZnode, opInstanceName);
return ZNodePaths.joinZNode(controller.abortZnode, opInstanceName);
}
public ZooKeeperWatcher getWatcher() {
@ -212,7 +213,7 @@ public abstract class ZKProcedureUtil
private boolean isMemberNode(final String path, final String statePath) {
int count = 0;
for (int i = statePath.length(); i < path.length(); ++i) {
count += (path.charAt(i) == ZKUtil.ZNODE_PATH_SEPARATOR) ? 1 : 0;
count += (path.charAt(i) == ZNodePaths.ZNODE_PATH_SEPARATOR) ? 1 : 0;
}
return count == 2;
}
@ -261,7 +262,7 @@ public abstract class ZKProcedureUtil
if (children == null) return;
for (String child : children) {
LOG.debug(prefix + child);
String node = ZKUtil.joinZNode(root.equals("/") ? "" : root, child);
String node = ZNodePaths.joinZNode(root.equals("/") ? "" : root, child);
logZKTree(node, prefix + "---");
}
}

View File

@ -162,6 +162,7 @@ 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.ZKUtil;
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;
@ -3523,7 +3524,7 @@ public class HRegionServer extends HasThread implements
}
private String getMyEphemeralNodePath() {
return ZKUtil.joinZNode(this.zooKeeper.znodePaths.rsZNode, getServerName().toString());
return ZNodePaths.joinZNode(this.zooKeeper.znodePaths.rsZNode, getServerName().toString());
}
private boolean isHealthCheckerConfigured() {

View File

@ -26,6 +26,7 @@ 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.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;
@ -67,7 +68,7 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable
super(watcher);
this.authManager = authManager;
String aclZnodeParent = conf.get("zookeeper.znode.acl.parent", ACL_NODE);
this.aclZNode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, aclZnodeParent);
this.aclZNode = ZNodePaths.joinZNode(watcher.znodePaths.baseZNode, aclZnodeParent);
executor = Executors.newSingleThreadExecutor(
new DaemonThreadFactory("zk-permission-watcher"));
}
@ -260,8 +261,8 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable
*/
public void writeToZookeeper(byte[] entry, byte[] permsData) {
String entryName = Bytes.toString(entry);
String zkNode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, ACL_NODE);
zkNode = ZKUtil.joinZNode(zkNode, entryName);
String zkNode = ZNodePaths.joinZNode(watcher.znodePaths.baseZNode, ACL_NODE);
zkNode = ZNodePaths.joinZNode(zkNode, entryName);
try {
ZKUtil.createWithParents(watcher, zkNode);
@ -278,8 +279,8 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable
* @param tableName
*/
public void deleteTableACLNode(final TableName tableName) {
String zkNode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, ACL_NODE);
zkNode = ZKUtil.joinZNode(zkNode, tableName.getNameAsString());
String zkNode = ZNodePaths.joinZNode(watcher.znodePaths.baseZNode, ACL_NODE);
zkNode = ZNodePaths.joinZNode(zkNode, tableName.getNameAsString());
try {
ZKUtil.deleteNode(watcher, zkNode);
@ -295,8 +296,8 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable
* Delete the acl notify node of namespace
*/
public void deleteNamespaceACLNode(final String namespace) {
String zkNode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, ACL_NODE);
zkNode = ZKUtil.joinZNode(zkNode, AccessControlLists.NAMESPACE_PREFIX + namespace);
String zkNode = ZNodePaths.joinZNode(watcher.znodePaths.baseZNode, ACL_NODE);
zkNode = ZNodePaths.joinZNode(zkNode, AccessControlLists.NAMESPACE_PREFIX + namespace);
try {
ZKUtil.deleteNode(watcher, zkNode);

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.util.Bytes;
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.ZKUtil;
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;
@ -301,7 +301,7 @@ public class AuthenticationTokenSecretManager
setDaemon(true);
setName("ZKSecretWatcher-leaderElector");
zkLeader = new ZKLeaderManager(watcher,
ZKUtil.joinZNode(zkWatcher.getRootKeyZNode(), "keymaster"),
ZNodePaths.joinZNode(zkWatcher.getRootKeyZNode(), "keymaster"),
Bytes.toBytes(serverName), this);
}

View File

@ -29,6 +29,7 @@ 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;
@ -52,8 +53,8 @@ public class ZKSecretWatcher extends ZooKeeperListener {
super(watcher);
this.secretManager = secretManager;
String keyZNodeParent = conf.get("zookeeper.znode.tokenauth.parent", DEFAULT_ROOT_NODE);
this.baseKeyZNode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, keyZNodeParent);
this.keysParentZNode = ZKUtil.joinZNode(baseKeyZNode, DEFAULT_KEYS_PARENT);
this.baseKeyZNode = ZNodePaths.joinZNode(watcher.znodePaths.baseZNode, keyZNodeParent);
this.keysParentZNode = ZNodePaths.joinZNode(baseKeyZNode, DEFAULT_KEYS_PARENT);
}
public void start() throws KeeperException {
@ -159,7 +160,7 @@ public class ZKSecretWatcher extends ZooKeeperListener {
}
private String getKeyNode(int keyId) {
return ZKUtil.joinZNode(keysParentZNode, Integer.toString(keyId));
return ZNodePaths.joinZNode(keysParentZNode, Integer.toString(keyId));
}
public void removeKeyFromZK(AuthenticationKey key) {

View File

@ -24,6 +24,7 @@ import org.apache.commons.logging.LogFactory;
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;
@ -53,8 +54,8 @@ public class ZKVisibilityLabelWatcher extends ZooKeeperListener {
String labelZnodeParent = conf.get(VISIBILITY_LABEL_ZK_PATH, DEFAULT_VISIBILITY_LABEL_NODE);
String userAuthsZnodeParent = conf.get(VISIBILITY_USER_AUTHS_ZK_PATH,
DEFAULT_VISIBILITY_USER_AUTHS_NODE);
this.labelZnode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, labelZnodeParent);
this.userAuthsZnode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, userAuthsZnodeParent);
this.labelZnode = ZNodePaths.joinZNode(watcher.znodePaths.baseZNode, labelZnodeParent);
this.userAuthsZnode = ZNodePaths.joinZNode(watcher.znodePaths.baseZNode, userAuthsZnodeParent);
}
public void start() throws KeeperException {

View File

@ -130,6 +130,7 @@ 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.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.ipc.RemoteException;
@ -703,7 +704,7 @@ public class HBaseFsck extends Configured implements Closeable {
*/
private boolean setMasterInMaintenanceMode() throws IOException {
RetryCounter retryCounter = createZNodeRetryCounterFactory.create();
hbckEphemeralNodePath = ZKUtil.joinZNode(
hbckEphemeralNodePath = ZNodePaths.joinZNode(
zkw.znodePaths.masterMaintZNode,
"hbck-" + Long.toString(EnvironmentEdgeManager.currentTime()));
do {

View File

@ -31,6 +31,7 @@ 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.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -94,7 +95,7 @@ public class ZKDataMigrator {
private static ZooKeeperProtos.DeprecatedTableState.State getTableState(
final ZooKeeperWatcher zkw, final TableName tableName)
throws KeeperException, InterruptedException {
String znode = ZKUtil.joinZNode(zkw.znodePaths.tableZNode, tableName.getNameAsString());
String znode = ZNodePaths.joinZNode(zkw.znodePaths.tableZNode, tableName.getNameAsString());
byte [] data = ZKUtil.getData(zkw, znode);
if (data == null || data.length <= 0) return null;
try {

View File

@ -82,7 +82,7 @@ public class RegionServerTracker extends ZooKeeperListener {
if (regionServers.get(sn) == null) {
RegionServerInfo.Builder rsInfoBuilder = RegionServerInfo.newBuilder();
try {
String nodePath = ZKUtil.joinZNode(watcher.znodePaths.rsZNode, n);
String nodePath = ZNodePaths.joinZNode(watcher.znodePaths.rsZNode, n);
byte[] data = ZKUtil.getData(watcher, nodePath);
if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) {
int magicLen = ProtobufUtil.lengthOfPBMagic();

View File

@ -52,9 +52,9 @@ public class SplitOrMergeTracker {
} catch (KeeperException e) {
throw new RuntimeException(e);
}
splitZnode = ZKUtil.joinZNode(watcher.znodePaths.switchZNode,
splitZnode = ZNodePaths.joinZNode(watcher.znodePaths.switchZNode,
conf.get("zookeeper.znode.switch.split", "split"));
mergeZnode = ZKUtil.joinZNode(watcher.znodePaths.switchZNode,
mergeZnode = ZNodePaths.joinZNode(watcher.znodePaths.switchZNode,
conf.get("zookeeper.znode.switch.merge", "merge"));
splitStateTracker = new SwitchStateTracker(watcher, splitZnode, abortable);
mergeStateTracker = new SwitchStateTracker(watcher, mergeZnode, abortable);

View File

@ -45,7 +45,7 @@ public class ZKSplitLog {
* @param filename log file name (only the basename)
*/
public static String getEncodedNodeName(ZooKeeperWatcher zkw, String filename) {
return ZKUtil.joinZNode(zkw.znodePaths.splitLogZNode, encode(filename));
return ZNodePaths.joinZNode(zkw.znodePaths.splitLogZNode, encode(filename));
}
public static String getFileName(String node) {
@ -70,7 +70,7 @@ public class ZKSplitLog {
}
public static String getRescanNode(ZooKeeperWatcher zkw) {
return ZKUtil.joinZNode(zkw.znodePaths.splitLogZNode, "RESCAN");
return ZNodePaths.joinZNode(zkw.znodePaths.splitLogZNode, "RESCAN");
}
/**

View File

@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
/**
* Test provides some very generic helpers which might be used across the tests
* *** for running UTs in hbase-server
*/
public abstract class GenericTestUtils {

View File

@ -56,6 +56,7 @@ 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.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
@ -135,13 +136,13 @@ public class TestMetaWithReplicas {
Configuration conf = TEST_UTIL.getConfiguration();
String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
String primaryMetaZnode = ZKUtil.joinZNode(baseZNode,
String primaryMetaZnode = ZNodePaths.joinZNode(baseZNode,
conf.get("zookeeper.znode.metaserver", "meta-region-server"));
// check that the data in the znode is parseable (this would also mean the znode exists)
byte[] data = ZKUtil.getData(zkw, primaryMetaZnode);
ProtobufUtil.toServerName(data);
for (int i = 1; i < 3; i++) {
String secZnode = ZKUtil.joinZNode(baseZNode,
String secZnode = ZNodePaths.joinZNode(baseZNode,
conf.get("zookeeper.znode.metaserver", "meta-region-server") + "-" + i);
String str = zkw.znodePaths.getZNodeForReplica(i);
assertTrue(str.equals(secZnode));
@ -171,7 +172,7 @@ public class TestMetaWithReplicas {
String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
String primaryMetaZnode = ZKUtil.joinZNode(baseZNode,
String primaryMetaZnode = ZNodePaths.joinZNode(baseZNode,
conf.get("zookeeper.znode.metaserver", "meta-region-server"));
byte[] data = ZKUtil.getData(zkw, primaryMetaZnode);
ServerName primary = ProtobufUtil.toServerName(data);
@ -402,7 +403,7 @@ public class TestMetaWithReplicas {
ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
String primaryMetaZnode = ZKUtil.joinZNode(baseZNode,
String primaryMetaZnode = ZNodePaths.joinZNode(baseZNode,
conf.get("zookeeper.znode.metaserver", "meta-region-server"));
// check that the data in the znode is parseable (this would also mean the znode exists)
byte[] data = ZKUtil.getData(zkw, primaryMetaZnode);

View File

@ -50,6 +50,7 @@ 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.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.Assert;
@ -322,7 +323,7 @@ public class TestAssignmentListener {
"zkWatcher-NewServerDrainTest", abortable, true);
String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
String drainingZNode = ZKUtil.joinZNode(baseZNode,
String drainingZNode = ZNodePaths.joinZNode(baseZNode,
conf.get("zookeeper.znode.draining.rs", "draining"));
HMaster master = Mockito.mock(HMaster.class);
@ -348,7 +349,7 @@ public class TestAssignmentListener {
// Create draining znodes for the draining servers, which would have been
// performed when the previous HMaster was running.
for (ServerName sn : drainingServers) {
String znode = ZKUtil.joinZNode(drainingZNode, sn.getServerName());
String znode = ZNodePaths.joinZNode(drainingZNode, sn.getServerName());
ZKUtil.createAndFailSilent(zooKeeper, znode);
}

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -95,7 +96,7 @@ public class TestMetaShutdownHandler {
// Delete the ephemeral node of the meta-carrying region server.
// This is trigger the expire of this region server on the master.
String rsEphemeralNodePath =
ZKUtil.joinZNode(master.getZooKeeper().znodePaths.rsZNode, metaServerName.toString());
ZNodePaths.joinZNode(master.getZooKeeper().znodePaths.rsZNode, metaServerName.toString());
ZKUtil.deleteNode(master.getZooKeeper(), rsEphemeralNodePath);
// Wait for SSH to finish
final ServerManager serverManager = master.getServerManager();

View File

@ -29,6 +29,7 @@ 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.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
@ -72,7 +73,7 @@ public class TestTableStateManager {
private void setTableStateInZK(ZooKeeperWatcher watcher, final TableName tableName,
final ZooKeeperProtos.DeprecatedTableState.State state)
throws KeeperException, IOException {
String znode = ZKUtil.joinZNode(watcher.znodePaths.tableZNode, tableName.getNameAsString());
String znode = ZNodePaths.joinZNode(watcher.znodePaths.tableZNode, tableName.getNameAsString());
if (ZKUtil.checkExists(watcher, znode) == -1) {
ZKUtil.createAndFailSilent(watcher, znode);
}

View File

@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
import org.apache.hadoop.hbase.wal.FSHLogProvider;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALProvider;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.After;
import org.junit.Before;
@ -124,7 +124,7 @@ public class TestCompactionInDeadRegionServer {
HRegion region = (HRegion) rsToSuspend.getRegions(TABLE_NAME).get(0);
ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
watcher.getRecoverableZooKeeper().delete(
ZKUtil.joinZNode(watcher.getZNodePaths().rsZNode, rsToSuspend.getServerName().toString()),
ZNodePaths.joinZNode(watcher.getZNodePaths().rsZNode, rsToSuspend.getServerName().toString()),
-1);
UTIL.waitFor(60000, 1000, new ExplainingPredicate<Exception>() {

View File

@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
@ -429,7 +430,7 @@ public class TestSplitLogWorker {
if (node.startsWith("RESCAN")) {
String name = ZKSplitLog.getEncodedNodeName(zkw, node);
String fn = ZKSplitLog.getFileName(name);
byte [] data = ZKUtil.getData(zkw, ZKUtil.joinZNode(zkw.znodePaths.splitLogZNode, fn));
byte [] data = ZKUtil.getData(zkw, ZNodePaths.joinZNode(zkw.znodePaths.splitLogZNode, fn));
slt = SplitLogTask.parseFrom(data);
assertTrue(slt.toString(), slt.isDone(SRV));
}
@ -485,9 +486,9 @@ public class TestSplitLogWorker {
RegionServerServices mockedRS = getRegionServer(RS);
// create two RS nodes
String rsPath = ZKUtil.joinZNode(zkw.znodePaths.rsZNode, RS.getServerName());
String rsPath = ZNodePaths.joinZNode(zkw.znodePaths.rsZNode, RS.getServerName());
zkw.getRecoverableZooKeeper().create(rsPath, null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
rsPath = ZKUtil.joinZNode(zkw.znodePaths.rsZNode, RS2.getServerName());
rsPath = ZNodePaths.joinZNode(zkw.znodePaths.rsZNode, RS2.getServerName());
zkw.getRecoverableZooKeeper().create(rsPath, null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
for (int i = 0; i < maxTasks; i++) {

View File

@ -75,6 +75,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.HFileTestUtil;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.After;
import org.junit.Before;
@ -203,10 +204,10 @@ public class TestMasterReplication {
putAndWait(row, famName, htables[0], htables[0]);
rollWALAndWait(utilities[0], table.getTableName(), row);
ZooKeeperWatcher zkw = utilities[0].getZooKeeperWatcher();
String queuesZnode =
ZKUtil.joinZNode(zkw.getZNodePaths().baseZNode, ZKUtil.joinZNode("replication", "rs"));
String queuesZnode = ZNodePaths.joinZNode(zkw.getZNodePaths().baseZNode,
ZNodePaths.joinZNode("replication", "rs"));
List<String> listChildrenNoWatch =
ZKUtil.listChildrenNoWatch(zkw, ZKUtil.joinZNode(queuesZnode, rsName.toString()));
ZKUtil.listChildrenNoWatch(zkw, ZNodePaths.joinZNode(queuesZnode, rsName.toString()));
assertEquals(0, listChildrenNoWatch.size());
}

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
@ -88,7 +89,7 @@ public class TestReplicationStateHBaseImpl {
utility.startMiniCluster();
zkw = HBaseTestingUtility.getZooKeeperWatcher(utility);
String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
replicationZNode = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName);
replicationZNode = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName);
}
@Before

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
@ -69,7 +70,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
zkw = HBaseTestingUtility.getZooKeeperWatcher(utility);
String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
replicationZNode = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName);
replicationZNode = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName);
KEY_ONE = initPeerClusterState("/hbase1");
KEY_TWO = initPeerClusterState("/hbase2");
}
@ -80,7 +81,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
Configuration testConf = new Configuration(conf);
testConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, baseZKNode);
ZooKeeperWatcher zkw1 = new ZooKeeperWatcher(testConf, "test1", null);
String fakeRs = ZKUtil.joinZNode(zkw1.znodePaths.rsZNode, "hostname1.example.org:1234");
String fakeRs = ZNodePaths.joinZNode(zkw1.znodePaths.rsZNode, "hostname1.example.org:1234");
ZKUtil.createWithParents(zkw1, fakeRs);
ZKClusterId.setClusterId(zkw1, new ClusterId());
return ZKConfig.getZooKeeperClusterKey(testConf);
@ -126,13 +127,13 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
@Test
public void testIsPeerPath_PathToChildOfPeerNode() {
String peerChild = ZKUtil.joinZNode(ZKUtil.joinZNode(rqZK.peersZNode, "1"), "child");
String peerChild = ZNodePaths.joinZNode(ZNodePaths.joinZNode(rqZK.peersZNode, "1"), "child");
assertFalse(rqZK.isPeerPath(peerChild));
}
@Test
public void testIsPeerPath_ActualPeerPath() {
String peerPath = ZKUtil.joinZNode(rqZK.peersZNode, "1");
String peerPath = ZNodePaths.joinZNode(rqZK.peersZNode, "1");
assertTrue(rqZK.isPeerPath(peerPath));
}

View File

@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.Before;
@ -88,7 +89,7 @@ public class TestReplicationTrackerZKImpl {
@Before
public void setUp() throws Exception {
zkw = HBaseTestingUtility.getZooKeeperWatcher(utility);
String fakeRs1 = ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234");
String fakeRs1 = ZNodePaths.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234");
try {
ZKClusterId.setClusterId(zkw, new ClusterId());
rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw);
@ -117,31 +118,34 @@ public class TestReplicationTrackerZKImpl {
// 1 region server
ZKUtil.createWithParents(zkw,
ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234"));
ZNodePaths.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234"));
assertEquals(1, rt.getListOfRegionServers().size());
// 2 region servers
ZKUtil.createWithParents(zkw,
ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234"));
ZNodePaths.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234"));
assertEquals(2, rt.getListOfRegionServers().size());
// 1 region server
ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234"));
ZKUtil.deleteNode(zkw,
ZNodePaths.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234"));
assertEquals(1, rt.getListOfRegionServers().size());
// 0 region server
ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234"));
ZKUtil.deleteNode(zkw,
ZNodePaths.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234"));
assertEquals(0, rt.getListOfRegionServers().size());
}
@Test(timeout = 30000)
public void testRegionServerRemovedEvent() throws Exception {
ZKUtil.createAndWatch(zkw,
ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234"),
ZNodePaths.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234"),
HConstants.EMPTY_BYTE_ARRAY);
rt.registerListener(new DummyReplicationListener());
// delete one
ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234"));
ZKUtil.deleteNode(zkw,
ZNodePaths.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234"));
// wait for event
while (rsRemovedCount.get() < 1) {
Thread.sleep(5);

View File

@ -438,8 +438,8 @@ public abstract class TestReplicationSourceManager {
public void testPeerRemovalCleanup() throws Exception{
String replicationSourceImplName = conf.get("replication.replicationsource.implementation");
final String peerId = "FakePeer";
final ReplicationPeerConfig peerConfig =
new ReplicationPeerConfig().setClusterKey("localhost:1:/hbase");
final ReplicationPeerConfig peerConfig = new ReplicationPeerConfig()
.setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase");
try {
DummyServer server = new DummyServer();
final ReplicationQueues rq =
@ -473,8 +473,8 @@ public abstract class TestReplicationSourceManager {
@Test
public void testRemovePeerMetricsCleanup() throws Exception {
final String peerId = "DummyPeer";
final ReplicationPeerConfig peerConfig =
new ReplicationPeerConfig().setClusterKey("localhost:1:/hbase");
final ReplicationPeerConfig peerConfig = new ReplicationPeerConfig()
.setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase");
try {
addPeerAndWait(peerId, peerConfig, true);

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.Assert;
@ -87,7 +88,7 @@ public class TestZKSecretWatcherRefreshKeys {
AuthenticationKey ak = new AuthenticationKey(key,
System.currentTimeMillis() + 600 * 1000, null);
ZKUtil.createWithParents(zk,
ZKUtil.joinZNode(watcher.getKeysParentZNode(), key.toString()),
ZNodePaths.joinZNode(watcher.getKeysParentZNode(), key.toString()),
Writables.getBytes(ak));
}
Assert.assertNull(keyManager.getCurrentKey());

View File

@ -88,7 +88,7 @@ public class TestZKMulti {
ZKUtil.multiOrSequential(zkw, new LinkedList<>(), false);
// single create
String path = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testSimpleMulti");
String path = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testSimpleMulti");
LinkedList<ZKUtilOp> singleCreate = new LinkedList<>();
singleCreate.add(ZKUtilOp.createAndFailSilent(path, new byte[0]));
ZKUtil.multiOrSequential(zkw, singleCreate, false);
@ -110,12 +110,12 @@ public class TestZKMulti {
@Test (timeout=60000)
public void testComplexMulti() throws Exception {
String path1 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti1");
String path2 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti2");
String path3 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti3");
String path4 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti4");
String path5 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti5");
String path6 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti6");
String path1 = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti1");
String path2 = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti2");
String path3 = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti3");
String path4 = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti4");
String path5 = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti5");
String path6 = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti6");
// create 4 nodes that we'll setData on or delete later
LinkedList<ZKUtilOp> create4Nodes = new LinkedList<>();
create4Nodes.add(ZKUtilOp.createAndFailSilent(path1, Bytes.toBytes(path1)));
@ -154,7 +154,7 @@ public class TestZKMulti {
public void testSingleFailure() throws Exception {
// try to delete a node that doesn't exist
boolean caughtNoNode = false;
String path = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureZ");
String path = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureZ");
LinkedList<ZKUtilOp> ops = new LinkedList<>();
ops.add(ZKUtilOp.deleteNodeFailSilent(path));
try {
@ -191,9 +191,9 @@ public class TestZKMulti {
@Test (timeout=60000)
public void testSingleFailureInMulti() throws Exception {
// try a multi where all but one operation succeeds
String pathA = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureInMultiA");
String pathB = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureInMultiB");
String pathC = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureInMultiC");
String pathA = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureInMultiA");
String pathB = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureInMultiB");
String pathC = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureInMultiC");
LinkedList<ZKUtilOp> ops = new LinkedList<>();
ops.add(ZKUtilOp.createAndFailSilent(pathA, Bytes.toBytes(pathA)));
ops.add(ZKUtilOp.createAndFailSilent(pathB, Bytes.toBytes(pathB)));
@ -213,17 +213,17 @@ public class TestZKMulti {
@Test (timeout=60000)
public void testMultiFailure() throws Exception {
String pathX = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureX");
String pathY = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureY");
String pathZ = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureZ");
String pathX = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureX");
String pathY = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureY");
String pathZ = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureZ");
// create X that we will use to fail create later
LinkedList<ZKUtilOp> ops = new LinkedList<>();
ops.add(ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathX)));
ZKUtil.multiOrSequential(zkw, ops, false);
// fail one of each create ,setData, delete
String pathV = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureV");
String pathW = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureW");
String pathV = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureV");
String pathW = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureW");
ops = new LinkedList<>();
ops.add(ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathX))); // fail -- already exists
ops.add(ZKUtilOp.setData(pathY, Bytes.toBytes(pathY))); // fail -- doesn't exist
@ -267,10 +267,10 @@ public class TestZKMulti {
@Test (timeout=60000)
public void testRunSequentialOnMultiFailure() throws Exception {
String path1 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "runSequential1");
String path2 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "runSequential2");
String path3 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "runSequential3");
String path4 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "runSequential4");
String path1 = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "runSequential1");
String path2 = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "runSequential2");
String path3 = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "runSequential3");
String path4 = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, "runSequential4");
// create some nodes that we will use later
LinkedList<ZKUtilOp> ops = new LinkedList<>();

View File

@ -100,7 +100,7 @@ public class TestZooKeeperNodeTracker {
ZKUtil.createAndFailSilent(zk, zk.znodePaths.baseZNode);
final String node =
ZKUtil.joinZNode(zk.znodePaths.baseZNode, new Long(rand.nextLong()).toString());
ZNodePaths.joinZNode(zk.znodePaths.baseZNode, new Long(rand.nextLong()).toString());
final byte [] dataOne = Bytes.toBytes("dataOne");
final byte [] dataTwo = Bytes.toBytes("dataTwo");