HBASE-11574. hbase:meta's regions can be replicated
This commit is contained in:
parent
0f6faaf5fc
commit
6b20a0f831
|
@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||||
import org.apache.hadoop.hbase.client.Result;
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||||
import org.apache.hadoop.hbase.master.RegionState;
|
import org.apache.hadoop.hbase.master.RegionState;
|
||||||
|
@ -219,12 +220,16 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
||||||
* first meta regions
|
* first meta regions
|
||||||
*/
|
*/
|
||||||
private HRegionInfo(long regionId, TableName tableName) {
|
private HRegionInfo(long regionId, TableName tableName) {
|
||||||
|
this(regionId, tableName, DEFAULT_REPLICA_ID);
|
||||||
|
}
|
||||||
|
|
||||||
|
public HRegionInfo(long regionId, TableName tableName, int replicaId) {
|
||||||
super();
|
super();
|
||||||
this.regionId = regionId;
|
this.regionId = regionId;
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
// Note: First Meta regions names are still in old format
|
this.replicaId = replicaId;
|
||||||
this.regionName = createRegionName(tableName, null,
|
// Note: First Meta region replicas names are in old format
|
||||||
regionId, false);
|
this.regionName = createRegionName(tableName, null, regionId, replicaId, false);
|
||||||
setHashCode();
|
setHashCode();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -914,7 +919,8 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
||||||
TableName tableName =
|
TableName tableName =
|
||||||
ProtobufUtil.toTableName(proto.getTableName());
|
ProtobufUtil.toTableName(proto.getTableName());
|
||||||
if (tableName.equals(TableName.META_TABLE_NAME)) {
|
if (tableName.equals(TableName.META_TABLE_NAME)) {
|
||||||
return FIRST_META_REGIONINFO;
|
return RegionReplicaUtil.getRegionInfoForReplica(FIRST_META_REGIONINFO,
|
||||||
|
proto.getReplicaId());
|
||||||
}
|
}
|
||||||
long regionId = proto.getRegionId();
|
long regionId = proto.getRegionId();
|
||||||
int replicaId = proto.hasReplicaId() ? proto.getReplicaId() : DEFAULT_REPLICA_ID;
|
int replicaId = proto.hasReplicaId() ? proto.getReplicaId() : DEFAULT_REPLICA_ID;
|
||||||
|
|
|
@ -1095,7 +1095,7 @@ public class MetaTableAccessor {
|
||||||
* Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
|
* Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
|
||||||
* does not add its daughter's as different rows, but adds information about the daughters
|
* does not add its daughter's as different rows, but adds information about the daughters
|
||||||
* in the same row as the parent. Use
|
* in the same row as the parent. Use
|
||||||
* {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName, int)
|
* {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName, int)}
|
||||||
* if you want to do that.
|
* if you want to do that.
|
||||||
* @param meta the Table for META
|
* @param meta the Table for META
|
||||||
* @param regionInfo region information
|
* @param regionInfo region information
|
||||||
|
@ -1117,7 +1117,7 @@ public class MetaTableAccessor {
|
||||||
* Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
|
* Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
|
||||||
* does not add its daughter's as different rows, but adds information about the daughters
|
* does not add its daughter's as different rows, but adds information about the daughters
|
||||||
* in the same row as the parent. Use
|
* in the same row as the parent. Use
|
||||||
* {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName, int)
|
* {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName, int)}
|
||||||
* if you want to do that.
|
* if you want to do that.
|
||||||
* @param connection connection we're using
|
* @param connection connection we're using
|
||||||
* @param regionInfo region information
|
* @param regionInfo region information
|
||||||
|
|
|
@ -551,6 +551,7 @@ final class ConnectionManager {
|
||||||
static class HConnectionImplementation implements ClusterConnection, Closeable {
|
static class HConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
static final Log LOG = LogFactory.getLog(HConnectionImplementation.class);
|
static final Log LOG = LogFactory.getLog(HConnectionImplementation.class);
|
||||||
private final long pause;
|
private final long pause;
|
||||||
|
private final boolean useMetaReplicas;
|
||||||
private final int numTries;
|
private final int numTries;
|
||||||
final int rpcTimeout;
|
final int rpcTimeout;
|
||||||
private NonceGenerator nonceGenerator = null;
|
private NonceGenerator nonceGenerator = null;
|
||||||
|
@ -674,6 +675,8 @@ final class ConnectionManager {
|
||||||
this.closed = false;
|
this.closed = false;
|
||||||
this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
|
this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
|
||||||
HConstants.DEFAULT_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.numTries = tableConfig.getRetriesNumber();
|
||||||
this.rpcTimeout = conf.getInt(
|
this.rpcTimeout = conf.getInt(
|
||||||
HConstants.HBASE_RPC_TIMEOUT_KEY,
|
HConstants.HBASE_RPC_TIMEOUT_KEY,
|
||||||
|
@ -1132,7 +1135,7 @@ final class ConnectionManager {
|
||||||
RegionLocations locations = null;
|
RegionLocations locations = null;
|
||||||
if (useCache) {
|
if (useCache) {
|
||||||
locations = getCachedLocation(tableName, metaCacheKey);
|
locations = getCachedLocation(tableName, metaCacheKey);
|
||||||
if (locations != null) {
|
if (locations != null && locations.getRegionLocation(replicaId) != null) {
|
||||||
return locations;
|
return locations;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1143,7 +1146,7 @@ final class ConnectionManager {
|
||||||
// same query while we were waiting on the lock.
|
// same query while we were waiting on the lock.
|
||||||
if (useCache) {
|
if (useCache) {
|
||||||
locations = getCachedLocation(tableName, metaCacheKey);
|
locations = getCachedLocation(tableName, metaCacheKey);
|
||||||
if (locations != null) {
|
if (locations != null && locations.getRegionLocation(replicaId) != null) {
|
||||||
return locations;
|
return locations;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1183,6 +1186,9 @@ final class ConnectionManager {
|
||||||
s.setStartRow(metaKey);
|
s.setStartRow(metaKey);
|
||||||
s.setSmall(true);
|
s.setSmall(true);
|
||||||
s.setCaching(1);
|
s.setCaching(1);
|
||||||
|
if (this.useMetaReplicas) {
|
||||||
|
s.setConsistency(Consistency.TIMELINE);
|
||||||
|
}
|
||||||
|
|
||||||
int localNumRetries = (retry ? numTries : 1);
|
int localNumRetries = (retry ? numTries : 1);
|
||||||
|
|
||||||
|
|
|
@ -421,9 +421,6 @@ public class HBaseAdmin implements Admin {
|
||||||
public HTableDescriptor getTableDescriptor(final TableName tableName)
|
public HTableDescriptor getTableDescriptor(final TableName tableName)
|
||||||
throws TableNotFoundException, IOException {
|
throws TableNotFoundException, IOException {
|
||||||
if (tableName == null) return null;
|
if (tableName == null) return null;
|
||||||
if (tableName.equals(TableName.META_TABLE_NAME)) {
|
|
||||||
return HTableDescriptor.META_TABLEDESC;
|
|
||||||
}
|
|
||||||
HTableDescriptor htd = executeCallable(new MasterCallable<HTableDescriptor>(getConnection()) {
|
HTableDescriptor htd = executeCallable(new MasterCallable<HTableDescriptor>(getConnection()) {
|
||||||
@Override
|
@Override
|
||||||
public HTableDescriptor call(int callTimeout) throws ServiceException {
|
public HTableDescriptor call(int callTimeout) throws ServiceException {
|
||||||
|
|
|
@ -49,6 +49,7 @@ class HConnectionKey {
|
||||||
HConstants.HBASE_META_SCANNER_CACHING,
|
HConstants.HBASE_META_SCANNER_CACHING,
|
||||||
HConstants.HBASE_CLIENT_INSTANCE_ID,
|
HConstants.HBASE_CLIENT_INSTANCE_ID,
|
||||||
HConstants.RPC_CODEC_CONF_KEY,
|
HConstants.RPC_CODEC_CONF_KEY,
|
||||||
|
HConstants.USE_META_REPLICAS,
|
||||||
RpcControllerFactory.CUSTOM_CONTROLLER_CONF_KEY};
|
RpcControllerFactory.CUSTOM_CONTROLLER_CONF_KEY};
|
||||||
|
|
||||||
private Map<String, String> properties;
|
private Map<String, String> properties;
|
||||||
|
|
|
@ -153,7 +153,9 @@ public final class MetaScanner {
|
||||||
try (Table metaTable = new HTable(TableName.META_TABLE_NAME, connection, null)) {
|
try (Table metaTable = new HTable(TableName.META_TABLE_NAME, connection, null)) {
|
||||||
if (row != null) {
|
if (row != null) {
|
||||||
// Scan starting at a particular row in a particular table
|
// 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) {
|
if (startRowResult == null) {
|
||||||
throw new TableNotFoundException("Cannot find row in " + metaTable.getName() +
|
throw new TableNotFoundException("Cannot find row in " + metaTable.getName() +
|
||||||
" for table: " + tableName + ", row=" + Bytes.toStringBinary(row));
|
" for table: " + tableName + ", row=" + Bytes.toStringBinary(row));
|
||||||
|
@ -177,6 +179,10 @@ public final class MetaScanner {
|
||||||
int scannerCaching = connection.getConfiguration()
|
int scannerCaching = connection.getConfiguration()
|
||||||
.getInt(HConstants.HBASE_META_SCANNER_CACHING,
|
.getInt(HConstants.HBASE_META_SCANNER_CACHING,
|
||||||
HConstants.DEFAULT_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) {
|
if (rowUpperLimit <= scannerCaching) {
|
||||||
scan.setSmall(true);
|
scan.setSmall(true);
|
||||||
}
|
}
|
||||||
|
@ -215,10 +221,13 @@ public final class MetaScanner {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private static Result getClosestRowOrBefore(final Table metaTable, final TableName userTableName,
|
private static Result getClosestRowOrBefore(final Table metaTable, final TableName userTableName,
|
||||||
final byte [] row)
|
final byte [] row, boolean useMetaReplicas)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
byte[] searchRow = HRegionInfo.createRegionName(userTableName, row, HConstants.NINES, false);
|
byte[] searchRow = HRegionInfo.createRegionName(userTableName, row, HConstants.NINES, false);
|
||||||
Scan scan = Scan.createGetClosestRowOrBeforeReverseScan(searchRow);
|
Scan scan = Scan.createGetClosestRowOrBeforeReverseScan(searchRow);
|
||||||
|
if (useMetaReplicas) {
|
||||||
|
scan.setConsistency(Consistency.TIMELINE);
|
||||||
|
}
|
||||||
try (ResultScanner resultScanner = metaTable.getScanner(scan)) {
|
try (ResultScanner resultScanner = metaTable.getScanner(scan)) {
|
||||||
return resultScanner.next();
|
return resultScanner.next();
|
||||||
}
|
}
|
||||||
|
|
|
@ -47,9 +47,13 @@ public class RegionReplicaUtil {
|
||||||
if (regionInfo.getReplicaId() == replicaId) {
|
if (regionInfo.getReplicaId() == replicaId) {
|
||||||
return regionInfo;
|
return regionInfo;
|
||||||
}
|
}
|
||||||
HRegionInfo replicaInfo = new HRegionInfo(regionInfo.getTable(), regionInfo.getStartKey(),
|
HRegionInfo replicaInfo;
|
||||||
regionInfo.getEndKey(), regionInfo.isSplit(), regionInfo.getRegionId(), replicaId);
|
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());
|
replicaInfo.setOffline(regionInfo.isOffline());
|
||||||
return replicaInfo;
|
return replicaInfo;
|
||||||
}
|
}
|
||||||
|
|
|
@ -272,7 +272,7 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
|
||||||
return 0; // not scheduling on other replicas for strong consistency
|
return 0; // not scheduling on other replicas for strong consistency
|
||||||
}
|
}
|
||||||
for (int id = min; id <= max; id++) {
|
for (int id = min; id <= max; id++) {
|
||||||
if (currentScannerCallable.getHRegionInfo().getReplicaId() == id) {
|
if (currentScannerCallable.id == id) {
|
||||||
continue; //this was already scheduled earlier
|
continue; //this was already scheduled earlier
|
||||||
}
|
}
|
||||||
ScannerCallable s = currentScannerCallable.getScannerCallableForReplica(id);
|
ScannerCallable s = currentScannerCallable.getScannerCallableForReplica(id);
|
||||||
|
|
|
@ -18,6 +18,7 @@
|
||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
@ -54,14 +55,32 @@ class ZooKeeperRegistry implements Registry {
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Looking up meta region location in ZK," + " connection=" + this);
|
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()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Looked up meta region location, connection=" + this +
|
if (servers == null) {
|
||||||
"; serverName=" + ((servername == null) ? "null" : servername));
|
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;
|
if (servers == null) return null;
|
||||||
HRegionLocation loc = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, servername, 0);
|
HRegionLocation[] locs = new HRegionLocation[servers.size()];
|
||||||
return new RegionLocations(new HRegionLocation[] {loc});
|
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) {
|
} catch (InterruptedException e) {
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
return null;
|
return null;
|
||||||
|
|
|
@ -30,12 +30,14 @@ import java.util.List;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.HConnection;
|
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.client.RetriesExhaustedException;
|
||||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||||
import org.apache.hadoop.hbase.ipc.FailedServerException;
|
import org.apache.hadoop.hbase.ipc.FailedServerException;
|
||||||
|
@ -75,9 +77,6 @@ import com.google.protobuf.InvalidProtocolBufferException;
|
||||||
public class MetaTableLocator {
|
public class MetaTableLocator {
|
||||||
private static final Log LOG = LogFactory.getLog(MetaTableLocator.class);
|
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
|
// only needed to allow non-timeout infinite waits to stop when cluster shuts down
|
||||||
private volatile boolean stopped = false;
|
private volatile boolean stopped = false;
|
||||||
|
|
||||||
|
@ -94,9 +93,21 @@ public class MetaTableLocator {
|
||||||
* @return meta table regions and their locations.
|
* @return meta table regions and their locations.
|
||||||
*/
|
*/
|
||||||
public List<Pair<HRegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw) {
|
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<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;
|
return list;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -105,8 +116,18 @@ public class MetaTableLocator {
|
||||||
* @return List of meta regions
|
* @return List of meta regions
|
||||||
*/
|
*/
|
||||||
public List<HRegionInfo> getMetaRegions(ZooKeeperWatcher zkw) {
|
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;
|
List<Pair<HRegionInfo, ServerName>> result;
|
||||||
result = getMetaRegionsAndLocations(zkw);
|
result = getMetaRegionsAndLocations(zkw, replicaId);
|
||||||
return getListOfHRegionInfos(result);
|
return getListOfHRegionInfos(result);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -134,17 +155,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
|
* Gets the meta region location, if available, and waits for up to the
|
||||||
* specified timeout if not immediately available.
|
* specified timeout if not immediately available.
|
||||||
* Given the zookeeper notification could be delayed, we will try to
|
* Given the zookeeper notification could be delayed, we will try to
|
||||||
* get the latest data.
|
* get the latest data.
|
||||||
|
* @param zkw
|
||||||
* @param timeout maximum time to wait, in millis
|
* @param timeout maximum time to wait, in millis
|
||||||
* @return server name for server hosting meta region formatted as per
|
* @return server name for server hosting meta region formatted as per
|
||||||
* {@link ServerName}, or null if none available
|
* {@link ServerName}, or null if none available
|
||||||
* @throws InterruptedException if interrupted while waiting
|
* @throws InterruptedException if interrupted while waiting
|
||||||
|
* @throws NotAllMetaRegionsOnlineException
|
||||||
*/
|
*/
|
||||||
public ServerName waitMetaRegionLocation(ZooKeeperWatcher zkw, long timeout)
|
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 {
|
throws InterruptedException, NotAllMetaRegionsOnlineException {
|
||||||
try {
|
try {
|
||||||
if (ZKUtil.checkExists(zkw, zkw.baseZNode) == -1) {
|
if (ZKUtil.checkExists(zkw, zkw.baseZNode) == -1) {
|
||||||
|
@ -156,7 +212,7 @@ public class MetaTableLocator {
|
||||||
} catch (KeeperException e) {
|
} catch (KeeperException e) {
|
||||||
throw new IllegalStateException("KeeperException while trying to check baseZNode:", 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) {
|
if (sn == null) {
|
||||||
throw new NotAllMetaRegionsOnlineException("Timed out; " + timeout + "ms");
|
throw new NotAllMetaRegionsOnlineException("Timed out; " + timeout + "ms");
|
||||||
|
@ -193,6 +249,8 @@ public class MetaTableLocator {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Verify <code>hbase:meta</code> is deployed and accessible.
|
* 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 timeout How long to wait on zk for meta address (passed through to
|
||||||
* the internal call to {@link #getMetaServerConnection}.
|
* the internal call to {@link #getMetaServerConnection}.
|
||||||
* @return True if the <code>hbase:meta</code> location is healthy.
|
* @return True if the <code>hbase:meta</code> location is healthy.
|
||||||
|
@ -201,10 +259,26 @@ public class MetaTableLocator {
|
||||||
*/
|
*/
|
||||||
public boolean verifyMetaRegionLocation(HConnection hConnection,
|
public boolean verifyMetaRegionLocation(HConnection hConnection,
|
||||||
ZooKeeperWatcher zkw, final long timeout)
|
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 {
|
throws InterruptedException, IOException {
|
||||||
AdminProtos.AdminService.BlockingInterface service = null;
|
AdminProtos.AdminService.BlockingInterface service = null;
|
||||||
try {
|
try {
|
||||||
service = getMetaServerConnection(hConnection, zkw, timeout);
|
service = getMetaServerConnection(hConnection, zkw, timeout, replicaId);
|
||||||
} catch (NotAllMetaRegionsOnlineException e) {
|
} catch (NotAllMetaRegionsOnlineException e) {
|
||||||
// Pass
|
// Pass
|
||||||
} catch (ServerNotRunningYetException e) {
|
} catch (ServerNotRunningYetException e) {
|
||||||
|
@ -215,7 +289,8 @@ public class MetaTableLocator {
|
||||||
// Pass -- server name sends us to a server that is dying or already dead.
|
// Pass -- server name sends us to a server that is dying or already dead.
|
||||||
}
|
}
|
||||||
return (service != null) && verifyRegionLocation(service,
|
return (service != null) && verifyRegionLocation(service,
|
||||||
getMetaRegionLocation(zkw), META_REGION_NAME);
|
getMetaRegionLocation(zkw, replicaId), RegionReplicaUtil.getRegionInfoForReplica(
|
||||||
|
HRegionInfo.FIRST_META_REGIONINFO, replicaId).getRegionName());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -271,16 +346,19 @@ public class MetaTableLocator {
|
||||||
* Gets a connection to the server hosting meta, as reported by ZooKeeper,
|
* Gets a connection to the server hosting meta, as reported by ZooKeeper,
|
||||||
* waiting up to the specified timeout for availability.
|
* waiting up to the specified timeout for availability.
|
||||||
* <p>WARNING: Does not retry. Use an {@link org.apache.hadoop.hbase.client.HTable} instead.
|
* <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 timeout How long to wait on meta location
|
||||||
|
* @param replicaId
|
||||||
* @return connection to server hosting meta
|
* @return connection to server hosting meta
|
||||||
* @throws InterruptedException
|
* @throws InterruptedException
|
||||||
* @throws NotAllMetaRegionsOnlineException if timed out waiting
|
* @throws NotAllMetaRegionsOnlineException if timed out waiting
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private AdminService.BlockingInterface getMetaServerConnection(HConnection hConnection,
|
private AdminService.BlockingInterface getMetaServerConnection(HConnection hConnection,
|
||||||
ZooKeeperWatcher zkw, long timeout)
|
ZooKeeperWatcher zkw, long timeout, int replicaId)
|
||||||
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
|
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
|
||||||
return getCachedConnection(hConnection, waitMetaRegionLocation(zkw, timeout));
|
return getCachedConnection(hConnection, waitMetaRegionLocation(zkw, replicaId, timeout));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -345,6 +423,20 @@ public class MetaTableLocator {
|
||||||
*/
|
*/
|
||||||
public static void setMetaLocation(ZooKeeperWatcher zookeeper,
|
public static void setMetaLocation(ZooKeeperWatcher zookeeper,
|
||||||
ServerName serverName, RegionState.State state) throws KeeperException {
|
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);
|
LOG.info("Setting hbase:meta region location in ZooKeeper as " + serverName);
|
||||||
// Make the MetaRegionServer pb and then get its bytes and save this as
|
// Make the MetaRegionServer pb and then get its bytes and save this as
|
||||||
// the znode content.
|
// the znode content.
|
||||||
|
@ -354,10 +446,15 @@ public class MetaTableLocator {
|
||||||
.setState(state.convert()).build();
|
.setState(state.convert()).build();
|
||||||
byte[] data = ProtobufUtil.prependPBMagic(pbrsr.toByteArray());
|
byte[] data = ProtobufUtil.prependPBMagic(pbrsr.toByteArray());
|
||||||
try {
|
try {
|
||||||
ZKUtil.setData(zookeeper, zookeeper.metaServerZNode, data);
|
ZKUtil.setData(zookeeper, zookeeper.getZNodeForReplica(replicaId), data);
|
||||||
} catch(KeeperException.NoNodeException nne) {
|
} catch(KeeperException.NoNodeException nne) {
|
||||||
LOG.debug("META region location doesn't existed, create it");
|
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
|
||||||
ZKUtil.createAndWatch(zookeeper, zookeeper.metaServerZNode, data);
|
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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -365,10 +462,22 @@ public class MetaTableLocator {
|
||||||
* Load the meta region state from the meta server ZNode.
|
* Load the meta region state from the meta server ZNode.
|
||||||
*/
|
*/
|
||||||
public static RegionState getMetaRegionState(ZooKeeperWatcher zkw) throws KeeperException {
|
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;
|
RegionState.State state = RegionState.State.OPEN;
|
||||||
ServerName serverName = null;
|
ServerName serverName = null;
|
||||||
try {
|
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)) {
|
if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) {
|
||||||
try {
|
try {
|
||||||
int prefixLen = ProtobufUtil.lengthOfPBMagic();
|
int prefixLen = ProtobufUtil.lengthOfPBMagic();
|
||||||
|
@ -396,7 +505,8 @@ public class MetaTableLocator {
|
||||||
if (serverName == null) {
|
if (serverName == null) {
|
||||||
state = RegionState.State.OFFLINE;
|
state = RegionState.State.OFFLINE;
|
||||||
}
|
}
|
||||||
return new RegionState(HRegionInfo.FIRST_META_REGIONINFO,
|
return new RegionState(
|
||||||
|
RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, replicaId),
|
||||||
state, serverName);
|
state, serverName);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -407,14 +517,53 @@ public class MetaTableLocator {
|
||||||
*/
|
*/
|
||||||
public void deleteMetaLocation(ZooKeeperWatcher zookeeper)
|
public void deleteMetaLocation(ZooKeeperWatcher zookeeper)
|
||||||
throws KeeperException {
|
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 {
|
try {
|
||||||
// Just delete the node. Don't need any watches.
|
// Just delete the node. Don't need any watches.
|
||||||
ZKUtil.deleteNode(zookeeper, zookeeper.metaServerZNode);
|
ZKUtil.deleteNode(zookeeper, zookeeper.getZNodeForReplica(replicaId));
|
||||||
} catch(KeeperException.NoNodeException nne) {
|
} catch(KeeperException.NoNodeException nne) {
|
||||||
// Has already been deleted
|
// 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.
|
* Wait until the meta region is available and is not in transition.
|
||||||
|
@ -425,6 +574,20 @@ public class MetaTableLocator {
|
||||||
*/
|
*/
|
||||||
public ServerName blockUntilAvailable(final ZooKeeperWatcher zkw,
|
public ServerName blockUntilAvailable(final ZooKeeperWatcher zkw,
|
||||||
final long timeout)
|
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 {
|
throws InterruptedException {
|
||||||
if (timeout < 0) throw new IllegalArgumentException();
|
if (timeout < 0) throw new IllegalArgumentException();
|
||||||
if (zkw == null) throw new IllegalArgumentException();
|
if (zkw == null) throw new IllegalArgumentException();
|
||||||
|
@ -432,7 +595,7 @@ public class MetaTableLocator {
|
||||||
ServerName sn = null;
|
ServerName sn = null;
|
||||||
try {
|
try {
|
||||||
while (true) {
|
while (true) {
|
||||||
sn = getMetaRegionLocation(zkw);
|
sn = getMetaRegionLocation(zkw, replicaId);
|
||||||
if (sn != null || sw.elapsedMillis()
|
if (sn != null || sw.elapsedMillis()
|
||||||
> timeout - HConstants.SOCKET_RETRY_WAIT_MS) {
|
> timeout - HConstants.SOCKET_RETRY_WAIT_MS) {
|
||||||
break;
|
break;
|
||||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.commons.lang.StringUtils;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
@ -966,7 +967,7 @@ public class ZKUtil {
|
||||||
// Certain znodes are accessed directly by the client,
|
// Certain znodes are accessed directly by the client,
|
||||||
// so they must be readable by non-authenticated clients
|
// so they must be readable by non-authenticated clients
|
||||||
if ((node.equals(zkw.baseZNode) == true) ||
|
if ((node.equals(zkw.baseZNode) == true) ||
|
||||||
(node.equals(zkw.metaServerZNode) == true) ||
|
(zkw.isAnyMetaReplicaZnode(node)) ||
|
||||||
(node.equals(zkw.getMasterAddressZNode()) == true) ||
|
(node.equals(zkw.getMasterAddressZNode()) == true) ||
|
||||||
(node.equals(zkw.clusterIdZNode) == true) ||
|
(node.equals(zkw.clusterIdZNode) == true) ||
|
||||||
(node.equals(zkw.rsZNode) == true) ||
|
(node.equals(zkw.rsZNode) == true) ||
|
||||||
|
@ -1699,6 +1700,13 @@ public class ZKUtil {
|
||||||
}
|
}
|
||||||
sb.append("\nRegion server holding hbase:meta: "
|
sb.append("\nRegion server holding hbase:meta: "
|
||||||
+ new MetaTableLocator().getMetaRegionLocation(zkw));
|
+ 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:");
|
sb.append("\nRegion servers:");
|
||||||
for (String child : listChildrenNoWatch(zkw, zkw.rsZNode)) {
|
for (String child : listChildrenNoWatch(zkw, zkw.rsZNode)) {
|
||||||
sb.append("\n ").append(child);
|
sb.append("\n ").append(child);
|
||||||
|
@ -1887,7 +1895,7 @@ public class ZKUtil {
|
||||||
" byte(s) of data from znode " + znode +
|
" byte(s) of data from znode " + znode +
|
||||||
(watcherSet? " and set watcher; ": "; data=") +
|
(watcherSet? " and set watcher; ": "; data=") +
|
||||||
(data == null? "null": data.length == 0? "empty": (
|
(data == null? "null": data.length == 0? "empty": (
|
||||||
znode.startsWith(zkw.metaServerZNode)?
|
znode.startsWith(ZooKeeperWatcher.META_ZNODE_PREFIX)?
|
||||||
getServerNameOrEmptyString(data):
|
getServerNameOrEmptyString(data):
|
||||||
znode.startsWith(zkw.backupMasterAddressesZNode)?
|
znode.startsWith(zkw.backupMasterAddressesZNode)?
|
||||||
getServerNameOrEmptyString(data):
|
getServerNameOrEmptyString(data):
|
||||||
|
|
|
@ -22,7 +22,9 @@ import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.concurrent.CopyOnWriteArrayList;
|
import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
|
||||||
|
@ -31,6 +33,7 @@ import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.Abortable;
|
import org.apache.hadoop.hbase.Abortable;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.zookeeper.KeeperException;
|
import org.apache.zookeeper.KeeperException;
|
||||||
|
@ -82,8 +85,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
||||||
|
|
||||||
// base znode for this cluster
|
// base znode for this cluster
|
||||||
public String baseZNode;
|
public String baseZNode;
|
||||||
// znode containing location of server hosting meta region
|
//znodes containing the locations of the servers hosting the meta replicas
|
||||||
public String metaServerZNode;
|
private Map<Integer,String> metaReplicaZnodes = new HashMap<Integer, String>();
|
||||||
// znode containing ephemeral nodes of the regionservers
|
// znode containing ephemeral nodes of the regionservers
|
||||||
public String rsZNode;
|
public String rsZNode;
|
||||||
// znode containing ephemeral nodes of the draining regionservers
|
// znode containing ephemeral nodes of the draining regionservers
|
||||||
|
@ -110,6 +113,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
||||||
// znode containing namespace descriptors
|
// znode containing namespace descriptors
|
||||||
public static String namespaceZNode = "namespace";
|
public static String namespaceZNode = "namespace";
|
||||||
|
|
||||||
|
public final static String META_ZNODE_PREFIX = "meta-region-server";
|
||||||
|
|
||||||
private final Configuration conf;
|
private final Configuration conf;
|
||||||
|
|
||||||
|
@ -200,8 +204,15 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
||||||
private void setNodeNames(Configuration conf) {
|
private void setNodeNames(Configuration conf) {
|
||||||
baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
|
baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
|
||||||
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
|
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
|
||||||
metaServerZNode = ZKUtil.joinZNode(baseZNode,
|
metaReplicaZnodes.put(0, ZKUtil.joinZNode(baseZNode,
|
||||||
conf.get("zookeeper.znode.metaserver", "meta-region-server"));
|
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,
|
rsZNode = ZKUtil.joinZNode(baseZNode,
|
||||||
conf.get("zookeeper.znode.rs", "rs"));
|
conf.get("zookeeper.znode.rs", "rs"));
|
||||||
drainingZNode = ZKUtil.joinZNode(baseZNode,
|
drainingZNode = ZKUtil.joinZNode(baseZNode,
|
||||||
|
@ -228,6 +239,75 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
||||||
conf.get("zookeeper.znode.namespace", "namespace"));
|
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.
|
* Register the specified listener to receive ZooKeeper events.
|
||||||
* @param listener
|
* @param listener
|
||||||
|
|
|
@ -894,6 +894,12 @@ public final class HConstants {
|
||||||
public static final String LOG_REPLAY_WAIT_REGION_TIMEOUT =
|
public static final String LOG_REPLAY_WAIT_REGION_TIMEOUT =
|
||||||
"hbase.master.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 name of the configuration parameter that specifies
|
||||||
* the number of bytes in a newly created checksum chunk.
|
* 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();
|
||||||
|
}
|
||||||
|
}
|
|
@ -1381,14 +1381,15 @@ public class AssignmentManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Assigns the hbase:meta region.
|
* Assigns the hbase:meta region or a replica.
|
||||||
* <p>
|
* <p>
|
||||||
* Assumes that hbase:meta is currently closed and is not being actively served by
|
* Assumes that hbase:meta is currently closed and is not being actively served by
|
||||||
* any RegionServer.
|
* any RegionServer.
|
||||||
|
* @param hri TODO
|
||||||
*/
|
*/
|
||||||
public void assignMeta() throws KeeperException {
|
public void assignMeta(HRegionInfo hri) throws KeeperException {
|
||||||
regionStates.updateRegionState(HRegionInfo.FIRST_META_REGIONINFO, State.OFFLINE);
|
regionStates.updateRegionState(hri, State.OFFLINE);
|
||||||
assign(HRegionInfo.FIRST_META_REGIONINFO);
|
assign(hri);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1968,6 +1969,15 @@ public class AssignmentManager {
|
||||||
return isCarryingRegion(serverName, HRegionInfo.FIRST_META_REGIONINFO);
|
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.
|
* Check if the shutdown server carries the specific region.
|
||||||
* @return whether the serverName currently hosts the region
|
* @return whether the serverName currently hosts the region
|
||||||
|
|
|
@ -57,12 +57,14 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
|
import org.apache.hadoop.hbase.HRegionLocation;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.MasterNotRunningException;
|
import org.apache.hadoop.hbase.MasterNotRunningException;
|
||||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||||
import org.apache.hadoop.hbase.NamespaceNotFoundException;
|
import org.apache.hadoop.hbase.NamespaceNotFoundException;
|
||||||
import org.apache.hadoop.hbase.PleaseHoldException;
|
import org.apache.hadoop.hbase.PleaseHoldException;
|
||||||
|
import org.apache.hadoop.hbase.RegionLocations;
|
||||||
import org.apache.hadoop.hbase.Server;
|
import org.apache.hadoop.hbase.Server;
|
||||||
import org.apache.hadoop.hbase.ServerLoad;
|
import org.apache.hadoop.hbase.ServerLoad;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
|
@ -75,6 +77,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.MetaScanner;
|
import org.apache.hadoop.hbase.client.MetaScanner;
|
||||||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
||||||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
|
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.client.Result;
|
||||||
import org.apache.hadoop.hbase.client.TableState;
|
import org.apache.hadoop.hbase.client.TableState;
|
||||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||||
|
@ -341,6 +344,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
LOG.info("hbase.rootdir=" + FSUtils.getRootDir(this.conf) +
|
LOG.info("hbase.rootdir=" + FSUtils.getRootDir(this.conf) +
|
||||||
", hbase.cluster.distributed=" + this.conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, false));
|
", 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);
|
Replication.decorateMasterConfiguration(this.conf);
|
||||||
|
|
||||||
// Hack! Maps DFSClient => Master for logs. HDFS made this
|
// Hack! Maps DFSClient => Master for logs. HDFS made this
|
||||||
|
@ -603,7 +609,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
|
|
||||||
// enable table descriptors cache
|
// enable table descriptors cache
|
||||||
this.tableDescriptors.setCacheOn();
|
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
|
// warm-up HTDs cache on master initialization
|
||||||
if (preLoadTableDescriptors) {
|
if (preLoadTableDescriptors) {
|
||||||
status.setStatus("Pre-loading table descriptors");
|
status.setStatus("Pre-loading table descriptors");
|
||||||
|
@ -693,7 +701,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
|
|
||||||
// Make sure meta assigned before proceeding.
|
// Make sure meta assigned before proceeding.
|
||||||
status.setStatus("Assigning Meta Region");
|
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
|
// check if master is shutting down because above assignMeta could return even hbase:meta isn't
|
||||||
// assigned when master is shutting down
|
// assigned when master is shutting down
|
||||||
if(isStopped()) return;
|
if(isStopped()) return;
|
||||||
|
@ -737,6 +745,15 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
LOG.info("Master has completed initialization");
|
LOG.info("Master has completed initialization");
|
||||||
configurationManager.registerObserver(this.balancer);
|
configurationManager.registerObserver(this.balancer);
|
||||||
initialized = true;
|
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);
|
||||||
|
|
||||||
status.setStatus("Starting quota manager");
|
status.setStatus("Starting quota manager");
|
||||||
initQuotaManager();
|
initQuotaManager();
|
||||||
|
|
||||||
|
@ -773,47 +790,81 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
return new ServerManager(master, services);
|
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);
|
||||||
|
serverManager.sendRegionClose(r.getServerName(), r.getRegion());
|
||||||
|
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.
|
* Check <code>hbase:meta</code> is assigned. If not, assign it.
|
||||||
* @param status MonitoredTask
|
* @param status MonitoredTask
|
||||||
* @param previouslyFailedMetaRSs
|
* @param previouslyFailedMetaRSs
|
||||||
|
* @param replicaId
|
||||||
* @throws InterruptedException
|
* @throws InterruptedException
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
* @throws KeeperException
|
* @throws KeeperException
|
||||||
*/
|
*/
|
||||||
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMetaRSs)
|
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMetaRSs, int replicaId)
|
||||||
throws InterruptedException, IOException, KeeperException {
|
throws InterruptedException, IOException, KeeperException {
|
||||||
// Work on meta region
|
// Work on meta region
|
||||||
int assigned = 0;
|
int assigned = 0;
|
||||||
long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
|
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.
|
// Get current meta state from zk.
|
||||||
RegionState metaState = MetaTableLocator.getMetaRegionState(getZooKeeper());
|
RegionState metaState = MetaTableLocator.getMetaRegionState(getZooKeeper(), replicaId);
|
||||||
|
HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO,
|
||||||
|
replicaId);
|
||||||
RegionStates regionStates = assignmentManager.getRegionStates();
|
RegionStates regionStates = assignmentManager.getRegionStates();
|
||||||
regionStates.createRegionState(HRegionInfo.FIRST_META_REGIONINFO,
|
regionStates.createRegionState(hri, metaState.getState(),
|
||||||
metaState.getState(), metaState.getServerName(), null);
|
metaState.getServerName(), null);
|
||||||
|
|
||||||
if (!metaState.isOpened() || !metaTableLocator.verifyMetaRegionLocation(
|
if (!metaState.isOpened() || !metaTableLocator.verifyMetaRegionLocation(
|
||||||
this.getConnection(), this.getZooKeeper(), timeout)) {
|
this.getConnection(), this.getZooKeeper(), timeout, replicaId)) {
|
||||||
ServerName currentMetaServer = metaState.getServerName();
|
ServerName currentMetaServer = metaState.getServerName();
|
||||||
if (serverManager.isServerOnline(currentMetaServer)) {
|
if (serverManager.isServerOnline(currentMetaServer)) {
|
||||||
LOG.info("Meta was in transition on " + currentMetaServer);
|
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
|
||||||
|
LOG.info("Meta was in transition on " + currentMetaServer);
|
||||||
|
} else {
|
||||||
|
LOG.info("Meta with replicaId " + replicaId + " was in transition on " +
|
||||||
|
currentMetaServer);
|
||||||
|
}
|
||||||
assignmentManager.processRegionsInTransition(Arrays.asList(metaState));
|
assignmentManager.processRegionsInTransition(Arrays.asList(metaState));
|
||||||
} else {
|
} else {
|
||||||
if (currentMetaServer != null) {
|
if (currentMetaServer != null) {
|
||||||
splitMetaLogBeforeAssignment(currentMetaServer);
|
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
|
||||||
regionStates.logSplit(HRegionInfo.FIRST_META_REGIONINFO);
|
splitMetaLogBeforeAssignment(currentMetaServer);
|
||||||
previouslyFailedMetaRSs.add(currentMetaServer);
|
regionStates.logSplit(HRegionInfo.FIRST_META_REGIONINFO);
|
||||||
|
previouslyFailedMetaRSs.add(currentMetaServer);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
LOG.info("Re-assigning hbase:meta, it was on " + currentMetaServer);
|
LOG.info("Re-assigning hbase:meta with replicaId, " + replicaId +
|
||||||
assignmentManager.assignMeta();
|
" it was on " + currentMetaServer);
|
||||||
|
assignmentManager.assignMeta(hri);
|
||||||
}
|
}
|
||||||
assigned++;
|
assigned++;
|
||||||
}
|
}
|
||||||
|
|
||||||
enableMeta(TableName.META_TABLE_NAME);
|
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) enableMeta(TableName.META_TABLE_NAME);
|
||||||
|
|
||||||
if ((RecoveryMode.LOG_REPLAY == this.getMasterFileSystem().getLogRecoveryMode())
|
if ((RecoveryMode.LOG_REPLAY == this.getMasterFileSystem().getLogRecoveryMode())
|
||||||
&& (!previouslyFailedMetaRSs.isEmpty())) {
|
&& (!previouslyFailedMetaRSs.isEmpty())) {
|
||||||
|
@ -826,10 +877,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
|
// 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.
|
// by SSH so that system tables can be assigned.
|
||||||
// No need to wait for meta is assigned = 0 when meta is just verified.
|
// No need to wait for meta is assigned = 0 when meta is just verified.
|
||||||
enableServerShutdownHandler(assigned != 0);
|
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) enableServerShutdownHandler(assigned != 0);
|
||||||
|
LOG.info("hbase:meta with replicaId " + replicaId + " assigned=" + assigned + ", location="
|
||||||
LOG.info("hbase:meta assigned=" + assigned + ", location="
|
+ metaTableLocator.getMetaRegionLocation(this.getZooKeeper(), replicaId));
|
||||||
+ metaTableLocator.getMetaRegionLocation(this.getZooKeeper()));
|
|
||||||
status.setStatus("META assigned.");
|
status.setStatus("META assigned.");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -170,7 +170,7 @@ public class RegionStateStore {
|
||||||
// persist meta state in MetaTableLocator (which in turn is zk storage currently)
|
// persist meta state in MetaTableLocator (which in turn is zk storage currently)
|
||||||
try {
|
try {
|
||||||
MetaTableLocator.setMetaLocation(server.getZooKeeper(),
|
MetaTableLocator.setMetaLocation(server.getZooKeeper(),
|
||||||
newState.getServerName(), newState.getState());
|
newState.getServerName(), hri.getReplicaId(), newState.getState());
|
||||||
return; // Done
|
return; // Done
|
||||||
} catch (KeeperException e) {
|
} catch (KeeperException e) {
|
||||||
throw new IOException("Failed to update meta ZNode", e);
|
throw new IOException("Failed to update meta ZNode", e);
|
||||||
|
|
|
@ -857,10 +857,11 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Check if a region belongs to some small system table.
|
* 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) {
|
public boolean shouldBeOnMaster(HRegionInfo region) {
|
||||||
return tablesOnMaster.contains(region.getTable().getNameAsString());
|
return tablesOnMaster.contains(region.getTable().getNameAsString())
|
||||||
|
&& region.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -150,7 +150,7 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
|
||||||
getLong("hbase.catalog.verification.timeout", 1000);
|
getLong("hbase.catalog.verification.timeout", 1000);
|
||||||
if (!server.getMetaTableLocator().verifyMetaRegionLocation(server.getConnection(),
|
if (!server.getMetaTableLocator().verifyMetaRegionLocation(server.getConnection(),
|
||||||
this.server.getZooKeeper(), timeout)) {
|
this.server.getZooKeeper(), timeout)) {
|
||||||
this.services.getAssignmentManager().assignMeta();
|
this.services.getAssignmentManager().assignMeta(HRegionInfo.FIRST_META_REGIONINFO);
|
||||||
} else if (serverName.equals(server.getMetaTableLocator().getMetaRegionLocation(
|
} else if (serverName.equals(server.getMetaTableLocator().getMetaRegionLocation(
|
||||||
this.server.getZooKeeper()))) {
|
this.server.getZooKeeper()))) {
|
||||||
throw new IOException("hbase:meta is onlined on the dead server "
|
throw new IOException("hbase:meta is onlined on the dead server "
|
||||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.Server;
|
import org.apache.hadoop.hbase.Server;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
|
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||||
import org.apache.hadoop.hbase.client.TableState;
|
import org.apache.hadoop.hbase.client.TableState;
|
||||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||||
import org.apache.hadoop.hbase.executor.EventType;
|
import org.apache.hadoop.hbase.executor.EventType;
|
||||||
|
@ -198,7 +199,17 @@ public class ServerShutdownHandler extends EventHandler {
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
resubmit(serverName, 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
|
// Clean out anything in regions in transition. Being conservative and
|
||||||
// doing after log splitting. Could do some states before -- OPENING?
|
// doing after log splitting. Could do some states before -- OPENING?
|
||||||
// OFFLINE? -- and then others after like CLOSING that depend on log
|
// OFFLINE? -- and then others after like CLOSING that depend on log
|
||||||
|
@ -208,8 +219,7 @@ public class ServerShutdownHandler extends EventHandler {
|
||||||
" region(s) that " + (serverName == null? "null": serverName) +
|
" region(s) that " + (serverName == null? "null": serverName) +
|
||||||
" was carrying (and " + regionsInTransition.size() +
|
" was carrying (and " + regionsInTransition.size() +
|
||||||
" regions(s) that were opening on this server)");
|
" regions(s) that were opening on this server)");
|
||||||
|
|
||||||
List<HRegionInfo> toAssignRegions = new ArrayList<HRegionInfo>();
|
|
||||||
toAssignRegions.addAll(regionsInTransition);
|
toAssignRegions.addAll(regionsInTransition);
|
||||||
|
|
||||||
// Iterate regions that were on this server and assign them
|
// Iterate regions that were on this server and assign them
|
||||||
|
|
|
@ -1167,6 +1167,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
||||||
"hbase.regionserver.optionalcacheflushinterval";
|
"hbase.regionserver.optionalcacheflushinterval";
|
||||||
/** Default interval for the memstore flush */
|
/** Default interval for the memstore flush */
|
||||||
public static final int DEFAULT_CACHE_FLUSH_INTERVAL = 3600000;
|
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 */
|
/** 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 =
|
public static final String MEMSTORE_FLUSH_PER_CHANGES =
|
||||||
|
@ -1799,18 +1800,23 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
||||||
&& (this.maxFlushedSeqId + this.flushPerChanges < this.sequenceId.get())) {
|
&& (this.maxFlushedSeqId + this.flushPerChanges < this.sequenceId.get())) {
|
||||||
return true;
|
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;
|
return false;
|
||||||
}
|
}
|
||||||
long now = EnvironmentEdgeManager.currentTime();
|
long now = EnvironmentEdgeManager.currentTime();
|
||||||
//if we flushed in the recent past, we don't need to do again now
|
//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;
|
return false;
|
||||||
}
|
}
|
||||||
//since we didn't flush in the recent past, flush now if certain conditions
|
//since we didn't flush in the recent past, flush now if certain conditions
|
||||||
//are met. Return true on first such memstore hit.
|
//are met. Return true on first such memstore hit.
|
||||||
for (Store s : this.getStores().values()) {
|
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
|
// we have an old enough edit in the memstore, flush
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
|
@ -470,6 +470,8 @@ public class HRegionServer extends HasThread implements
|
||||||
checkCodecs(this.conf);
|
checkCodecs(this.conf);
|
||||||
this.userProvider = UserProvider.instantiate(conf);
|
this.userProvider = UserProvider.instantiate(conf);
|
||||||
FSUtils.setupShortCircuitRead(this.conf);
|
FSUtils.setupShortCircuitRead(this.conf);
|
||||||
|
// Disable usage of meta replicas in the regionserver
|
||||||
|
this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
|
||||||
|
|
||||||
// Config'ed params
|
// Config'ed params
|
||||||
this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
|
this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
|
||||||
|
@ -788,11 +790,19 @@ public class HRegionServer extends HasThread implements
|
||||||
rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress(
|
rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress(
|
||||||
rpcServices.isa.getAddress(), 0));
|
rpcServices.isa.getAddress(), 0));
|
||||||
|
|
||||||
|
boolean onlyMetaRefresh = false;
|
||||||
int storefileRefreshPeriod = conf.getInt(
|
int storefileRefreshPeriod = conf.getInt(
|
||||||
StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD
|
StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD
|
||||||
, StorefileRefresherChore.DEFAULT_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) {
|
if (storefileRefreshPeriod > 0) {
|
||||||
this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod, this, this);
|
this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod,
|
||||||
|
onlyMetaRefresh, this, this);
|
||||||
}
|
}
|
||||||
registerConfigurationObservers();
|
registerConfigurationObservers();
|
||||||
}
|
}
|
||||||
|
@ -1739,7 +1749,8 @@ public class HRegionServer extends HasThread implements
|
||||||
WAL wal;
|
WAL wal;
|
||||||
LogRoller roller = walRoller;
|
LogRoller roller = walRoller;
|
||||||
//_ROOT_ and hbase:meta regions have separate WAL.
|
//_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();
|
roller = ensureMetaWALRoller();
|
||||||
wal = walFactory.getMetaWAL(regionInfo.getEncodedNameAsBytes());
|
wal = walFactory.getMetaWAL(regionInfo.getEncodedNameAsBytes());
|
||||||
} else if (regionInfo == null) {
|
} else if (regionInfo == null) {
|
||||||
|
@ -1837,7 +1848,8 @@ public class HRegionServer extends HasThread implements
|
||||||
Preconditions.checkArgument(hris != null && hris.length == 1);
|
Preconditions.checkArgument(hris != null && hris.length == 1);
|
||||||
if (hris[0].isMetaRegion()) {
|
if (hris[0].isMetaRegion()) {
|
||||||
try {
|
try {
|
||||||
MetaTableLocator.setMetaLocation(getZooKeeper(), serverName, State.OPEN);
|
MetaTableLocator.setMetaLocation(getZooKeeper(), serverName,
|
||||||
|
hris[0].getReplicaId(),State.OPEN);
|
||||||
} catch (KeeperException e) {
|
} catch (KeeperException e) {
|
||||||
LOG.info("Failed to update meta location", e);
|
LOG.info("Failed to update meta location", e);
|
||||||
return false;
|
return false;
|
||||||
|
|
|
@ -53,19 +53,28 @@ public class StorefileRefresherChore extends Chore {
|
||||||
= "hbase.regionserver.storefile.refresh.period";
|
= "hbase.regionserver.storefile.refresh.period";
|
||||||
static final int DEFAULT_REGIONSERVER_STOREFILE_REFRESH_PERIOD = 0; //disabled by default
|
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 HRegionServer regionServer;
|
||||||
private long hfileTtl;
|
private long hfileTtl;
|
||||||
private int period;
|
private int period;
|
||||||
|
private boolean onlyMetaRefresh = true;
|
||||||
|
|
||||||
//ts of last time regions store files are refreshed
|
//ts of last time regions store files are refreshed
|
||||||
private Map<String, Long> lastRefreshTimes; // encodedName -> long
|
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);
|
super("StorefileRefresherChore", period, stoppable);
|
||||||
this.period = period;
|
this.period = period;
|
||||||
this.regionServer = regionServer;
|
this.regionServer = regionServer;
|
||||||
this.hfileTtl = this.regionServer.getConfiguration().getLong(
|
this.hfileTtl = this.regionServer.getConfiguration().getLong(
|
||||||
TimeToLiveHFileCleaner.TTL_CONF_KEY, TimeToLiveHFileCleaner.DEFAULT_TTL);
|
TimeToLiveHFileCleaner.TTL_CONF_KEY, TimeToLiveHFileCleaner.DEFAULT_TTL);
|
||||||
|
this.onlyMetaRefresh = onlyMetaRefresh;
|
||||||
if (period > hfileTtl / 2) {
|
if (period > hfileTtl / 2) {
|
||||||
throw new RuntimeException(REGIONSERVER_STOREFILE_REFRESH_PERIOD +
|
throw new RuntimeException(REGIONSERVER_STOREFILE_REFRESH_PERIOD +
|
||||||
" should be set smaller than half of " + TimeToLiveHFileCleaner.TTL_CONF_KEY);
|
" 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
|
// skip checking for this region if it can accept writes
|
||||||
continue;
|
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();
|
String encodedName = r.getRegionInfo().getEncodedName();
|
||||||
long time = EnvironmentEdgeManager.currentTime();
|
long time = EnvironmentEdgeManager.currentTime();
|
||||||
if (!lastRefreshTimes.containsKey(encodedName)) {
|
if (!lastRefreshTimes.containsKey(encodedName)) {
|
||||||
|
|
|
@ -126,6 +126,7 @@ import org.apache.hadoop.hbase.wal.WAL;
|
||||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||||
import org.apache.hadoop.hbase.wal.WALSplitter;
|
import org.apache.hadoop.hbase.wal.WALSplitter;
|
||||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||||
|
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||||
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
@ -306,6 +307,8 @@ public class HBaseFsck extends Configured implements Closeable {
|
||||||
setConf(HBaseConfiguration.create(getConf()));
|
setConf(HBaseConfiguration.create(getConf()));
|
||||||
// disable blockcache for tool invocation, see HBASE-10500
|
// disable blockcache for tool invocation, see HBASE-10500
|
||||||
getConf().setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0);
|
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);
|
errors = getErrorReporter(conf);
|
||||||
|
|
||||||
int numThreads = conf.getInt("hbasefsck.numthreads", MAX_NUM_THREADS);
|
int numThreads = conf.getInt("hbasefsck.numthreads", MAX_NUM_THREADS);
|
||||||
|
@ -1608,28 +1611,29 @@ public class HBaseFsck extends Configured implements Closeable {
|
||||||
* Record the location of the hbase:meta region as found in ZooKeeper.
|
* Record the location of the hbase:meta region as found in ZooKeeper.
|
||||||
*/
|
*/
|
||||||
private boolean recordMetaRegion() throws IOException {
|
private boolean recordMetaRegion() throws IOException {
|
||||||
HRegionLocation metaLocation = connection.locateRegion(
|
RegionLocations rl = ((ClusterConnection)connection).locateRegion(TableName.META_TABLE_NAME,
|
||||||
TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW);
|
HConstants.EMPTY_START_ROW, false, false);
|
||||||
|
if (rl == null) {
|
||||||
// Check if Meta region is valid and existing
|
|
||||||
if (metaLocation == null || metaLocation.getRegionInfo() == null ||
|
|
||||||
metaLocation.getHostname() == null) {
|
|
||||||
errors.reportError(ERROR_CODE.NULL_META_REGION,
|
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;
|
return false;
|
||||||
}
|
}
|
||||||
ServerName sn;
|
for (HRegionLocation metaLocation : rl.getRegionLocations()) {
|
||||||
try {
|
// Check if Meta region is valid and existing
|
||||||
sn = getMetaRegionServerName();
|
if (metaLocation == null || metaLocation.getRegionInfo() == null ||
|
||||||
} catch (KeeperException e) {
|
metaLocation.getHostname() == null) {
|
||||||
throw new IOException(e);
|
errors.reportError(ERROR_CODE.NULL_META_REGION,
|
||||||
}
|
"META region or some of its attributes are null.");
|
||||||
MetaEntry m = new MetaEntry(metaLocation.getRegionInfo(), sn, System.currentTimeMillis());
|
return false;
|
||||||
HbckInfo hbckInfo = regionInfoMap.get(metaLocation.getRegionInfo().getEncodedName());
|
}
|
||||||
if (hbckInfo == null) {
|
ServerName sn = metaLocation.getServerName();
|
||||||
regionInfoMap.put(metaLocation.getRegionInfo().getEncodedName(), new HbckInfo(m));
|
MetaEntry m = new MetaEntry(metaLocation.getRegionInfo(), sn, System.currentTimeMillis());
|
||||||
} else {
|
HbckInfo hbckInfo = regionInfoMap.get(metaLocation.getRegionInfo().getEncodedName());
|
||||||
hbckInfo.metaEntry = m;
|
if (hbckInfo == null) {
|
||||||
|
regionInfoMap.put(metaLocation.getRegionInfo().getEncodedName(), new HbckInfo(m));
|
||||||
|
} else {
|
||||||
|
hbckInfo.metaEntry = m;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -1650,12 +1654,12 @@ public class HBaseFsck extends Configured implements Closeable {
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
private ServerName getMetaRegionServerName()
|
private ServerName getMetaRegionServerName(int replicaId)
|
||||||
throws IOException, KeeperException {
|
throws IOException, KeeperException {
|
||||||
ZooKeeperWatcher zkw = createZooKeeperWatcher();
|
ZooKeeperWatcher zkw = createZooKeeperWatcher();
|
||||||
ServerName sn = null;
|
ServerName sn = null;
|
||||||
try {
|
try {
|
||||||
sn = new MetaTableLocator().getMetaRegionLocation(zkw);
|
sn = new MetaTableLocator().getMetaRegionLocation(zkw, replicaId);
|
||||||
} finally {
|
} finally {
|
||||||
zkw.close();
|
zkw.close();
|
||||||
}
|
}
|
||||||
|
@ -3027,55 +3031,83 @@ public class HBaseFsck extends Configured implements Closeable {
|
||||||
* If there are inconsistencies (i.e. zero or more than one regions
|
* 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.
|
* pretend to be holding the hbase:meta) try to fix that and report an error.
|
||||||
* @throws IOException from HBaseFsckRepair functions
|
* @throws IOException from HBaseFsckRepair functions
|
||||||
* @throws KeeperException
|
* @throws KeeperException
|
||||||
* @throws InterruptedException
|
* @throws InterruptedException
|
||||||
*/
|
*/
|
||||||
boolean checkMetaRegion() throws IOException, KeeperException, 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()) {
|
for (HbckInfo value : regionInfoMap.values()) {
|
||||||
if (value.metaEntry != null && value.metaEntry.isMetaRegion()) {
|
if (value.metaEntry != null && value.metaEntry.isMetaRegion()) {
|
||||||
metaRegions.add(value);
|
metaRegions.put(value.getReplicaId(), value);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
int metaReplication = admin.getTableDescriptor(TableName.META_TABLE_NAME)
|
||||||
// There will be always one entry in regionInfoMap corresponding to hbase:meta
|
.getRegionReplication();
|
||||||
// Check the deployed servers. It should be exactly one server.
|
boolean noProblem = true;
|
||||||
List<ServerName> servers = new ArrayList<ServerName>();
|
// There will be always entries in regionInfoMap corresponding to hbase:meta & its replicas
|
||||||
HbckInfo metaHbckInfo = null;
|
// Check the deployed servers. It should be exactly one server for each replica.
|
||||||
if (!metaRegions.isEmpty()) {
|
for (int i = 0; i < metaReplication; i++) {
|
||||||
metaHbckInfo = metaRegions.get(0);
|
HbckInfo metaHbckInfo = metaRegions.remove(i);
|
||||||
servers = metaHbckInfo.deployedOn;
|
List<ServerName> servers = new ArrayList<ServerName>();
|
||||||
}
|
if (metaHbckInfo != null) {
|
||||||
if (servers.size() != 1) {
|
servers = metaHbckInfo.deployedOn;
|
||||||
if (servers.size() == 0) {
|
}
|
||||||
errors.reportError(ERROR_CODE.NO_META_REGION, "hbase:meta is not found on any region.");
|
if (servers.size() != 1) {
|
||||||
if (shouldFixAssignments()) {
|
noProblem = false;
|
||||||
errors.print("Trying to fix a problem with hbase:meta..");
|
if (servers.size() == 0) {
|
||||||
setShouldRerun();
|
assignMetaReplica(i);
|
||||||
// try to fix it (treat it as unassigned region)
|
} else if (servers.size() > 1) {
|
||||||
HBaseFsckRepair.fixUnassigned(admin, HRegionInfo.FIRST_META_REGIONINFO);
|
errors
|
||||||
HBaseFsckRepair.waitUntilAssigned(admin, HRegionInfo.FIRST_META_REGIONINFO);
|
.reportError(ERROR_CODE.MULTI_META_REGION, "hbase:meta, replicaId " +
|
||||||
}
|
metaHbckInfo.getReplicaId() + " is found on more than one region.");
|
||||||
} else if (servers.size() > 1) {
|
if (shouldFixAssignments()) {
|
||||||
errors
|
errors.print("Trying to fix a problem with hbase:meta, replicaId " +
|
||||||
.reportError(ERROR_CODE.MULTI_META_REGION, "hbase:meta is found on more than one region.");
|
metaHbckInfo.getReplicaId() +"..");
|
||||||
if (shouldFixAssignments()) {
|
setShouldRerun();
|
||||||
if (metaHbckInfo == null) {
|
// try fix it (treat is a dupe assignment)
|
||||||
errors.print(
|
HBaseFsckRepair.fixMultiAssignment(connection, metaHbckInfo.metaEntry, servers);
|
||||||
"Unable to fix problem with hbase:meta due to hbase:meta region info missing");
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
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
|
// unassign whatever is remaining in metaRegions. They are excess replicas.
|
||||||
return true;
|
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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -28,12 +28,14 @@
|
||||||
import="org.apache.hadoop.hbase.ServerName"
|
import="org.apache.hadoop.hbase.ServerName"
|
||||||
import="org.apache.hadoop.hbase.ServerLoad"
|
import="org.apache.hadoop.hbase.ServerLoad"
|
||||||
import="org.apache.hadoop.hbase.RegionLoad"
|
import="org.apache.hadoop.hbase.RegionLoad"
|
||||||
|
import="org.apache.hadoop.hbase.HConstants"
|
||||||
import="org.apache.hadoop.hbase.master.HMaster"
|
import="org.apache.hadoop.hbase.master.HMaster"
|
||||||
import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator"
|
import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator"
|
||||||
import="org.apache.hadoop.hbase.util.Bytes"
|
import="org.apache.hadoop.hbase.util.Bytes"
|
||||||
import="org.apache.hadoop.hbase.util.FSUtils"
|
import="org.apache.hadoop.hbase.util.FSUtils"
|
||||||
import="org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState"
|
import="org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState"
|
||||||
import="org.apache.hadoop.hbase.TableName"
|
import="org.apache.hadoop.hbase.TableName"
|
||||||
|
import="org.apache.hadoop.hbase.client.RegionReplicaUtil"
|
||||||
import="org.apache.hadoop.hbase.HBaseConfiguration" %>
|
import="org.apache.hadoop.hbase.HBaseConfiguration" %>
|
||||||
<%
|
<%
|
||||||
HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
|
HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
|
||||||
|
@ -53,6 +55,8 @@
|
||||||
ServerName rl = metaTableLocator.getMetaRegionLocation(master.getZooKeeper());
|
ServerName rl = metaTableLocator.getMetaRegionLocation(master.getZooKeeper());
|
||||||
boolean showFragmentation = conf.getBoolean("hbase.master.ui.fragmentation.enabled", false);
|
boolean showFragmentation = conf.getBoolean("hbase.master.ui.fragmentation.enabled", false);
|
||||||
boolean readOnly = conf.getBoolean("hbase.master.ui.readonly", 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;
|
Map<String, Integer> frags = null;
|
||||||
if (showFragmentation) {
|
if (showFragmentation) {
|
||||||
frags = FSUtils.getTableFragmentation(master);
|
frags = FSUtils.getTableFragmentation(master);
|
||||||
|
@ -202,11 +206,14 @@
|
||||||
%>
|
%>
|
||||||
<%= tableHeader %>
|
<%= tableHeader %>
|
||||||
<%
|
<%
|
||||||
// NOTE: Presumes one meta region only.
|
// NOTE: Presumes meta with one or more replicas
|
||||||
HRegionInfo meta = HRegionInfo.FIRST_META_REGIONINFO;
|
for (int j = 0; j < numMetaReplicas; j++) {
|
||||||
ServerName metaLocation = metaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), 1);
|
HRegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
|
||||||
for (int i = 0; i < 1; i++) {
|
HRegionInfo.FIRST_META_REGIONINFO, j);
|
||||||
String url = "//" + metaLocation.getHostname() + ":" + master.getRegionServerInfoPort(metaLocation) + "/";
|
ServerName metaLocation = metaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1);
|
||||||
|
for (int i = 0; i < 1; i++) {
|
||||||
|
String url = "//" + metaLocation.getHostname() + ":" +
|
||||||
|
master.getRegionServerInfoPort(metaLocation) + "/";
|
||||||
%>
|
%>
|
||||||
<tr>
|
<tr>
|
||||||
<td><%= escapeXml(meta.getRegionNameAsString()) %></td>
|
<td><%= escapeXml(meta.getRegionNameAsString()) %></td>
|
||||||
|
@ -217,6 +224,7 @@
|
||||||
<td>-</td>
|
<td>-</td>
|
||||||
</tr>
|
</tr>
|
||||||
<% } %>
|
<% } %>
|
||||||
|
<%} %>
|
||||||
</table>
|
</table>
|
||||||
<%} else {
|
<%} else {
|
||||||
Admin admin = master.getConnection().getAdmin();
|
Admin admin = master.getConnection().getAdmin();
|
||||||
|
|
|
@ -0,0 +1,393 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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(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[]{});
|
||||||
|
}
|
||||||
|
}
|
|
@ -262,8 +262,8 @@ public class TestMasterNoCluster {
|
||||||
TESTUTIL.getConfiguration());
|
TESTUTIL.getConfiguration());
|
||||||
HMaster master = new HMaster(conf, cp) {
|
HMaster master = new HMaster(conf, cp) {
|
||||||
@Override
|
@Override
|
||||||
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMeatRSs) {
|
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMeatRSs, int replicaId)
|
||||||
}
|
{ }
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void initializeZKBasedSystemTrackers() throws IOException,
|
void initializeZKBasedSystemTrackers() throws IOException,
|
||||||
|
|
|
@ -33,22 +33,28 @@ import junit.framework.TestCase;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.Cell;
|
import org.apache.hadoop.hbase.Cell;
|
||||||
import org.apache.hadoop.hbase.CellUtil;
|
import org.apache.hadoop.hbase.CellUtil;
|
||||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
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.KeepDeletedCells;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValueTestUtil;
|
import org.apache.hadoop.hbase.KeyValueTestUtil;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||||
|
import org.apache.hadoop.hbase.TableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||||
import org.apache.hadoop.hbase.client.Scan;
|
import org.apache.hadoop.hbase.client.Scan;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdge;
|
import org.apache.hadoop.hbase.util.EnvironmentEdge;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
|
@ -928,6 +934,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, TableDescriptor.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 {
|
private class EnvironmentEdgeForMemstoreTest implements EnvironmentEdge {
|
||||||
long t = 1234;
|
long t = 1234;
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -145,7 +145,7 @@ public class TestStoreFileRefresherChore {
|
||||||
boolean isStale = false;
|
boolean isStale = false;
|
||||||
public StaleStorefileRefresherChore(int period, HRegionServer regionServer,
|
public StaleStorefileRefresherChore(int period, HRegionServer regionServer,
|
||||||
Stoppable stoppable) {
|
Stoppable stoppable) {
|
||||||
super(period, regionServer, stoppable);
|
super(period, false, regionServer, stoppable);
|
||||||
}
|
}
|
||||||
@Override
|
@Override
|
||||||
protected boolean isRegionStale(String encodedName, long time) {
|
protected boolean isRegionStale(String encodedName, long time) {
|
||||||
|
|
Loading…
Reference in New Issue