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:
parent
31234eb862
commit
72270866cb
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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.
|
||||
*/
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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 + "---");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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() {
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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()) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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];
|
||||
|
|
|
@ -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 + "---");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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 {
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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>() {
|
||||
|
||||
|
|
|
@ -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++) {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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<>();
|
||||
|
|
|
@ -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");
|
||||
|
|
Loading…
Reference in New Issue