HBASE-11574. hbase:meta's regions can be replicated
This commit is contained in:
parent
7b4058a41a
commit
c66fdf1e83
|
@ -35,6 +35,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
|
@ -242,12 +243,16 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
* first meta regions
|
||||
*/
|
||||
private HRegionInfo(long regionId, TableName tableName) {
|
||||
this(regionId, tableName, DEFAULT_REPLICA_ID);
|
||||
}
|
||||
|
||||
public HRegionInfo(long regionId, TableName tableName, int replicaId) {
|
||||
super();
|
||||
this.regionId = regionId;
|
||||
this.tableName = tableName;
|
||||
// Note: First Meta regions names are still in old format
|
||||
this.regionName = createRegionName(tableName, null,
|
||||
regionId, false);
|
||||
this.replicaId = replicaId;
|
||||
// Note: First Meta region replicas names are in old format
|
||||
this.regionName = createRegionName(tableName, null, regionId, replicaId, false);
|
||||
setHashCode();
|
||||
}
|
||||
|
||||
|
@ -1017,7 +1022,8 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
TableName tableName =
|
||||
ProtobufUtil.toTableName(proto.getTableName());
|
||||
if (tableName.equals(TableName.META_TABLE_NAME)) {
|
||||
return FIRST_META_REGIONINFO;
|
||||
return RegionReplicaUtil.getRegionInfoForReplica(FIRST_META_REGIONINFO,
|
||||
proto.getReplicaId());
|
||||
}
|
||||
long regionId = proto.getRegionId();
|
||||
int replicaId = proto.hasReplicaId() ? proto.getReplicaId() : DEFAULT_REPLICA_ID;
|
||||
|
|
|
@ -535,6 +535,7 @@ class ConnectionManager {
|
|||
static class HConnectionImplementation implements ClusterConnection, Closeable {
|
||||
static final Log LOG = LogFactory.getLog(HConnectionImplementation.class);
|
||||
private final long pause;
|
||||
private final boolean useMetaReplicas;
|
||||
private final int numTries;
|
||||
final int rpcTimeout;
|
||||
private NonceGenerator nonceGenerator = null;
|
||||
|
@ -658,6 +659,8 @@ class ConnectionManager {
|
|||
this.closed = false;
|
||||
this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
|
||||
HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
|
||||
this.useMetaReplicas = conf.getBoolean(HConstants.USE_META_REPLICAS,
|
||||
HConstants.DEFAULT_USE_META_REPLICAS);
|
||||
this.numTries = tableConfig.getRetriesNumber();
|
||||
this.rpcTimeout = conf.getInt(
|
||||
HConstants.HBASE_RPC_TIMEOUT_KEY,
|
||||
|
@ -1115,7 +1118,7 @@ class ConnectionManager {
|
|||
RegionLocations locations = null;
|
||||
if (useCache) {
|
||||
locations = getCachedLocation(tableName, metaCacheKey);
|
||||
if (locations != null) {
|
||||
if (locations != null && locations.getRegionLocation(replicaId) != null) {
|
||||
return locations;
|
||||
}
|
||||
}
|
||||
|
@ -1126,7 +1129,7 @@ class ConnectionManager {
|
|||
// same query while we were waiting on the lock.
|
||||
if (useCache) {
|
||||
locations = getCachedLocation(tableName, metaCacheKey);
|
||||
if (locations != null) {
|
||||
if (locations != null && locations.getRegionLocation(replicaId) != null) {
|
||||
return locations;
|
||||
}
|
||||
}
|
||||
|
@ -1166,6 +1169,9 @@ class ConnectionManager {
|
|||
s.setStartRow(metaKey);
|
||||
s.setSmall(true);
|
||||
s.setCaching(1);
|
||||
if (this.useMetaReplicas) {
|
||||
s.setConsistency(Consistency.TIMELINE);
|
||||
}
|
||||
|
||||
int localNumRetries = (retry ? numTries : 1);
|
||||
|
||||
|
|
|
@ -417,9 +417,6 @@ public class HBaseAdmin implements Admin {
|
|||
public HTableDescriptor getTableDescriptor(final TableName tableName)
|
||||
throws TableNotFoundException, IOException {
|
||||
if (tableName == null) return null;
|
||||
if (tableName.equals(TableName.META_TABLE_NAME)) {
|
||||
return HTableDescriptor.META_TABLEDESC;
|
||||
}
|
||||
HTableDescriptor htd = executeCallable(new MasterCallable<HTableDescriptor>(getConnection()) {
|
||||
@Override
|
||||
public HTableDescriptor call(int callTimeout) throws ServiceException {
|
||||
|
|
|
@ -49,6 +49,7 @@ class HConnectionKey {
|
|||
HConstants.HBASE_META_SCANNER_CACHING,
|
||||
HConstants.HBASE_CLIENT_INSTANCE_ID,
|
||||
HConstants.RPC_CODEC_CONF_KEY,
|
||||
HConstants.USE_META_REPLICAS,
|
||||
RpcControllerFactory.CUSTOM_CONTROLLER_CONF_KEY};
|
||||
|
||||
private Map<String, String> properties;
|
||||
|
|
|
@ -150,7 +150,9 @@ public class MetaScanner {
|
|||
try (Table metaTable = new HTable(TableName.META_TABLE_NAME, connection, null)) {
|
||||
if (row != null) {
|
||||
// Scan starting at a particular row in a particular table
|
||||
Result startRowResult = getClosestRowOrBefore(metaTable, tableName, row);
|
||||
Result startRowResult = getClosestRowOrBefore(metaTable, tableName, row,
|
||||
connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS,
|
||||
HConstants.DEFAULT_USE_META_REPLICAS));
|
||||
if (startRowResult == null) {
|
||||
throw new TableNotFoundException("Cannot find row in " + metaTable.getName() +
|
||||
" for table: " + tableName + ", row=" + Bytes.toStringBinary(row));
|
||||
|
@ -174,6 +176,10 @@ public class MetaScanner {
|
|||
int scannerCaching = connection.getConfiguration()
|
||||
.getInt(HConstants.HBASE_META_SCANNER_CACHING,
|
||||
HConstants.DEFAULT_HBASE_META_SCANNER_CACHING);
|
||||
if (connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS,
|
||||
HConstants.DEFAULT_USE_META_REPLICAS)) {
|
||||
scan.setConsistency(Consistency.TIMELINE);
|
||||
}
|
||||
if (rowUpperLimit <= scannerCaching) {
|
||||
scan.setSmall(true);
|
||||
}
|
||||
|
@ -212,10 +218,13 @@ public class MetaScanner {
|
|||
* @throws IOException
|
||||
*/
|
||||
private static Result getClosestRowOrBefore(final Table metaTable, final TableName userTableName,
|
||||
final byte [] row)
|
||||
final byte [] row, boolean useMetaReplicas)
|
||||
throws IOException {
|
||||
byte[] searchRow = HRegionInfo.createRegionName(userTableName, row, HConstants.NINES, false);
|
||||
Scan scan = Scan.createGetClosestRowOrBeforeReverseScan(searchRow);
|
||||
if (useMetaReplicas) {
|
||||
scan.setConsistency(Consistency.TIMELINE);
|
||||
}
|
||||
try (ResultScanner resultScanner = metaTable.getScanner(scan)) {
|
||||
return resultScanner.next();
|
||||
}
|
||||
|
|
|
@ -47,9 +47,13 @@ public class RegionReplicaUtil {
|
|||
if (regionInfo.getReplicaId() == replicaId) {
|
||||
return regionInfo;
|
||||
}
|
||||
HRegionInfo replicaInfo = new HRegionInfo(regionInfo.getTable(), regionInfo.getStartKey(),
|
||||
regionInfo.getEndKey(), regionInfo.isSplit(), regionInfo.getRegionId(), replicaId);
|
||||
|
||||
HRegionInfo replicaInfo;
|
||||
if (regionInfo.isMetaRegion()) {
|
||||
replicaInfo = new HRegionInfo(regionInfo.getRegionId(), regionInfo.getTable(), replicaId);
|
||||
} else {
|
||||
replicaInfo = new HRegionInfo(regionInfo.getTable(), regionInfo.getStartKey(),
|
||||
regionInfo.getEndKey(), regionInfo.isSplit(), regionInfo.getRegionId(), replicaId);
|
||||
}
|
||||
replicaInfo.setOffline(regionInfo.isOffline());
|
||||
return replicaInfo;
|
||||
}
|
||||
|
|
|
@ -272,7 +272,7 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
|
|||
return 0; // not scheduling on other replicas for strong consistency
|
||||
}
|
||||
for (int id = min; id <= max; id++) {
|
||||
if (currentScannerCallable.getHRegionInfo().getReplicaId() == id) {
|
||||
if (currentScannerCallable.id == id) {
|
||||
continue; //this was already scheduled earlier
|
||||
}
|
||||
ScannerCallable s = currentScannerCallable.getScannerCallableForReplica(id);
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -57,14 +58,32 @@ class ZooKeeperRegistry implements Registry {
|
|||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Looking up meta region location in ZK," + " connection=" + this);
|
||||
}
|
||||
ServerName servername = new MetaTableLocator().blockUntilAvailable(zkw, hci.rpcTimeout);
|
||||
List<ServerName> servers = new MetaTableLocator().blockUntilAvailable(zkw, hci.rpcTimeout,
|
||||
hci.getConfiguration());
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Looked up meta region location, connection=" + this +
|
||||
"; serverName=" + ((servername == null) ? "null" : servername));
|
||||
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 (servername == null) return null;
|
||||
HRegionLocation loc = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, servername, 0);
|
||||
return new RegionLocations(new HRegionLocation[] {loc});
|
||||
if (servers == null) return null;
|
||||
HRegionLocation[] locs = new HRegionLocation[servers.size()];
|
||||
int i = 0;
|
||||
for (ServerName server : servers) {
|
||||
HRegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
|
||||
HRegionInfo.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;
|
||||
|
|
|
@ -23,10 +23,12 @@ import com.google.protobuf.InvalidProtocolBufferException;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
||||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
|
@ -77,9 +79,6 @@ import javax.annotation.Nullable;
|
|||
public class MetaTableLocator {
|
||||
private static final Log LOG = LogFactory.getLog(MetaTableLocator.class);
|
||||
|
||||
static final byte [] META_REGION_NAME =
|
||||
HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
|
||||
|
||||
// only needed to allow non-timeout infinite waits to stop when cluster shuts down
|
||||
private volatile boolean stopped = false;
|
||||
|
||||
|
@ -96,9 +95,21 @@ public class MetaTableLocator {
|
|||
* @return meta table regions and their locations.
|
||||
*/
|
||||
public List<Pair<HRegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw) {
|
||||
ServerName serverName = new MetaTableLocator().getMetaRegionLocation(zkw);
|
||||
return getMetaRegionsAndLocations(zkw, HRegionInfo.DEFAULT_REPLICA_ID);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param zkw
|
||||
* @param replicaId
|
||||
* @return meta table regions and their locations.
|
||||
*/
|
||||
public List<Pair<HRegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw,
|
||||
int replicaId) {
|
||||
ServerName serverName = getMetaRegionLocation(zkw, replicaId);
|
||||
List<Pair<HRegionInfo, ServerName>> list = new ArrayList<Pair<HRegionInfo, ServerName>>();
|
||||
list.add(new Pair<HRegionInfo, ServerName>(HRegionInfo.FIRST_META_REGIONINFO, serverName));
|
||||
list.add(new Pair<HRegionInfo, ServerName>(RegionReplicaUtil.getRegionInfoForReplica(
|
||||
HRegionInfo.FIRST_META_REGIONINFO, replicaId), serverName));
|
||||
return list;
|
||||
}
|
||||
|
||||
|
@ -107,8 +118,18 @@ public class MetaTableLocator {
|
|||
* @return List of meta regions
|
||||
*/
|
||||
public List<HRegionInfo> getMetaRegions(ZooKeeperWatcher zkw) {
|
||||
return getMetaRegions(zkw, HRegionInfo.DEFAULT_REPLICA_ID);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param zkw
|
||||
* @param replicaId
|
||||
* @return List of meta regions
|
||||
*/
|
||||
public List<HRegionInfo> getMetaRegions(ZooKeeperWatcher zkw, int replicaId) {
|
||||
List<Pair<HRegionInfo, ServerName>> result;
|
||||
result = getMetaRegionsAndLocations(zkw);
|
||||
result = getMetaRegionsAndLocations(zkw, replicaId);
|
||||
return getListOfHRegionInfos(result);
|
||||
}
|
||||
|
||||
|
@ -136,17 +157,52 @@ public class MetaTableLocator {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the meta region location, if available. Does not block.
|
||||
* @param zkw
|
||||
* @param replicaId
|
||||
* @return server name
|
||||
*/
|
||||
public ServerName getMetaRegionLocation(final ZooKeeperWatcher zkw, int replicaId) {
|
||||
try {
|
||||
RegionState state = getMetaRegionState(zkw, replicaId);
|
||||
return state.isOpened() ? state.getServerName() : null;
|
||||
} catch (KeeperException ke) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the meta region location, if available, and waits for up to the
|
||||
* specified timeout if not immediately available.
|
||||
* Given the zookeeper notification could be delayed, we will try to
|
||||
* get the latest data.
|
||||
* @param zkw
|
||||
* @param timeout maximum time to wait, in millis
|
||||
* @return server name for server hosting meta region formatted as per
|
||||
* {@link ServerName}, or null if none available
|
||||
* @throws InterruptedException if interrupted while waiting
|
||||
* @throws NotAllMetaRegionsOnlineException
|
||||
*/
|
||||
public ServerName waitMetaRegionLocation(ZooKeeperWatcher zkw, long timeout)
|
||||
throws InterruptedException, NotAllMetaRegionsOnlineException {
|
||||
return waitMetaRegionLocation(zkw, HRegionInfo.DEFAULT_REPLICA_ID, timeout);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the meta region location, if available, and waits for up to the
|
||||
* specified timeout if not immediately available.
|
||||
* Given the zookeeper notification could be delayed, we will try to
|
||||
* get the latest data.
|
||||
* @param zkw
|
||||
* @param replicaId
|
||||
* @param timeout maximum time to wait, in millis
|
||||
* @return server name for server hosting meta region formatted as per
|
||||
* {@link ServerName}, or null if none available
|
||||
* @throws InterruptedException
|
||||
* @throws NotAllMetaRegionsOnlineException
|
||||
*/
|
||||
public ServerName waitMetaRegionLocation(ZooKeeperWatcher zkw, int replicaId, long timeout)
|
||||
throws InterruptedException, NotAllMetaRegionsOnlineException {
|
||||
try {
|
||||
if (ZKUtil.checkExists(zkw, zkw.baseZNode) == -1) {
|
||||
|
@ -158,7 +214,7 @@ public class MetaTableLocator {
|
|||
} catch (KeeperException e) {
|
||||
throw new IllegalStateException("KeeperException while trying to check baseZNode:", e);
|
||||
}
|
||||
ServerName sn = blockUntilAvailable(zkw, timeout);
|
||||
ServerName sn = blockUntilAvailable(zkw, replicaId, timeout);
|
||||
|
||||
if (sn == null) {
|
||||
throw new NotAllMetaRegionsOnlineException("Timed out; " + timeout + "ms");
|
||||
|
@ -195,6 +251,8 @@ public class MetaTableLocator {
|
|||
|
||||
/**
|
||||
* Verify <code>hbase:meta</code> is deployed and accessible.
|
||||
* @param hConnection
|
||||
* @param zkw
|
||||
* @param timeout How long to wait on zk for meta address (passed through to
|
||||
* the internal call to {@link #getMetaServerConnection}.
|
||||
* @return True if the <code>hbase:meta</code> location is healthy.
|
||||
|
@ -203,10 +261,26 @@ public class MetaTableLocator {
|
|||
*/
|
||||
public boolean verifyMetaRegionLocation(HConnection hConnection,
|
||||
ZooKeeperWatcher zkw, final long timeout)
|
||||
throws InterruptedException, IOException {
|
||||
return verifyMetaRegionLocation(hConnection, zkw, timeout, HRegionInfo.DEFAULT_REPLICA_ID);
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify <code>hbase:meta</code> is deployed and accessible.
|
||||
* @param hConnection
|
||||
* @param zkw
|
||||
* @param timeout How long to wait on zk for meta address (passed through to
|
||||
* @param replicaId
|
||||
* @return True if the <code>hbase:meta</code> location is healthy.
|
||||
* @throws InterruptedException
|
||||
* @throws IOException
|
||||
*/
|
||||
public boolean verifyMetaRegionLocation(HConnection hConnection,
|
||||
ZooKeeperWatcher zkw, final long timeout, int replicaId)
|
||||
throws InterruptedException, IOException {
|
||||
AdminProtos.AdminService.BlockingInterface service = null;
|
||||
try {
|
||||
service = getMetaServerConnection(hConnection, zkw, timeout);
|
||||
service = getMetaServerConnection(hConnection, zkw, timeout, replicaId);
|
||||
} catch (NotAllMetaRegionsOnlineException e) {
|
||||
// Pass
|
||||
} catch (ServerNotRunningYetException e) {
|
||||
|
@ -217,7 +291,8 @@ public class MetaTableLocator {
|
|||
// Pass -- server name sends us to a server that is dying or already dead.
|
||||
}
|
||||
return (service != null) && verifyRegionLocation(service,
|
||||
getMetaRegionLocation(zkw), META_REGION_NAME);
|
||||
getMetaRegionLocation(zkw, replicaId), RegionReplicaUtil.getRegionInfoForReplica(
|
||||
HRegionInfo.FIRST_META_REGIONINFO, replicaId).getRegionName());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -273,16 +348,19 @@ public class MetaTableLocator {
|
|||
* Gets a connection to the server hosting meta, as reported by ZooKeeper,
|
||||
* waiting up to the specified timeout for availability.
|
||||
* <p>WARNING: Does not retry. Use an {@link org.apache.hadoop.hbase.client.HTable} instead.
|
||||
* @param hConnection
|
||||
* @param zkw
|
||||
* @param timeout How long to wait on meta location
|
||||
* @param replicaId
|
||||
* @return connection to server hosting meta
|
||||
* @throws InterruptedException
|
||||
* @throws NotAllMetaRegionsOnlineException if timed out waiting
|
||||
* @throws IOException
|
||||
*/
|
||||
private AdminService.BlockingInterface getMetaServerConnection(HConnection hConnection,
|
||||
ZooKeeperWatcher zkw, long timeout)
|
||||
ZooKeeperWatcher zkw, long timeout, int replicaId)
|
||||
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
|
||||
return getCachedConnection(hConnection, waitMetaRegionLocation(zkw, timeout));
|
||||
return getCachedConnection(hConnection, waitMetaRegionLocation(zkw, replicaId, timeout));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -347,6 +425,20 @@ public class MetaTableLocator {
|
|||
*/
|
||||
public static void setMetaLocation(ZooKeeperWatcher zookeeper,
|
||||
ServerName serverName, RegionState.State state) throws KeeperException {
|
||||
setMetaLocation(zookeeper, serverName, HRegionInfo.DEFAULT_REPLICA_ID, state);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the location of <code>hbase:meta</code> in ZooKeeper to the
|
||||
* specified server address.
|
||||
* @param zookeeper
|
||||
* @param serverName
|
||||
* @param replicaId
|
||||
* @param state
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public static void setMetaLocation(ZooKeeperWatcher zookeeper,
|
||||
ServerName serverName, int replicaId, RegionState.State state) throws KeeperException {
|
||||
LOG.info("Setting hbase:meta region location in ZooKeeper as " + serverName);
|
||||
// Make the MetaRegionServer pb and then get its bytes and save this as
|
||||
// the znode content.
|
||||
|
@ -356,10 +448,15 @@ public class MetaTableLocator {
|
|||
.setState(state.convert()).build();
|
||||
byte[] data = ProtobufUtil.prependPBMagic(pbrsr.toByteArray());
|
||||
try {
|
||||
ZKUtil.setData(zookeeper, zookeeper.metaServerZNode, data);
|
||||
ZKUtil.setData(zookeeper, zookeeper.getZNodeForReplica(replicaId), data);
|
||||
} catch(KeeperException.NoNodeException nne) {
|
||||
LOG.debug("META region location doesn't existed, create it");
|
||||
ZKUtil.createAndWatch(zookeeper, zookeeper.metaServerZNode, data);
|
||||
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
|
||||
LOG.debug("META region location doesn't exist, create it");
|
||||
} else {
|
||||
LOG.debug("META region location doesn't exist for replicaId " + replicaId +
|
||||
", create it");
|
||||
}
|
||||
ZKUtil.createAndWatch(zookeeper, zookeeper.getZNodeForReplica(replicaId), data);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -367,10 +464,22 @@ public class MetaTableLocator {
|
|||
* Load the meta region state from the meta server ZNode.
|
||||
*/
|
||||
public static RegionState getMetaRegionState(ZooKeeperWatcher zkw) throws KeeperException {
|
||||
return getMetaRegionState(zkw, HRegionInfo.DEFAULT_REPLICA_ID);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load the meta region state from the meta server ZNode.
|
||||
* @param zkw
|
||||
* @param replicaId
|
||||
* @return regionstate
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public static RegionState getMetaRegionState(ZooKeeperWatcher zkw, int replicaId)
|
||||
throws KeeperException {
|
||||
RegionState.State state = RegionState.State.OPEN;
|
||||
ServerName serverName = null;
|
||||
try {
|
||||
byte[] data = ZKUtil.getData(zkw, zkw.metaServerZNode);
|
||||
byte[] data = ZKUtil.getData(zkw, zkw.getZNodeForReplica(replicaId));
|
||||
if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) {
|
||||
try {
|
||||
int prefixLen = ProtobufUtil.lengthOfPBMagic();
|
||||
|
@ -398,7 +507,8 @@ public class MetaTableLocator {
|
|||
if (serverName == null) {
|
||||
state = RegionState.State.OFFLINE;
|
||||
}
|
||||
return new RegionState(HRegionInfo.FIRST_META_REGIONINFO,
|
||||
return new RegionState(
|
||||
RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, replicaId),
|
||||
state, serverName);
|
||||
}
|
||||
|
||||
|
@ -409,14 +519,53 @@ public class MetaTableLocator {
|
|||
*/
|
||||
public void deleteMetaLocation(ZooKeeperWatcher zookeeper)
|
||||
throws KeeperException {
|
||||
LOG.info("Deleting hbase:meta region location in ZooKeeper");
|
||||
deleteMetaLocation(zookeeper, HRegionInfo.DEFAULT_REPLICA_ID);
|
||||
}
|
||||
|
||||
public void deleteMetaLocation(ZooKeeperWatcher zookeeper, int replicaId)
|
||||
throws KeeperException {
|
||||
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
|
||||
LOG.info("Deleting hbase:meta region location in ZooKeeper");
|
||||
} else {
|
||||
LOG.info("Deleting hbase:meta for " + replicaId + " region location in ZooKeeper");
|
||||
}
|
||||
try {
|
||||
// Just delete the node. Don't need any watches.
|
||||
ZKUtil.deleteNode(zookeeper, zookeeper.metaServerZNode);
|
||||
ZKUtil.deleteNode(zookeeper, zookeeper.getZNodeForReplica(replicaId));
|
||||
} catch(KeeperException.NoNodeException nne) {
|
||||
// Has already been deleted
|
||||
}
|
||||
}
|
||||
/**
|
||||
* Wait until the primary meta region is available. Get the secondary
|
||||
* locations as well but don't block for those.
|
||||
* @param zkw
|
||||
* @param timeout
|
||||
* @param conf
|
||||
* @return ServerName or null if we timed out.
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
public List<ServerName> blockUntilAvailable(final ZooKeeperWatcher zkw,
|
||||
final long timeout, Configuration conf)
|
||||
throws InterruptedException {
|
||||
int numReplicasConfigured = 1;
|
||||
try {
|
||||
List<String> metaReplicaNodes = zkw.getMetaReplicaNodes();
|
||||
numReplicasConfigured = metaReplicaNodes.size();
|
||||
} catch (KeeperException e) {
|
||||
LOG.warn("Got ZK exception " + e);
|
||||
}
|
||||
List<ServerName> servers = new ArrayList<ServerName>(numReplicasConfigured);
|
||||
ServerName server = blockUntilAvailable(zkw, timeout);
|
||||
if (server == null) return null;
|
||||
servers.add(server);
|
||||
|
||||
for (int replicaId = 1; replicaId < numReplicasConfigured; replicaId++) {
|
||||
// return all replica locations for the meta
|
||||
servers.add(getMetaRegionLocation(zkw, replicaId));
|
||||
}
|
||||
return servers;
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait until the meta region is available and is not in transition.
|
||||
|
@ -427,6 +576,20 @@ public class MetaTableLocator {
|
|||
*/
|
||||
public ServerName blockUntilAvailable(final ZooKeeperWatcher zkw,
|
||||
final long timeout)
|
||||
throws InterruptedException {
|
||||
return blockUntilAvailable(zkw, HRegionInfo.DEFAULT_REPLICA_ID, timeout);
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait until the meta region is available and is not in transition.
|
||||
* @param zkw
|
||||
* @param replicaId
|
||||
* @param timeout
|
||||
* @return ServerName or null if we timed out.
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
public ServerName blockUntilAvailable(final ZooKeeperWatcher zkw, int replicaId,
|
||||
final long timeout)
|
||||
throws InterruptedException {
|
||||
if (timeout < 0) throw new IllegalArgumentException();
|
||||
if (zkw == null) throw new IllegalArgumentException();
|
||||
|
@ -434,7 +597,7 @@ public class MetaTableLocator {
|
|||
ServerName sn = null;
|
||||
try {
|
||||
while (true) {
|
||||
sn = getMetaRegionLocation(zkw);
|
||||
sn = getMetaRegionLocation(zkw, replicaId);
|
||||
if (sn != null || sw.elapsedMillis()
|
||||
> timeout - HConstants.SOCKET_RETRY_WAIT_MS) {
|
||||
break;
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -968,7 +969,7 @@ public class ZKUtil {
|
|||
// Certain znodes are accessed directly by the client,
|
||||
// so they must be readable by non-authenticated clients
|
||||
if ((node.equals(zkw.baseZNode) == true) ||
|
||||
(node.equals(zkw.metaServerZNode) == true) ||
|
||||
(zkw.isAnyMetaReplicaZnode(node)) ||
|
||||
(node.equals(zkw.getMasterAddressZNode()) == true) ||
|
||||
(node.equals(zkw.clusterIdZNode) == true) ||
|
||||
(node.equals(zkw.rsZNode) == true) ||
|
||||
|
@ -1702,6 +1703,13 @@ public class ZKUtil {
|
|||
}
|
||||
sb.append("\nRegion server holding hbase:meta: "
|
||||
+ new MetaTableLocator().getMetaRegionLocation(zkw));
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
int numMetaReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
|
||||
HConstants.DEFAULT_META_REPLICA_NUM);
|
||||
for (int i = 1; i < numMetaReplicas; i++) {
|
||||
sb.append("\nRegion server holding hbase:meta, replicaId " + i + " "
|
||||
+ new MetaTableLocator().getMetaRegionLocation(zkw, i));
|
||||
}
|
||||
sb.append("\nRegion servers:");
|
||||
for (String child : listChildrenNoWatch(zkw, zkw.rsZNode)) {
|
||||
sb.append("\n ").append(child);
|
||||
|
@ -1892,7 +1900,7 @@ public class ZKUtil {
|
|||
(data == null? "null": data.length == 0? "empty": (
|
||||
znode.startsWith(zkw.assignmentZNode)?
|
||||
ZKAssign.toString(data): // We should not be doing this reaching into another class
|
||||
znode.startsWith(zkw.metaServerZNode)?
|
||||
znode.startsWith(ZooKeeperWatcher.META_ZNODE_PREFIX)?
|
||||
getServerNameOrEmptyString(data):
|
||||
znode.startsWith(zkw.backupMasterAddressesZNode)?
|
||||
getServerNameOrEmptyString(data):
|
||||
|
|
|
@ -21,7 +21,10 @@ package org.apache.hadoop.hbase.zookeeper;
|
|||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
|
@ -31,6 +34,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.WatchedEvent;
|
||||
|
@ -81,8 +85,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
|
||||
// base znode for this cluster
|
||||
public String baseZNode;
|
||||
// znode containing location of server hosting meta region
|
||||
public String metaServerZNode;
|
||||
//znodes containing the locations of the servers hosting the meta replicas
|
||||
private Map<Integer,String> metaReplicaZnodes = new HashMap<Integer, String>();
|
||||
// znode containing ephemeral nodes of the regionservers
|
||||
public String rsZNode;
|
||||
// znode containing ephemeral nodes of the draining regionservers
|
||||
|
@ -117,6 +121,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
add(new ACL(ZooDefs.Perms.ALL,ZooDefs.Ids.AUTH_IDS));
|
||||
}};
|
||||
|
||||
public final static String META_ZNODE_PREFIX = "meta-region-server";
|
||||
|
||||
private final Configuration conf;
|
||||
|
||||
private final Exception constructorCaller;
|
||||
|
@ -209,8 +215,15 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
private void setNodeNames(Configuration conf) {
|
||||
baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
|
||||
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
|
||||
metaServerZNode = ZKUtil.joinZNode(baseZNode,
|
||||
conf.get("zookeeper.znode.metaserver", "meta-region-server"));
|
||||
metaReplicaZnodes.put(0, ZKUtil.joinZNode(baseZNode,
|
||||
conf.get("zookeeper.znode.metaserver", "meta-region-server")));
|
||||
int numMetaReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
|
||||
HConstants.DEFAULT_META_REPLICA_NUM);
|
||||
for (int i = 1; i < numMetaReplicas; i++) {
|
||||
String str = ZKUtil.joinZNode(baseZNode,
|
||||
conf.get("zookeeper.znode.metaserver", "meta-region-server") + "-" + i);
|
||||
metaReplicaZnodes.put(i, str);
|
||||
}
|
||||
rsZNode = ZKUtil.joinZNode(baseZNode,
|
||||
conf.get("zookeeper.znode.rs", "rs"));
|
||||
drainingZNode = ZKUtil.joinZNode(baseZNode,
|
||||
|
@ -239,6 +252,75 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
conf.get("zookeeper.znode.namespace", "namespace"));
|
||||
}
|
||||
|
||||
/**
|
||||
* Is the znode of any meta replica
|
||||
* @param node
|
||||
* @return true or false
|
||||
*/
|
||||
public boolean isAnyMetaReplicaZnode(String node) {
|
||||
if (metaReplicaZnodes.values().contains(node)) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Is it the default meta replica's znode
|
||||
* @param node
|
||||
* @return true or false
|
||||
*/
|
||||
public boolean isDefaultMetaReplicaZnode(String node) {
|
||||
if (getZNodeForReplica(HRegionInfo.DEFAULT_REPLICA_ID).equals(node)) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the znodes corresponding to the meta replicas from ZK
|
||||
* @return list of znodes
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public List<String> getMetaReplicaNodes() throws KeeperException {
|
||||
List<String> childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, baseZNode);
|
||||
List<String> metaReplicaNodes = new ArrayList<String>(2);
|
||||
String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server");
|
||||
for (String child : childrenOfBaseNode) {
|
||||
if (child.startsWith(pattern)) metaReplicaNodes.add(child);
|
||||
}
|
||||
return metaReplicaNodes;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the znode string corresponding to a replicaId
|
||||
* @param replicaId
|
||||
* @return znode
|
||||
*/
|
||||
public String getZNodeForReplica(int replicaId) {
|
||||
String str = metaReplicaZnodes.get(replicaId);
|
||||
// return a newly created path but don't update the cache of paths
|
||||
// This is mostly needed for tests that attempt to create meta replicas
|
||||
// from outside the master
|
||||
if (str == null) {
|
||||
str = ZKUtil.joinZNode(baseZNode,
|
||||
conf.get("zookeeper.znode.metaserver", "meta-region-server") + "-" + replicaId);
|
||||
}
|
||||
return str;
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse the meta replicaId from the passed znode
|
||||
* @param znode
|
||||
* @return replicaId
|
||||
*/
|
||||
public int getMetaReplicaIdFromZnode(String znode) {
|
||||
String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server");
|
||||
if (znode.equals(pattern)) return HRegionInfo.DEFAULT_REPLICA_ID;
|
||||
// the non-default replicas are of the pattern meta-region-server-<replicaId>
|
||||
String nonDefaultPattern = pattern + "-";
|
||||
return Integer.parseInt(znode.substring(nonDefaultPattern.length()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Register the specified listener to receive ZooKeeper events.
|
||||
* @param listener
|
||||
|
|
|
@ -886,6 +886,12 @@ public final class HConstants {
|
|||
public static final String LOG_REPLAY_WAIT_REGION_TIMEOUT =
|
||||
"hbase.master.log.replay.wait.region.timeout";
|
||||
|
||||
/** Conf key for enabling meta replication */
|
||||
public static final String USE_META_REPLICAS = "hbase.meta.replicas.use";
|
||||
public static final boolean DEFAULT_USE_META_REPLICAS = false;
|
||||
public static final String META_REPLICAS_NUM = "hbase.meta.replica.count";
|
||||
public static final int DEFAULT_META_REPLICA_NUM = 1;
|
||||
|
||||
/**
|
||||
* The name of the configuration parameter that specifies
|
||||
* the number of bytes in a newly created checksum chunk.
|
||||
|
|
|
@ -0,0 +1,108 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.client.TestMetaWithReplicas;
|
||||
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.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* An integration test that starts the cluster with three replicas for the meta
|
||||
* It then creates a table, flushes the meta, kills the server holding the primary.
|
||||
* After that a client issues put/get requests on the created table - the other
|
||||
* replicas of the meta would be used to get the location of the region of the created
|
||||
* table.
|
||||
*/
|
||||
@Category(IntegrationTests.class)
|
||||
public class IntegrationTestMetaReplicas {
|
||||
static final Log LOG = LogFactory.getLog(IntegrationTestMetaReplicas.class);
|
||||
/**
|
||||
* Util to get at the cluster.
|
||||
*/
|
||||
private static IntegrationTestingUtility util;
|
||||
|
||||
@BeforeClass
|
||||
public static void setUp() throws Exception {
|
||||
// Set up the integration test util
|
||||
if (util == null) {
|
||||
util = new IntegrationTestingUtility();
|
||||
}
|
||||
util.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
|
||||
util.getConfiguration().setInt(
|
||||
StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 1000);
|
||||
// Make sure there are three servers.
|
||||
util.initializeCluster(3);
|
||||
ZooKeeperWatcher zkw = util.getZooKeeperWatcher();
|
||||
Configuration conf = util.getConfiguration();
|
||||
String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
|
||||
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
|
||||
String primaryMetaZnode = ZKUtil.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);
|
||||
ServerName.parseFrom(data);
|
||||
waitUntilZnodeAvailable(1);
|
||||
waitUntilZnodeAvailable(2);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardown() throws Exception {
|
||||
//Clean everything up.
|
||||
util.restoreCluster();
|
||||
util = null;
|
||||
}
|
||||
|
||||
private static void waitUntilZnodeAvailable(int replicaId) throws Exception {
|
||||
String znode = util.getZooKeeperWatcher().getZNodeForReplica(replicaId);
|
||||
int i = 0;
|
||||
while (i < 1000) {
|
||||
if (ZKUtil.checkExists(util.getZooKeeperWatcher(), znode) == -1) {
|
||||
Thread.sleep(100);
|
||||
i++;
|
||||
} else break;
|
||||
}
|
||||
if (i == 1000) throw new IOException("znode for meta replica " + replicaId + " not available");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testShutdownHandling() throws Exception {
|
||||
// This test creates a table, flushes the meta (with 3 replicas), kills the
|
||||
// server holding the primary meta replica. Then it does a put/get into/from
|
||||
// the test table. The put/get operations would use the replicas to locate the
|
||||
// location of the test table's region
|
||||
TestMetaWithReplicas.shutdownMetaAndDoValidations(util);
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
setUp();
|
||||
new IntegrationTestMetaReplicas().testShutdownHandling();
|
||||
teardown();
|
||||
}
|
||||
}
|
|
@ -2633,18 +2633,19 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
/**
|
||||
* Assigns the hbase:meta region.
|
||||
* Assigns the hbase:meta region or a replica.
|
||||
* <p>
|
||||
* Assumes that hbase:meta is currently closed and is not being actively served by
|
||||
* any RegionServer.
|
||||
* <p>
|
||||
* Forcibly unsets the current meta region location in ZooKeeper and assigns
|
||||
* hbase:meta to a random RegionServer.
|
||||
* @param hri TODO
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public void assignMeta() throws KeeperException {
|
||||
this.server.getMetaTableLocator().deleteMetaLocation(this.watcher);
|
||||
assign(HRegionInfo.FIRST_META_REGIONINFO, true);
|
||||
public void assignMeta(HRegionInfo hri) throws KeeperException {
|
||||
this.server.getMetaTableLocator().deleteMetaLocation(this.watcher, hri.getReplicaId());
|
||||
assign(hri, true);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -3273,6 +3274,15 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
return isCarryingRegion(serverName, HRegionInfo.FIRST_META_REGIONINFO);
|
||||
}
|
||||
|
||||
public boolean isCarryingMetaReplica(ServerName serverName, int replicaId) {
|
||||
return isCarryingRegion(serverName,
|
||||
RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, replicaId));
|
||||
}
|
||||
|
||||
public boolean isCarryingMetaReplica(ServerName serverName, HRegionInfo metaHri) {
|
||||
return isCarryingRegion(serverName, metaHri);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if the shutdown server carries the specific region.
|
||||
* We have a bunch of places that store region location
|
||||
|
|
|
@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
|||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MasterNotRunningException;
|
||||
import org.apache.hadoop.hbase.MetaMigrationConvertingToPB;
|
||||
|
@ -64,6 +65,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
|
|||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.NamespaceNotFoundException;
|
||||
import org.apache.hadoop.hbase.PleaseHoldException;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerLoad;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
|
@ -76,6 +78,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.client.MetaScanner;
|
||||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
||||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -107,6 +110,7 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
|||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
|
||||
import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
|
@ -122,6 +126,7 @@ import org.apache.hadoop.hbase.util.CompressionTest;
|
|||
import org.apache.hadoop.hbase.util.ConfigUtil;
|
||||
import org.apache.hadoop.hbase.util.EncryptionTest;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.HBaseFsckRepair;
|
||||
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
|
||||
import org.apache.hadoop.hbase.util.HasThread;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
@ -337,6 +342,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
LOG.info("hbase.rootdir=" + FSUtils.getRootDir(this.conf) +
|
||||
", hbase.cluster.distributed=" + this.conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, false));
|
||||
|
||||
// Disable usage of meta replicas in the master
|
||||
this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
|
||||
|
||||
Replication.decorateMasterConfiguration(this.conf);
|
||||
|
||||
// Hack! Maps DFSClient => Master for logs. HDFS made this
|
||||
|
@ -592,7 +600,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
|
||||
// enable table descriptors cache
|
||||
this.tableDescriptors.setCacheOn();
|
||||
|
||||
// set the META's descriptor to the correct replication
|
||||
this.tableDescriptors.get(TableName.META_TABLE_NAME).setRegionReplication(
|
||||
conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM));
|
||||
// warm-up HTDs cache on master initialization
|
||||
if (preLoadTableDescriptors) {
|
||||
status.setStatus("Pre-loading table descriptors");
|
||||
|
@ -679,7 +689,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
|
||||
// Make sure meta assigned before proceeding.
|
||||
status.setStatus("Assigning Meta Region");
|
||||
assignMeta(status, previouslyFailedMetaRSs);
|
||||
assignMeta(status, previouslyFailedMetaRSs, HRegionInfo.DEFAULT_REPLICA_ID);
|
||||
// check if master is shutting down because above assignMeta could return even hbase:meta isn't
|
||||
// assigned when master is shutting down
|
||||
if(isStopped()) return;
|
||||
|
@ -730,6 +740,16 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
LOG.info("Master has completed initialization");
|
||||
configurationManager.registerObserver(this.balancer);
|
||||
initialized = true;
|
||||
|
||||
// assign the meta replicas
|
||||
Set<ServerName> EMPTY_SET = new HashSet<ServerName>();
|
||||
int numReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
|
||||
HConstants.DEFAULT_META_REPLICA_NUM);
|
||||
for (int i = 1; i < numReplicas; i++) {
|
||||
assignMeta(status, EMPTY_SET, i);
|
||||
}
|
||||
unassignExcessMetaReplica(zooKeeper, numReplicas);
|
||||
|
||||
// clear the dead servers with same host name and port of online server because we are not
|
||||
// removing dead server with same hostname and port of rs which is trying to check in before
|
||||
// master initialization. See HBASE-5916.
|
||||
|
@ -763,34 +783,64 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
return new ServerManager(master, services);
|
||||
}
|
||||
|
||||
private void unassignExcessMetaReplica(ZooKeeperWatcher zkw, int numMetaReplicasConfigured) {
|
||||
// unassign the unneeded replicas (for e.g., if the previous master was configured
|
||||
// with a replication of 3 and now it is 2, we need to unassign the 1 unneeded replica)
|
||||
try {
|
||||
List<String> metaReplicaZnodes = zooKeeper.getMetaReplicaNodes();
|
||||
for (String metaReplicaZnode : metaReplicaZnodes) {
|
||||
int replicaId = zooKeeper.getMetaReplicaIdFromZnode(metaReplicaZnode);
|
||||
if (replicaId >= numMetaReplicasConfigured) {
|
||||
RegionState r = MetaTableLocator.getMetaRegionState(zkw, replicaId);
|
||||
LOG.info("Closing excess replica of meta region " + r.getRegion());
|
||||
// send a close and wait for a max of 30 seconds
|
||||
ServerManager.closeRegionSilentlyAndWait(getConnection(), r.getServerName(),
|
||||
r.getRegion(), 30000);
|
||||
ZKUtil.deleteNode(zkw, zkw.getZNodeForReplica(replicaId));
|
||||
}
|
||||
}
|
||||
} catch (Exception ex) {
|
||||
// ignore the exception since we don't want the master to be wedged due to potential
|
||||
// issues in the cleanup of the extra regions. We can do that cleanup via hbck or manually
|
||||
LOG.warn("Ignoring exception " + ex);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check <code>hbase:meta</code> is assigned. If not, assign it.
|
||||
* @param status MonitoredTask
|
||||
* @param previouslyFailedMetaRSs
|
||||
* @param replicaId
|
||||
* @throws InterruptedException
|
||||
* @throws IOException
|
||||
* @throws KeeperException
|
||||
*/
|
||||
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMetaRSs)
|
||||
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMetaRSs, int replicaId)
|
||||
throws InterruptedException, IOException, KeeperException {
|
||||
// Work on meta region
|
||||
int assigned = 0;
|
||||
long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
|
||||
status.setStatus("Assigning hbase:meta region");
|
||||
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
|
||||
status.setStatus("Assigning hbase:meta region");
|
||||
} else {
|
||||
status.setStatus("Assigning hbase:meta region, replicaId " + replicaId);
|
||||
}
|
||||
// Get current meta state from zk.
|
||||
RegionStates regionStates = assignmentManager.getRegionStates();
|
||||
RegionState metaState = MetaTableLocator.getMetaRegionState(getZooKeeper());
|
||||
RegionState metaState = MetaTableLocator.getMetaRegionState(getZooKeeper(), replicaId);
|
||||
HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO,
|
||||
replicaId);
|
||||
ServerName currentMetaServer = metaState.getServerName();
|
||||
if (!ConfigUtil.useZKForAssignment(conf)) {
|
||||
regionStates.createRegionState(HRegionInfo.FIRST_META_REGIONINFO, metaState.getState(),
|
||||
regionStates.createRegionState(hri, metaState.getState(),
|
||||
currentMetaServer, null);
|
||||
} else {
|
||||
regionStates.createRegionState(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
regionStates.createRegionState(hri);
|
||||
}
|
||||
boolean rit = this.assignmentManager.
|
||||
processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
processRegionInTransitionAndBlockUntilAssigned(hri);
|
||||
boolean metaRegionLocation = metaTableLocator.verifyMetaRegionLocation(
|
||||
this.getConnection(), this.getZooKeeper(), timeout);
|
||||
this.getConnection(), this.getZooKeeper(), timeout, replicaId);
|
||||
if (!metaRegionLocation || !metaState.isOpened()) {
|
||||
// Meta location is not verified. It should be in transition, or offline.
|
||||
// We will wait for it to be assigned in enableSSHandWaitForMeta below.
|
||||
|
@ -811,10 +861,12 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
LOG.info("Forcing expire of " + currentMetaServer);
|
||||
serverManager.expireServer(currentMetaServer);
|
||||
}
|
||||
splitMetaLogBeforeAssignment(currentMetaServer);
|
||||
previouslyFailedMetaRSs.add(currentMetaServer);
|
||||
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
|
||||
splitMetaLogBeforeAssignment(currentMetaServer);
|
||||
previouslyFailedMetaRSs.add(currentMetaServer);
|
||||
}
|
||||
}
|
||||
assignmentManager.assignMeta();
|
||||
assignmentManager.assignMeta(hri);
|
||||
}
|
||||
} else {
|
||||
// Region already assigned. We didn't assign it. Add to in-memory state.
|
||||
|
@ -824,7 +876,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
HRegionInfo.FIRST_META_REGIONINFO, currentMetaServer);
|
||||
}
|
||||
|
||||
enableMeta(TableName.META_TABLE_NAME);
|
||||
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) enableMeta(TableName.META_TABLE_NAME);
|
||||
|
||||
if ((RecoveryMode.LOG_REPLAY == this.getMasterFileSystem().getLogRecoveryMode())
|
||||
&& (!previouslyFailedMetaRSs.isEmpty())) {
|
||||
|
@ -837,10 +889,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
// if the meta region server is died at this time, we need it to be re-assigned
|
||||
// by SSH so that system tables can be assigned.
|
||||
// No need to wait for meta is assigned = 0 when meta is just verified.
|
||||
enableServerShutdownHandler(assigned != 0);
|
||||
|
||||
LOG.info("hbase:meta assigned=" + assigned + ", rit=" + rit +
|
||||
", location=" + metaTableLocator.getMetaRegionLocation(this.getZooKeeper()));
|
||||
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) enableServerShutdownHandler(assigned != 0);
|
||||
LOG.info("hbase:meta with replicaId " + replicaId + " assigned=" + assigned + ", rit=" + rit +
|
||||
", location=" + metaTableLocator.getMetaRegionLocation(this.getZooKeeper(), replicaId));
|
||||
status.setStatus("META assigned.");
|
||||
}
|
||||
|
||||
|
@ -852,13 +903,15 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
assignmentManager.processRegionInTransitionZkLess();
|
||||
} else {
|
||||
if (currentServer != null) {
|
||||
splitMetaLogBeforeAssignment(currentServer);
|
||||
regionStates.logSplit(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
previouslyFailedRs.add(currentServer);
|
||||
if (regionState.getRegion().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
|
||||
splitMetaLogBeforeAssignment(currentServer);
|
||||
regionStates.logSplit(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
previouslyFailedRs.add(currentServer);
|
||||
}
|
||||
}
|
||||
LOG.info("Re-assigning hbase:meta, it was on " + currentServer);
|
||||
regionStates.updateRegionState(HRegionInfo.FIRST_META_REGIONINFO, State.OFFLINE);
|
||||
assignmentManager.assignMeta();
|
||||
regionStates.updateRegionState(regionState.getRegion(), State.OFFLINE);
|
||||
assignmentManager.assignMeta(regionState.getRegion());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -176,19 +176,19 @@ public class RegionStateStore {
|
|||
|
||||
HRegionInfo hri = newState.getRegion();
|
||||
try {
|
||||
// update meta before checking for initialization.
|
||||
// meta state stored in zk.
|
||||
if (hri.isMetaRegion()) {
|
||||
// persist meta state in MetaTableLocator (which in turn is zk storage currently)
|
||||
try {
|
||||
MetaTableLocator.setMetaLocation(server.getZooKeeper(),
|
||||
newState.getServerName(), newState.getState());
|
||||
return; // Done
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Failed to update meta ZNode", e);
|
||||
// update meta before checking for initialization.
|
||||
// meta state stored in zk.
|
||||
if (hri.isMetaRegion()) {
|
||||
// persist meta state in MetaTableLocator (which in turn is zk storage currently)
|
||||
try {
|
||||
MetaTableLocator.setMetaLocation(server.getZooKeeper(),
|
||||
newState.getServerName(), hri.getReplicaId(), newState.getState());
|
||||
return; // Done
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Failed to update meta ZNode", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
if (!initialized || !shouldPersistStateChange(hri, newState, oldState)) {
|
||||
return;
|
||||
}
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.ClockOutOfSyncException;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.RegionLoad;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerLoad;
|
||||
|
@ -776,6 +777,35 @@ public class ServerManager {
|
|||
return sendRegionClose(server, region, versionOfClosingNode, null, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Contacts a region server and waits up to timeout ms
|
||||
* to close the region. This bypasses the active hmaster.
|
||||
*/
|
||||
public static void closeRegionSilentlyAndWait(ClusterConnection connection,
|
||||
ServerName server, HRegionInfo region, long timeout) throws IOException, InterruptedException {
|
||||
AdminService.BlockingInterface rs = connection.getAdmin(server);
|
||||
try {
|
||||
ProtobufUtil.closeRegion(rs, server, region.getRegionName(), false);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Exception when closing region: " + region.getRegionNameAsString(), e);
|
||||
}
|
||||
long expiration = timeout + System.currentTimeMillis();
|
||||
while (System.currentTimeMillis() < expiration) {
|
||||
try {
|
||||
HRegionInfo rsRegion =
|
||||
ProtobufUtil.getRegionInfo(rs, region.getRegionName());
|
||||
if (rsRegion == null) return;
|
||||
} catch (IOException ioe) {
|
||||
if (ioe instanceof NotServingRegionException) // no need to retry again
|
||||
return;
|
||||
LOG.warn("Exception when retrieving regioninfo from: " + region.getRegionNameAsString(), ioe);
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
throw new IOException("Region " + region + " failed to close within"
|
||||
+ " timeout " + timeout);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends an MERGE REGIONS RPC to the specified server to merge the specified
|
||||
* regions.
|
||||
|
|
|
@ -842,10 +842,11 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
|
|||
|
||||
/**
|
||||
* Check if a region belongs to some small system table.
|
||||
* If so, it may be expected to be put on the master regionserver.
|
||||
* If so, the primary replica may be expected to be put on the master regionserver.
|
||||
*/
|
||||
public boolean shouldBeOnMaster(HRegionInfo region) {
|
||||
return tablesOnMaster.contains(region.getTable().getNameAsString());
|
||||
return tablesOnMaster.contains(region.getTable().getNameAsString())
|
||||
&& region.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -156,7 +156,7 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
|
|||
getLong("hbase.catalog.verification.timeout", 1000);
|
||||
if (!server.getMetaTableLocator().verifyMetaRegionLocation(server.getConnection(),
|
||||
this.server.getZooKeeper(), timeout)) {
|
||||
this.services.getAssignmentManager().assignMeta();
|
||||
this.services.getAssignmentManager().assignMeta(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
} else if (serverName.equals(server.getMetaTableLocator().getMetaRegionLocation(
|
||||
this.server.getZooKeeper()))) {
|
||||
throw new IOException("hbase:meta is onlined on the dead server "
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
|
|||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
|
@ -218,7 +219,17 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
} catch (IOException ioe) {
|
||||
resubmit(serverName, ioe);
|
||||
}
|
||||
|
||||
List<HRegionInfo> toAssignRegions = new ArrayList<HRegionInfo>();
|
||||
int replicaCount = services.getConfiguration().getInt(HConstants.META_REPLICAS_NUM,
|
||||
HConstants.DEFAULT_META_REPLICA_NUM);
|
||||
for (int i = 1; i < replicaCount; i++) {
|
||||
HRegionInfo metaHri =
|
||||
RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, i);
|
||||
if (am.isCarryingMetaReplica(serverName, metaHri)) {
|
||||
LOG.info("Reassigning meta replica" + metaHri + " that was on " + serverName);
|
||||
toAssignRegions.add(metaHri);
|
||||
}
|
||||
}
|
||||
// Clean out anything in regions in transition. Being conservative and
|
||||
// doing after log splitting. Could do some states before -- OPENING?
|
||||
// OFFLINE? -- and then others after like CLOSING that depend on log
|
||||
|
@ -228,8 +239,7 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
" region(s) that " + (serverName == null? "null": serverName) +
|
||||
" was carrying (and " + regionsInTransition.size() +
|
||||
" regions(s) that were opening on this server)");
|
||||
|
||||
List<HRegionInfo> toAssignRegions = new ArrayList<HRegionInfo>();
|
||||
|
||||
toAssignRegions.addAll(regionsInTransition);
|
||||
|
||||
// Iterate regions that were on this server and assign them
|
||||
|
|
|
@ -1172,6 +1172,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
"hbase.regionserver.optionalcacheflushinterval";
|
||||
/** Default interval for the memstore flush */
|
||||
public static final int DEFAULT_CACHE_FLUSH_INTERVAL = 3600000;
|
||||
public static final int META_CACHE_FLUSH_INTERVAL = 300000; // 5 minutes
|
||||
|
||||
/** Conf key to force a flush if there are already enough changes for one region in memstore */
|
||||
public static final String MEMSTORE_FLUSH_PER_CHANGES =
|
||||
|
@ -1797,18 +1798,23 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
&& (this.maxFlushedSeqId + this.flushPerChanges < this.sequenceId.get())) {
|
||||
return true;
|
||||
}
|
||||
if (flushCheckInterval <= 0) { //disabled
|
||||
long modifiedFlushCheckInterval = flushCheckInterval;
|
||||
if (getRegionInfo().isMetaRegion() &&
|
||||
getRegionInfo().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
|
||||
modifiedFlushCheckInterval = META_CACHE_FLUSH_INTERVAL;
|
||||
}
|
||||
if (modifiedFlushCheckInterval <= 0) { //disabled
|
||||
return false;
|
||||
}
|
||||
long now = EnvironmentEdgeManager.currentTime();
|
||||
//if we flushed in the recent past, we don't need to do again now
|
||||
if ((now - getEarliestFlushTimeForAllStores() < flushCheckInterval)) {
|
||||
if ((now - getEarliestFlushTimeForAllStores() < modifiedFlushCheckInterval)) {
|
||||
return false;
|
||||
}
|
||||
//since we didn't flush in the recent past, flush now if certain conditions
|
||||
//are met. Return true on first such memstore hit.
|
||||
for (Store s : this.getStores().values()) {
|
||||
if (s.timeOfOldestEdit() < now - flushCheckInterval) {
|
||||
if (s.timeOfOldestEdit() < now - modifiedFlushCheckInterval) {
|
||||
// we have an old enough edit in the memstore, flush
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -466,6 +466,8 @@ public class HRegionServer extends HasThread implements
|
|||
checkCodecs(this.conf);
|
||||
this.userProvider = UserProvider.instantiate(conf);
|
||||
FSUtils.setupShortCircuitRead(this.conf);
|
||||
// Disable usage of meta replicas in the regionserver
|
||||
this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
|
||||
|
||||
// Config'ed params
|
||||
this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
|
||||
|
@ -783,11 +785,19 @@ public class HRegionServer extends HasThread implements
|
|||
rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress(
|
||||
rpcServices.isa.getAddress(), 0));
|
||||
|
||||
boolean onlyMetaRefresh = false;
|
||||
int storefileRefreshPeriod = conf.getInt(
|
||||
StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD
|
||||
, StorefileRefresherChore.DEFAULT_REGIONSERVER_STOREFILE_REFRESH_PERIOD);
|
||||
if (storefileRefreshPeriod == 0) {
|
||||
storefileRefreshPeriod = conf.getInt(
|
||||
StorefileRefresherChore.REGIONSERVER_META_STOREFILE_REFRESH_PERIOD,
|
||||
StorefileRefresherChore.DEFAULT_REGIONSERVER_STOREFILE_REFRESH_PERIOD);
|
||||
onlyMetaRefresh = true;
|
||||
}
|
||||
if (storefileRefreshPeriod > 0) {
|
||||
this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod, this, this);
|
||||
this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod,
|
||||
onlyMetaRefresh, this, this);
|
||||
}
|
||||
registerConfigurationObservers();
|
||||
}
|
||||
|
@ -1726,7 +1736,8 @@ public class HRegionServer extends HasThread implements
|
|||
WAL wal;
|
||||
LogRoller roller = walRoller;
|
||||
//_ROOT_ and hbase:meta regions have separate WAL.
|
||||
if (regionInfo != null && regionInfo.isMetaTable()) {
|
||||
if (regionInfo != null && regionInfo.isMetaTable() &&
|
||||
regionInfo.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
|
||||
roller = ensureMetaWALRoller();
|
||||
wal = walFactory.getMetaWAL(regionInfo.getEncodedNameAsBytes());
|
||||
} else if (regionInfo == null) {
|
||||
|
@ -1801,7 +1812,8 @@ public class HRegionServer extends HasThread implements
|
|||
|
||||
// Update ZK, or META
|
||||
if (r.getRegionInfo().isMetaRegion()) {
|
||||
MetaTableLocator.setMetaLocation(getZooKeeper(), serverName, State.OPEN);
|
||||
MetaTableLocator.setMetaLocation(getZooKeeper(), serverName, r.getRegionInfo().getReplicaId(),
|
||||
State.OPEN);
|
||||
} else if (useZKForAssignment) {
|
||||
MetaTableAccessor.updateRegionLocation(getConnection(), r.getRegionInfo(),
|
||||
this.serverName, openSeqNum);
|
||||
|
|
|
@ -53,19 +53,28 @@ public class StorefileRefresherChore extends Chore {
|
|||
= "hbase.regionserver.storefile.refresh.period";
|
||||
static final int DEFAULT_REGIONSERVER_STOREFILE_REFRESH_PERIOD = 0; //disabled by default
|
||||
|
||||
/**
|
||||
* Whether all storefiles should be refreshed, as opposed to just hbase:meta's
|
||||
* Meta region doesn't have WAL replication for replicas enabled yet
|
||||
*/
|
||||
public static final String REGIONSERVER_META_STOREFILE_REFRESH_PERIOD
|
||||
= "hbase.regionserver.meta.storefile.refresh.period";
|
||||
private HRegionServer regionServer;
|
||||
private long hfileTtl;
|
||||
private int period;
|
||||
private boolean onlyMetaRefresh = true;
|
||||
|
||||
//ts of last time regions store files are refreshed
|
||||
private Map<String, Long> lastRefreshTimes; // encodedName -> long
|
||||
|
||||
public StorefileRefresherChore(int period, HRegionServer regionServer, Stoppable stoppable) {
|
||||
public StorefileRefresherChore(int period, boolean onlyMetaRefresh, HRegionServer regionServer,
|
||||
Stoppable stoppable) {
|
||||
super("StorefileRefresherChore", period, stoppable);
|
||||
this.period = period;
|
||||
this.regionServer = regionServer;
|
||||
this.hfileTtl = this.regionServer.getConfiguration().getLong(
|
||||
TimeToLiveHFileCleaner.TTL_CONF_KEY, TimeToLiveHFileCleaner.DEFAULT_TTL);
|
||||
this.onlyMetaRefresh = onlyMetaRefresh;
|
||||
if (period > hfileTtl / 2) {
|
||||
throw new RuntimeException(REGIONSERVER_STOREFILE_REFRESH_PERIOD +
|
||||
" should be set smaller than half of " + TimeToLiveHFileCleaner.TTL_CONF_KEY);
|
||||
|
@ -80,6 +89,9 @@ public class StorefileRefresherChore extends Chore {
|
|||
// skip checking for this region if it can accept writes
|
||||
continue;
|
||||
}
|
||||
// don't refresh unless enabled for all files, or it the meta region
|
||||
// meta region don't have WAL replication for replicas enabled yet
|
||||
if (onlyMetaRefresh && !r.getRegionInfo().isMetaTable()) continue;
|
||||
String encodedName = r.getRegionInfo().getEncodedName();
|
||||
long time = EnvironmentEdgeManager.currentTime();
|
||||
if (!lastRefreshTimes.containsKey(encodedName)) {
|
||||
|
|
|
@ -124,6 +124,7 @@ import org.apache.hadoop.hbase.util.hbck.TableLockChecker;
|
|||
import org.apache.hadoop.hbase.wal.WALSplitter;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
|
@ -305,6 +306,8 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
setConf(HBaseConfiguration.create(getConf()));
|
||||
// disable blockcache for tool invocation, see HBASE-10500
|
||||
getConf().setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0);
|
||||
// Disable usage of meta replicas in hbck
|
||||
getConf().setBoolean(HConstants.USE_META_REPLICAS, false);
|
||||
errors = getErrorReporter(conf);
|
||||
|
||||
int numThreads = conf.getInt("hbasefsck.numthreads", MAX_NUM_THREADS);
|
||||
|
@ -1601,28 +1604,29 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
* Record the location of the hbase:meta region as found in ZooKeeper.
|
||||
*/
|
||||
private boolean recordMetaRegion() throws IOException {
|
||||
HRegionLocation metaLocation = connection.locateRegion(
|
||||
TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW);
|
||||
|
||||
// Check if Meta region is valid and existing
|
||||
if (metaLocation == null || metaLocation.getRegionInfo() == null ||
|
||||
metaLocation.getHostname() == null) {
|
||||
RegionLocations rl = ((ClusterConnection)connection).locateRegion(TableName.META_TABLE_NAME,
|
||||
HConstants.EMPTY_START_ROW, false, false);
|
||||
if (rl == null) {
|
||||
errors.reportError(ERROR_CODE.NULL_META_REGION,
|
||||
"META region or some of its attributes are null.");
|
||||
"META region or some of its attributes are null.");
|
||||
return false;
|
||||
}
|
||||
ServerName sn;
|
||||
try {
|
||||
sn = getMetaRegionServerName();
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
MetaEntry m = new MetaEntry(metaLocation.getRegionInfo(), sn, System.currentTimeMillis());
|
||||
HbckInfo hbckInfo = regionInfoMap.get(metaLocation.getRegionInfo().getEncodedName());
|
||||
if (hbckInfo == null) {
|
||||
regionInfoMap.put(metaLocation.getRegionInfo().getEncodedName(), new HbckInfo(m));
|
||||
} else {
|
||||
hbckInfo.metaEntry = m;
|
||||
for (HRegionLocation metaLocation : rl.getRegionLocations()) {
|
||||
// Check if Meta region is valid and existing
|
||||
if (metaLocation == null || metaLocation.getRegionInfo() == null ||
|
||||
metaLocation.getHostname() == null) {
|
||||
errors.reportError(ERROR_CODE.NULL_META_REGION,
|
||||
"META region or some of its attributes are null.");
|
||||
return false;
|
||||
}
|
||||
ServerName sn = metaLocation.getServerName();
|
||||
MetaEntry m = new MetaEntry(metaLocation.getRegionInfo(), sn, System.currentTimeMillis());
|
||||
HbckInfo hbckInfo = regionInfoMap.get(metaLocation.getRegionInfo().getEncodedName());
|
||||
if (hbckInfo == null) {
|
||||
regionInfoMap.put(metaLocation.getRegionInfo().getEncodedName(), new HbckInfo(m));
|
||||
} else {
|
||||
hbckInfo.metaEntry = m;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
@ -1643,12 +1647,12 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
});
|
||||
}
|
||||
|
||||
private ServerName getMetaRegionServerName()
|
||||
private ServerName getMetaRegionServerName(int replicaId)
|
||||
throws IOException, KeeperException {
|
||||
ZooKeeperWatcher zkw = createZooKeeperWatcher();
|
||||
ServerName sn = null;
|
||||
try {
|
||||
sn = new MetaTableLocator().getMetaRegionLocation(zkw);
|
||||
sn = new MetaTableLocator().getMetaRegionLocation(zkw, replicaId);
|
||||
} finally {
|
||||
zkw.close();
|
||||
}
|
||||
|
@ -3021,55 +3025,83 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
* If there are inconsistencies (i.e. zero or more than one regions
|
||||
* pretend to be holding the hbase:meta) try to fix that and report an error.
|
||||
* @throws IOException from HBaseFsckRepair functions
|
||||
* @throws KeeperException
|
||||
* @throws InterruptedException
|
||||
* @throws KeeperException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
boolean checkMetaRegion() throws IOException, KeeperException, InterruptedException {
|
||||
List<HbckInfo> metaRegions = Lists.newArrayList();
|
||||
Map<Integer, HbckInfo> metaRegions = new HashMap<Integer, HbckInfo>();
|
||||
for (HbckInfo value : regionInfoMap.values()) {
|
||||
if (value.metaEntry != null && value.metaEntry.isMetaRegion()) {
|
||||
metaRegions.add(value);
|
||||
metaRegions.put(value.getReplicaId(), value);
|
||||
}
|
||||
}
|
||||
|
||||
// There will be always one entry in regionInfoMap corresponding to hbase:meta
|
||||
// Check the deployed servers. It should be exactly one server.
|
||||
List<ServerName> servers = new ArrayList<ServerName>();
|
||||
HbckInfo metaHbckInfo = null;
|
||||
if (!metaRegions.isEmpty()) {
|
||||
metaHbckInfo = metaRegions.get(0);
|
||||
servers = metaHbckInfo.deployedOn;
|
||||
}
|
||||
if (servers.size() != 1) {
|
||||
if (servers.size() == 0) {
|
||||
errors.reportError(ERROR_CODE.NO_META_REGION, "hbase:meta is not found on any region.");
|
||||
if (shouldFixAssignments()) {
|
||||
errors.print("Trying to fix a problem with hbase:meta..");
|
||||
setShouldRerun();
|
||||
// try to fix it (treat it as unassigned region)
|
||||
HBaseFsckRepair.fixUnassigned(admin, HRegionInfo.FIRST_META_REGIONINFO);
|
||||
HBaseFsckRepair.waitUntilAssigned(admin, HRegionInfo.FIRST_META_REGIONINFO);
|
||||
}
|
||||
} else if (servers.size() > 1) {
|
||||
errors
|
||||
.reportError(ERROR_CODE.MULTI_META_REGION, "hbase:meta is found on more than one region.");
|
||||
if (shouldFixAssignments()) {
|
||||
if (metaHbckInfo == null) {
|
||||
errors.print(
|
||||
"Unable to fix problem with hbase:meta due to hbase:meta region info missing");
|
||||
return false;
|
||||
int metaReplication = admin.getTableDescriptor(TableName.META_TABLE_NAME)
|
||||
.getRegionReplication();
|
||||
boolean noProblem = true;
|
||||
// There will be always entries in regionInfoMap corresponding to hbase:meta & its replicas
|
||||
// Check the deployed servers. It should be exactly one server for each replica.
|
||||
for (int i = 0; i < metaReplication; i++) {
|
||||
HbckInfo metaHbckInfo = metaRegions.remove(i);
|
||||
List<ServerName> servers = new ArrayList<ServerName>();
|
||||
if (metaHbckInfo != null) {
|
||||
servers = metaHbckInfo.deployedOn;
|
||||
}
|
||||
if (servers.size() != 1) {
|
||||
noProblem = false;
|
||||
if (servers.size() == 0) {
|
||||
assignMetaReplica(i);
|
||||
} else if (servers.size() > 1) {
|
||||
errors
|
||||
.reportError(ERROR_CODE.MULTI_META_REGION, "hbase:meta, replicaId " +
|
||||
metaHbckInfo.getReplicaId() + " is found on more than one region.");
|
||||
if (shouldFixAssignments()) {
|
||||
errors.print("Trying to fix a problem with hbase:meta, replicaId " +
|
||||
metaHbckInfo.getReplicaId() +"..");
|
||||
setShouldRerun();
|
||||
// try fix it (treat is a dupe assignment)
|
||||
HBaseFsckRepair.fixMultiAssignment(connection, metaHbckInfo.metaEntry, servers);
|
||||
}
|
||||
errors.print("Trying to fix a problem with hbase:meta..");
|
||||
setShouldRerun();
|
||||
// try fix it (treat is a dupe assignment)
|
||||
HBaseFsckRepair.fixMultiAssignment(connection, metaHbckInfo.metaEntry, servers);
|
||||
}
|
||||
}
|
||||
// rerun hbck with hopefully fixed META
|
||||
return false;
|
||||
}
|
||||
// no errors, so continue normally
|
||||
return true;
|
||||
// unassign whatever is remaining in metaRegions. They are excess replicas.
|
||||
for (Map.Entry<Integer, HbckInfo> entry : metaRegions.entrySet()) {
|
||||
noProblem = false;
|
||||
errors.reportError(ERROR_CODE.SHOULD_NOT_BE_DEPLOYED,
|
||||
"hbase:meta replicas are deployed in excess. Configured " + metaReplication +
|
||||
", deployed " + metaRegions.size());
|
||||
if (shouldFixAssignments()) {
|
||||
errors.print("Trying to undeploy excess replica, replicaId: " + entry.getKey() +
|
||||
" of hbase:meta..");
|
||||
setShouldRerun();
|
||||
unassignMetaReplica(entry.getValue());
|
||||
}
|
||||
}
|
||||
// if noProblem is false, rerun hbck with hopefully fixed META
|
||||
// if noProblem is true, no errors, so continue normally
|
||||
return noProblem;
|
||||
}
|
||||
|
||||
private void unassignMetaReplica(HbckInfo hi) throws IOException, InterruptedException,
|
||||
KeeperException {
|
||||
undeployRegions(hi);
|
||||
ZooKeeperWatcher zkw = createZooKeeperWatcher();
|
||||
ZKUtil.deleteNode(zkw, zkw.getZNodeForReplica(hi.metaEntry.getReplicaId()));
|
||||
}
|
||||
|
||||
private void assignMetaReplica(int replicaId)
|
||||
throws IOException, KeeperException, InterruptedException {
|
||||
errors.reportError(ERROR_CODE.NO_META_REGION, "hbase:meta, replicaId " +
|
||||
replicaId +" is not found on any region.");
|
||||
if (shouldFixAssignments()) {
|
||||
errors.print("Trying to fix a problem with hbase:meta..");
|
||||
setShouldRerun();
|
||||
// try to fix it (treat it as unassigned region)
|
||||
HRegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
|
||||
HRegionInfo.FIRST_META_REGIONINFO, replicaId);
|
||||
HBaseFsckRepair.fixUnassigned(admin, h);
|
||||
HBaseFsckRepair.waitUntilAssigned(admin, h);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
|
@ -44,8 +45,7 @@ import org.apache.hadoop.hbase.client.HTable;
|
|||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||
import org.apache.hadoop.hbase.master.ServerManager;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
|
@ -159,29 +159,10 @@ public class HBaseFsckRepair {
|
|||
@SuppressWarnings("deprecation")
|
||||
public static void closeRegionSilentlyAndWait(HConnection connection,
|
||||
ServerName server, HRegionInfo region) throws IOException, InterruptedException {
|
||||
AdminService.BlockingInterface rs = connection.getAdmin(server);
|
||||
try {
|
||||
ProtobufUtil.closeRegion(rs, server, region.getRegionName(), false);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Exception when closing region: " + region.getRegionNameAsString(), e);
|
||||
}
|
||||
long timeout = connection.getConfiguration()
|
||||
.getLong("hbase.hbck.close.timeout", 120000);
|
||||
long expiration = timeout + System.currentTimeMillis();
|
||||
while (System.currentTimeMillis() < expiration) {
|
||||
try {
|
||||
HRegionInfo rsRegion =
|
||||
ProtobufUtil.getRegionInfo(rs, region.getRegionName());
|
||||
if (rsRegion == null) return;
|
||||
} catch (IOException ioe) {
|
||||
if (ioe instanceof NotServingRegionException) // no need to retry again
|
||||
return;
|
||||
LOG.warn("Exception when retrieving regioninfo from: " + region.getRegionNameAsString(), ioe);
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
throw new IOException("Region " + region + " failed to close within"
|
||||
+ " timeout " + timeout);
|
||||
ServerManager.closeRegionSilentlyAndWait((ClusterConnection)connection, server,
|
||||
region, timeout);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -29,6 +29,7 @@
|
|||
import="org.apache.hadoop.hbase.ServerName"
|
||||
import="org.apache.hadoop.hbase.ServerLoad"
|
||||
import="org.apache.hadoop.hbase.RegionLoad"
|
||||
import="org.apache.hadoop.hbase.HConstants"
|
||||
import="org.apache.hadoop.hbase.master.HMaster"
|
||||
import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator"
|
||||
import="org.apache.hadoop.hbase.util.Bytes"
|
||||
|
@ -55,6 +56,8 @@
|
|||
ServerName rl = metaTableLocator.getMetaRegionLocation(master.getZooKeeper());
|
||||
boolean showFragmentation = conf.getBoolean("hbase.master.ui.fragmentation.enabled", false);
|
||||
boolean readOnly = conf.getBoolean("hbase.master.ui.readonly", false);
|
||||
int numMetaReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
|
||||
HConstants.DEFAULT_META_REPLICA_NUM);
|
||||
Map<String, Integer> frags = null;
|
||||
if (showFragmentation) {
|
||||
frags = FSUtils.getTableFragmentation(master);
|
||||
|
@ -204,11 +207,14 @@
|
|||
%>
|
||||
<%= tableHeader %>
|
||||
<%
|
||||
// NOTE: Presumes one meta region only.
|
||||
HRegionInfo meta = HRegionInfo.FIRST_META_REGIONINFO;
|
||||
ServerName metaLocation = metaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), 1);
|
||||
for (int i = 0; i < 1; i++) {
|
||||
String url = "//" + metaLocation.getHostname() + ":" + master.getRegionServerInfoPort(metaLocation) + "/";
|
||||
// NOTE: Presumes meta with one or more replicas
|
||||
for (int j = 0; j < numMetaReplicas; j++) {
|
||||
HRegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
|
||||
HRegionInfo.FIRST_META_REGIONINFO, j);
|
||||
ServerName metaLocation = metaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1);
|
||||
for (int i = 0; i < 1; i++) {
|
||||
String url = "//" + metaLocation.getHostname() + ":" +
|
||||
master.getRegionServerInfoPort(metaLocation) + "/";
|
||||
%>
|
||||
<tr>
|
||||
<td><%= escapeXml(meta.getRegionNameAsString()) %></td>
|
||||
|
@ -219,6 +225,7 @@
|
|||
<td>-</td>
|
||||
</tr>
|
||||
<% } %>
|
||||
<%} %>
|
||||
</table>
|
||||
<%} else {
|
||||
Admin admin = master.getConnection().getAdmin();
|
||||
|
|
|
@ -0,0 +1,394 @@
|
|||
/**
|
||||
*
|
||||
* 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 static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
|
||||
import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.HBaseFsck;
|
||||
import org.apache.hadoop.hbase.util.HBaseFsckRepair;
|
||||
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
|
||||
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.ZooKeeperWatcher;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Tests the scenarios where replicas are enabled for the meta table
|
||||
*/
|
||||
@Category(MediumTests.class)
|
||||
public class TestMetaWithReplicas {
|
||||
static final Log LOG = LogFactory.getLog(TestMetaWithReplicas.class);
|
||||
private final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setInt("zookeeper.session.timeout", 30000);
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
|
||||
TEST_UTIL.getConfiguration().setInt(
|
||||
StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 1000);
|
||||
TEST_UTIL.startMiniCluster(3);
|
||||
// disable the balancer
|
||||
LoadBalancerTracker l = new LoadBalancerTracker(TEST_UTIL.getZooKeeperWatcher(),
|
||||
new Abortable() {
|
||||
boolean aborted = false;
|
||||
@Override
|
||||
public boolean isAborted() {
|
||||
return aborted;
|
||||
}
|
||||
@Override
|
||||
public void abort(String why, Throwable e) {
|
||||
aborted = true;
|
||||
}
|
||||
});
|
||||
l.setBalancerOn(false);
|
||||
for (int replicaId = 1; replicaId < 3; replicaId ++) {
|
||||
HRegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO,
|
||||
replicaId);
|
||||
TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().waitForAssignment(h);
|
||||
}
|
||||
LOG.debug("All meta replicas assigned");
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetaHTDReplicaCount() throws Exception {
|
||||
assertTrue(TEST_UTIL.getHBaseAdmin().getTableDescriptor(TableName.META_TABLE_NAME)
|
||||
.getRegionReplication() == 3);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testZookeeperNodesForReplicas() throws Exception {
|
||||
// Checks all the znodes exist when meta's replicas are enabled
|
||||
ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
|
||||
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
|
||||
String primaryMetaZnode = ZKUtil.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);
|
||||
ServerName.parseFrom(data);
|
||||
for (int i = 1; i < 3; i++) {
|
||||
String secZnode = ZKUtil.joinZNode(baseZNode,
|
||||
conf.get("zookeeper.znode.metaserver", "meta-region-server") + "-" + i);
|
||||
String str = zkw.getZNodeForReplica(i);
|
||||
assertTrue(str.equals(secZnode));
|
||||
// check that the data in the znode is parseable (this would also mean the znode exists)
|
||||
data = ZKUtil.getData(zkw, secZnode);
|
||||
ServerName.parseFrom(data);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testShutdownHandling() throws Exception {
|
||||
// This test creates a table, flushes the meta (with 3 replicas), kills the
|
||||
// server holding the primary meta replica. Then it does a put/get into/from
|
||||
// the test table. The put/get operations would use the replicas to locate the
|
||||
// location of the test table's region
|
||||
shutdownMetaAndDoValidations(TEST_UTIL);
|
||||
}
|
||||
|
||||
public static void shutdownMetaAndDoValidations(HBaseTestingUtility util) throws Exception {
|
||||
// This test creates a table, flushes the meta (with 3 replicas), kills the
|
||||
// server holding the primary meta replica. Then it does a put/get into/from
|
||||
// the test table. The put/get operations would use the replicas to locate the
|
||||
// location of the test table's region
|
||||
ZooKeeperWatcher zkw = util.getZooKeeperWatcher();
|
||||
Configuration conf = util.getConfiguration();
|
||||
conf.setBoolean(HConstants.USE_META_REPLICAS, true);
|
||||
|
||||
String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
|
||||
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
|
||||
String primaryMetaZnode = ZKUtil.joinZNode(baseZNode,
|
||||
conf.get("zookeeper.znode.metaserver", "meta-region-server"));
|
||||
byte[] data = ZKUtil.getData(zkw, primaryMetaZnode);
|
||||
ServerName primary = ServerName.parseFrom(data);
|
||||
|
||||
byte[] TABLE = Bytes.toBytes("testShutdownHandling");
|
||||
byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
|
||||
if (util.getHBaseAdmin().tableExists(TABLE)) {
|
||||
util.getHBaseAdmin().disableTable(TABLE);
|
||||
util.getHBaseAdmin().deleteTable(TABLE);
|
||||
}
|
||||
Table htable = util.createTable(TABLE, FAMILIES, conf);
|
||||
|
||||
util.getHBaseAdmin().flush(TableName.META_TABLE_NAME);
|
||||
Thread.sleep(conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD,
|
||||
30000) * 6);
|
||||
Connection c = ConnectionFactory.createConnection(util.getConfiguration());
|
||||
List<HRegionInfo> regions = MetaTableAccessor.getTableRegions(zkw, c,
|
||||
TableName.valueOf(TABLE));
|
||||
HRegionLocation hrl = MetaTableAccessor.getRegionLocation(c, regions.get(0));
|
||||
// Ensure that the primary server for test table is not the same one as the primary
|
||||
// of the meta region since we will be killing the srv holding the meta's primary...
|
||||
// We want to be able to write to the test table even when the meta is not present ..
|
||||
// If the servers are the same, then move the test table's region out of the server
|
||||
// to another random server
|
||||
if (hrl.getServerName().equals(primary)) {
|
||||
util.getHBaseAdmin().move(hrl.getRegionInfo().getEncodedNameAsBytes(), null);
|
||||
// wait for the move to complete
|
||||
do {
|
||||
Thread.sleep(10);
|
||||
hrl = MetaTableAccessor.getRegionLocation(c, regions.get(0));
|
||||
} while (primary.equals(hrl.getServerName()));
|
||||
util.getHBaseAdmin().flush(TableName.META_TABLE_NAME);
|
||||
Thread.sleep(conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD,
|
||||
30000) * 3);
|
||||
}
|
||||
ServerName master = util.getHBaseClusterInterface().getClusterStatus().getMaster();
|
||||
// kill the master so that regionserver recovery is not triggered at all
|
||||
// for the meta server
|
||||
util.getHBaseClusterInterface().stopMaster(master);
|
||||
util.getHBaseClusterInterface().waitForMasterToStop(master, 60000);
|
||||
if (!master.equals(primary)) {
|
||||
util.getHBaseClusterInterface().killRegionServer(primary);
|
||||
util.getHBaseClusterInterface().waitForRegionServerToStop(primary, 60000);
|
||||
}
|
||||
((ClusterConnection)c).clearRegionCache();
|
||||
htable.close();
|
||||
htable = c.getTable(TableName.valueOf(TABLE));
|
||||
byte[] row = "test".getBytes();
|
||||
Put put = new Put(row);
|
||||
put.add("foo".getBytes(), row, row);
|
||||
BufferedMutator m = c.getBufferedMutator(TableName.valueOf(TABLE));
|
||||
m.mutate(put);
|
||||
m.flush();
|
||||
// Try to do a get of the row that was just put
|
||||
Get get = new Get(row);
|
||||
Result r = htable.get(get);
|
||||
assertTrue(Arrays.equals(r.getRow(), row));
|
||||
// now start back the killed servers and disable use of replicas. That would mean
|
||||
// calls go to the primary
|
||||
util.getHBaseClusterInterface().startMaster(master.getHostname(), 0);
|
||||
util.getHBaseClusterInterface().startRegionServer(primary.getHostname(), 0);
|
||||
util.getHBaseClusterInterface().waitForActiveAndReadyMaster();
|
||||
((ClusterConnection)c).clearRegionCache();
|
||||
htable.close();
|
||||
conf.setBoolean(HConstants.USE_META_REPLICAS, false);
|
||||
htable = c.getTable(TableName.valueOf(TABLE));
|
||||
r = htable.get(get);
|
||||
assertTrue(Arrays.equals(r.getRow(), row));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testChangingReplicaCount() throws Exception {
|
||||
// tests changing the replica count across master restarts
|
||||
// reduce the replica count from 3 to 2
|
||||
stopMasterAndValidateReplicaCount(3, 2);
|
||||
// increase the replica count from 2 to 3
|
||||
stopMasterAndValidateReplicaCount(2, 3);
|
||||
}
|
||||
|
||||
private void stopMasterAndValidateReplicaCount(int originalReplicaCount, int newReplicaCount)
|
||||
throws Exception {
|
||||
ServerName sn = TEST_UTIL.getHBaseClusterInterface().getClusterStatus().getMaster();
|
||||
TEST_UTIL.getHBaseClusterInterface().stopMaster(sn);
|
||||
TEST_UTIL.getHBaseClusterInterface().waitForMasterToStop(sn, 60000);
|
||||
List<String> metaZnodes = TEST_UTIL.getZooKeeperWatcher().getMetaReplicaNodes();
|
||||
assert(metaZnodes.size() == originalReplicaCount); //we should have what was configured before
|
||||
TEST_UTIL.getHBaseClusterInterface().getConf().setInt(HConstants.META_REPLICAS_NUM,
|
||||
newReplicaCount);
|
||||
TEST_UTIL.getHBaseClusterInterface().startMaster(sn.getHostname(), 0);
|
||||
TEST_UTIL.getHBaseClusterInterface().waitForActiveAndReadyMaster();
|
||||
int count = 0;
|
||||
do {
|
||||
metaZnodes = TEST_UTIL.getZooKeeperWatcher().getMetaReplicaNodes();
|
||||
Thread.sleep(10);
|
||||
count++;
|
||||
// wait for the count to be different from the originalReplicaCount. When the
|
||||
// replica count is reduced, that will happen when the master unassigns excess
|
||||
// replica, and deletes the excess znodes
|
||||
} while (metaZnodes.size() == originalReplicaCount && count < 1000);
|
||||
assert(metaZnodes.size() == newReplicaCount);
|
||||
// also check if hbck returns without errors
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM,
|
||||
newReplicaCount);
|
||||
HBaseFsck hbck = HbckTestingUtil.doFsck(TEST_UTIL.getConfiguration(), false);
|
||||
HbckTestingUtil.assertNoErrors(hbck);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHBaseFsckWithMetaReplicas() throws Exception {
|
||||
HBaseFsck hbck = HbckTestingUtil.doFsck(TEST_UTIL.getConfiguration(), false);
|
||||
HbckTestingUtil.assertNoErrors(hbck);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHBaseFsckWithFewerMetaReplicas() throws Exception {
|
||||
ClusterConnection c = (ClusterConnection)ConnectionFactory.createConnection(
|
||||
TEST_UTIL.getConfiguration());
|
||||
RegionLocations rl = c.locateRegion(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW,
|
||||
false, false);
|
||||
HBaseFsckRepair.closeRegionSilentlyAndWait(c,
|
||||
rl.getRegionLocation(1).getServerName(), rl.getRegionLocation(1).getRegionInfo());
|
||||
// check that problem exists
|
||||
HBaseFsck hbck = doFsck(TEST_UTIL.getConfiguration(), false);
|
||||
assertErrors(hbck, new ERROR_CODE[]{ERROR_CODE.UNKNOWN,ERROR_CODE.NO_META_REGION});
|
||||
// fix the problem
|
||||
hbck = doFsck(TEST_UTIL.getConfiguration(), true);
|
||||
// run hbck again to make sure we don't see any errors
|
||||
hbck = doFsck(TEST_UTIL.getConfiguration(), false);
|
||||
assertErrors(hbck, new ERROR_CODE[]{});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHBaseFsckWithFewerMetaReplicaZnodes() throws Exception {
|
||||
ClusterConnection c = (ClusterConnection)ConnectionFactory.createConnection(
|
||||
TEST_UTIL.getConfiguration());
|
||||
RegionLocations rl = c.locateRegion(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW,
|
||||
false, false);
|
||||
HBaseFsckRepair.closeRegionSilentlyAndWait(c,
|
||||
rl.getRegionLocation(2).getServerName(), rl.getRegionLocation(2).getRegionInfo());
|
||||
ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
|
||||
ZKUtil.deleteNode(zkw, zkw.getZNodeForReplica(2));
|
||||
// check that problem exists
|
||||
HBaseFsck hbck = doFsck(TEST_UTIL.getConfiguration(), false);
|
||||
assertErrors(hbck, new ERROR_CODE[]{ERROR_CODE.UNKNOWN,ERROR_CODE.NO_META_REGION});
|
||||
// fix the problem
|
||||
hbck = doFsck(TEST_UTIL.getConfiguration(), true);
|
||||
// run hbck again to make sure we don't see any errors
|
||||
hbck = doFsck(TEST_UTIL.getConfiguration(), false);
|
||||
assertErrors(hbck, new ERROR_CODE[]{});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAccessingUnknownTables() throws Exception {
|
||||
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
|
||||
conf.setBoolean(HConstants.USE_META_REPLICAS, true);
|
||||
Table table = TEST_UTIL.getConnection().getTable(TableName.valueOf("RandomTable"));
|
||||
Get get = new Get(Bytes.toBytes("foo"));
|
||||
try {
|
||||
table.get(get);
|
||||
} catch (TableNotFoundException t) {
|
||||
return;
|
||||
}
|
||||
fail("Expected TableNotFoundException");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetaAddressChange() throws Exception {
|
||||
// checks that even when the meta's location changes, the various
|
||||
// caches update themselves. Uses the master operations to test
|
||||
// this
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
|
||||
String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
|
||||
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
|
||||
String primaryMetaZnode = ZKUtil.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);
|
||||
ServerName currentServer = ServerName.parseFrom(data);
|
||||
Collection<ServerName> liveServers = TEST_UTIL.getHBaseAdmin().getClusterStatus().getServers();
|
||||
ServerName moveToServer = null;
|
||||
for (ServerName s : liveServers) {
|
||||
if (!currentServer.equals(s)) {
|
||||
moveToServer = s;
|
||||
}
|
||||
}
|
||||
assert(moveToServer != null);
|
||||
String tableName = "randomTable5678";
|
||||
TEST_UTIL.createTable(TableName.valueOf(tableName), "f");
|
||||
assertTrue(TEST_UTIL.getHBaseAdmin().tableExists(tableName));
|
||||
TEST_UTIL.getHBaseAdmin().move(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
|
||||
Bytes.toBytes(moveToServer.getServerName()));
|
||||
int i = 0;
|
||||
do {
|
||||
Thread.sleep(10);
|
||||
data = ZKUtil.getData(zkw, primaryMetaZnode);
|
||||
currentServer = ServerName.parseFrom(data);
|
||||
i++;
|
||||
} while (!moveToServer.equals(currentServer) && i < 1000); //wait for 10 seconds overall
|
||||
assert(i != 1000);
|
||||
TEST_UTIL.getHBaseAdmin().disableTable("randomTable5678");
|
||||
assertTrue(TEST_UTIL.getHBaseAdmin().isTableDisabled("randomTable5678"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testShutdownOfReplicaHolder() throws Exception {
|
||||
// checks that the when the server holding meta replica is shut down, the meta replica
|
||||
// can be recovered
|
||||
RegionLocations rl = ConnectionManager.getConnectionInternal(TEST_UTIL.getConfiguration()).
|
||||
locateRegion(TableName.META_TABLE_NAME, Bytes.toBytes(""), false, true);
|
||||
HRegionLocation hrl = rl.getRegionLocation(1);
|
||||
ServerName oldServer = hrl.getServerName();
|
||||
TEST_UTIL.getHBaseClusterInterface().killRegionServer(oldServer);
|
||||
int i = 0;
|
||||
do {
|
||||
LOG.debug("Waiting for the replica " + hrl.getRegionInfo() + " to come up");
|
||||
Thread.sleep(30000); //wait for the detection/recovery
|
||||
rl = ConnectionManager.getConnectionInternal(TEST_UTIL.getConfiguration()).
|
||||
locateRegion(TableName.META_TABLE_NAME, Bytes.toBytes(""), false, true);
|
||||
hrl = rl.getRegionLocation(1);
|
||||
i++;
|
||||
} while ((hrl == null || hrl.getServerName().equals(oldServer)) && i < 3);
|
||||
assertTrue(i != 3);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHBaseFsckWithExcessMetaReplicas() throws Exception {
|
||||
HBaseFsck hbck = new HBaseFsck(TEST_UTIL.getConfiguration());
|
||||
// Create a meta replica (this will be the 4th one) and assign it
|
||||
HRegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
|
||||
HRegionInfo.FIRST_META_REGIONINFO, 3);
|
||||
// create in-memory state otherwise master won't assign
|
||||
TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager()
|
||||
.getRegionStates().createRegionState(h);
|
||||
TEST_UTIL.getMiniHBaseCluster().getMaster().assignRegion(h);
|
||||
HBaseFsckRepair.waitUntilAssigned(TEST_UTIL.getHBaseAdmin(), h);
|
||||
// check that problem exists
|
||||
hbck = doFsck(TEST_UTIL.getConfiguration(), false);
|
||||
assertErrors(hbck, new ERROR_CODE[]{ERROR_CODE.UNKNOWN, ERROR_CODE.SHOULD_NOT_BE_DEPLOYED});
|
||||
// fix the problem
|
||||
hbck = doFsck(TEST_UTIL.getConfiguration(), true);
|
||||
// run hbck again to make sure we don't see any errors
|
||||
hbck = doFsck(TEST_UTIL.getConfiguration(), false);
|
||||
assertErrors(hbck, new ERROR_CODE[]{});
|
||||
}
|
||||
}
|
|
@ -267,8 +267,8 @@ public class TestMasterNoCluster {
|
|||
TESTUTIL.getConfiguration());
|
||||
HMaster master = new HMaster(conf, cp) {
|
||||
@Override
|
||||
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMeatRSs) {
|
||||
}
|
||||
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMeatRSs, int replicaId)
|
||||
{ }
|
||||
|
||||
@Override
|
||||
void initializeZKBasedSystemTrackers() throws IOException,
|
||||
|
|
|
@ -33,21 +33,26 @@ import junit.framework.TestCase;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeepDeletedCells;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueTestUtil;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdge;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
|
@ -927,6 +932,36 @@ public class TestDefaultMemStore extends TestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testShouldFlushMeta() throws Exception {
|
||||
// write an edit in the META and ensure the shouldFlush (that the periodic memstore
|
||||
// flusher invokes) returns true after META_CACHE_FLUSH_INTERVAL (even though
|
||||
// the MEMSTORE_PERIODIC_FLUSH_INTERVAL is set to a higher value)
|
||||
Configuration conf = new Configuration();
|
||||
conf.setInt(HRegion.MEMSTORE_PERIODIC_FLUSH_INTERVAL, HRegion.META_CACHE_FLUSH_INTERVAL * 10);
|
||||
HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU(conf);
|
||||
Path testDir = hbaseUtility.getDataTestDir();
|
||||
EnvironmentEdgeForMemstoreTest edge = new EnvironmentEdgeForMemstoreTest();
|
||||
EnvironmentEdgeManager.injectEdge(edge);
|
||||
edge.setCurrentTimeMillis(1234);
|
||||
WALFactory wFactory = new WALFactory(conf, null, "1234");
|
||||
HRegion meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO, testDir,
|
||||
conf, HTableDescriptor.metaTableDescriptor(conf),
|
||||
wFactory.getMetaWAL(HRegionInfo.FIRST_META_REGIONINFO.
|
||||
getEncodedNameAsBytes()));
|
||||
HRegionInfo hri = new HRegionInfo(TableName.valueOf("testShouldFlushMeta"),
|
||||
Bytes.toBytes("row_0200"), Bytes.toBytes("row_0300"));
|
||||
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("testShouldFlushMeta"));
|
||||
desc.addFamily(new HColumnDescriptor("foo".getBytes()));
|
||||
HRegion r =
|
||||
HRegion.createHRegion(hri, testDir, conf, desc,
|
||||
wFactory.getWAL(hri.getEncodedNameAsBytes()));
|
||||
HRegion.addRegionToMETA(meta, r);
|
||||
edge.setCurrentTimeMillis(1234 + 100);
|
||||
assertTrue(meta.shouldFlush() == false);
|
||||
edge.setCurrentTimeMillis(edge.currentTime() + HRegion.META_CACHE_FLUSH_INTERVAL + 1);
|
||||
assertTrue(meta.shouldFlush() == true);
|
||||
}
|
||||
|
||||
private class EnvironmentEdgeForMemstoreTest implements EnvironmentEdge {
|
||||
long t = 1234;
|
||||
@Override
|
||||
|
|
|
@ -144,7 +144,7 @@ public class TestStoreFileRefresherChore {
|
|||
boolean isStale = false;
|
||||
public StaleStorefileRefresherChore(int period, HRegionServer regionServer,
|
||||
Stoppable stoppable) {
|
||||
super(period, regionServer, stoppable);
|
||||
super(period, false, regionServer, stoppable);
|
||||
}
|
||||
@Override
|
||||
protected boolean isRegionStale(String encodedName, long time) {
|
||||
|
|
Loading…
Reference in New Issue