HBASE-18839 Apply RegionInfo to code base

This commit is contained in:
Chia-Ping Tsai 2017-09-28 16:16:21 +08:00
parent d26b8f8ddd
commit 6693f45faf
278 changed files with 4862 additions and 4656 deletions

View File

@ -43,7 +43,6 @@ import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
@ -53,15 +52,16 @@ import org.apache.hadoop.hbase.backup.HBackupFileSystem;
import org.apache.hadoop.hbase.backup.RestoreRequest;
import org.apache.hadoop.hbase.backup.impl.BackupManifest;
import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A collection for methods used by multiple classes to backup HBase tables.
@ -150,11 +150,11 @@ public final class BackupUtils {
LOG.debug("Attempting to copy table info for:" + table + " target: " + target
+ " descriptor: " + orig);
LOG.debug("Finished copying tableinfo.");
List<HRegionInfo> regions = null;
List<RegionInfo> regions = null;
regions = MetaTableAccessor.getTableRegions(conn, table);
// For each region, write the region info to disk
LOG.debug("Starting to write region info for table " + table);
for (HRegionInfo regionInfo : regions) {
for (RegionInfo regionInfo : regions) {
Path regionDir =
HRegion.getRegionDir(new Path(backupInfo.getTableBackupDir(table)), regionInfo);
regionDir = new Path(backupInfo.getTableBackupDir(table), regionDir.getName());
@ -168,8 +168,8 @@ public final class BackupUtils {
* Write the .regioninfo file on-disk.
*/
public static void writeRegioninfoOnFilesystem(final Configuration conf, final FileSystem fs,
final Path regionInfoDir, HRegionInfo regionInfo) throws IOException {
final byte[] content = regionInfo.toDelimitedByteArray();
final Path regionInfoDir, RegionInfo regionInfo) throws IOException {
final byte[] content = RegionInfo.toDelimitedByteArray(regionInfo);
Path regionInfoFile = new Path(regionInfoDir, "." + HConstants.REGIONINFO_QUALIFIER_STR);
// First check to get the permissions
FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);

View File

@ -38,20 +38,21 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.MetaTableAccessor.CollectingVisitor;
import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.RawAsyncTable;
import org.apache.hadoop.hbase.client.RawScanResultConsumer;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.client.Scan.ReadType;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The asynchronous meta table accessor. Used to read/write region and assignment information store
@ -112,7 +113,7 @@ public class AsyncMetaTableAccessor {
RawAsyncTable metaTable, byte[] regionName) {
CompletableFuture<Optional<HRegionLocation>> future = new CompletableFuture<>();
try {
HRegionInfo parsedRegionInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionName);
RegionInfo parsedRegionInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionName);
metaTable.get(
new Get(MetaTableAccessor.getMetaKeyForRegion(parsedRegionInfo))
.addFamily(HConstants.CATALOG_FAMILY)).whenComplete(
@ -151,14 +152,14 @@ public class AsyncMetaTableAccessor {
results
.stream()
.filter(result -> !result.isEmpty())
.filter(result -> MetaTableAccessor.getHRegionInfo(result) != null)
.filter(result -> MetaTableAccessor.getRegionInfo(result) != null)
.forEach(
result -> {
getRegionLocations(result).ifPresent(
locations -> {
for (HRegionLocation location : locations.getRegionLocations()) {
if (location != null
&& encodedRegionNameStr.equals(location.getRegionInfo()
&& encodedRegionNameStr.equals(location.getRegion()
.getEncodedName())) {
future.complete(Optional.of(location));
return;
@ -218,27 +219,27 @@ public class AsyncMetaTableAccessor {
* @return the list of regioninfos and server. The return value will be wrapped by a
* {@link CompletableFuture}.
*/
private static CompletableFuture<List<Pair<HRegionInfo, ServerName>>> getTableRegionsAndLocations(
private static CompletableFuture<List<Pair<RegionInfo, ServerName>>> getTableRegionsAndLocations(
RawAsyncTable metaTable, final Optional<TableName> tableName,
final boolean excludeOfflinedSplitParents) {
CompletableFuture<List<Pair<HRegionInfo, ServerName>>> future = new CompletableFuture<>();
CompletableFuture<List<Pair<RegionInfo, ServerName>>> future = new CompletableFuture<>();
if (tableName.filter((t) -> t.equals(TableName.META_TABLE_NAME)).isPresent()) {
future.completeExceptionally(new IOException(
"This method can't be used to locate meta regions;" + " use MetaTableLocator instead"));
}
// Make a version of CollectingVisitor that collects HRegionInfo and ServerAddress
CollectingVisitor<Pair<HRegionInfo, ServerName>> visitor = new CollectingVisitor<Pair<HRegionInfo, ServerName>>() {
// Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
CollectingVisitor<Pair<RegionInfo, ServerName>> visitor = new CollectingVisitor<Pair<RegionInfo, ServerName>>() {
private Optional<RegionLocations> current = null;
@Override
public boolean visit(Result r) throws IOException {
current = getRegionLocations(r);
if (!current.isPresent() || current.get().getRegionLocation().getRegionInfo() == null) {
LOG.warn("No serialized HRegionInfo in " + r);
if (!current.isPresent() || current.get().getRegionLocation().getRegion() == null) {
LOG.warn("No serialized RegionInfo in " + r);
return true;
}
HRegionInfo hri = current.get().getRegionLocation().getRegionInfo();
RegionInfo hri = current.get().getRegionLocation().getRegion();
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
// Else call super and add this Result to the collection.
return super.visit(r);
@ -251,7 +252,7 @@ public class AsyncMetaTableAccessor {
}
for (HRegionLocation loc : current.get().getRegionLocations()) {
if (loc != null) {
this.results.add(new Pair<HRegionInfo, ServerName>(loc.getRegionInfo(), loc
this.results.add(new Pair<RegionInfo, ServerName>(loc.getRegionInfo(), loc
.getServerName()));
}
}
@ -381,7 +382,7 @@ public class AsyncMetaTableAccessor {
*/
private static Optional<RegionLocations> getRegionLocations(final Result r) {
if (r == null) return Optional.empty();
Optional<HRegionInfo> regionInfo = getHRegionInfo(r, getRegionInfoColumn());
Optional<RegionInfo> regionInfo = getHRegionInfo(r, getRegionInfoColumn());
if (!regionInfo.isPresent()) return Optional.empty();
List<HRegionLocation> locations = new ArrayList<HRegionLocation>(1);
@ -427,11 +428,11 @@ public class AsyncMetaTableAccessor {
* @param replicaId the replicaId for the HRegionLocation
* @return HRegionLocation parsed from the given meta row Result for the given replicaId
*/
private static HRegionLocation getRegionLocation(final Result r, final HRegionInfo regionInfo,
private static HRegionLocation getRegionLocation(final Result r, final RegionInfo regionInfo,
final int replicaId) {
Optional<ServerName> serverName = getServerName(r, replicaId);
long seqNum = getSeqNumDuringOpen(r, replicaId);
HRegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
RegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
return new HRegionLocation(replicaInfo, serverName.orElse(null), seqNum);
}
@ -521,16 +522,16 @@ public class AsyncMetaTableAccessor {
}
/**
* Returns the HRegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
* Returns the RegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
* <code>qualifier</code> of the catalog table result.
* @param r a Result object from the catalog table scan
* @param qualifier Column family qualifier
* @return An HRegionInfo instance.
* @return An RegionInfo instance.
*/
private static Optional<HRegionInfo> getHRegionInfo(final Result r, byte[] qualifier) {
private static Optional<RegionInfo> getHRegionInfo(final Result r, byte[] qualifier) {
Cell cell = r.getColumnLatestCell(getCatalogFamily(), qualifier);
if (cell == null) return Optional.empty();
return Optional.ofNullable(HRegionInfo.parseFromOrNull(cell.getValueArray(),
return Optional.ofNullable(RegionInfo.parseFromOrNull(cell.getValueArray(),
cell.getValueOffset(), cell.getValueLength()));
}
@ -575,7 +576,7 @@ public class AsyncMetaTableAccessor {
return replicaId == 0
? HConstants.SERVER_QUALIFIER
: Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@ -587,7 +588,7 @@ public class AsyncMetaTableAccessor {
return replicaId == 0
? HConstants.STARTCODE_QUALIFIER
: Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@ -599,7 +600,7 @@ public class AsyncMetaTableAccessor {
return replicaId == 0
? HConstants.SEQNUM_QUALIFIER
: Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**

View File

@ -28,17 +28,18 @@ import java.util.stream.Collectors;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionInfoDisplay;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.client.RegionInfoDisplay;
import org.apache.hadoop.io.DataInputBuffer;
/**
* Information about a region. A region is a range of keys in the whole keyspace of a table, an
@ -308,7 +309,7 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
*
* @param other
*/
public HRegionInfo(HRegionInfo other) {
public HRegionInfo(RegionInfo other) {
super();
this.endKey = other.getEndKey();
this.offLine = other.isOffline();
@ -710,13 +711,13 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
* @param info the HRegionInfo to convert
* @return the converted RegionInfo
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use toProtoRegionInfo(org.apache.hadoop.hbase.client.RegionInfo)
* Use toRegionInfo(org.apache.hadoop.hbase.client.RegionInfo)
* in org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.
*/
@Deprecated
@InterfaceAudience.Private
public static HBaseProtos.RegionInfo convert(final HRegionInfo info) {
return ProtobufUtil.toProtoRegionInfo(info);
return ProtobufUtil.toRegionInfo(info);
}
/**

View File

@ -18,11 +18,13 @@
*/
package org.apache.hadoop.hbase;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ImmutableHRegionInfo;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Data structure to hold HRegionInfo and the address for the hosting
* Data structure to hold RegionInfo and the address for the hosting
* HRegionServer. Immutable. Comparable, but we compare the 'location' only:
* i.e. the hostname and port, and *not* the regioninfo. This means two
* instances are the same if they refer to the same 'location' (the same
@ -36,15 +38,15 @@ import org.apache.hadoop.hbase.util.Addressing;
*/
@InterfaceAudience.Public
public class HRegionLocation implements Comparable<HRegionLocation> {
private final HRegionInfo regionInfo;
private final RegionInfo regionInfo;
private final ServerName serverName;
private final long seqNum;
public HRegionLocation(HRegionInfo regionInfo, ServerName serverName) {
public HRegionLocation(RegionInfo regionInfo, ServerName serverName) {
this(regionInfo, serverName, HConstants.NO_SEQNUM);
}
public HRegionLocation(HRegionInfo regionInfo, ServerName serverName, long seqNum) {
public HRegionLocation(RegionInfo regionInfo, ServerName serverName, long seqNum) {
this.regionInfo = regionInfo;
this.serverName = serverName;
this.seqNum = seqNum;
@ -84,8 +86,20 @@ public class HRegionLocation implements Comparable<HRegionLocation> {
return this.serverName.hashCode();
}
/** @return HRegionInfo */
/**
*
* @return Immutable HRegionInfo
* @deprecated Since 2.0.0. Will remove in 3.0.0. Use {@link #getRegion()}} instead.
*/
@Deprecated
public HRegionInfo getRegionInfo(){
return regionInfo == null ? null : new ImmutableHRegionInfo(regionInfo);
}
/**
* @return regionInfo
*/
public RegionInfo getRegion(){
return regionInfo;
}

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
@ -36,7 +37,6 @@ import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Consistency;
@ -44,6 +44,8 @@ import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.RegionServerCallable;
@ -66,6 +68,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ExceptionUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
@ -152,9 +155,9 @@ public class MetaTableAccessor {
static {
// Copy the prefix from FIRST_META_REGIONINFO into META_REGION_PREFIX.
// FIRST_META_REGIONINFO == 'hbase:meta,,1'. META_REGION_PREFIX == 'hbase:meta,'
int len = HRegionInfo.FIRST_META_REGIONINFO.getRegionName().length - 2;
int len = RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName().length - 2;
META_REGION_PREFIX = new byte [len];
System.arraycopy(HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), 0,
System.arraycopy(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(), 0,
META_REGION_PREFIX, 0, len);
}
@ -168,9 +171,9 @@ public class MetaTableAccessor {
* @deprecated use {@link #getTableRegionsAndLocations}, region can have multiple locations
*/
@Deprecated
public static NavigableMap<HRegionInfo, ServerName> allTableRegions(
public static NavigableMap<RegionInfo, ServerName> allTableRegions(
Connection connection, final TableName tableName) throws IOException {
final NavigableMap<HRegionInfo, ServerName> regions = new TreeMap<>();
final NavigableMap<RegionInfo, ServerName> regions = new TreeMap<>();
Visitor visitor = new TableVisitorBase(tableName) {
@Override
public boolean visitInternal(Result result) throws IOException {
@ -178,7 +181,7 @@ public class MetaTableAccessor {
if (locations == null) return true;
for (HRegionLocation loc : locations.getRegionLocations()) {
if (loc != null) {
HRegionInfo regionInfo = loc.getRegionInfo();
RegionInfo regionInfo = loc.getRegionInfo();
regions.put(regionInfo, loc.getServerName());
}
}
@ -300,12 +303,12 @@ public class MetaTableAccessor {
* Gets the region info and assignment for the specified region.
* @param connection connection we're using
* @param regionName Region to lookup.
* @return Location and HRegionInfo for <code>regionName</code>
* @return Location and RegionInfo for <code>regionName</code>
* @throws IOException
* @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
*/
@Deprecated
public static Pair<HRegionInfo, ServerName> getRegion(Connection connection, byte [] regionName)
public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte [] regionName)
throws IOException {
HRegionLocation location = getRegionLocation(connection, regionName);
return location == null
@ -323,7 +326,7 @@ public class MetaTableAccessor {
public static HRegionLocation getRegionLocation(Connection connection,
byte[] regionName) throws IOException {
byte[] row = regionName;
HRegionInfo parsedInfo = null;
RegionInfo parsedInfo = null;
try {
parsedInfo = parseRegionInfoFromRegionName(regionName);
row = getMetaKeyForRegion(parsedInfo);
@ -347,7 +350,7 @@ public class MetaTableAccessor {
* @throws IOException
*/
public static HRegionLocation getRegionLocation(Connection connection,
HRegionInfo regionInfo) throws IOException {
RegionInfo regionInfo) throws IOException {
byte[] row = getMetaKeyForRegion(regionInfo);
Get get = new Get(row);
get.addFamily(HConstants.CATALOG_FAMILY);
@ -356,7 +359,7 @@ public class MetaTableAccessor {
}
/** Returns the row key to use for this regionInfo */
public static byte[] getMetaKeyForRegion(HRegionInfo regionInfo) {
public static byte[] getMetaKeyForRegion(RegionInfo regionInfo) {
return RegionReplicaUtil.getRegionInfoForDefaultReplica(regionInfo).getRegionName();
}
@ -364,12 +367,17 @@ public class MetaTableAccessor {
* is stored in the name, so the returned object should only be used for the fields
* in the regionName.
*/
public static HRegionInfo parseRegionInfoFromRegionName(byte[] regionName) throws IOException {
byte[][] fields = HRegionInfo.parseRegionName(regionName);
public static RegionInfo parseRegionInfoFromRegionName(byte[] regionName) throws IOException {
byte[][] fields = RegionInfo.parseRegionName(regionName);
long regionId = Long.parseLong(Bytes.toString(fields[2]));
int replicaId = fields.length > 3 ? Integer.parseInt(Bytes.toString(fields[3]), 16) : 0;
return new HRegionInfo(TableName.valueOf(fields[0]), fields[1], fields[1], false, regionId,
replicaId);
return RegionInfoBuilder.newBuilder(TableName.valueOf(fields[0]))
.setStartKey(fields[1])
.setEndKey(fields[2])
.setSplit(false)
.setRegionId(regionId)
.setReplicaId(replicaId)
.build();
}
/**
@ -392,11 +400,11 @@ public class MetaTableAccessor {
* @throws IOException
*/
@Nullable
public static Pair<HRegionInfo, HRegionInfo> getRegionsFromMergeQualifier(
public static Pair<RegionInfo, RegionInfo> getRegionsFromMergeQualifier(
Connection connection, byte[] regionName) throws IOException {
Result result = getRegionResult(connection, regionName);
HRegionInfo mergeA = getHRegionInfo(result, HConstants.MERGEA_QUALIFIER);
HRegionInfo mergeB = getHRegionInfo(result, HConstants.MERGEB_QUALIFIER);
RegionInfo mergeA = getRegionInfo(result, HConstants.MERGEA_QUALIFIER);
RegionInfo mergeB = getRegionInfo(result, HConstants.MERGEB_QUALIFIER);
if (mergeA == null && mergeB == null) {
return null;
}
@ -429,15 +437,15 @@ public class MetaTableAccessor {
* @throws IOException
*/
@VisibleForTesting
public static List<HRegionInfo> getAllRegions(Connection connection,
public static List<RegionInfo> getAllRegions(Connection connection,
boolean excludeOfflinedSplitParents)
throws IOException {
List<Pair<HRegionInfo, ServerName>> result;
List<Pair<RegionInfo, ServerName>> result;
result = getTableRegionsAndLocations(connection, null,
excludeOfflinedSplitParents);
return getListOfHRegionInfos(result);
return getListOfRegionInfos(result);
}
@ -446,10 +454,10 @@ public class MetaTableAccessor {
* to get meta table regions, use methods in MetaTableLocator instead.
* @param connection connection we're using
* @param tableName table we're looking for
* @return Ordered list of {@link HRegionInfo}.
* @return Ordered list of {@link RegionInfo}.
* @throws IOException
*/
public static List<HRegionInfo> getTableRegions(Connection connection, TableName tableName)
public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName)
throws IOException {
return getTableRegions(connection, tableName, false);
}
@ -461,22 +469,21 @@ public class MetaTableAccessor {
* @param tableName table we're looking for
* @param excludeOfflinedSplitParents If true, do not include offlined split
* parents in the return.
* @return Ordered list of {@link HRegionInfo}.
* @return Ordered list of {@link RegionInfo}.
* @throws IOException
*/
public static List<HRegionInfo> getTableRegions(Connection connection,
public static List<RegionInfo> getTableRegions(Connection connection,
TableName tableName, final boolean excludeOfflinedSplitParents)
throws IOException {
List<Pair<HRegionInfo, ServerName>> result =
List<Pair<RegionInfo, ServerName>> result =
getTableRegionsAndLocations(connection, tableName, excludeOfflinedSplitParents);
return getListOfHRegionInfos(result);
return getListOfRegionInfos(result);
}
@Nullable
static List<HRegionInfo> getListOfHRegionInfos(final List<Pair<HRegionInfo, ServerName>> pairs) {
if (pairs == null || pairs.isEmpty()) return null;
List<HRegionInfo> result = new ArrayList<>(pairs.size());
for (Pair<HRegionInfo, ServerName> pair: pairs) {
static List<RegionInfo> getListOfRegionInfos(final List<Pair<RegionInfo, ServerName>> pairs) {
if (pairs == null || pairs.isEmpty()) return Collections.EMPTY_LIST;
List<RegionInfo> result = new ArrayList<>(pairs.size());
for (Pair<RegionInfo, ServerName> pair: pairs) {
result.add(pair.getFirst());
}
return result;
@ -488,7 +495,7 @@ public class MetaTableAccessor {
* @return True if <code>current</code> tablename is equal to
* <code>tableName</code>
*/
static boolean isInsideTable(final HRegionInfo current, final TableName tableName) {
static boolean isInsideTable(final RegionInfo current, final TableName tableName) {
return tableName.equals(current.getTable());
}
@ -587,7 +594,7 @@ public class MetaTableAccessor {
* @return Return list of regioninfos and server.
* @throws IOException
*/
public static List<Pair<HRegionInfo, ServerName>>
public static List<Pair<RegionInfo, ServerName>>
getTableRegionsAndLocations(Connection connection, TableName tableName)
throws IOException {
return getTableRegionsAndLocations(connection, tableName, true);
@ -601,26 +608,26 @@ public class MetaTableAccessor {
* @return Return list of regioninfos and server addresses.
* @throws IOException
*/
public static List<Pair<HRegionInfo, ServerName>> getTableRegionsAndLocations(
public static List<Pair<RegionInfo, ServerName>> getTableRegionsAndLocations(
Connection connection, @Nullable final TableName tableName,
final boolean excludeOfflinedSplitParents) throws IOException {
if (tableName != null && tableName.equals(TableName.META_TABLE_NAME)) {
throw new IOException("This method can't be used to locate meta regions;"
+ " use MetaTableLocator instead");
}
// Make a version of CollectingVisitor that collects HRegionInfo and ServerAddress
CollectingVisitor<Pair<HRegionInfo, ServerName>> visitor =
new CollectingVisitor<Pair<HRegionInfo, ServerName>>() {
// Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
CollectingVisitor<Pair<RegionInfo, ServerName>> visitor =
new CollectingVisitor<Pair<RegionInfo, ServerName>>() {
private RegionLocations current = null;
@Override
public boolean visit(Result r) throws IOException {
current = getRegionLocations(r);
if (current == null || current.getRegionLocation().getRegionInfo() == null) {
LOG.warn("No serialized HRegionInfo in " + r);
LOG.warn("No serialized RegionInfo in " + r);
return true;
}
HRegionInfo hri = current.getRegionLocation().getRegionInfo();
RegionInfo hri = current.getRegionLocation().getRegionInfo();
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
// Else call super and add this Result to the collection.
return super.visit(r);
@ -652,10 +659,10 @@ public class MetaTableAccessor {
* catalog regions).
* @throws IOException
*/
public static NavigableMap<HRegionInfo, Result>
public static NavigableMap<RegionInfo, Result>
getServerUserRegions(Connection connection, final ServerName serverName)
throws IOException {
final NavigableMap<HRegionInfo, Result> hris = new TreeMap<>();
final NavigableMap<RegionInfo, Result> hris = new TreeMap<>();
// Fill the above hris map with entries from hbase:meta that have the passed
// servername.
CollectingVisitor<Result> v = new CollectingVisitor<Result>() {
@ -741,9 +748,9 @@ public class MetaTableAccessor {
startRow =
getTableStartRowForMeta(tableName, QueryType.REGION);
if (row != null) {
HRegionInfo closestRi =
RegionInfo closestRi =
getClosestRegionInfo(connection, tableName, row);
startRow = HRegionInfo
startRow = RegionInfo
.createRegionName(tableName, closestRi.getStartKey(), HConstants.ZEROES, false);
}
stopRow =
@ -813,11 +820,11 @@ public class MetaTableAccessor {
* @throws java.io.IOException
*/
@NonNull
public static HRegionInfo getClosestRegionInfo(Connection connection,
public static RegionInfo getClosestRegionInfo(Connection connection,
@NonNull final TableName tableName,
@NonNull final byte[] row)
throws IOException {
byte[] searchRow = HRegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
byte[] searchRow = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
Scan scan = getMetaScan(connection, 1);
scan.setReversed(true);
scan.setStartRow(searchRow);
@ -827,9 +834,9 @@ public class MetaTableAccessor {
throw new TableNotFoundException("Cannot find row in META " +
" for table: " + tableName + ", row=" + Bytes.toStringBinary(row));
}
HRegionInfo regionInfo = getHRegionInfo(result);
RegionInfo regionInfo = getRegionInfo(result);
if (regionInfo == null) {
throw new IOException("HRegionInfo was null or empty in Meta for " +
throw new IOException("RegionInfo was null or empty in Meta for " +
tableName + ", row=" + Bytes.toStringBinary(row));
}
return regionInfo;
@ -879,7 +886,7 @@ public class MetaTableAccessor {
return replicaId == 0
? HConstants.SERVER_QUALIFIER
: Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@ -892,7 +899,7 @@ public class MetaTableAccessor {
return replicaId == 0
? HConstants.STARTCODE_QUALIFIER
: Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@ -905,7 +912,7 @@ public class MetaTableAccessor {
return replicaId == 0
? HConstants.SEQNUM_QUALIFIER
: Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@ -975,7 +982,7 @@ public class MetaTableAccessor {
@Nullable
public static RegionLocations getRegionLocations(final Result r) {
if (r == null) return null;
HRegionInfo regionInfo = getHRegionInfo(r, getRegionInfoColumn());
RegionInfo regionInfo = getRegionInfo(r, getRegionInfoColumn());
if (regionInfo == null) return null;
List<HRegionLocation> locations = new ArrayList<>(1);
@ -1021,37 +1028,37 @@ public class MetaTableAccessor {
* @param replicaId the replicaId for the HRegionLocation
* @return HRegionLocation parsed from the given meta row Result for the given replicaId
*/
private static HRegionLocation getRegionLocation(final Result r, final HRegionInfo regionInfo,
private static HRegionLocation getRegionLocation(final Result r, final RegionInfo regionInfo,
final int replicaId) {
ServerName serverName = getServerName(r, replicaId);
long seqNum = getSeqNumDuringOpen(r, replicaId);
HRegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
RegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
return new HRegionLocation(replicaInfo, serverName, seqNum);
}
/**
* Returns HRegionInfo object from the column
* Returns RegionInfo object from the column
* HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog
* table Result.
* @param data a Result object from the catalog table scan
* @return HRegionInfo or null
* @return RegionInfo or null
*/
public static HRegionInfo getHRegionInfo(Result data) {
return getHRegionInfo(data, HConstants.REGIONINFO_QUALIFIER);
public static RegionInfo getRegionInfo(Result data) {
return getRegionInfo(data, HConstants.REGIONINFO_QUALIFIER);
}
/**
* Returns the HRegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
* Returns the RegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
* <code>qualifier</code> of the catalog table result.
* @param r a Result object from the catalog table scan
* @param qualifier Column family qualifier
* @return An HRegionInfo instance or null.
* @return An RegionInfo instance or null.
*/
@Nullable
private static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier) {
private static RegionInfo getRegionInfo(final Result r, byte [] qualifier) {
Cell cell = r.getColumnLatestCell(getCatalogFamily(), qualifier);
if (cell == null) return null;
return HRegionInfo.parseFromOrNull(cell.getValueArray(),
return RegionInfo.parseFromOrNull(cell.getValueArray(),
cell.getValueOffset(), cell.getValueLength());
}
@ -1059,12 +1066,12 @@ public class MetaTableAccessor {
* Returns the daughter regions by reading the corresponding columns of the catalog table
* Result.
* @param data a Result object from the catalog table scan
* @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
* @return a pair of RegionInfo or PairOfSameType(null, null) if the region is not a split
* parent
*/
public static PairOfSameType<HRegionInfo> getDaughterRegions(Result data) {
HRegionInfo splitA = getHRegionInfo(data, HConstants.SPLITA_QUALIFIER);
HRegionInfo splitB = getHRegionInfo(data, HConstants.SPLITB_QUALIFIER);
public static PairOfSameType<RegionInfo> getDaughterRegions(Result data) {
RegionInfo splitA = getRegionInfo(data, HConstants.SPLITA_QUALIFIER);
RegionInfo splitB = getRegionInfo(data, HConstants.SPLITB_QUALIFIER);
return new PairOfSameType<>(splitA, splitB);
}
@ -1073,12 +1080,12 @@ public class MetaTableAccessor {
* Returns the merge regions by reading the corresponding columns of the catalog table
* Result.
* @param data a Result object from the catalog table scan
* @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
* @return a pair of RegionInfo or PairOfSameType(null, null) if the region is not a split
* parent
*/
public static PairOfSameType<HRegionInfo> getMergeRegions(Result data) {
HRegionInfo mergeA = getHRegionInfo(data, HConstants.MERGEA_QUALIFIER);
HRegionInfo mergeB = getHRegionInfo(data, HConstants.MERGEB_QUALIFIER);
public static PairOfSameType<RegionInfo> getMergeRegions(Result data) {
RegionInfo mergeA = getRegionInfo(data, HConstants.MERGEA_QUALIFIER);
RegionInfo mergeB = getRegionInfo(data, HConstants.MERGEB_QUALIFIER);
return new PairOfSameType<>(mergeA, mergeB);
}
@ -1222,7 +1229,7 @@ public class MetaTableAccessor {
@Override
public boolean visit(Result rowResult) throws IOException {
HRegionInfo info = getHRegionInfo(rowResult);
RegionInfo info = getRegionInfo(rowResult);
if (info == null) {
return true;
}
@ -1251,7 +1258,7 @@ public class MetaTableAccessor {
@Override
public final boolean visit(Result rowResult) throws IOException {
HRegionInfo info = getHRegionInfo(rowResult);
RegionInfo info = getRegionInfo(rowResult);
if (info == null) {
return true;
}
@ -1298,7 +1305,7 @@ public class MetaTableAccessor {
/**
* Generates and returns a Put containing the region into for the catalog table
*/
public static Put makePutFromRegionInfo(HRegionInfo regionInfo)
public static Put makePutFromRegionInfo(RegionInfo regionInfo)
throws IOException {
return makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime());
}
@ -1306,7 +1313,7 @@ public class MetaTableAccessor {
/**
* Generates and returns a Put containing the region into for the catalog table
*/
public static Put makePutFromRegionInfo(HRegionInfo regionInfo, long ts)
public static Put makePutFromRegionInfo(RegionInfo regionInfo, long ts)
throws IOException {
Put put = new Put(regionInfo.getRegionName(), ts);
addRegionInfo(put, regionInfo);
@ -1317,7 +1324,7 @@ public class MetaTableAccessor {
* Generates and returns a Delete containing the region info for the catalog
* table
*/
public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo) {
public static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo) {
long now = EnvironmentEdgeManager.currentTime();
return makeDeleteFromRegionInfo(regionInfo, now);
}
@ -1326,7 +1333,7 @@ public class MetaTableAccessor {
* Generates and returns a Delete containing the region info for the catalog
* table
*/
public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo, long ts) {
public static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo, long ts) {
if (regionInfo == null) {
throw new IllegalArgumentException("Can't make a delete for null region");
}
@ -1356,14 +1363,14 @@ public class MetaTableAccessor {
/**
* Adds split daughters to the Put
*/
public static Put addDaughtersToPut(Put put, HRegionInfo splitA, HRegionInfo splitB) {
public static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo splitB) {
if (splitA != null) {
put.addImmutable(
HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER, splitA.toByteArray());
HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER, RegionInfo.toByteArray(splitA));
}
if (splitB != null) {
put.addImmutable(
HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER, splitB.toByteArray());
HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER, RegionInfo.toByteArray(splitB));
}
return put;
}
@ -1505,7 +1512,7 @@ public class MetaTableAccessor {
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(Connection connection,
HRegionInfo regionInfo)
RegionInfo regionInfo)
throws IOException {
putToMetaTable(connection, makePutFromRegionInfo(regionInfo));
LOG.info("Added " + regionInfo.getRegionNameAsString());
@ -1518,7 +1525,7 @@ public class MetaTableAccessor {
* @param regionInfo region information
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(Table meta, HRegionInfo regionInfo) throws IOException {
public static void addRegionToMeta(Table meta, RegionInfo regionInfo) throws IOException {
addRegionToMeta(meta, regionInfo, null, null);
}
@ -1526,7 +1533,7 @@ public class MetaTableAccessor {
* 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
* in the same row as the parent. Use
* {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName,int,boolean)}
* {@link #splitRegion(Connection, RegionInfo, RegionInfo, RegionInfo, ServerName,int,boolean)}
* if you want to do that.
* @param meta the Table for META
* @param regionInfo region information
@ -1534,8 +1541,8 @@ public class MetaTableAccessor {
* @param splitB second split daughter of the parent regionInfo
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(Table meta, HRegionInfo regionInfo,
HRegionInfo splitA, HRegionInfo splitB) throws IOException {
public static void addRegionToMeta(Table meta, RegionInfo regionInfo,
RegionInfo splitA, RegionInfo splitB) throws IOException {
Put put = makePutFromRegionInfo(regionInfo);
addDaughtersToPut(put, splitA, splitB);
meta.put(put);
@ -1551,7 +1558,7 @@ public class MetaTableAccessor {
* 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
* in the same row as the parent. Use
* {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName,int,boolean)}
* {@link #splitRegion(Connection, RegionInfo, RegionInfo, RegionInfo, ServerName,int,boolean)}
* if you want to do that.
* @param connection connection we're using
* @param regionInfo region information
@ -1559,8 +1566,8 @@ public class MetaTableAccessor {
* @param splitB second split daughter of the parent regionInfo
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(Connection connection, HRegionInfo regionInfo,
HRegionInfo splitA, HRegionInfo splitB) throws IOException {
public static void addRegionToMeta(Connection connection, RegionInfo regionInfo,
RegionInfo splitA, RegionInfo splitB) throws IOException {
Table meta = getMetaHTable(connection);
try {
addRegionToMeta(meta, regionInfo, splitA, splitB);
@ -1576,7 +1583,7 @@ public class MetaTableAccessor {
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionsToMeta(Connection connection,
List<HRegionInfo> regionInfos, int regionReplication)
List<RegionInfo> regionInfos, int regionReplication)
throws IOException {
addRegionsToMeta(connection, regionInfos, regionReplication, HConstants.LATEST_TIMESTAMP);
}
@ -1589,10 +1596,10 @@ public class MetaTableAccessor {
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionsToMeta(Connection connection,
List<HRegionInfo> regionInfos, int regionReplication, long ts)
List<RegionInfo> regionInfos, int regionReplication, long ts)
throws IOException {
List<Put> puts = new ArrayList<>();
for (HRegionInfo regionInfo : regionInfos) {
for (RegionInfo regionInfo : regionInfos) {
if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
Put put = makePutFromRegionInfo(regionInfo, ts);
// Add empty locations for region replicas so that number of replicas can be cached
@ -1614,7 +1621,7 @@ public class MetaTableAccessor {
* @param openSeqNum the latest sequence number obtained when the region was open
*/
public static void addDaughter(final Connection connection,
final HRegionInfo regionInfo, final ServerName sn, final long openSeqNum)
final RegionInfo regionInfo, final ServerName sn, final long openSeqNum)
throws NotAllMetaRegionsOnlineException, IOException {
long now = EnvironmentEdgeManager.currentTime();
Put put = new Put(regionInfo.getRegionName(), now);
@ -1640,23 +1647,21 @@ public class MetaTableAccessor {
* @param saveBarrier true if need save replication barrier in meta, used for serial replication
* @throws IOException
*/
public static void mergeRegions(final Connection connection, HRegionInfo mergedRegion,
HRegionInfo regionA, HRegionInfo regionB, ServerName sn, int regionReplication,
public static void mergeRegions(final Connection connection, RegionInfo mergedRegion,
RegionInfo regionA, RegionInfo regionB, ServerName sn, int regionReplication,
long masterSystemTime, boolean saveBarrier)
throws IOException {
Table meta = getMetaHTable(connection);
try {
HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
// use the maximum of what master passed us vs local time.
long time = Math.max(EnvironmentEdgeManager.currentTime(), masterSystemTime);
// Put for parent
Put putOfMerged = makePutFromRegionInfo(copyOfMerged, time);
Put putOfMerged = makePutFromRegionInfo(mergedRegion, time);
putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER,
regionA.toByteArray());
RegionInfo.toByteArray(regionA));
putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER,
regionB.toByteArray());
RegionInfo.toByteArray(regionB));
// Deletes for merging regions
Delete deleteA = makeDeleteFromRegionInfo(regionA, time);
@ -1707,17 +1712,15 @@ public class MetaTableAccessor {
* @param sn the location of the region
* @param saveBarrier true if need save replication barrier in meta, used for serial replication
*/
public static void splitRegion(final Connection connection, HRegionInfo parent,
HRegionInfo splitA, HRegionInfo splitB, ServerName sn, int regionReplication,
public static void splitRegion(final Connection connection, RegionInfo parent,
RegionInfo splitA, RegionInfo splitB, ServerName sn, int regionReplication,
boolean saveBarrier) throws IOException {
Table meta = getMetaHTable(connection);
try {
HRegionInfo copyOfParent = new HRegionInfo(parent);
copyOfParent.setOffline(true);
copyOfParent.setSplit(true);
//Put for parent
Put putParent = makePutFromRegionInfo(copyOfParent);
Put putParent = makePutFromRegionInfo(RegionInfoBuilder.newBuilder(parent)
.setOffline(true)
.setSplit(true).build());
addDaughtersToPut(putParent, splitA, splitB);
//Puts for daughters
@ -1878,7 +1881,7 @@ public class MetaTableAccessor {
* @throws IOException
*/
public static void updateRegionLocation(Connection connection,
HRegionInfo regionInfo, ServerName sn, long openSeqNum,
RegionInfo regionInfo, ServerName sn, long openSeqNum,
long masterSystemTime)
throws IOException {
updateLocation(connection, regionInfo, sn, openSeqNum, masterSystemTime);
@ -1920,7 +1923,7 @@ public class MetaTableAccessor {
* if the server is down on other end.
*/
private static void updateLocation(final Connection connection,
HRegionInfo regionInfo, ServerName sn, long openSeqNum,
RegionInfo regionInfo, ServerName sn, long openSeqNum,
long masterSystemTime)
throws IOException {
@ -1943,7 +1946,7 @@ public class MetaTableAccessor {
* @throws IOException
*/
public static void deleteRegion(Connection connection,
HRegionInfo regionInfo)
RegionInfo regionInfo)
throws IOException {
long time = EnvironmentEdgeManager.currentTime();
Delete delete = new Delete(regionInfo.getRegionName());
@ -1959,7 +1962,7 @@ public class MetaTableAccessor {
* @throws IOException
*/
public static void deleteRegions(Connection connection,
List<HRegionInfo> regionsInfo) throws IOException {
List<RegionInfo> regionsInfo) throws IOException {
deleteRegions(connection, regionsInfo, EnvironmentEdgeManager.currentTime());
}
/**
@ -1968,10 +1971,10 @@ public class MetaTableAccessor {
* @param regionsInfo list of regions to be deleted from META
* @throws IOException
*/
public static void deleteRegions(Connection connection, List<HRegionInfo> regionsInfo, long ts)
public static void deleteRegions(Connection connection, List<RegionInfo> regionsInfo, long ts)
throws IOException {
List<Delete> deletes = new ArrayList<>(regionsInfo.size());
for (HRegionInfo hri: regionsInfo) {
for (RegionInfo hri: regionsInfo) {
Delete e = new Delete(hri.getRegionName());
e.addFamily(getCatalogFamily(), ts);
deletes.add(e);
@ -1991,26 +1994,26 @@ public class MetaTableAccessor {
* @throws IOException
*/
public static void mutateRegions(Connection connection,
final List<HRegionInfo> regionsToRemove,
final List<HRegionInfo> regionsToAdd)
final List<RegionInfo> regionsToRemove,
final List<RegionInfo> regionsToAdd)
throws IOException {
List<Mutation> mutation = new ArrayList<>();
if (regionsToRemove != null) {
for (HRegionInfo hri: regionsToRemove) {
for (RegionInfo hri: regionsToRemove) {
mutation.add(makeDeleteFromRegionInfo(hri));
}
}
if (regionsToAdd != null) {
for (HRegionInfo hri: regionsToAdd) {
for (RegionInfo hri: regionsToAdd) {
mutation.add(makePutFromRegionInfo(hri));
}
}
mutateMetaTable(connection, mutation);
if (regionsToRemove != null && regionsToRemove.size() > 0) {
LOG.debug("Deleted " + HRegionInfo.getShortNameToLog(regionsToRemove));
LOG.debug("Deleted " + RegionInfo.getShortNameToLog(regionsToRemove));
}
if (regionsToAdd != null && regionsToAdd.size() > 0) {
LOG.debug("Added " + HRegionInfo.getShortNameToLog(regionsToAdd));
LOG.debug("Added " + RegionInfo.getShortNameToLog(regionsToAdd));
}
}
@ -2021,7 +2024,7 @@ public class MetaTableAccessor {
* @throws IOException
*/
public static void overwriteRegions(Connection connection,
List<HRegionInfo> regionInfos, int regionReplication) throws IOException {
List<RegionInfo> regionInfos, int regionReplication) throws IOException {
// use master time for delete marker and the Put
long now = EnvironmentEdgeManager.currentTime();
deleteRegions(connection, regionInfos, now);
@ -2045,7 +2048,7 @@ public class MetaTableAccessor {
* @throws IOException
*/
public static void deleteMergeQualifiers(Connection connection,
final HRegionInfo mergedRegion) throws IOException {
final RegionInfo mergedRegion) throws IOException {
long time = EnvironmentEdgeManager.currentTime();
Delete delete = new Delete(mergedRegion.getRegionName());
delete.addColumns(getCatalogFamily(), HConstants.MERGEA_QUALIFIER, time);
@ -2057,10 +2060,10 @@ public class MetaTableAccessor {
+ Bytes.toStringBinary(HConstants.MERGEB_QUALIFIER));
}
public static Put addRegionInfo(final Put p, final HRegionInfo hri)
public static Put addRegionInfo(final Put p, final RegionInfo hri)
throws IOException {
p.addImmutable(getCatalogFamily(), HConstants.REGIONINFO_QUALIFIER,
hri.toByteArray());
RegionInfo.toByteArray(hri));
return p;
}

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase;
import java.util.Collection;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.util.Bytes;
@ -143,14 +144,14 @@ public class RegionLocations {
*/
public RegionLocations remove(HRegionLocation location) {
if (location == null) return this;
if (location.getRegionInfo() == null) return this;
if (location.getRegion() == null) return this;
int replicaId = location.getRegionInfo().getReplicaId();
if (replicaId >= locations.length) return this;
// check whether something to remove. HRL.compareTo() compares ONLY the
// serverName. We want to compare the HRI's as well.
if (locations[replicaId] == null
|| !location.getRegionInfo().equals(locations[replicaId].getRegionInfo())
|| RegionInfo.COMPARATOR.compare(location.getRegion(), locations[replicaId].getRegion()) != 0
|| !location.equals(locations[replicaId])) {
return this;
}

View File

@ -17,9 +17,9 @@
*/
package org.apache.hadoop.hbase.client;
import java.util.List;
import java.util.Collection;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
@ -29,20 +29,19 @@ import java.util.regex.Pattern;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.quotas.QuotaFilter;
import org.apache.hadoop.hbase.quotas.QuotaSettings;
import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallable;
import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.client.security.SecurityCapability;
import org.apache.hadoop.hbase.quotas.QuotaFilter;
import org.apache.hadoop.hbase.quotas.QuotaSettings;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import com.google.protobuf.RpcChannel;
@ -306,12 +305,12 @@ public interface AsyncAdmin {
/**
* Get all the online regions on a region server.
*/
CompletableFuture<List<HRegionInfo>> getOnlineRegions(ServerName serverName);
CompletableFuture<List<RegionInfo>> getOnlineRegions(ServerName serverName);
/**
* Get the regions of a given table.
*/
CompletableFuture<List<HRegionInfo>> getTableRegions(TableName tableName);
CompletableFuture<List<RegionInfo>> getTableRegions(TableName tableName);
/**
* Flush a table.

View File

@ -32,12 +32,10 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallable;
import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.client.security.SecurityCapability;
@ -46,6 +44,7 @@ import org.apache.hadoop.hbase.quotas.QuotaSettings;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import com.google.protobuf.RpcChannel;
@ -215,12 +214,12 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
}
@Override
public CompletableFuture<List<HRegionInfo>> getOnlineRegions(ServerName serverName) {
public CompletableFuture<List<RegionInfo>> getOnlineRegions(ServerName serverName) {
return wrap(rawAdmin.getOnlineRegions(serverName));
}
@Override
public CompletableFuture<List<HRegionInfo>> getTableRegions(TableName tableName) {
public CompletableFuture<List<RegionInfo>> getTableRegions(TableName tableName) {
return wrap(rawAdmin.getTableRegions(tableName));
}

View File

@ -57,7 +57,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory;
import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
@ -67,6 +66,20 @@ import org.apache.hadoop.hbase.ipc.RpcClientFactory;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ExceptionUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
@ -110,19 +123,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Remov
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ExceptionUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import edu.umd.cs.findbugs.annotations.Nullable;
@ -600,13 +600,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
LOG.debug("Table " + tableName + " not enabled");
return false;
}
List<Pair<HRegionInfo, ServerName>> locations =
List<Pair<RegionInfo, ServerName>> locations =
MetaTableAccessor.getTableRegionsAndLocations(this, tableName, true);
int notDeployed = 0;
int regionCount = 0;
for (Pair<HRegionInfo, ServerName> pair : locations) {
HRegionInfo info = pair.getFirst();
for (Pair<RegionInfo, ServerName> pair : locations) {
RegionInfo info = pair.getFirst();
if (pair.getSecond() == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Table " + tableName + " has not deployed region " + pair.getFirst()
@ -675,10 +675,10 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
@Override
public List<HRegionLocation> locateRegions(final TableName tableName,
final boolean useCache, final boolean offlined) throws IOException {
List<HRegionInfo> regions = MetaTableAccessor
List<RegionInfo> regions = MetaTableAccessor
.getTableRegions(this, tableName, !offlined);
final List<HRegionLocation> locations = new ArrayList<>();
for (HRegionInfo regionInfo : regions) {
for (RegionInfo regionInfo : regions) {
RegionLocations list = locateRegion(tableName, regionInfo.getStartKey(), useCache, true);
if (list != null) {
for (HRegionLocation loc : list.getRegionLocations()) {

View File

@ -22,16 +22,16 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* A Callable for flushRegion() RPC.
@ -52,7 +52,7 @@ public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionR
}
public FlushRegionCallable(ClusterConnection connection,
RpcControllerFactory rpcControllerFactory, HRegionInfo regionInfo,
RpcControllerFactory rpcControllerFactory, RegionInfo regionInfo,
boolean writeFlushWalMarker) {
this(connection, rpcControllerFactory, regionInfo.getTable(), regionInfo.getRegionName(),
regionInfo.getStartKey(), writeFlushWalMarker);

View File

@ -40,6 +40,7 @@ import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -69,8 +70,6 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.client.security.SecurityCapability;
@ -86,6 +85,26 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@ -184,28 +203,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRe
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StringUtils;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Message;
import com.google.protobuf.RpcController;
import java.util.stream.Collectors;
/**
* HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that
@ -399,12 +400,26 @@ public class HBaseAdmin implements Admin {
@Override
public List<RegionInfo> getRegions(final ServerName sn) throws IOException {
return getOnlineRegions(sn).stream().collect(Collectors.toList());
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
// TODO: There is no timeout on this controller. Set one!
HBaseRpcController controller = rpcControllerFactory.newController();
return ProtobufUtil.getOnlineRegions(controller, admin);
}
@Override
public List<RegionInfo> getRegions(final TableName tableName) throws IOException {
return getTableRegions(tableName).stream().collect(Collectors.toList());
ZooKeeperWatcher zookeeper =
new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
try {
if (TableName.META_TABLE_NAME.equals(tableName)) {
return new MetaTableLocator().getMetaRegions(zookeeper);
} else {
return MetaTableAccessor.getTableRegions(connection, tableName, true);
}
} finally {
zookeeper.close();
}
}
private static class AbortProcedureFuture extends ProcedureFuture<Boolean> {
@ -1164,13 +1179,7 @@ public class HBaseAdmin implements Admin {
@Deprecated
@Override
public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
// TODO: There is no timeout on this controller. Set one!
HBaseRpcController controller = rpcControllerFactory.newController();
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(controller, admin);
return onlineRegions == null ? null : onlineRegions.stream()
.map(hri -> new ImmutableHRegionInfo(hri))
.collect(Collectors.toList());
return getRegions(sn).stream().map(ImmutableHRegionInfo::new).collect(Collectors.toList());
}
@Override
@ -1185,14 +1194,14 @@ public class HBaseAdmin implements Admin {
@Override
public void flushRegion(final byte[] regionName) throws IOException {
Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
if (regionServerPair == null) {
throw new IllegalArgumentException("Unknown regionname: " + Bytes.toStringBinary(regionName));
}
if (regionServerPair.getSecond() == null) {
throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
}
final HRegionInfo hRegionInfo = regionServerPair.getFirst();
final RegionInfo hRegionInfo = regionServerPair.getFirst();
ServerName serverName = regionServerPair.getSecond();
final AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
Callable<Void> callable = new Callable<Void>() {
@ -1304,13 +1313,13 @@ public class HBaseAdmin implements Admin {
checkTableExists(tableName);
zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
List<Pair<HRegionInfo, ServerName>> pairs;
List<Pair<RegionInfo, ServerName>> pairs;
if (TableName.META_TABLE_NAME.equals(tableName)) {
pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
} else {
pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
}
for (Pair<HRegionInfo, ServerName> pair: pairs) {
for (Pair<RegionInfo, ServerName> pair: pairs) {
if (pair.getFirst().isOffline()) continue;
if (pair.getSecond() == null) continue;
try {
@ -1344,7 +1353,7 @@ public class HBaseAdmin implements Admin {
*/
private void compactRegion(final byte[] regionName, final byte[] columnFamily,final boolean major)
throws IOException {
Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
if (regionServerPair == null) {
throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
}
@ -1354,7 +1363,7 @@ public class HBaseAdmin implements Admin {
compact(regionServerPair.getSecond(), regionServerPair.getFirst(), major, columnFamily);
}
private void compact(final ServerName sn, final HRegionInfo hri,
private void compact(final ServerName sn, final RegionInfo hri,
final boolean major, final byte [] family)
throws IOException {
final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
@ -1655,7 +1664,7 @@ public class HBaseAdmin implements Admin {
}
TableName tableName = null;
Pair<HRegionInfo, ServerName> pair;
Pair<RegionInfo, ServerName> pair;
for(int i = 0; i < nameofRegionsToMerge.length; i++) {
pair = getRegion(nameofRegionsToMerge[i]);
@ -1750,7 +1759,7 @@ public class HBaseAdmin implements Admin {
throws IOException {
byte[] encodedNameofRegionToSplit = HRegionInfo.isEncodedRegionName(regionName) ?
regionName : HRegionInfo.encodeRegionName(regionName).getBytes();
Pair<HRegionInfo, ServerName> pair = getRegion(regionName);
Pair<RegionInfo, ServerName> pair = getRegion(regionName);
if (pair != null) {
if (pair.getFirst() != null &&
pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
@ -1762,11 +1771,10 @@ public class HBaseAdmin implements Admin {
+ Bytes.toStringBinary(encodedNameofRegionToSplit));
}
HRegionInfo hri = pair.getFirst();
return splitRegionAsync(hri, splitPoint);
return splitRegionAsync(pair.getFirst(), splitPoint);
}
Future<Void> splitRegionAsync(HRegionInfo hri, byte[] splitPoint) throws IOException {
Future<Void> splitRegionAsync(RegionInfo hri, byte[] splitPoint) throws IOException {
TableName tableName = hri.getTable();
if (hri.getStartKey() != null && splitPoint != null &&
Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) {
@ -1824,7 +1832,7 @@ public class HBaseAdmin implements Admin {
checkTableExists(tableName);
zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
List<Pair<HRegionInfo, ServerName>> pairs;
List<Pair<RegionInfo, ServerName>> pairs;
if (TableName.META_TABLE_NAME.equals(tableName)) {
pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
} else {
@ -1833,10 +1841,10 @@ public class HBaseAdmin implements Admin {
if (splitPoint == null) {
LOG.info("SplitPoint is null, will find bestSplitPoint from Region");
}
for (Pair<HRegionInfo, ServerName> pair: pairs) {
for (Pair<RegionInfo, ServerName> pair: pairs) {
// May not be a server for a particular row
if (pair.getSecond() == null) continue;
HRegionInfo r = pair.getFirst();
RegionInfo r = pair.getFirst();
// check for parents
if (r.isSplitParent()) continue;
// if a split point given, only split that particular region
@ -1854,7 +1862,7 @@ public class HBaseAdmin implements Admin {
@Override
public void splitRegion(final byte[] regionName, final byte [] splitPoint) throws IOException {
Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
if (regionServerPair == null) {
throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
}
@ -1921,19 +1929,18 @@ public class HBaseAdmin implements Admin {
* Throw IllegalArgumentException if <code>regionName</code> is null.
* @throws IOException
*/
Pair<HRegionInfo, ServerName> getRegion(final byte[] regionName) throws IOException {
Pair<RegionInfo, ServerName> getRegion(final byte[] regionName) throws IOException {
if (regionName == null) {
throw new IllegalArgumentException("Pass a table name or region name");
}
Pair<HRegionInfo, ServerName> pair =
MetaTableAccessor.getRegion(connection, regionName);
Pair<RegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionName);
if (pair == null) {
final AtomicReference<Pair<HRegionInfo, ServerName>> result = new AtomicReference<>(null);
final AtomicReference<Pair<RegionInfo, ServerName>> result = new AtomicReference<>(null);
final String encodedName = Bytes.toString(regionName);
MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
@Override
public boolean visit(Result data) throws IOException {
HRegionInfo info = MetaTableAccessor.getHRegionInfo(data);
RegionInfo info = MetaTableAccessor.getRegionInfo(data);
if (info == null) {
LOG.warn("No serialized HRegionInfo in " + data);
return true;
@ -1979,7 +1986,7 @@ public class HBaseAdmin implements Admin {
return HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
}
byte[] tmp = regionNameOrEncodedRegionName;
Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
if (regionServerPair != null && regionServerPair.getFirst() != null) {
tmp = regionServerPair.getFirst().getRegionName();
}
@ -2373,23 +2380,10 @@ public class HBaseAdmin implements Admin {
@Deprecated
@Override
public List<HRegionInfo> getTableRegions(final TableName tableName)
throws IOException {
ZooKeeperWatcher zookeeper =
new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
List<HRegionInfo> regions = null;
try {
if (TableName.META_TABLE_NAME.equals(tableName)) {
regions = new MetaTableLocator().getMetaRegions(zookeeper);
} else {
regions = MetaTableAccessor.getTableRegions(connection, tableName, true);
}
} finally {
zookeeper.close();
}
return regions == null ? null : regions.stream()
.map(hri -> new ImmutableHRegionInfo(hri))
.collect(Collectors.toList());
throws IOException {
return getRegions(tableName).stream()
.map(ImmutableHRegionInfo::new)
.collect(Collectors.toList());
}
@Override
@ -2499,7 +2493,7 @@ public class HBaseAdmin implements Admin {
@Override
public CompactionState getCompactionStateForRegion(final byte[] regionName)
throws IOException {
final Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
final Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
if (regionServerPair == null) {
throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
}
@ -3320,7 +3314,7 @@ public class HBaseAdmin implements Admin {
default:
ZooKeeperWatcher zookeeper = null;
try {
List<Pair<HRegionInfo, ServerName>> pairs;
List<Pair<RegionInfo, ServerName>> pairs;
if (TableName.META_TABLE_NAME.equals(tableName)) {
zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
@ -3328,7 +3322,7 @@ public class HBaseAdmin implements Admin {
} else {
pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
}
for (Pair<HRegionInfo, ServerName> pair: pairs) {
for (Pair<RegionInfo, ServerName> pair: pairs) {
if (pair.getFirst().isOffline()) continue;
if (pair.getSecond() == null) continue;
final ServerName sn = pair.getSecond();

View File

@ -22,7 +22,6 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
@ -30,9 +29,11 @@ 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.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* An implementation of {@link RegionLocator}. Used to view region location information for a single
@ -84,10 +85,10 @@ public class HRegionLocator implements RegionLocator {
@Override
public List<HRegionLocation> getAllRegionLocations() throws IOException {
TableName tableName = getName();
List<Pair<HRegionInfo, ServerName>> locations =
List<Pair<RegionInfo, ServerName>> locations =
MetaTableAccessor.getTableRegionsAndLocations(this.connection, tableName);
ArrayList<HRegionLocation> regions = new ArrayList<>(locations.size());
for (Pair<HRegionInfo, ServerName> entry : locations) {
for (Pair<RegionInfo, ServerName> entry : locations) {
regions.add(new HRegionLocation(entry.getFirst(), entry.getSecond()));
}

View File

@ -19,9 +19,6 @@
*/
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
import java.io.IOException;
import java.util.AbstractMap.SimpleEntry;
import java.util.ArrayList;
@ -43,13 +40,15 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
* HTableMultiplexer provides a thread-safe non blocking PUT API across all the tables.
@ -207,7 +206,7 @@ public class HTableMultiplexer {
LinkedBlockingQueue<PutStatus> queue = getQueue(loc);
// Generate a MultiPutStatus object and offer it into the queue
PutStatus s = new PutStatus(loc.getRegionInfo(), put, maxAttempts);
PutStatus s = new PutStatus(loc.getRegion(), put, maxAttempts);
return queue.offer(s);
}
@ -372,11 +371,11 @@ public class HTableMultiplexer {
@VisibleForTesting
static class PutStatus {
final HRegionInfo regionInfo;
final RegionInfo regionInfo;
final Put put;
final int maxAttempCount;
public PutStatus(HRegionInfo regionInfo, Put put, int maxAttempCount) {
public PutStatus(RegionInfo regionInfo, Put put, int maxAttempCount) {
this.regionInfo = regionInfo;
this.put = put;
this.maxAttempCount = maxAttempCount;

View File

@ -33,7 +33,7 @@ public class ImmutableHRegionInfo extends HRegionInfo {
*
* @param other
*/
public ImmutableHRegionInfo(HRegionInfo other) {
public ImmutableHRegionInfo(RegionInfo other) {
super(other);
}

View File

@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
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.MetaTableAccessor.QueryType;
@ -61,7 +60,6 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.AdminRequestCallerBuilder;
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder;
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.ServerRequestCallerBuilder;
@ -78,6 +76,15 @@ import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
import org.apache.hadoop.hbase.shaded.io.netty.util.TimerTask;
@ -240,13 +247,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Remov
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.hadoop.hbase.util.Pair;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.Message;
@ -595,7 +595,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
int notDeployed = 0;
int regionCount = 0;
for (HRegionLocation location : locations) {
HRegionInfo info = location.getRegionInfo();
RegionInfo info = location.getRegionInfo();
if (location.getServerName() == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Table " + tableName + " has not deployed region "
@ -739,10 +739,10 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
}
@Override
public CompletableFuture<List<HRegionInfo>> getOnlineRegions(ServerName serverName) {
return this.<List<HRegionInfo>> newAdminCaller()
public CompletableFuture<List<RegionInfo>> getOnlineRegions(ServerName serverName) {
return this.<List<RegionInfo>> newAdminCaller()
.action((controller, stub) -> this
.<GetOnlineRegionRequest, GetOnlineRegionResponse, List<HRegionInfo>> adminCall(
.<GetOnlineRegionRequest, GetOnlineRegionResponse, List<RegionInfo>> adminCall(
controller, stub, RequestConverter.buildGetOnlineRegionRequest(),
(s, c, req, done) -> s.getOnlineRegion(c, req, done),
resp -> ProtobufUtil.getRegionInfos(resp)))
@ -750,7 +750,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
}
@Override
public CompletableFuture<List<HRegionInfo>> getTableRegions(TableName tableName) {
public CompletableFuture<List<RegionInfo>> getTableRegions(TableName tableName) {
if (tableName.equals(META_TABLE_NAME)) {
return connection.getLocator().getRegionLocation(tableName, null, null, operationTimeoutNs)
.thenApply(loc -> Arrays.asList(loc.getRegionInfo()));
@ -807,7 +807,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return;
}
HRegionInfo regionInfo = location.getRegionInfo();
RegionInfo regionInfo = location.getRegionInfo();
this.<Void> newAdminCaller()
.serverName(serverName)
.action(
@ -973,7 +973,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
/**
* Compact the region at specific region server.
*/
private CompletableFuture<Void> compact(final ServerName sn, final HRegionInfo hri,
private CompletableFuture<Void> compact(final ServerName sn, final RegionInfo hri,
final boolean major, Optional<byte[]> columnFamily) {
return this
.<Void> newAdminCaller()
@ -987,8 +987,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private byte[] toEncodeRegionName(byte[] regionName) {
try {
return HRegionInfo.isEncodedRegionName(regionName) ? regionName
: Bytes.toBytes(HRegionInfo.encodeRegionName(regionName));
return RegionInfo.isEncodedRegionName(regionName) ? regionName
: Bytes.toBytes(RegionInfo.encodeRegionName(regionName));
} catch (IOException e) {
return regionName;
}
@ -1002,8 +1002,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
result.completeExceptionally(err);
return;
}
HRegionInfo regionInfo = location.getRegionInfo();
if (regionInfo.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
RegionInfo regionInfo = location.getRegionInfo();
if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
result.completeExceptionally(new IllegalArgumentException(
"Can't invoke merge on non-default regions directly"));
return;
@ -1138,14 +1138,14 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
if (results != null && !results.isEmpty()) {
List<CompletableFuture<Void>> splitFutures = new ArrayList<>();
for (Result r : results) {
if (r.isEmpty() || MetaTableAccessor.getHRegionInfo(r) == null) continue;
if (r.isEmpty() || MetaTableAccessor.getRegionInfo(r) == null) continue;
RegionLocations rl = MetaTableAccessor.getRegionLocations(r);
if (rl != null) {
for (HRegionLocation h : rl.getRegionLocations()) {
if (h != null && h.getServerName() != null) {
HRegionInfo hri = h.getRegionInfo();
RegionInfo hri = h.getRegion();
if (hri == null || hri.isSplitParent()
|| hri.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID)
|| hri.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID)
continue;
splitFutures.add(split(hri, Optional.empty()));
}
@ -1202,8 +1202,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
CompletableFuture<Void> future = new CompletableFuture<>();
getRegionLocation(regionName).whenComplete(
(location, err) -> {
HRegionInfo regionInfo = location.getRegionInfo();
if (regionInfo.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
RegionInfo regionInfo = location.getRegionInfo();
if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
future.completeExceptionally(new IllegalArgumentException(
"Can't split replicas directly. "
+ "Replicas are auto-split when their primary is split."));
@ -1226,7 +1226,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return future;
}
private CompletableFuture<Void> split(final HRegionInfo hri,
private CompletableFuture<Void> split(final RegionInfo hri,
Optional<byte[]> splitPoint) {
if (hri.getStartKey() != null && splitPoint.isPresent()
&& Bytes.compareTo(hri.getStartKey(), splitPoint.get()) == 0) {
@ -2051,7 +2051,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
}
try {
CompletableFuture<Optional<HRegionLocation>> future;
if (HRegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
future = AsyncMetaTableAccessor.getRegionLocationWithEncodedName(metaTable,
regionNameOrEncodedRegionName);
} else {
@ -2087,19 +2087,19 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
* @param regionNameOrEncodedRegionName
* @return region info, wrapped by a {@link CompletableFuture}
*/
private CompletableFuture<HRegionInfo> getRegionInfo(byte[] regionNameOrEncodedRegionName) {
private CompletableFuture<RegionInfo> getRegionInfo(byte[] regionNameOrEncodedRegionName) {
if (regionNameOrEncodedRegionName == null) {
return failedFuture(new IllegalArgumentException("Passed region name can't be null"));
}
if (Bytes.equals(regionNameOrEncodedRegionName,
HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName())
|| Bytes.equals(regionNameOrEncodedRegionName,
HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
return CompletableFuture.completedFuture(HRegionInfo.FIRST_META_REGIONINFO);
RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
return CompletableFuture.completedFuture(RegionInfoBuilder.FIRST_META_REGIONINFO);
}
CompletableFuture<HRegionInfo> future = new CompletableFuture<>();
CompletableFuture<RegionInfo> future = new CompletableFuture<>();
getRegionLocation(regionNameOrEncodedRegionName).whenComplete((location, err) -> {
if (err != null) {
future.completeExceptionally(err);

View File

@ -17,16 +17,15 @@
*/
package org.apache.hadoop.hbase.client;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcChannel;
import com.google.protobuf.RpcController;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcChannel;
import com.google.protobuf.RpcController;
/**
* A low level asynchronous table.
* <p>
@ -134,10 +133,10 @@ public interface RawAsyncTable extends AsyncTableBase {
* As the locating itself also takes some time, the implementation may want to send rpc calls on
* the fly, which means we do not know how many regions we have when we get the return value of
* the rpc calls, so we need an {@link #onComplete()} which is used to tell you that we have
* passed all the return values to you(through the {@link #onRegionComplete(HRegionInfo, Object)}
* or {@link #onRegionError(HRegionInfo, Throwable)} calls), i.e, there will be no
* {@link #onRegionComplete(HRegionInfo, Object)} or
* {@link #onRegionError(HRegionInfo, Throwable)} calls in the future.
* passed all the return values to you(through the {@link #onRegionComplete(RegionInfo, Object)}
* or {@link #onRegionError(RegionInfo, Throwable)} calls), i.e, there will be no
* {@link #onRegionComplete(RegionInfo, Object)} or
* {@link #onRegionError(RegionInfo, Throwable)} calls in the future.
* <p>
* Here is a pseudo code to describe a typical implementation of a range coprocessor service
* method to help you better understand how the {@link CoprocessorCallback} will be called. The
@ -179,18 +178,18 @@ public interface RawAsyncTable extends AsyncTableBase {
* @param region the region that the response belongs to
* @param resp the response of the coprocessor call
*/
void onRegionComplete(HRegionInfo region, R resp);
void onRegionComplete(RegionInfo region, R resp);
/**
* @param region the region that the error belongs to
* @param error the response error of the coprocessor call
*/
void onRegionError(HRegionInfo region, Throwable error);
void onRegionError(RegionInfo region, Throwable error);
/**
* Indicate that all responses of the regions have been notified by calling
* {@link #onRegionComplete(HRegionInfo, Object)} or
* {@link #onRegionError(HRegionInfo, Throwable)}.
* {@link #onRegionComplete(RegionInfo, Object)} or
* {@link #onRegionError(RegionInfo, Throwable)}.
*/
void onComplete();

View File

@ -23,8 +23,6 @@ import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
import com.google.protobuf.RpcChannel;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@ -38,13 +36,15 @@ import java.util.function.Function;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.SingleRequestCallerBuilder;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@ -58,8 +58,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequ
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import com.google.protobuf.RpcChannel;
/**
* The implementation of RawAsyncTable.
@ -439,7 +439,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
}
private <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
CoprocessorCallable<S, R> callable, HRegionInfo region, byte[] row) {
CoprocessorCallable<S, R> callable, RegionInfo region, byte[] row) {
RegionCoprocessorRpcChannelImpl channel = new RegionCoprocessorRpcChannelImpl(conn, tableName,
region, row, rpcTimeoutNs, operationTimeoutNs);
S stub = stubMaker.apply(channel);
@ -461,7 +461,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
return coprocessorService(stubMaker, callable, null, row);
}
private boolean locateFinished(HRegionInfo region, byte[] endKey, boolean endKeyInclusive) {
private boolean locateFinished(RegionInfo region, byte[] endKey, boolean endKeyInclusive) {
if (isEmptyStopRow(endKey)) {
if (isEmptyStopRow(region.getEndKey())) {
return true;
@ -488,7 +488,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
return;
}
unfinishedRequest.incrementAndGet();
HRegionInfo region = loc.getRegionInfo();
RegionInfo region = loc.getRegionInfo();
if (locateFinished(region, endKey, endKeyInclusive)) {
locateFinished.set(true);
} else {

View File

@ -17,27 +17,27 @@
*/
package org.apache.hadoop.hbase.client;
import com.google.protobuf.Descriptors.MethodDescriptor;
import com.google.protobuf.Message;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcChannel;
import com.google.protobuf.RpcController;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.Descriptors.MethodDescriptor;
import com.google.protobuf.Message;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcChannel;
import com.google.protobuf.RpcController;
/**
* The implementation of a region based coprocessor rpc channel.
@ -49,7 +49,7 @@ class RegionCoprocessorRpcChannelImpl implements RpcChannel {
private final TableName tableName;
private final HRegionInfo region;
private final RegionInfo region;
private final byte[] row;
@ -57,7 +57,7 @@ class RegionCoprocessorRpcChannelImpl implements RpcChannel {
private final long operationTimeoutNs;
RegionCoprocessorRpcChannelImpl(AsyncConnectionImpl conn, TableName tableName, HRegionInfo region,
RegionCoprocessorRpcChannelImpl(AsyncConnectionImpl conn, TableName tableName, RegionInfo region,
byte[] row, long rpcTimeoutNs, long operationTimeoutNs) {
this.conn = conn;
this.tableName = tableName;

View File

@ -18,17 +18,6 @@
*/
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.MD5Hash;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import java.io.DataInputStream;
import java.io.IOException;
import java.util.ArrayList;
@ -36,9 +25,21 @@ import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.util.ByteArrayHashKey;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.HashKey;
import org.apache.hadoop.hbase.util.JenkinsHash;
import org.apache.hadoop.hbase.util.MD5Hash;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
/**
* Information about a region. A region is a range of keys in the whole keyspace
@ -436,7 +437,7 @@ public interface RegionInfo {
* @see #parseFrom(byte[])
*/
static byte [] toByteArray(RegionInfo ri) {
byte [] bytes = ProtobufUtil.toProtoRegionInfo(ri).toByteArray();
byte [] bytes = ProtobufUtil.toRegionInfo(ri).toByteArray();
return ProtobufUtil.prependPBMagic(bytes);
}
@ -691,7 +692,7 @@ public interface RegionInfo {
* @throws IOException
*/
static byte [] toDelimitedByteArray(RegionInfo ri) throws IOException {
return ProtobufUtil.toDelimitedByteArray(ProtobufUtil.toProtoRegionInfo(ri));
return ProtobufUtil.toDelimitedByteArray(ProtobufUtil.toRegionInfo(ri));
}
/**

View File

@ -18,16 +18,17 @@
*/
package org.apache.hadoop.hbase.client;
import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.ArrayUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import java.util.Arrays;
@InterfaceAudience.Private
public class RegionInfoBuilder {
private static final Log LOG = LogFactory.getLog(RegionInfoBuilder.class);
@ -52,7 +53,15 @@ public class RegionInfoBuilder {
public static final RegionInfo FIRST_META_REGIONINFO =
new MutableRegionInfo(1L, TableName.META_TABLE_NAME, RegionInfo.DEFAULT_REPLICA_ID);
private MutableRegionInfo content = null;
private final TableName tableName;
private byte[] startKey = HConstants.EMPTY_START_ROW;
private byte[] endKey = HConstants.EMPTY_END_ROW;
private long regionId = System.currentTimeMillis();
private int replicaId = RegionInfo.DEFAULT_REPLICA_ID;
private boolean offLine = false;
private boolean split = false;
private byte[] regionName = null;
private String encodedName = null;
public static RegionInfoBuilder newBuilder(TableName tableName) {
return new RegionInfoBuilder(tableName);
@ -63,52 +72,54 @@ public class RegionInfoBuilder {
}
private RegionInfoBuilder(TableName tableName) {
this.content = new MutableRegionInfo(tableName);
this.tableName = tableName;
}
private RegionInfoBuilder(RegionInfo regionInfo) {
this.content = new MutableRegionInfo(regionInfo);
this.tableName = regionInfo.getTable();
this.startKey = regionInfo.getStartKey();
this.endKey = regionInfo.getEndKey();
this.offLine = regionInfo.isOffline();
this.split = regionInfo.isSplit();
this.regionId = regionInfo.getRegionId();
this.replicaId = regionInfo.getReplicaId();
this.regionName = regionInfo.getRegionName();
this.encodedName = regionInfo.getEncodedName();
}
public RegionInfoBuilder setStartKey(byte[] startKey) {
content.setStartKey(startKey);
this.startKey = startKey;
return this;
}
public RegionInfoBuilder setEndKey(byte[] endKey) {
content.setEndKey(endKey);
this.endKey = endKey;
return this;
}
public RegionInfoBuilder setRegionId(long regionId) {
content.setRegionId(regionId);
this.regionId = regionId;
return this;
}
public RegionInfoBuilder setReplicaId(int replicaId) {
content.setReplicaId(replicaId);
this.replicaId = replicaId;
return this;
}
public RegionInfoBuilder setSplit(boolean isSplit) {
content.setSplit(isSplit);
public RegionInfoBuilder setSplit(boolean split) {
this.split = split;
return this;
}
public RegionInfoBuilder setOffline(boolean isOffline) {
content.setOffline(isOffline);
public RegionInfoBuilder setOffline(boolean offLine) {
this.offLine = offLine;
return this;
}
public RegionInfo build() {
RegionInfo ri = new MutableRegionInfo(content);
// Run a late check that we are not creating default meta region.
if (ri.getTable().equals(TableName.META_TABLE_NAME) &&
ri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
throw new IllegalArgumentException("Cannot create the default meta region; " +
"use static define FIRST_META_REGIONINFO");
}
return new MutableRegionInfo(content);
return new MutableRegionInfo(tableName, startKey, endKey, split,
regionId, replicaId, offLine, regionName, encodedName);
}
/**
@ -144,26 +155,49 @@ public class RegionInfoBuilder {
// but now table state is kept up in zookeeper as of 0.90.0 HBase.
private boolean offLine = false;
private boolean split = false;
private long regionId = -1;
private int replicaId = RegionInfo.DEFAULT_REPLICA_ID;
private transient byte [] regionName = HConstants.EMPTY_BYTE_ARRAY;
private byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
private int hashCode = -1;
private String encodedName;
private byte [] encodedNameAsBytes;
// Current TableName
private TableName tableName;
private final long regionId;
private final int replicaId;
private final byte[] regionName;
private final byte[] startKey;
private final byte[] endKey;
private final int hashCode;
private final String encodedName;
private final byte[] encodedNameAsBytes;
private final TableName tableName;
private void setHashCode() {
int result = Arrays.hashCode(this.regionName);
result ^= this.regionId;
result ^= Arrays.hashCode(this.startKey);
result ^= Arrays.hashCode(this.endKey);
result ^= Boolean.valueOf(this.offLine).hashCode();
result ^= Arrays.hashCode(this.tableName.getName());
result ^= this.replicaId;
this.hashCode = result;
private static int generateHashCode(final TableName tableName, final byte[] startKey,
final byte[] endKey, final long regionId,
final int replicaId, boolean offLine, byte[] regionName) {
int result = Arrays.hashCode(regionName);
result ^= regionId;
result ^= Arrays.hashCode(checkStartKey(startKey));
result ^= Arrays.hashCode(checkEndKey(endKey));
result ^= Boolean.valueOf(offLine).hashCode();
result ^= Arrays.hashCode(tableName.getName());
result ^= replicaId;
return result;
}
private static byte[] checkStartKey(byte[] startKey) {
return startKey == null? HConstants.EMPTY_START_ROW: startKey;
}
private static byte[] checkEndKey(byte[] endKey) {
return endKey == null? HConstants.EMPTY_END_ROW: endKey;
}
private static TableName checkTableName(TableName tableName) {
if (tableName == null) {
throw new IllegalArgumentException("TableName cannot be null");
}
return tableName;
}
private static int checkReplicaId(int regionId) {
if (regionId > MAX_REPLICA_ID) {
throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
}
return regionId;
}
/**
@ -171,162 +205,57 @@ public class RegionInfoBuilder {
* first meta regions
*/
private MutableRegionInfo(long regionId, TableName tableName, int replicaId) {
// This constructor is currently private for making hbase:meta region only.
super();
this.regionId = regionId;
this.tableName = tableName;
this.replicaId = replicaId;
// Note: First Meta region replicas names are in old format so we pass false here.
this.regionName =
RegionInfo.createRegionName(tableName, null, regionId, replicaId, false);
setHashCode();
this(tableName,
HConstants.EMPTY_START_ROW,
HConstants.EMPTY_END_ROW,
false,
regionId,
replicaId,
false,
RegionInfo.createRegionName(tableName, null, regionId, replicaId, false));
}
MutableRegionInfo(final TableName tableName) {
this(tableName, null, null);
}
/**
* Construct MutableRegionInfo with explicit parameters
*
* @param tableName the table name
* @param startKey first key in region
* @param endKey end of key range
* @throws IllegalArgumentException
*/
MutableRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey)
throws IllegalArgumentException {
this(tableName, startKey, endKey, false);
}
/**
* Construct MutableRegionInfo with explicit parameters
*
* @param tableName the table descriptor
* @param startKey first key in region
* @param endKey end of key range
* @param split true if this region has split and we have daughter regions
* regions that may or may not hold references to this region.
* @throws IllegalArgumentException
*/
MutableRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey,
final boolean split)
throws IllegalArgumentException {
this(tableName, startKey, endKey, split, System.currentTimeMillis());
}
/**
* Construct MutableRegionInfo with explicit parameters
*
* @param tableName the table descriptor
* @param startKey first key in region
* @param endKey end of key range
* @param split true if this region has split and we have daughter regions
* regions that may or may not hold references to this region.
* @param regionid Region id to use.
* @throws IllegalArgumentException
*/
MutableRegionInfo(final TableName tableName, final byte[] startKey,
final byte[] endKey, final boolean split, final long regionid)
throws IllegalArgumentException {
this(tableName, startKey, endKey, split, regionid, RegionInfo.DEFAULT_REPLICA_ID);
final byte[] endKey, final boolean split, final long regionId,
final int replicaId, boolean offLine, byte[] regionName) {
this(checkTableName(tableName),
checkStartKey(startKey),
checkEndKey(endKey),
split, regionId,
checkReplicaId(replicaId),
offLine,
regionName,
RegionInfo.encodeRegionName(regionName));
}
/**
* Construct MutableRegionInfo with explicit parameters
*
* @param tableName the table descriptor
* @param startKey first key in region
* @param endKey end of key range
* @param split true if this region has split and we have daughter regions
* regions that may or may not hold references to this region.
* @param regionid Region id to use.
* @param replicaId the replicaId to use
* @throws IllegalArgumentException
*/
MutableRegionInfo(final TableName tableName, final byte[] startKey,
final byte[] endKey, final boolean split, final long regionid,
final int replicaId)
throws IllegalArgumentException {
super();
if (tableName == null) {
throw new IllegalArgumentException("TableName cannot be null");
}
this.tableName = tableName;
this.offLine = false;
this.regionId = regionid;
this.replicaId = replicaId;
if (this.replicaId > MAX_REPLICA_ID) {
throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
}
this.regionName = RegionInfo.createRegionName(this.tableName, startKey, regionId, replicaId,
!this.tableName.equals(TableName.META_TABLE_NAME));
final byte[] endKey, final boolean split, final long regionId,
final int replicaId, boolean offLine, byte[] regionName, String encodedName) {
this.tableName = checkTableName(tableName);
this.startKey = checkStartKey(startKey);
this.endKey = checkEndKey(endKey);
this.split = split;
this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone();
this.startKey = startKey == null?
HConstants.EMPTY_START_ROW: startKey.clone();
this.tableName = tableName;
setHashCode();
}
/**
* Construct MutableRegionInfo.
* Only for RegionInfoBuilder to use.
* @param other
*/
MutableRegionInfo(MutableRegionInfo other, boolean isMetaRegion) {
super();
if (other.getTable() == null) {
throw new IllegalArgumentException("TableName cannot be null");
}
this.tableName = other.getTable();
this.offLine = other.isOffline();
this.regionId = other.getRegionId();
this.replicaId = other.getReplicaId();
if (this.replicaId > MAX_REPLICA_ID) {
throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
}
if(isMetaRegion) {
// Note: First Meta region replicas names are in old format
this.regionName = RegionInfo.createRegionName(
other.getTable(), null, other.getRegionId(),
other.getReplicaId(), false);
this.regionId = regionId;
this.replicaId = checkReplicaId(replicaId);
this.offLine = offLine;
if (ArrayUtils.isEmpty(regionName)) {
this.regionName = RegionInfo.createRegionName(this.tableName, this.startKey, this.regionId, this.replicaId,
!this.tableName.equals(TableName.META_TABLE_NAME));
this.encodedName = RegionInfo.encodeRegionName(this.regionName);
} else {
this.regionName = RegionInfo.createRegionName(
other.getTable(), other.getStartKey(), other.getRegionId(),
other.getReplicaId(), true);
this.regionName = regionName;
this.encodedName = encodedName;
}
this.split = other.isSplit();
this.endKey = other.getEndKey() == null? HConstants.EMPTY_END_ROW: other.getEndKey().clone();
this.startKey = other.getStartKey() == null?
HConstants.EMPTY_START_ROW: other.getStartKey().clone();
this.tableName = other.getTable();
setHashCode();
this.hashCode = generateHashCode(
this.tableName,
this.startKey,
this.endKey,
this.regionId,
this.replicaId,
this.offLine,
this.regionName);
this.encodedNameAsBytes = Bytes.toBytes(this.encodedName);
}
/**
* Construct a copy of RegionInfo as MutableRegionInfo.
* Only for RegionInfoBuilder to use.
* @param regionInfo
*/
MutableRegionInfo(RegionInfo regionInfo) {
super();
this.endKey = regionInfo.getEndKey();
this.offLine = regionInfo.isOffline();
this.regionId = regionInfo.getRegionId();
this.regionName = regionInfo.getRegionName();
this.split = regionInfo.isSplit();
this.startKey = regionInfo.getStartKey();
this.hashCode = regionInfo.hashCode();
this.encodedName = regionInfo.getEncodedName();
this.tableName = regionInfo.getTable();
this.replicaId = regionInfo.getReplicaId();
}
/**
* @return Return a short, printable name for this region
* (usually encoded name) for us logging.
@ -342,15 +271,6 @@ public class RegionInfoBuilder {
return regionId;
}
/**
* set region id.
* @param regionId
* @return MutableRegionInfo
*/
public MutableRegionInfo setRegionId(long regionId) {
this.regionId = regionId;
return this;
}
/**
* @return the regionName as an array of bytes.
@ -361,16 +281,6 @@ public class RegionInfoBuilder {
return regionName;
}
/**
* set region name.
* @param regionName
* @return MutableRegionInfo
*/
public MutableRegionInfo setRegionName(byte[] regionName) {
this.regionName = regionName;
return this;
}
/**
* @return Region name as a String for use in logging, etc.
*/
@ -389,18 +299,12 @@ public class RegionInfoBuilder {
/** @return the encoded region name */
@Override
public synchronized String getEncodedName() {
if (this.encodedName == null) {
this.encodedName = RegionInfo.encodeRegionName(this.regionName);
}
public String getEncodedName() {
return this.encodedName;
}
@Override
public synchronized byte [] getEncodedNameAsBytes() {
if (this.encodedNameAsBytes == null) {
this.encodedNameAsBytes = Bytes.toBytes(getEncodedName());
}
public byte [] getEncodedNameAsBytes() {
return this.encodedNameAsBytes;
}
@ -410,14 +314,6 @@ public class RegionInfoBuilder {
return startKey;
}
/**
* @param startKey
* @return MutableRegionInfo
*/
public MutableRegionInfo setStartKey(byte[] startKey) {
this.startKey = startKey;
return this;
}
/** @return the endKey */
@Override
@ -425,27 +321,12 @@ public class RegionInfoBuilder {
return endKey;
}
/**
* @param endKey
* @return MutableRegionInfo
*/
public MutableRegionInfo setEndKey(byte[] endKey) {
this.endKey = endKey;
return this;
}
/**
* Get current table name of the region
* @return TableName
*/
@Override
public TableName getTable() {
// This method name should be getTableName but there was already a method getTableName
// that returned a byte array. It is unfortunate given everywhere else, getTableName returns
// a TableName instance.
if (tableName == null || tableName.getName().length == 0) {
tableName = RegionInfo.getTable(getRegionName());
}
return this.tableName;
}
@ -560,11 +441,6 @@ public class RegionInfoBuilder {
return replicaId;
}
public MutableRegionInfo setReplicaId(int replicaId) {
this.replicaId = replicaId;
return this;
}
/**
* @see java.lang.Object#toString()
*/

View File

@ -21,9 +21,8 @@ package org.apache.hadoop.hbase.client;
import java.util.Collection;
import java.util.Iterator;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Utility methods which contain the logic for regions and replicas.
@ -49,30 +48,6 @@ public class RegionReplicaUtil {
*/
static final int DEFAULT_REPLICA_ID = 0;
/**
* Returns the HRegionInfo for the given replicaId. HRegionInfo's correspond to
* a range of a table, but more than one "instance" of the same range can be
* deployed which are differentiated by the replicaId.
* @param replicaId the replicaId to use
* @return an HRegionInfo object corresponding to the same range (table, start and
* end key), but for the given replicaId.
*/
@Deprecated // Deprecated for HBase-2.0.0, use #getRegionInfoForReplica
public static HRegionInfo getRegionInfoForReplica(HRegionInfo regionInfo, int replicaId) {
if (regionInfo.getReplicaId() == replicaId) {
return regionInfo;
}
HRegionInfo replicaInfo;
if (regionInfo.isMetaRegion()) {
replicaInfo = new HRegionInfo(regionInfo.getRegionId(), regionInfo.getTable(), replicaId);
} else {
replicaInfo = new HRegionInfo(regionInfo.getTable(), regionInfo.getStartKey(),
regionInfo.getEndKey(), regionInfo.isSplit(), regionInfo.getRegionId(), replicaId);
}
replicaInfo.setOffline(regionInfo.isOffline());
return replicaInfo;
}
/**
* Returns the RegionInfo for the given replicaId.
* RegionInfo's correspond to a range of a table, but more than one
@ -87,31 +62,33 @@ public class RegionReplicaUtil {
if (regionInfo.getReplicaId() == replicaId) {
return regionInfo;
}
RegionInfoBuilder replicaInfo;
RegionInfo ri;
if (regionInfo.isMetaRegion()) {
ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
return RegionInfoBuilder.newBuilder(regionInfo.getTable())
.setRegionId(regionInfo.getRegionId())
.setReplicaId(replicaId)
.setOffline(regionInfo.isOffline())
.build();
} else {
replicaInfo = RegionInfoBuilder.newBuilder(regionInfo.getTable())
return RegionInfoBuilder.newBuilder(regionInfo.getTable())
.setStartKey(regionInfo.getStartKey())
.setEndKey(regionInfo.getEndKey())
.setSplit(regionInfo.isSplit())
.setRegionId(regionInfo.getRegionId())
.setReplicaId(replicaId);
replicaInfo.setOffline(regionInfo.isOffline());
ri = replicaInfo.build();
.setReplicaId(replicaId)
.setOffline(regionInfo.isOffline())
.build();
}
return ri;
}
/**
* Returns the HRegionInfo for the default replicaId (0). HRegionInfo's correspond to
* Returns the RegionInfo for the default replicaId (0). RegionInfo's correspond to
* a range of a table, but more than one "instance" of the same range can be
* deployed which are differentiated by the replicaId.
* @return an HRegionInfo object corresponding to the same range (table, start and
* @return an RegionInfo object corresponding to the same range (table, start and
* end key), but for the default replicaId.
*/
public static HRegionInfo getRegionInfoForDefaultReplica(HRegionInfo regionInfo) {
public static RegionInfo getRegionInfoForDefaultReplica(RegionInfo regionInfo) {
return getRegionInfoForReplica(regionInfo, DEFAULT_REPLICA_ID);
}
@ -121,7 +98,7 @@ public class RegionReplicaUtil {
}
/** @return true if this region is a default replica for the region */
public static boolean isDefaultReplica(HRegionInfo hri) {
public static boolean isDefaultReplica(RegionInfo hri) {
return hri.getReplicaId() == DEFAULT_REPLICA_ID;
}
@ -129,22 +106,22 @@ public class RegionReplicaUtil {
* Removes the non-default replicas from the passed regions collection
* @param regions
*/
public static void removeNonDefaultRegions(Collection<HRegionInfo> regions) {
Iterator<HRegionInfo> iterator = regions.iterator();
public static void removeNonDefaultRegions(Collection<RegionInfo> regions) {
Iterator<RegionInfo> iterator = regions.iterator();
while (iterator.hasNext()) {
HRegionInfo hri = iterator.next();
RegionInfo hri = iterator.next();
if (!RegionReplicaUtil.isDefaultReplica(hri)) {
iterator.remove();
}
}
}
public static boolean isReplicasForSameRegion(HRegionInfo regionInfoA, HRegionInfo regionInfoB) {
public static boolean isReplicasForSameRegion(RegionInfo regionInfoA, RegionInfo regionInfoB) {
return compareRegionInfosWithoutReplicaId(regionInfoA, regionInfoB) == 0;
}
private static int compareRegionInfosWithoutReplicaId(HRegionInfo regionInfoA,
HRegionInfo regionInfoB) {
private static int compareRegionInfosWithoutReplicaId(RegionInfo regionInfoA,
RegionInfo regionInfoB) {
int result = regionInfoA.getTable().compareTo(regionInfoB.getTable());
if (result != 0) {
return result;

View File

@ -22,14 +22,13 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
/**
@ -76,8 +75,8 @@ class ZooKeeperRegistry implements Registry {
HRegionLocation[] locs = new HRegionLocation[servers.size()];
int i = 0;
for (ServerName server : servers) {
HRegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
HRegionInfo.FIRST_META_REGIONINFO, i);
RegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
RegionInfoBuilder.FIRST_META_REGIONINFO, i);
if (server == null) locs[i++] = null;
else locs[i++] = new HRegionLocation(h, server, 0);
}

View File

@ -19,10 +19,12 @@ package org.apache.hadoop.hbase.master;
import java.util.Date;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
/**
@ -161,27 +163,27 @@ public class RegionState {
}
private final long stamp;
private final HRegionInfo hri;
private final RegionInfo hri;
private final ServerName serverName;
private final State state;
// The duration of region in transition
private long ritDuration;
public RegionState(HRegionInfo region, State state) {
public RegionState(RegionInfo region, State state) {
this(region, state, System.currentTimeMillis(), null);
}
public RegionState(HRegionInfo region,
public RegionState(RegionInfo region,
State state, ServerName serverName) {
this(region, state, System.currentTimeMillis(), serverName);
}
public RegionState(HRegionInfo region,
public RegionState(RegionInfo region,
State state, long stamp, ServerName serverName) {
this(region, state, stamp, serverName, 0);
}
public RegionState(HRegionInfo region, State state, long stamp, ServerName serverName,
public RegionState(RegionInfo region, State state, long stamp, ServerName serverName,
long ritDuration) {
this.hri = region;
this.state = state;
@ -198,7 +200,7 @@ public class RegionState {
return stamp;
}
public HRegionInfo getRegion() {
public RegionInfo getRegion() {
return hri;
}
@ -381,7 +383,7 @@ public class RegionState {
*/
public ClusterStatusProtos.RegionState convert() {
ClusterStatusProtos.RegionState.Builder regionState = ClusterStatusProtos.RegionState.newBuilder();
regionState.setRegionInfo(HRegionInfo.convert(hri));
regionState.setRegionInfo(ProtobufUtil.toRegionInfo(hri));
regionState.setState(state.convert());
regionState.setStamp(getStamp());
return regionState.build();
@ -393,7 +395,7 @@ public class RegionState {
* @return the RegionState
*/
public static RegionState convert(ClusterStatusProtos.RegionState proto) {
return new RegionState(HRegionInfo.convert(proto.getRegionInfo()),
return new RegionState(ProtobufUtil.toRegionInfo(proto.getRegionInfo()),
State.convert(proto.getState()), proto.getStamp(), null);
}
@ -407,7 +409,8 @@ public class RegionState {
return false;
}
RegionState tmp = (RegionState)obj;
return tmp.hri.equals(hri) && tmp.state == state
return RegionInfo.COMPARATOR.compare(tmp.hri, hri) == 0 && tmp.state == state
&& ((serverName != null && serverName.equals(tmp.serverName))
|| (tmp.serverName == null && serverName == null));
}

View File

@ -25,6 +25,7 @@ import java.lang.reflect.Method;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
@ -56,7 +57,6 @@ import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerLoad;
@ -1721,14 +1721,14 @@ public final class ProtobufUtil {
* @return the retrieved region info
* @throws IOException
*/
public static HRegionInfo getRegionInfo(final RpcController controller,
public static org.apache.hadoop.hbase.client.RegionInfo getRegionInfo(final RpcController controller,
final AdminService.BlockingInterface admin, final byte[] regionName) throws IOException {
try {
GetRegionInfoRequest request =
RequestConverter.buildGetRegionInfoRequest(regionName);
GetRegionInfoResponse response =
admin.getRegionInfo(controller, request);
return HRegionInfo.convert(response.getRegionInfo());
return toRegionInfo(response.getRegionInfo());
} catch (ServiceException se) {
throw getRemoteException(se);
}
@ -1787,7 +1787,7 @@ public final class ProtobufUtil {
*
*/
public static void warmupRegion(final RpcController controller,
final AdminService.BlockingInterface admin, final HRegionInfo regionInfo) throws IOException {
final AdminService.BlockingInterface admin, final org.apache.hadoop.hbase.client.RegionInfo regionInfo) throws IOException {
try {
WarmupRegionRequest warmupRegionRequest =
@ -1806,7 +1806,7 @@ public final class ProtobufUtil {
* @throws IOException
*/
public static void openRegion(final RpcController controller,
final AdminService.BlockingInterface admin, ServerName server, final HRegionInfo region)
final AdminService.BlockingInterface admin, ServerName server, final org.apache.hadoop.hbase.client.RegionInfo region)
throws IOException {
OpenRegionRequest request =
RequestConverter.buildOpenRegionRequest(server, region, null, null);
@ -1825,7 +1825,7 @@ public final class ProtobufUtil {
* @return a list of online region info
* @throws IOException
*/
public static List<HRegionInfo> getOnlineRegions(final AdminService.BlockingInterface admin)
public static List<org.apache.hadoop.hbase.client.RegionInfo> getOnlineRegions(final AdminService.BlockingInterface admin)
throws IOException {
return getOnlineRegions(null, admin);
}
@ -1835,7 +1835,7 @@ public final class ProtobufUtil {
* server using admin protocol.
* @return a list of online region info
*/
public static List<HRegionInfo> getOnlineRegions(final RpcController controller,
public static List<org.apache.hadoop.hbase.client.RegionInfo> getOnlineRegions(final RpcController controller,
final AdminService.BlockingInterface admin)
throws IOException {
GetOnlineRegionRequest request = RequestConverter.buildGetOnlineRegionRequest();
@ -1852,13 +1852,13 @@ public final class ProtobufUtil {
* Get the list of region info from a GetOnlineRegionResponse
*
* @param proto the GetOnlineRegionResponse
* @return the list of region info or null if <code>proto</code> is null
* @return the list of region info or empty if <code>proto</code> is null
*/
public static List<HRegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
if (proto == null) return null;
List<HRegionInfo> regionInfos = new ArrayList<>(proto.getRegionInfoList().size());
public static List<org.apache.hadoop.hbase.client.RegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
if (proto == null) return Collections.EMPTY_LIST;
List<org.apache.hadoop.hbase.client.RegionInfo> regionInfos = new ArrayList<>(proto.getRegionInfoList().size());
for (RegionInfo regionInfo: proto.getRegionInfoList()) {
regionInfos.add(HRegionInfo.convert(regionInfo));
regionInfos.add(toRegionInfo(regionInfo));
}
return regionInfos;
}
@ -1950,7 +1950,7 @@ public final class ProtobufUtil {
RegionSpecifierType type = regionSpecifier.getType();
switch (type) {
case REGION_NAME:
return HRegionInfo.encodeRegionName(value.toByteArray());
return org.apache.hadoop.hbase.client.RegionInfo.encodeRegionName(value.toByteArray());
case ENCODED_REGION_NAME:
return value.toStringUtf8();
default:
@ -2074,12 +2074,14 @@ public final class ProtobufUtil {
return b.build();
}
public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] family,
public static CompactionDescriptor toCompactionDescriptor(
org.apache.hadoop.hbase.client.RegionInfo info, byte[] family,
List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
return toCompactionDescriptor(info, null, family, inputPaths, outputPaths, storeDir);
}
public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] regionName,
public static CompactionDescriptor toCompactionDescriptor(
org.apache.hadoop.hbase.client.RegionInfo info, byte[] regionName,
byte[] family, List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
// compaction descriptor contains relative paths.
// input / output paths are relative to the store dir
@ -2100,7 +2102,7 @@ public final class ProtobufUtil {
return builder.build();
}
public static FlushDescriptor toFlushDescriptor(FlushAction action, HRegionInfo hri,
public static FlushDescriptor toFlushDescriptor(FlushAction action, org.apache.hadoop.hbase.client.RegionInfo hri,
long flushSeqId, Map<byte[], List<Path>> committedFiles) {
FlushDescriptor.Builder desc = FlushDescriptor.newBuilder()
.setAction(action)
@ -2125,7 +2127,7 @@ public final class ProtobufUtil {
}
public static RegionEventDescriptor toRegionEventDescriptor(
EventType eventType, HRegionInfo hri, long seqId, ServerName server,
EventType eventType, org.apache.hadoop.hbase.client.RegionInfo hri, long seqId, ServerName server,
Map<byte[], List<Path>> storeFiles) {
final byte[] tableNameAsBytes = hri.getTable().getName();
final byte[] encodedNameAsBytes = hri.getEncodedNameAsBytes();
@ -3315,7 +3317,7 @@ public final class ProtobufUtil {
* @param info the RegionInfo to convert
* @return the converted Proto RegionInfo
*/
public static HBaseProtos.RegionInfo toProtoRegionInfo(final org.apache.hadoop.hbase.client.RegionInfo info) {
public static HBaseProtos.RegionInfo toRegionInfo(final org.apache.hadoop.hbase.client.RegionInfo info) {
if (info == null) return null;
HBaseProtos.RegionInfo.Builder builder = HBaseProtos.RegionInfo.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));

View File

@ -30,11 +30,9 @@ import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Action;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
@ -46,6 +44,7 @@ import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionCoprocessorServiceExec;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Row;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
@ -54,6 +53,12 @@ import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.security.token.Token;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
@ -70,7 +75,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavor
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest;
@ -81,6 +85,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationPr
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
@ -97,12 +102,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTabl
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
@ -115,17 +117,19 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormaliz
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
@ -140,10 +144,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetRe
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.security.token.Token;
/**
* Helper utility to build protocol buffer requests,
@ -886,10 +886,10 @@ public final class RequestConverter {
* @return a protocol buffer OpenRegionRequest
*/
public static OpenRegionRequest
buildOpenRegionRequest(ServerName server, final List<Pair<HRegionInfo,
buildOpenRegionRequest(ServerName server, final List<Pair<RegionInfo,
List<ServerName>>> regionOpenInfos, Boolean openForReplay) {
OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
for (Pair<HRegionInfo, List<ServerName>> regionOpenInfo: regionOpenInfos) {
for (Pair<RegionInfo, List<ServerName>> regionOpenInfo: regionOpenInfos) {
builder.addOpenInfo(buildRegionOpenInfo(regionOpenInfo.getFirst(),
regionOpenInfo.getSecond(), openForReplay));
}
@ -911,7 +911,7 @@ public final class RequestConverter {
* @return a protocol buffer OpenRegionRequest
*/
public static OpenRegionRequest buildOpenRegionRequest(ServerName server,
final HRegionInfo region, List<ServerName> favoredNodes,
final RegionInfo region, List<ServerName> favoredNodes,
Boolean openForReplay) {
OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
builder.addOpenInfo(buildRegionOpenInfo(region, favoredNodes,
@ -929,12 +929,12 @@ public final class RequestConverter {
* @return a protocol buffer UpdateFavoredNodesRequest
*/
public static UpdateFavoredNodesRequest buildUpdateFavoredNodesRequest(
final List<Pair<HRegionInfo, List<ServerName>>> updateRegionInfos) {
final List<Pair<RegionInfo, List<ServerName>>> updateRegionInfos) {
UpdateFavoredNodesRequest.Builder ubuilder = UpdateFavoredNodesRequest.newBuilder();
if (updateRegionInfos != null && !updateRegionInfos.isEmpty()) {
RegionUpdateInfo.Builder builder = RegionUpdateInfo.newBuilder();
for (Pair<HRegionInfo, List<ServerName>> pair : updateRegionInfos) {
builder.setRegion(HRegionInfo.convert(pair.getFirst()));
for (Pair<RegionInfo, List<ServerName>> pair : updateRegionInfos) {
builder.setRegion(ProtobufUtil.toRegionInfo(pair.getFirst()));
for (ServerName server : pair.getSecond()) {
builder.addFavoredNodes(ProtobufUtil.toServerName(server));
}
@ -950,9 +950,9 @@ public final class RequestConverter {
*
* @param regionInfo Region we are warming up
*/
public static WarmupRegionRequest buildWarmupRegionRequest(final HRegionInfo regionInfo) {
public static WarmupRegionRequest buildWarmupRegionRequest(final RegionInfo regionInfo) {
WarmupRegionRequest.Builder builder = WarmupRegionRequest.newBuilder();
builder.setRegionInfo(HRegionInfo.convert(regionInfo));
builder.setRegionInfo(ProtobufUtil.toRegionInfo(regionInfo));
return builder.build();
}
@ -1184,11 +1184,11 @@ public final class RequestConverter {
return builder.build();
}
public static SplitTableRegionRequest buildSplitTableRegionRequest(final HRegionInfo regionInfo,
public static SplitTableRegionRequest buildSplitTableRegionRequest(final RegionInfo regionInfo,
final byte[] splitRow, final long nonceGroup, final long nonce)
throws DeserializationException {
SplitTableRegionRequest.Builder builder = SplitTableRegionRequest.newBuilder();
builder.setRegionInfo(HRegionInfo.convert(regionInfo));
builder.setRegionInfo(ProtobufUtil.toRegionInfo(regionInfo));
if (splitRow != null) {
builder.setSplitRow(UnsafeByteOperations.unsafeWrap(splitRow));
}
@ -1611,10 +1611,10 @@ public final class RequestConverter {
* Create a RegionOpenInfo based on given region info and version of offline node
*/
public static RegionOpenInfo buildRegionOpenInfo(
final HRegionInfo region,
final RegionInfo region,
final List<ServerName> favoredNodes, Boolean openForReplay) {
RegionOpenInfo.Builder builder = RegionOpenInfo.newBuilder();
builder.setRegion(HRegionInfo.convert(region));
builder.setRegion(ProtobufUtil.toRegionInfo(region));
if (favoredNodes != null) {
for (ServerName server : favoredNodes) {
builder.addFavoredNodes(ProtobufUtil.toServerName(server));

View File

@ -28,16 +28,19 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.SingleResponse;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
@ -55,9 +58,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCata
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.util.StringUtils;
import edu.umd.cs.findbugs.annotations.Nullable;
@ -237,7 +237,7 @@ public final class ResponseConverter {
* @param proto the GetOnlineRegionResponse
* @return the list of region info
*/
public static List<HRegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
public static List<RegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
if (proto == null || proto.getRegionInfoCount() == 0) return null;
return ProtobufUtil.getRegionInfos(proto);
}
@ -280,10 +280,10 @@ public final class ResponseConverter {
* @return the response
*/
public static GetOnlineRegionResponse buildGetOnlineRegionResponse(
final List<HRegionInfo> regions) {
final List<RegionInfo> regions) {
GetOnlineRegionResponse.Builder builder = GetOnlineRegionResponse.newBuilder();
for (HRegionInfo region: regions) {
builder.addRegionInfo(HRegionInfo.convert(region));
for (RegionInfo region: regions) {
builder.addRegionInfo(ProtobufUtil.toRegionInfo(region));
}
return builder.build();
}

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hbase.zookeeper;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
import java.io.EOFException;
import java.io.IOException;
import java.net.ConnectException;
@ -27,19 +25,19 @@ import java.net.SocketException;
import java.net.SocketTimeoutException;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
@ -47,17 +45,20 @@ import org.apache.hadoop.hbase.ipc.FailedServerException;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.zookeeper.KeeperException;
/**
* Utility class to perform operation (get/wait for/verify/set/delete) on znode in ZooKeeper
@ -93,8 +94,8 @@ public class MetaTableLocator {
* @param zkw ZooKeeper watcher to be used
* @return meta table regions and their locations.
*/
public List<Pair<HRegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw) {
return getMetaRegionsAndLocations(zkw, HRegionInfo.DEFAULT_REPLICA_ID);
public List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw) {
return getMetaRegionsAndLocations(zkw, RegionInfo.DEFAULT_REPLICA_ID);
}
/**
@ -103,12 +104,12 @@ public class MetaTableLocator {
* @param replicaId
* @return meta table regions and their locations.
*/
public List<Pair<HRegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw,
public List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw,
int replicaId) {
ServerName serverName = getMetaRegionLocation(zkw, replicaId);
List<Pair<HRegionInfo, ServerName>> list = new ArrayList<>(1);
List<Pair<RegionInfo, ServerName>> list = new ArrayList<>(1);
list.add(new Pair<>(RegionReplicaUtil.getRegionInfoForReplica(
HRegionInfo.FIRST_META_REGIONINFO, replicaId), serverName));
RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), serverName));
return list;
}
@ -116,8 +117,8 @@ public class MetaTableLocator {
* @param zkw ZooKeeper watcher to be used
* @return List of meta regions
*/
public List<HRegionInfo> getMetaRegions(ZooKeeperWatcher zkw) {
return getMetaRegions(zkw, HRegionInfo.DEFAULT_REPLICA_ID);
public List<RegionInfo> getMetaRegions(ZooKeeperWatcher zkw) {
return getMetaRegions(zkw, RegionInfo.DEFAULT_REPLICA_ID);
}
/**
@ -126,17 +127,17 @@ public class MetaTableLocator {
* @param replicaId
* @return List of meta regions
*/
public List<HRegionInfo> getMetaRegions(ZooKeeperWatcher zkw, int replicaId) {
List<Pair<HRegionInfo, ServerName>> result;
public List<RegionInfo> getMetaRegions(ZooKeeperWatcher zkw, int replicaId) {
List<Pair<RegionInfo, ServerName>> result;
result = getMetaRegionsAndLocations(zkw, replicaId);
return getListOfHRegionInfos(result);
return getListOfRegionInfos(result);
}
private List<HRegionInfo> getListOfHRegionInfos(
final List<Pair<HRegionInfo, ServerName>> pairs) {
if (pairs == null || pairs.isEmpty()) return null;
List<HRegionInfo> result = new ArrayList<>(pairs.size());
for (Pair<HRegionInfo, ServerName> pair: pairs) {
private List<RegionInfo> getListOfRegionInfos(
final List<Pair<RegionInfo, ServerName>> pairs) {
if (pairs == null || pairs.isEmpty()) return Collections.EMPTY_LIST;
List<RegionInfo> result = new ArrayList<>(pairs.size());
for (Pair<RegionInfo, ServerName> pair: pairs) {
result.add(pair.getFirst());
}
return result;
@ -185,7 +186,7 @@ public class MetaTableLocator {
*/
public ServerName waitMetaRegionLocation(ZooKeeperWatcher zkw, long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException {
return waitMetaRegionLocation(zkw, HRegionInfo.DEFAULT_REPLICA_ID, timeout);
return waitMetaRegionLocation(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
}
/**
@ -261,7 +262,7 @@ public class MetaTableLocator {
public boolean verifyMetaRegionLocation(ClusterConnection hConnection,
ZooKeeperWatcher zkw, final long timeout)
throws InterruptedException, IOException {
return verifyMetaRegionLocation(hConnection, zkw, timeout, HRegionInfo.DEFAULT_REPLICA_ID);
return verifyMetaRegionLocation(hConnection, zkw, timeout, RegionInfo.DEFAULT_REPLICA_ID);
}
/**
@ -291,7 +292,7 @@ public class MetaTableLocator {
}
return (service != null) && verifyRegionLocation(connection, service,
getMetaRegionLocation(zkw, replicaId), RegionReplicaUtil.getRegionInfoForReplica(
HRegionInfo.FIRST_META_REGIONINFO, replicaId).getRegionName());
RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId).getRegionName());
}
/**
@ -425,7 +426,7 @@ public class MetaTableLocator {
*/
public static void setMetaLocation(ZooKeeperWatcher zookeeper,
ServerName serverName, RegionState.State state) throws KeeperException {
setMetaLocation(zookeeper, serverName, HRegionInfo.DEFAULT_REPLICA_ID, state);
setMetaLocation(zookeeper, serverName, RegionInfo.DEFAULT_REPLICA_ID, state);
}
/**
@ -456,7 +457,7 @@ public class MetaTableLocator {
ZKUtil.setData(zookeeper,
zookeeper.znodePaths.getZNodeForReplica(replicaId), data);
} catch(KeeperException.NoNodeException nne) {
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
LOG.debug("META region location doesn't exist, create it");
} else {
LOG.debug("META region location doesn't exist for replicaId=" + replicaId +
@ -470,7 +471,7 @@ public class MetaTableLocator {
* Load the meta region state from the meta server ZNode.
*/
public static RegionState getMetaRegionState(ZooKeeperWatcher zkw) throws KeeperException {
return getMetaRegionState(zkw, HRegionInfo.DEFAULT_REPLICA_ID);
return getMetaRegionState(zkw, RegionInfo.DEFAULT_REPLICA_ID);
}
/**
@ -514,7 +515,7 @@ public class MetaTableLocator {
state = RegionState.State.OFFLINE;
}
return new RegionState(
RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, replicaId),
RegionReplicaUtil.getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId),
state, serverName);
}
@ -525,12 +526,12 @@ public class MetaTableLocator {
*/
public void deleteMetaLocation(ZooKeeperWatcher zookeeper)
throws KeeperException {
deleteMetaLocation(zookeeper, HRegionInfo.DEFAULT_REPLICA_ID);
deleteMetaLocation(zookeeper, RegionInfo.DEFAULT_REPLICA_ID);
}
public void deleteMetaLocation(ZooKeeperWatcher zookeeper, int replicaId)
throws KeeperException {
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
LOG.info("Deleting hbase:meta region location in ZooKeeper");
} else {
LOG.info("Deleting hbase:meta for " + replicaId + " region location in ZooKeeper");
@ -586,7 +587,7 @@ public class MetaTableLocator {
public ServerName blockUntilAvailable(final ZooKeeperWatcher zkw,
final long timeout)
throws InterruptedException {
return blockUntilAvailable(zkw, HRegionInfo.DEFAULT_REPLICA_ID, timeout);
return blockUntilAvailable(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
}
/**

View File

@ -19,6 +19,9 @@
package org.apache.hadoop.hbase.client;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.io.InterruptedIOException;
@ -58,26 +61,23 @@ import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.AsyncProcessTask.ListRowAccess;
import org.apache.hadoop.hbase.client.AsyncProcessTask.SubmittedRows;
import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.Ignore;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestRule;
import org.mockito.Mockito;
import org.apache.hadoop.hbase.client.AsyncProcessTask.SubmittedRows;
import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@Category({ClientTests.class, MediumTests.class})
public class TestAsyncProcess {
@ -106,9 +106,9 @@ public class TestAsyncProcess {
private static final HRegionLocation loc3 = new HRegionLocation(hri3, sn2);
// Replica stuff
private static final HRegionInfo hri1r1 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1),
hri1r2 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 2);
private static final HRegionInfo hri2r1 = RegionReplicaUtil.getRegionInfoForReplica(hri2, 1);
private static final RegionInfo hri1r1 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1);
private static final RegionInfo hri1r2 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 2);
private static final RegionInfo hri2r1 = RegionReplicaUtil.getRegionInfoForReplica(hri2, 1);
private static final RegionLocations hrls1 = new RegionLocations(new HRegionLocation(hri1, sn),
new HRegionLocation(hri1r1, sn2), new HRegionLocation(hri1r2, sn3));
private static final RegionLocations hrls2 = new RegionLocations(new HRegionLocation(hri2, sn2),
@ -355,8 +355,8 @@ public class TestAsyncProcess {
private Map<ServerName, Long> customPrimarySleepMs = new HashMap<>();
private final AtomicLong replicaCalls = new AtomicLong(0);
public void addFailures(HRegionInfo... hris) {
for (HRegionInfo hri : hris) {
public void addFailures(RegionInfo... hris) {
for (RegionInfo hri : hris) {
failures.add(hri.getRegionName());
}
}

View File

@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.client.coprocessor;
import static org.apache.hadoop.hbase.client.coprocessor.AggregationHelper.getParsedGenericInstance;
import static org.apache.hadoop.hbase.client.coprocessor.AggregationHelper.validateArgAndGetPB;
import com.google.protobuf.Message;
import java.io.IOException;
import java.util.Map;
import java.util.NavigableMap;
@ -31,11 +29,10 @@ import java.util.TreeMap;
import java.util.concurrent.CompletableFuture;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RawAsyncTable;
import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallback;
import org.apache.hadoop.hbase.client.RawScanResultConsumer;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
@ -44,6 +41,9 @@ import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateRespo
import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateService;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
import com.google.protobuf.Message;
/**
* This client class is for invoking the aggregate functions deployed on the Region Server side via
@ -73,7 +73,7 @@ public class AsyncAggregationClient {
}
@Override
public synchronized void onRegionError(HRegionInfo region, Throwable error) {
public synchronized void onRegionError(RegionInfo region, Throwable error) {
completeExceptionally(error);
}
@ -82,11 +82,11 @@ public class AsyncAggregationClient {
completeExceptionally(error);
}
protected abstract void aggregate(HRegionInfo region, AggregateResponse resp)
protected abstract void aggregate(RegionInfo region, AggregateResponse resp)
throws IOException;
@Override
public synchronized void onRegionComplete(HRegionInfo region, AggregateResponse resp) {
public synchronized void onRegionComplete(RegionInfo region, AggregateResponse resp) {
try {
aggregate(region, resp);
} catch (IOException e) {
@ -135,7 +135,7 @@ public class AsyncAggregationClient {
private R max;
@Override
protected void aggregate(HRegionInfo region, AggregateResponse resp) throws IOException {
protected void aggregate(RegionInfo region, AggregateResponse resp) throws IOException {
if (resp.getFirstPartCount() > 0) {
R result = getCellValueFromProto(ci, resp, 0);
if (max == null || (result != null && ci.compare(max, result) < 0)) {
@ -171,7 +171,7 @@ public class AsyncAggregationClient {
private R min;
@Override
protected void aggregate(HRegionInfo region, AggregateResponse resp) throws IOException {
protected void aggregate(RegionInfo region, AggregateResponse resp) throws IOException {
if (resp.getFirstPartCount() > 0) {
R result = getCellValueFromProto(ci, resp, 0);
if (min == null || (result != null && ci.compare(min, result) > 0)) {
@ -208,7 +208,7 @@ public class AsyncAggregationClient {
private long count;
@Override
protected void aggregate(HRegionInfo region, AggregateResponse resp) throws IOException {
protected void aggregate(RegionInfo region, AggregateResponse resp) throws IOException {
count += resp.getFirstPart(0).asReadOnlyByteBuffer().getLong();
}
@ -239,7 +239,7 @@ public class AsyncAggregationClient {
private S sum;
@Override
protected void aggregate(HRegionInfo region, AggregateResponse resp) throws IOException {
protected void aggregate(RegionInfo region, AggregateResponse resp) throws IOException {
if (resp.getFirstPartCount() > 0) {
S s = getPromotedValueFromProto(ci, resp, 0);
sum = ci.add(sum, s);
@ -276,7 +276,7 @@ public class AsyncAggregationClient {
long count = 0L;
@Override
protected void aggregate(HRegionInfo region, AggregateResponse resp) throws IOException {
protected void aggregate(RegionInfo region, AggregateResponse resp) throws IOException {
if (resp.getFirstPartCount() > 0) {
sum = ci.add(sum, getPromotedValueFromProto(ci, resp, 0));
count += resp.getSecondPart().asReadOnlyByteBuffer().getLong();
@ -315,7 +315,7 @@ public class AsyncAggregationClient {
private long count;
@Override
protected void aggregate(HRegionInfo region, AggregateResponse resp) throws IOException {
protected void aggregate(RegionInfo region, AggregateResponse resp) throws IOException {
if (resp.getFirstPartCount() > 0) {
sum = ci.add(sum, getPromotedValueFromProto(ci, resp, 0));
sumSq = ci.add(sumSq, getPromotedValueFromProto(ci, resp, 1));
@ -357,7 +357,7 @@ public class AsyncAggregationClient {
private final NavigableMap<byte[], S> map = new TreeMap<>(Bytes.BYTES_COMPARATOR);
@Override
protected void aggregate(HRegionInfo region, AggregateResponse resp) throws IOException {
protected void aggregate(RegionInfo region, AggregateResponse resp) throws IOException {
if (resp.getFirstPartCount() > 0) {
map.put(region.getStartKey(), getPromotedValueFromProto(ci, resp, firstPartIndex));
}

View File

@ -18,9 +18,6 @@
*/
package org.apache.hadoop.hbase.coprocessor;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
import com.google.protobuf.Service;
import java.io.Closeable;
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
@ -39,12 +36,10 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
@ -64,7 +59,6 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.security.token.FsDelegationToken;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.util.ArrayUtils;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
@ -77,6 +71,14 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
import com.google.protobuf.Service;
/**
* Export an HBase table. Writes content to sequence files up in HDFS. Use
@ -179,7 +181,7 @@ public class Export extends ExportProtos.ExportService implements RegionCoproces
}
private static SequenceFile.Writer.Option getOutputPath(final Configuration conf,
final HRegionInfo info, final ExportProtos.ExportRequest request) throws IOException {
final RegionInfo info, final ExportProtos.ExportRequest request) throws IOException {
Path file = new Path(request.getOutputPath(), "export-" + info.getEncodedName());
FileSystem fs = file.getFileSystem(conf);
if (fs.exists(file)) {
@ -189,7 +191,7 @@ public class Export extends ExportProtos.ExportService implements RegionCoproces
}
private static List<SequenceFile.Writer.Option> getWriterOptions(final Configuration conf,
final HRegionInfo info, final ExportProtos.ExportRequest request) throws IOException {
final RegionInfo info, final ExportProtos.ExportRequest request) throws IOException {
List<SequenceFile.Writer.Option> rval = new LinkedList<>();
rval.add(SequenceFile.Writer.keyClass(ImmutableBytesWritable.class));
rval.add(SequenceFile.Writer.valueClass(Result.class));
@ -341,7 +343,7 @@ public class Export extends ExportProtos.ExportService implements RegionCoproces
}
}
private Scan validateKey(final HRegionInfo region, final ExportProtos.ExportRequest request) throws IOException {
private Scan validateKey(final RegionInfo region, final ExportProtos.ExportRequest request) throws IOException {
Scan scan = ProtobufUtil.toScan(request.getScan());
byte[] regionStartKey = region.getStartKey();
byte[] originStartKey = scan.getStartRow();

View File

@ -18,19 +18,25 @@
package org.apache.hadoop.hbase.coprocessor.example;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.ArrayList;
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.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
@ -41,7 +47,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
@ -51,13 +56,6 @@ import org.junit.After;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
@Category(MediumTests.class)
public class TestRefreshHFilesEndpoint {
private static final Log LOG = LogFactory.getLog(TestRefreshHFilesEndpoint.class);
@ -139,7 +137,7 @@ public class TestRefreshHFilesEndpoint {
HStoreWithFaultyRefreshHFilesAPI store;
public HRegionForRefreshHFilesEP(final Path tableDir, final WAL wal, final FileSystem fs,
final Configuration confParam, final HRegionInfo regionInfo,
final Configuration confParam, final RegionInfo regionInfo,
final TableDescriptor htd, final RegionServerServices rsServices) {
super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices);
}

View File

@ -18,11 +18,17 @@
package org.apache.hadoop.hbase.mapreduce;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.lang.reflect.Method;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
@ -34,13 +40,9 @@ import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.yetus.audience.InterfaceAudience;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.lang.reflect.Method;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* TableSnapshotInputFormat allows a MapReduce job to run over a table snapshot. The job
@ -120,10 +122,18 @@ public class TableSnapshotInputFormat extends InputFormat<ImmutableBytesWritable
delegate.readFields(in);
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegion()}
*/
@Deprecated
public HRegionInfo getRegionInfo() {
return delegate.getRegionInfo();
}
public RegionInfo getRegion() {
return delegate.getRegionInfo();
}
}
@VisibleForTesting

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.mapreduce.JobUtil;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
@ -113,7 +113,7 @@ public class CompactionTool extends Configured implements Tool {
Path regionDir = path.getParent();
Path tableDir = regionDir.getParent();
TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
RegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
compactStoreFiles(tableDir, htd, hri,
path.getName(), compactOnce, major);
} else if (isRegionDir(fs, path)) {
@ -139,7 +139,7 @@ public class CompactionTool extends Configured implements Tool {
private void compactRegion(final Path tableDir, final TableDescriptor htd,
final Path regionDir, final boolean compactOnce, final boolean major)
throws IOException {
HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
RegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
for (Path familyDir: FSUtils.getFamilyDirs(fs, regionDir)) {
compactStoreFiles(tableDir, htd, hri, familyDir.getName(), compactOnce, major);
}
@ -151,7 +151,7 @@ public class CompactionTool extends Configured implements Tool {
* no more compactions are needed. Uses the Configuration settings provided.
*/
private void compactStoreFiles(final Path tableDir, final TableDescriptor htd,
final HRegionInfo hri, final String familyName, final boolean compactOnce,
final RegionInfo hri, final String familyName, final boolean compactOnce,
final boolean major) throws IOException {
HStore store = getStore(conf, fs, tableDir, htd, hri, familyName, tmpDir);
LOG.info("Compact table=" + htd.getTableName() +
@ -183,7 +183,7 @@ public class CompactionTool extends Configured implements Tool {
* the store dir to compact as source.
*/
private static HStore getStore(final Configuration conf, final FileSystem fs,
final Path tableDir, final TableDescriptor htd, final HRegionInfo hri,
final Path tableDir, final TableDescriptor htd, final RegionInfo hri,
final String familyName, final Path tempDir) throws IOException {
HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, hri) {
@Override

View File

@ -19,9 +19,9 @@
package org.apache.hadoop.hbase.snapshot;
import java.io.BufferedInputStream;
import java.io.FileNotFoundException;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
@ -34,7 +34,6 @@ import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Option;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -44,18 +43,16 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.io.FileLink;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.WALLink;
import org.apache.hadoop.hbase.io.hadoopbackport.ThrottledInputStream;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
@ -64,18 +61,22 @@ import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
import org.apache.hadoop.mapreduce.security.TokenCache;
import org.apache.hadoop.hbase.io.hadoopbackport.ThrottledInputStream;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
/**
* Export the specified snapshot to a given FileSystem.
@ -566,7 +567,7 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
SnapshotReferenceUtil.visitReferencedFiles(conf, fs, snapshotDir, snapshotDesc,
new SnapshotReferenceUtil.SnapshotVisitor() {
@Override
public void storeFile(final HRegionInfo regionInfo, final String family,
public void storeFile(final RegionInfo regionInfo, final String family,
final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
// for storeFile.hasReference() case, copied as part of the manifest
if (!storeFile.hasReference()) {

View File

@ -40,11 +40,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeepDeletedCells;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
@ -53,6 +52,7 @@ import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@ -65,13 +65,13 @@ import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.Import.KeyValueImporter;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.LauncherSecurityManager;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.mapreduce.Mapper.Context;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.ToolRunner;
@ -739,7 +739,7 @@ public class TestImportExport {
Table importTable = UTIL.createTable(TableName.valueOf(importTableName), FAMILYA, 3);
// Register the wal listener for the import table
HRegionInfo region = UTIL.getHBaseCluster().getRegionServerThreads().get(0).getRegionServer()
RegionInfo region = UTIL.getHBaseCluster().getRegionServerThreads().get(0).getRegionServer()
.getRegions(importTable.getName()).get(0).getRegionInfo();
TableWALActionListener walListener = new TableWALActionListener(region);
WAL wal = UTIL.getMiniHBaseCluster().getRegionServer(0).getWAL(region);
@ -773,15 +773,15 @@ public class TestImportExport {
}
/**
* This listens to the {@link #visitLogEntryBeforeWrite(HRegionInfo, WALKey, WALEdit)} to
* This listens to the {@link #visitLogEntryBeforeWrite(RegionInfo, WALKey, WALEdit)} to
* identify that an entry is written to the Write Ahead Log for the given table.
*/
private static class TableWALActionListener extends WALActionsListener.Base {
private HRegionInfo regionInfo;
private RegionInfo regionInfo;
private boolean isVisited = false;
public TableWALActionListener(HRegionInfo region) {
public TableWALActionListener(RegionInfo region) {
this.regionInfo = region;
}

View File

@ -18,7 +18,11 @@
package org.apache.hadoop.hbase.replication;
import static org.junit.Assert.*;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.ArrayList;
@ -37,7 +41,6 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
@ -48,6 +51,8 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@ -56,11 +61,9 @@ import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
import org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.replication.regionserver.Replication;
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSource;
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@ -70,6 +73,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.mapreduce.Job;
import org.junit.Before;
@ -79,6 +83,7 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
@Category({ReplicationTests.class, LargeTests.class})
public class TestReplicationSmallTests extends TestReplicationBase {
@ -753,8 +758,10 @@ public class TestReplicationSmallTests extends TestReplicationBase {
public void testCompactionWALEdits() throws Exception {
WALProtos.CompactionDescriptor compactionDescriptor =
WALProtos.CompactionDescriptor.getDefaultInstance();
HRegionInfo hri = new HRegionInfo(htable1.getName(),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
RegionInfo hri = RegionInfoBuilder.newBuilder(htable1.getName())
.setStartKey(HConstants.EMPTY_START_ROW)
.setEndKey(HConstants.EMPTY_END_ROW)
.build();
WALEdit edit = WALEdit.createCompaction(hri, compactionDescriptor);
Replication.scopeWALEdits(new WALKey(), edit,
htable1.getConfiguration(), null);
@ -822,7 +829,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
final TableName tableName = htable1.getName();
HRegion region = utility1.getMiniHBaseCluster().getRegions(tableName).get(0);
HRegionInfo hri = region.getRegionInfo();
RegionInfo hri = region.getRegionInfo();
NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
for (byte[] fam : htable1.getTableDescriptor().getFamiliesKeys()) {
scopes.put(fam, 1);
@ -989,7 +996,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
final List<Path> emptyWalPaths = new ArrayList<>();
long ts = System.currentTimeMillis();
for (int i = 0; i < numRs; i++) {
HRegionInfo regionInfo =
RegionInfo regionInfo =
utility1.getHBaseCluster().getRegions(htable1.getName()).get(0).getRegionInfo();
WAL wal = utility1.getHBaseCluster().getRegionServer(i).getWAL(regionInfo);
Path currentWalPath = AbstractFSWALProvider.getCurrentFileName(wal);
@ -1012,7 +1019,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
// roll the original wal, which enqueues a new wal behind our empty wal
for (int i = 0; i < numRs; i++) {
HRegionInfo regionInfo =
RegionInfo regionInfo =
utility1.getHBaseCluster().getRegions(htable1.getName()).get(0).getRegionInfo();
WAL wal = utility1.getHBaseCluster().getRegionServer(i).getWAL(regionInfo);
wal.rollWriter(true);

View File

@ -39,12 +39,10 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -53,13 +51,16 @@ import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.Ignore;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.junit.rules.TestRule;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
/**
* Test Export Snapshot Tool
*/
@ -138,7 +139,7 @@ public class TestExportSnapshot {
}
protected interface RegionPredicate {
boolean evaluate(final HRegionInfo regionInfo);
boolean evaluate(final RegionInfo regionInfo);
}
protected RegionPredicate getBypassRegionPredicate() {
@ -314,7 +315,7 @@ public class TestExportSnapshot {
SnapshotReferenceUtil.visitReferencedFiles(conf, fs, exportedSnapshot,
new SnapshotReferenceUtil.SnapshotVisitor() {
@Override
public void storeFile(final HRegionInfo regionInfo, final String family,
public void storeFile(final RegionInfo regionInfo, final String family,
final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
if (bypassregionPredicate != null && bypassregionPredicate.evaluate(regionInfo))
return;

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.snapshot;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -57,7 +57,7 @@ public class TestMobExportSnapshot extends TestExportSnapshot {
protected RegionPredicate getBypassRegionPredicate() {
return new RegionPredicate() {
@Override
public boolean evaluate(final HRegionInfo regionInfo) {
public boolean evaluate(final RegionInfo regionInfo) {
return MobUtils.isMobRegionInfo(regionInfo);
}
};

View File

@ -19,30 +19,28 @@
package org.apache.hadoop.hbase.rest;
import java.io.IOException;
import java.util.Map;
import javax.ws.rs.GET;
import javax.ws.rs.Produces;
import javax.ws.rs.core.CacheControl;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.UriInfo;
import javax.ws.rs.core.Response.ResponseBuilder;
import javax.ws.rs.core.UriInfo;
import java.io.IOException;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.rest.model.TableInfoModel;
import org.apache.hadoop.hbase.rest.model.TableRegionModel;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class RegionsResource extends ResourceBase {
@ -81,11 +79,11 @@ public class RegionsResource extends ResourceBase {
Connection connection = ConnectionFactory.createConnection(servlet.getConfiguration());
@SuppressWarnings("deprecation")
Map<HRegionInfo, ServerName> regions = MetaTableAccessor
Map<RegionInfo, ServerName> regions = MetaTableAccessor
.allTableRegions(connection, tableName);
connection.close();
for (Map.Entry<HRegionInfo,ServerName> e: regions.entrySet()) {
HRegionInfo hri = e.getKey();
for (Map.Entry<RegionInfo,ServerName> e: regions.entrySet()) {
RegionInfo hri = e.getKey();
ServerName addr = e.getValue();
model.add(
new TableRegionModel(tableName.getNameAsString(), hri.getRegionId(),

View File

@ -31,11 +31,10 @@ import java.util.Set;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.LoadBalancer;
@ -48,6 +47,8 @@ import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
import org.apache.hadoop.hbase.master.locking.LockManager;
import org.apache.hadoop.hbase.net.Address;
import org.apache.hadoop.hbase.procedure2.LockType;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
@ -114,9 +115,9 @@ public class RSGroupAdminServer implements RSGroupAdmin {
/**
* @return List of Regions associated with this <code>server</code>.
*/
private List<HRegionInfo> getRegions(final Address server) {
LinkedList<HRegionInfo> regions = new LinkedList<>();
for (Map.Entry<HRegionInfo, ServerName> el :
private List<RegionInfo> getRegions(final Address server) {
LinkedList<RegionInfo> regions = new LinkedList<>();
for (Map.Entry<RegionInfo, ServerName> el :
master.getAssignmentManager().getRegionStates().getRegionAssignments().entrySet()) {
if (el.getValue() == null) continue;
if (el.getValue().getAddress().equals(server)) {
@ -131,7 +132,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
return regions;
}
private void addRegion(final LinkedList<HRegionInfo> regions, HRegionInfo hri) {
private void addRegion(final LinkedList<RegionInfo> regions, RegionInfo hri) {
// If meta, move it last otherwise other unassigns fail because meta is not
// online for them to update state in. This is dodgy. Needs to be made more
// robust. See TODO below.
@ -206,8 +207,8 @@ public class RSGroupAdminServer implements RSGroupAdmin {
for (Iterator<Address> iter = allSevers.iterator(); iter.hasNext();) {
Address rs = iter.next();
// Get regions that are associated with this server and filter regions by tables.
List<HRegionInfo> regions = new ArrayList<>();
for (HRegionInfo region : getRegions(rs)) {
List<RegionInfo> regions = new ArrayList<>();
for (RegionInfo region : getRegions(rs)) {
if (!tables.contains(region.getTable())) {
regions.add(region);
}
@ -216,7 +217,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
LOG.info("Unassigning " + regions.size() +
" region(s) from " + rs + " for server move to " + targetGroupName);
if (!regions.isEmpty()) {
for (HRegionInfo region: regions) {
for (RegionInfo region: regions) {
// Regions might get assigned from tables of target group so we need to filter
if (!targetGrp.containsTable(region.getTable())) {
this.master.getAssignmentManager().unassign(region);
@ -259,7 +260,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
} catch (InterruptedException e) {
throw new IOException("Interrupted when waiting for table lock", e);
}
for (HRegionInfo region :
for (RegionInfo region :
master.getAssignmentManager().getRegionStates().getRegionsOfTable(table)) {
ServerName sn = master.getAssignmentManager().getRegionStates().getRegionServerOfRegion(region);
if (!servers.contains(sn.getAddress())) {
@ -334,7 +335,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
for (Iterator<Address> iter = editableMovedServers.iterator(); iter.hasNext();) {
Address rs = iter.next();
// Get regions that are associated with this server.
List<HRegionInfo> regions = getRegions(rs);
List<RegionInfo> regions = getRegions(rs);
// Unassign regions for a server
// TODO: This is problematic especially if hbase:meta is in the mix.
@ -345,7 +346,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
" region(s) from " + rs + " for server move to " + targetGroupName);
if (!regions.isEmpty()) {
// TODO bulk unassign or throttled unassign?
for (HRegionInfo region: regions) {
for (RegionInfo region: regions) {
// Regions might get assigned from tables of target group so we need to filter
if (!targetGrp.containsTable(region.getTable())) {
this.master.getAssignmentManager().unassign(region);
@ -426,7 +427,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
} catch (InterruptedException e) {
throw new IOException("Interrupted when waiting for table lock", e);
}
for (HRegionInfo region :
for (RegionInfo region :
master.getAssignmentManager().getRegionStates().getRegionsOfTable(table)) {
master.getAssignmentManager().unassign(region);
}
@ -517,7 +518,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
//We balance per group instead of per table
List<RegionPlan> plans = new ArrayList<>();
for(Map.Entry<TableName, Map<ServerName, List<HRegionInfo>>> tableMap:
for(Map.Entry<TableName, Map<ServerName, List<RegionInfo>>> tableMap:
getRSGroupAssignmentsByTable(groupName).entrySet()) {
LOG.info("Creating partial plan for table " + tableMap.getKey() + ": "
+ tableMap.getValue());
@ -599,7 +600,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
Map<String, RegionState> rit = Maps.newTreeMap();
AssignmentManager am = master.getAssignmentManager();
for(TableName tableName : getRSGroupInfo(groupName).getTables()) {
for(HRegionInfo regionInfo: am.getRegionStates().getRegionsOfTable(tableName)) {
for(RegionInfo regionInfo: am.getRegionStates().getRegionsOfTable(tableName)) {
RegionState state = am.getRegionStates().getRegionTransitionState(regionInfo);
if(state != null) {
rit.put(regionInfo.getEncodedName(), state);
@ -609,16 +610,16 @@ public class RSGroupAdminServer implements RSGroupAdmin {
return rit;
}
private Map<TableName, Map<ServerName, List<HRegionInfo>>>
private Map<TableName, Map<ServerName, List<RegionInfo>>>
getRSGroupAssignmentsByTable(String groupName) throws IOException {
Map<TableName, Map<ServerName, List<HRegionInfo>>> result = Maps.newHashMap();
Map<TableName, Map<ServerName, List<RegionInfo>>> result = Maps.newHashMap();
RSGroupInfo rsGroupInfo = getRSGroupInfo(groupName);
Map<TableName, Map<ServerName, List<HRegionInfo>>> assignments = Maps.newHashMap();
for(Map.Entry<HRegionInfo, ServerName> entry:
Map<TableName, Map<ServerName, List<RegionInfo>>> assignments = Maps.newHashMap();
for(Map.Entry<RegionInfo, ServerName> entry:
master.getAssignmentManager().getRegionStates().getRegionAssignments().entrySet()) {
TableName currTable = entry.getKey().getTable();
ServerName currServer = entry.getValue();
HRegionInfo currRegion = entry.getKey();
RegionInfo currRegion = entry.getKey();
if (rsGroupInfo.getTables().contains(currTable)) {
assignments.putIfAbsent(currTable, new HashMap<>());
assignments.get(currTable).putIfAbsent(currServer, new ArrayList<>());
@ -626,7 +627,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
}
}
Map<ServerName, List<HRegionInfo>> serverMap = Maps.newHashMap();
Map<ServerName, List<RegionInfo>> serverMap = Maps.newHashMap();
for(ServerName serverName: master.getServerManager().getOnlineServers().keySet()) {
if(rsGroupInfo.getServers().contains(serverName.getAddress())) {
serverMap.put(serverName, Collections.emptyList());

View File

@ -18,13 +18,6 @@
package org.apache.hadoop.hbase.rsgroup;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.LinkedListMultimap;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
@ -43,10 +36,9 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.MasterServices;
@ -54,6 +46,14 @@ import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
import org.apache.hadoop.hbase.net.Address;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.LinkedListMultimap;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
/**
* GroupBasedLoadBalancer, used when Region Server Grouping is configured (HBase-6721)
@ -106,31 +106,31 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
@Override
public List<RegionPlan> balanceCluster(TableName tableName, Map<ServerName, List<HRegionInfo>>
public List<RegionPlan> balanceCluster(TableName tableName, Map<ServerName, List<RegionInfo>>
clusterState) throws HBaseIOException {
return balanceCluster(clusterState);
}
@Override
public List<RegionPlan> balanceCluster(Map<ServerName, List<HRegionInfo>> clusterState)
public List<RegionPlan> balanceCluster(Map<ServerName, List<RegionInfo>> clusterState)
throws HBaseIOException {
if (!isOnline()) {
throw new ConstraintException(RSGroupInfoManager.RSGROUP_TABLE_NAME +
" is not online, unable to perform balance");
}
Map<ServerName,List<HRegionInfo>> correctedState = correctAssignments(clusterState);
Map<ServerName,List<RegionInfo>> correctedState = correctAssignments(clusterState);
List<RegionPlan> regionPlans = new ArrayList<>();
List<HRegionInfo> misplacedRegions = correctedState.get(LoadBalancer.BOGUS_SERVER_NAME);
for (HRegionInfo regionInfo : misplacedRegions) {
List<RegionInfo> misplacedRegions = correctedState.get(LoadBalancer.BOGUS_SERVER_NAME);
for (RegionInfo regionInfo : misplacedRegions) {
regionPlans.add(new RegionPlan(regionInfo, null, null));
}
try {
List<RSGroupInfo> rsgi = rsGroupInfoManager.listRSGroups();
for (RSGroupInfo info: rsgi) {
Map<ServerName, List<HRegionInfo>> groupClusterState = new HashMap<>();
Map<TableName, Map<ServerName, List<HRegionInfo>>> groupClusterLoad = new HashMap<>();
Map<ServerName, List<RegionInfo>> groupClusterState = new HashMap<>();
Map<TableName, Map<ServerName, List<RegionInfo>>> groupClusterLoad = new HashMap<>();
for (Address sName : info.getServers()) {
for(ServerName curr: clusterState.keySet()) {
if(curr.getAddress().equals(sName)) {
@ -154,15 +154,15 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
@Override
public Map<ServerName, List<HRegionInfo>> roundRobinAssignment(
List<HRegionInfo> regions, List<ServerName> servers) throws HBaseIOException {
Map<ServerName, List<HRegionInfo>> assignments = Maps.newHashMap();
ListMultimap<String,HRegionInfo> regionMap = ArrayListMultimap.create();
public Map<ServerName, List<RegionInfo>> roundRobinAssignment(
List<RegionInfo> regions, List<ServerName> servers) throws HBaseIOException {
Map<ServerName, List<RegionInfo>> assignments = Maps.newHashMap();
ListMultimap<String,RegionInfo> regionMap = ArrayListMultimap.create();
ListMultimap<String,ServerName> serverMap = ArrayListMultimap.create();
generateGroupMaps(regions, servers, regionMap, serverMap);
for(String groupKey : regionMap.keySet()) {
if (regionMap.get(groupKey).size() > 0) {
Map<ServerName, List<HRegionInfo>> result =
Map<ServerName, List<RegionInfo>> result =
this.internalBalancer.roundRobinAssignment(
regionMap.get(groupKey),
serverMap.get(groupKey));
@ -181,13 +181,13 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
@Override
public Map<ServerName, List<HRegionInfo>> retainAssignment(
Map<HRegionInfo, ServerName> regions, List<ServerName> servers) throws HBaseIOException {
public Map<ServerName, List<RegionInfo>> retainAssignment(
Map<RegionInfo, ServerName> regions, List<ServerName> servers) throws HBaseIOException {
try {
Map<ServerName, List<HRegionInfo>> assignments = new TreeMap<>();
ListMultimap<String, HRegionInfo> groupToRegion = ArrayListMultimap.create();
Set<HRegionInfo> misplacedRegions = getMisplacedRegions(regions);
for (HRegionInfo region : regions.keySet()) {
Map<ServerName, List<RegionInfo>> assignments = new TreeMap<>();
ListMultimap<String, RegionInfo> groupToRegion = ArrayListMultimap.create();
Set<RegionInfo> misplacedRegions = getMisplacedRegions(regions);
for (RegionInfo region : regions.keySet()) {
if (!misplacedRegions.contains(region)) {
String groupName = rsGroupInfoManager.getRSGroupOfTable(region.getTable());
groupToRegion.put(groupName, region);
@ -196,11 +196,11 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
// Now the "groupToRegion" map has only the regions which have correct
// assignments.
for (String key : groupToRegion.keySet()) {
Map<HRegionInfo, ServerName> currentAssignmentMap = new TreeMap<HRegionInfo, ServerName>();
List<HRegionInfo> regionList = groupToRegion.get(key);
Map<RegionInfo, ServerName> currentAssignmentMap = new TreeMap<RegionInfo, ServerName>();
List<RegionInfo> regionList = groupToRegion.get(key);
RSGroupInfo info = rsGroupInfoManager.getRSGroup(key);
List<ServerName> candidateList = filterOfflineServers(info, servers);
for (HRegionInfo region : regionList) {
for (RegionInfo region : regionList) {
currentAssignmentMap.put(region, regions.get(region));
}
if(candidateList.size() > 0) {
@ -209,7 +209,7 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
}
for (HRegionInfo region : misplacedRegions) {
for (RegionInfo region : misplacedRegions) {
String groupName = rsGroupInfoManager.getRSGroupOfTable(region.getTable());;
RSGroupInfo info = rsGroupInfoManager.getRSGroup(groupName);
List<ServerName> candidateList = filterOfflineServers(info, servers);
@ -235,9 +235,9 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
@Override
public ServerName randomAssignment(HRegionInfo region,
public ServerName randomAssignment(RegionInfo region,
List<ServerName> servers) throws HBaseIOException {
ListMultimap<String,HRegionInfo> regionMap = LinkedListMultimap.create();
ListMultimap<String,RegionInfo> regionMap = LinkedListMultimap.create();
ListMultimap<String,ServerName> serverMap = LinkedListMultimap.create();
generateGroupMaps(Lists.newArrayList(region), servers, regionMap, serverMap);
List<ServerName> filteredServers = serverMap.get(regionMap.keySet().iterator().next());
@ -245,12 +245,12 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
private void generateGroupMaps(
List<HRegionInfo> regions,
List<RegionInfo> regions,
List<ServerName> servers,
ListMultimap<String, HRegionInfo> regionMap,
ListMultimap<String, RegionInfo> regionMap,
ListMultimap<String, ServerName> serverMap) throws HBaseIOException {
try {
for (HRegionInfo region : regions) {
for (RegionInfo region : regions) {
String groupName = rsGroupInfoManager.getRSGroupOfTable(region.getTable());
if (groupName == null) {
LOG.warn("Group for table "+region.getTable()+" is null");
@ -301,11 +301,11 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
return finalList;
}
private Set<HRegionInfo> getMisplacedRegions(
Map<HRegionInfo, ServerName> regions) throws IOException {
Set<HRegionInfo> misplacedRegions = new HashSet<>();
for(Map.Entry<HRegionInfo, ServerName> region : regions.entrySet()) {
HRegionInfo regionInfo = region.getKey();
private Set<RegionInfo> getMisplacedRegions(
Map<RegionInfo, ServerName> regions) throws IOException {
Set<RegionInfo> misplacedRegions = new HashSet<>();
for(Map.Entry<RegionInfo, ServerName> region : regions.entrySet()) {
RegionInfo regionInfo = region.getKey();
ServerName assignedServer = region.getValue();
RSGroupInfo info = rsGroupInfoManager.getRSGroup(rsGroupInfoManager.
getRSGroupOfTable(regionInfo.getTable()));
@ -323,17 +323,17 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
return misplacedRegions;
}
private Map<ServerName, List<HRegionInfo>> correctAssignments(
Map<ServerName, List<HRegionInfo>> existingAssignments)
private Map<ServerName, List<RegionInfo>> correctAssignments(
Map<ServerName, List<RegionInfo>> existingAssignments)
throws HBaseIOException{
Map<ServerName, List<HRegionInfo>> correctAssignments = new TreeMap<>();
List<HRegionInfo> misplacedRegions = new LinkedList<>();
Map<ServerName, List<RegionInfo>> correctAssignments = new TreeMap<>();
List<RegionInfo> misplacedRegions = new LinkedList<>();
correctAssignments.put(LoadBalancer.BOGUS_SERVER_NAME, new LinkedList<>());
for (Map.Entry<ServerName, List<HRegionInfo>> assignments : existingAssignments.entrySet()){
for (Map.Entry<ServerName, List<RegionInfo>> assignments : existingAssignments.entrySet()){
ServerName sName = assignments.getKey();
correctAssignments.put(sName, new LinkedList<>());
List<HRegionInfo> regions = assignments.getValue();
for (HRegionInfo region : regions) {
List<RegionInfo> regions = assignments.getValue();
for (RegionInfo region : regions) {
RSGroupInfo info = null;
try {
info = rsGroupInfoManager.getRSGroup(
@ -352,7 +352,7 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
//TODO bulk unassign?
//unassign misplaced regions, so that they are assigned to correct groups.
for(HRegionInfo info: misplacedRegions) {
for(RegionInfo info: misplacedRegions) {
try {
this.masterServices.getAssignmentManager().unassign(info);
} catch (IOException e) {
@ -395,15 +395,15 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
}
@Override
public void setClusterLoad(Map<TableName, Map<ServerName, List<HRegionInfo>>> clusterLoad) {
public void setClusterLoad(Map<TableName, Map<ServerName, List<RegionInfo>>> clusterLoad) {
}
@Override
public void regionOnline(HRegionInfo regionInfo, ServerName sn) {
public void regionOnline(RegionInfo regionInfo, ServerName sn) {
}
@Override
public void regionOffline(HRegionInfo regionInfo) {
public void regionOffline(RegionInfo regionInfo) {
}
@Override

View File

@ -42,19 +42,18 @@ import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MetaTableAccessor.DefaultVisitorBase;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
@ -74,16 +73,18 @@ import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos;
import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
import org.apache.hadoop.hbase.security.access.AccessControlLists;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import com.google.protobuf.ServiceException;
/**
@ -639,8 +640,8 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager {
}
private boolean waitForGroupTableOnline() {
final List<HRegionInfo> foundRegions = new LinkedList<>();
final List<HRegionInfo> assignedRegions = new LinkedList<>();
final List<RegionInfo> foundRegions = new LinkedList<>();
final List<RegionInfo> assignedRegions = new LinkedList<>();
final AtomicBoolean found = new AtomicBoolean(false);
final TableStateManager tsm = masterServices.getTableStateManager();
boolean createSent = false;
@ -659,7 +660,7 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager {
MetaTableAccessor.Visitor visitor = new DefaultVisitorBase() {
@Override
public boolean visitInternal(Result row) throws IOException {
HRegionInfo info = MetaTableAccessor.getHRegionInfo(row);
RegionInfo info = MetaTableAccessor.getRegionInfo(row);
if (info != null) {
Cell serverCell =
row.getColumnLatestCell(HConstants.CATALOG_FAMILY,

View File

@ -17,35 +17,9 @@
*/
package org.apache.hadoop.hbase.master.balancer;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.rsgroup.RSGroupBasedLoadBalancer;
import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
import org.apache.hadoop.hbase.rsgroup.RSGroupInfoManager;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.net.Address;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.FileNotFoundException;
import java.io.IOException;
@ -61,9 +35,37 @@ import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.net.Address;
import org.apache.hadoop.hbase.rsgroup.RSGroupBasedLoadBalancer;
import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
import org.apache.hadoop.hbase.rsgroup.RSGroupInfoManager;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
//TODO use stochastic based load balancer instead
@Category(SmallTests.class)
@ -114,7 +116,7 @@ public class TestRSGroupBasedLoadBalancer {
*/
@Test
public void testBalanceCluster() throws Exception {
Map<ServerName, List<HRegionInfo>> servers = mockClusterServers();
Map<ServerName, List<RegionInfo>> servers = mockClusterServers();
ArrayListMultimap<String, ServerAndLoad> list = convertToGroupBasedMap(servers);
LOG.info("Mock Cluster : " + printStats(list));
List<RegionPlan> plans = loadBalancer.balanceCluster(servers);
@ -169,11 +171,11 @@ public class TestRSGroupBasedLoadBalancer {
* @throws java.io.IOException
* @throws java.io.FileNotFoundException
*/
private void assertImmediateAssignment(List<HRegionInfo> regions,
private void assertImmediateAssignment(List<RegionInfo> regions,
List<ServerName> servers,
Map<HRegionInfo, ServerName> assignments)
Map<RegionInfo, ServerName> assignments)
throws IOException {
for (HRegionInfo region : regions) {
for (RegionInfo region : regions) {
assertTrue(assignments.containsKey(region));
ServerName server = assignments.get(region);
TableName tableName = region.getTable();
@ -197,8 +199,8 @@ public class TestRSGroupBasedLoadBalancer {
*/
@Test
public void testBulkAssignment() throws Exception {
List<HRegionInfo> regions = randomRegions(25);
Map<ServerName, List<HRegionInfo>> assignments = loadBalancer
List<RegionInfo> regions = randomRegions(25);
Map<ServerName, List<RegionInfo>> assignments = loadBalancer
.roundRobinAssignment(regions, servers);
//test empty region/servers scenario
//this should not throw an NPE
@ -207,8 +209,8 @@ public class TestRSGroupBasedLoadBalancer {
//test regular scenario
assertTrue(assignments.keySet().size() == servers.size());
for (ServerName sn : assignments.keySet()) {
List<HRegionInfo> regionAssigned = assignments.get(sn);
for (HRegionInfo region : regionAssigned) {
List<RegionInfo> regionAssigned = assignments.get(sn);
for (RegionInfo region : regionAssigned) {
TableName tableName = region.getTable();
String groupName =
getMockedGroupInfoManager().getRSGroupOfTable(tableName);
@ -233,16 +235,16 @@ public class TestRSGroupBasedLoadBalancer {
@Test
public void testRetainAssignment() throws Exception {
// Test simple case where all same servers are there
Map<ServerName, List<HRegionInfo>> currentAssignments = mockClusterServers();
Map<HRegionInfo, ServerName> inputForTest = new HashMap<>();
Map<ServerName, List<RegionInfo>> currentAssignments = mockClusterServers();
Map<RegionInfo, ServerName> inputForTest = new HashMap<>();
for (ServerName sn : currentAssignments.keySet()) {
for (HRegionInfo region : currentAssignments.get(sn)) {
for (RegionInfo region : currentAssignments.get(sn)) {
inputForTest.put(region, sn);
}
}
//verify region->null server assignment is handled
inputForTest.put(randomRegions(1).get(0), null);
Map<ServerName, List<HRegionInfo>> newAssignment = loadBalancer
Map<ServerName, List<RegionInfo>> newAssignment = loadBalancer
.retainAssignment(inputForTest, servers);
assertRetainedAssignment(inputForTest, servers, newAssignment);
}
@ -255,9 +257,9 @@ public class TestRSGroupBasedLoadBalancer {
public void testRoundRobinAssignment() throws Exception {
List<ServerName> onlineServers = new ArrayList<ServerName>(servers.size());
onlineServers.addAll(servers);
List<HRegionInfo> regions = randomRegions(25);
List<RegionInfo> regions = randomRegions(25);
int bogusRegion = 0;
for(HRegionInfo region : regions){
for(RegionInfo region : regions){
String group = tableMap.get(region.getTable());
if("dg3".equals(group) || "dg4".equals(group)){
bogusRegion++;
@ -273,7 +275,7 @@ public class TestRSGroupBasedLoadBalancer {
it.remove();
}
}
Map<ServerName, List<HRegionInfo>> assignments = loadBalancer
Map<ServerName, List<RegionInfo>> assignments = loadBalancer
.roundRobinAssignment(regions, onlineServers);
assertEquals(bogusRegion, assignments.get(LoadBalancer.BOGUS_SERVER_NAME).size());
}
@ -294,17 +296,17 @@ public class TestRSGroupBasedLoadBalancer {
* @throws java.io.FileNotFoundException
*/
private void assertRetainedAssignment(
Map<HRegionInfo, ServerName> existing, List<ServerName> servers,
Map<ServerName, List<HRegionInfo>> assignment)
Map<RegionInfo, ServerName> existing, List<ServerName> servers,
Map<ServerName, List<RegionInfo>> assignment)
throws FileNotFoundException, IOException {
// Verify condition 1, every region assigned, and to online server
Set<ServerName> onlineServerSet = new TreeSet<>(servers);
Set<HRegionInfo> assignedRegions = new TreeSet<>();
for (Map.Entry<ServerName, List<HRegionInfo>> a : assignment.entrySet()) {
Set<RegionInfo> assignedRegions = new TreeSet<>(RegionInfo.COMPARATOR);
for (Map.Entry<ServerName, List<RegionInfo>> a : assignment.entrySet()) {
assertTrue(
"Region assigned to server that was not listed as online",
onlineServerSet.contains(a.getKey()));
for (HRegionInfo r : a.getValue())
for (RegionInfo r : a.getValue())
assignedRegions.add(r);
}
assertEquals(existing.size(), assignedRegions.size());
@ -315,9 +317,9 @@ public class TestRSGroupBasedLoadBalancer {
onlineHostNames.add(s.getHostname());
}
for (Map.Entry<ServerName, List<HRegionInfo>> a : assignment.entrySet()) {
for (Map.Entry<ServerName, List<RegionInfo>> a : assignment.entrySet()) {
ServerName currentServer = a.getKey();
for (HRegionInfo r : a.getValue()) {
for (RegionInfo r : a.getValue()) {
ServerName oldAssignedServer = existing.get(r);
TableName tableName = r.getTable();
String groupName =
@ -374,7 +376,7 @@ public class TestRSGroupBasedLoadBalancer {
}
private ArrayListMultimap<String, ServerAndLoad> convertToGroupBasedMap(
final Map<ServerName, List<HRegionInfo>> serversMap) throws IOException {
final Map<ServerName, List<RegionInfo>> serversMap) throws IOException {
ArrayListMultimap<String, ServerAndLoad> loadMap = ArrayListMultimap
.create();
for (RSGroupInfo gInfo : getMockedGroupInfoManager().listRSGroups()) {
@ -387,7 +389,7 @@ public class TestRSGroupBasedLoadBalancer {
break;
}
}
List<HRegionInfo> regions = serversMap.get(actual);
List<RegionInfo> regions = serversMap.get(actual);
assertTrue("No load for " + actual, regions != null);
loadMap.put(gInfo.getName(),
new ServerAndLoad(actual, regions.size()));
@ -434,12 +436,12 @@ public class TestRSGroupBasedLoadBalancer {
}
}
private Map<ServerName, List<HRegionInfo>> mockClusterServers() throws IOException {
private Map<ServerName, List<RegionInfo>> mockClusterServers() throws IOException {
assertTrue(servers.size() == regionAssignment.length);
Map<ServerName, List<HRegionInfo>> assignment = new TreeMap<>();
Map<ServerName, List<RegionInfo>> assignment = new TreeMap<>();
for (int i = 0; i < servers.size(); i++) {
int numRegions = regionAssignment[i];
List<HRegionInfo> regions = assignedRegions(numRegions, servers.get(i));
List<RegionInfo> regions = assignedRegions(numRegions, servers.get(i));
assignment.put(servers.get(i), regions);
}
return assignment;
@ -449,10 +451,10 @@ public class TestRSGroupBasedLoadBalancer {
* Generate a list of regions evenly distributed between the tables.
*
* @param numRegions The number of regions to be generated.
* @return List of HRegionInfo.
* @return List of RegionInfo.
*/
private List<HRegionInfo> randomRegions(int numRegions) {
List<HRegionInfo> regions = new ArrayList<>(numRegions);
private List<RegionInfo> randomRegions(int numRegions) {
List<RegionInfo> regions = new ArrayList<>(numRegions);
byte[] start = new byte[16];
byte[] end = new byte[16];
rand.nextBytes(start);
@ -462,9 +464,12 @@ public class TestRSGroupBasedLoadBalancer {
Bytes.putInt(start, 0, numRegions << 1);
Bytes.putInt(end, 0, (numRegions << 1) + 1);
int tableIndex = (i + regionIdx) % tables.length;
HRegionInfo hri = new HRegionInfo(
tables[tableIndex], start, end, false, regionId++);
regions.add(hri);
regions.add(RegionInfoBuilder.newBuilder(tables[tableIndex])
.setStartKey(start)
.setEndKey(end)
.setSplit(false)
.setRegionId(regionId++)
.build());
}
return regions;
}
@ -477,18 +482,20 @@ public class TestRSGroupBasedLoadBalancer {
* @return the list of regions
* @throws java.io.IOException Signals that an I/O exception has occurred.
*/
private List<HRegionInfo> assignedRegions(int numRegions, ServerName sn) throws IOException {
List<HRegionInfo> regions = new ArrayList<>(numRegions);
private List<RegionInfo> assignedRegions(int numRegions, ServerName sn) throws IOException {
List<RegionInfo> regions = new ArrayList<>(numRegions);
byte[] start = new byte[16];
byte[] end = new byte[16];
Bytes.putInt(start, 0, numRegions << 1);
Bytes.putInt(end, 0, (numRegions << 1) + 1);
for (int i = 0; i < numRegions; i++) {
TableName tableName = getTableName(sn);
HRegionInfo hri = new HRegionInfo(
tableName, start, end, false,
regionId++);
regions.add(hri);
regions.add(RegionInfoBuilder.newBuilder(tableName)
.setStartKey(start)
.setEndKey(end)
.setSplit(false)
.setRegionId(regionId++)
.build());
}
return regions;
}

View File

@ -36,32 +36,32 @@ import java.util.TreeMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseCluster;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.net.Address;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestName;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
import org.junit.rules.TestName;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
public abstract class TestRSGroupsBase {
protected static final Log LOG = LogFactory.getLog(TestRSGroupsBase.class);
@ -166,7 +166,7 @@ public abstract class TestRSGroupsBase {
for(RegionLoad rl : status.getLoad(serverName).getRegionsLoad().values()) {
TableName tableName = null;
try {
tableName = HRegionInfo.getTable(rl.getName());
tableName = RegionInfo.getTable(rl.getName());
} catch (IllegalArgumentException e) {
LOG.warn("Failed parse a table name from regionname=" +
Bytes.toStringBinary(rl.getName()));
@ -417,7 +417,7 @@ public abstract class TestRSGroupsBase {
Map<ServerName,List<String>> assignMap =
getTableServerRegionMap().get(tableName);
final ServerName first = assignMap.entrySet().iterator().next().getKey();
for(HRegionInfo region: admin.getTableRegions(tableName)) {
for(RegionInfo region: admin.getTableRegions(tableName)) {
if(!assignMap.get(first).contains(region)) {
admin.move(region.getEncodedNameAsBytes(), Bytes.toBytes(first.getServerName()));
}
@ -514,7 +514,7 @@ public abstract class TestRSGroupsBase {
});
// Lets move this region to the new group.
TEST_UTIL.getAdmin().move(Bytes.toBytes(HRegionInfo.encodeRegionName(Bytes.toBytes(targetRegion))),
TEST_UTIL.getAdmin().move(Bytes.toBytes(RegionInfo.encodeRegionName(Bytes.toBytes(targetRegion))),
Bytes.toBytes(targetServer.getServerName()));
TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
@Override
@ -587,7 +587,7 @@ public abstract class TestRSGroupsBase {
appInfo.getServers().iterator().next().toString());
AdminProtos.AdminService.BlockingInterface targetRS =
((ClusterConnection) admin.getConnection()).getAdmin(targetServer);
HRegionInfo targetRegion = ProtobufUtil.getOnlineRegions(targetRS).get(0);
RegionInfo targetRegion = ProtobufUtil.getOnlineRegions(targetRS).get(0);
Assert.assertEquals(1, ProtobufUtil.getOnlineRegions(targetRS).size());
try {
@ -728,7 +728,7 @@ public abstract class TestRSGroupsBase {
//get server which is not a member of new group
ServerName targetServer = null;
for(ServerName server : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers()) {
if(!newGroup.containsServer(server.getAddress()) &&
if(!newGroup.containsServer(server.getAddress()) &&
!rsGroupAdmin.getRSGroupInfo("master").containsServer(server.getAddress())) {
targetServer = server;
break;
@ -780,7 +780,7 @@ public abstract class TestRSGroupsBase {
List<String> regionList = getTableRegionMap().get(tableName);
for(String region : regionList) {
// Lets move this region to the targetServer
TEST_UTIL.getAdmin().move(Bytes.toBytes(HRegionInfo.encodeRegionName(Bytes.toBytes(region))),
TEST_UTIL.getAdmin().move(Bytes.toBytes(RegionInfo.encodeRegionName(Bytes.toBytes(region))),
Bytes.toBytes(targetServer.getServerName()));
}

View File

@ -30,9 +30,9 @@ AssignmentManager assignmentManager = null;
</%args>
<%import>
java.util.*;
org.apache.hadoop.hbase.client.RegionInfo;
org.apache.hadoop.hbase.HBaseConfiguration;
org.apache.hadoop.hbase.HConstants;
org.apache.hadoop.hbase.HRegionInfo;
org.apache.hadoop.hbase.HTableDescriptor;
org.apache.hadoop.hbase.NamespaceDescriptor;
org.apache.hadoop.hbase.ServerLoad;
@ -435,7 +435,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
<%for HTableDescriptor htDesc : tables%>
<%java>
TableName tableName = htDesc.getTableName();
Map<RegionState.State, List<HRegionInfo>> tableRegions =
Map<RegionState.State, List<RegionInfo>> tableRegions =
master.getAssignmentManager().getRegionStates()
.getRegionByStateOfTable(tableName);
int openRegionsCount = tableRegions.get(RegionState.State.OPEN).size();
@ -444,7 +444,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
int failedRegionsCount = tableRegions.get(RegionState.State.FAILED_OPEN).size()
+ tableRegions.get(RegionState.State.FAILED_CLOSE).size();
int otherRegionsCount = 0;
for (List<HRegionInfo> list: tableRegions.values()) {
for (List<RegionInfo> list: tableRegions.values()) {
otherRegionsCount += list.size();
}
// now subtract known states

View File

@ -26,7 +26,7 @@ String bcv = "";
<%import>
java.util.*;
org.apache.hadoop.hbase.regionserver.HRegionServer;
org.apache.hadoop.hbase.HRegionInfo;
org.apache.hadoop.hbase.client.RegionInfo;
org.apache.hadoop.hbase.ServerName;
org.apache.hadoop.hbase.HBaseConfiguration;
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@ -44,7 +44,7 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
<%java>
ServerInfo serverInfo = ProtobufUtil.getServerInfo(null, regionServer.getRSRpcServices());
ServerName serverName = ProtobufUtil.toServerName(serverInfo.getServerName());
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(regionServer.getRSRpcServices());
List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(regionServer.getRSRpcServices());
MasterAddressTracker masterAddressTracker = regionServer.getMasterAddressTracker();
ServerName masterServerName = masterAddressTracker == null ? null
: masterAddressTracker.getMasterAddress();

View File

@ -18,14 +18,15 @@
</%doc>
<%args>
HRegionServer regionServer;
List<HRegionInfo> onlineRegions;
List<RegionInfo> onlineRegions;
</%args>
<%import>
java.util.*;
org.apache.commons.lang3.time.FastDateFormat;
org.apache.hadoop.hbase.regionserver.HRegionServer;
org.apache.hadoop.hbase.util.Bytes;
org.apache.hadoop.hbase.HRegionInfo;
org.apache.hadoop.hbase.client.RegionInfo;
org.apache.hadoop.hbase.client.RegionInfoDisplay;
org.apache.hadoop.hbase.regionserver.Region;
org.apache.hadoop.hbase.ServerName;
org.apache.hadoop.hbase.HBaseConfiguration;
@ -39,7 +40,7 @@
<%if (onlineRegions != null && onlineRegions.size() > 0) %>
<%java>
Collections.sort(onlineRegions);
Collections.sort(onlineRegions, RegionInfo.COMPARATOR);
</%java>
<div class="tabbable">
@ -85,7 +86,7 @@
<%def baseInfo>
<%args>
List<HRegionInfo> onlineRegions;
List<RegionInfo> onlineRegions;
</%args>
<table class="table table-striped">
<tr>
@ -95,16 +96,16 @@
<th>ReplicaID</th>
</tr>
<%for HRegionInfo r: onlineRegions %>
<%for RegionInfo r: onlineRegions %>
<tr>
<%java>
String displayName = HRegionInfo.getRegionNameAsStringForDisplay(r,
String displayName = RegionInfoDisplay.getRegionNameAsStringForDisplay(r,
regionServer.getConfiguration());
</%java>
<td><a href="region.jsp?name=<% r.getEncodedName() %>"><% displayName %></a></td>
<td><% Bytes.toStringBinary(HRegionInfo.getStartKeyForDisplay(r,
<td><% Bytes.toStringBinary(RegionInfoDisplay.getStartKeyForDisplay(r,
regionServer.getConfiguration())) %></td>
<td><% Bytes.toStringBinary(HRegionInfo.getEndKeyForDisplay(r,
<td><% Bytes.toStringBinary(RegionInfoDisplay.getEndKeyForDisplay(r,
regionServer.getConfiguration())) %></td>
<td><% r.getReplicaId() %></td>
</tr>
@ -114,7 +115,7 @@
<%def requestStats>
<%args>
List<HRegionInfo> onlineRegions;
List<RegionInfo> onlineRegions;
</%args>
<table class="table table-striped">
<tr>
@ -124,12 +125,12 @@
<th>Write Request Count</th>
</tr>
<%for HRegionInfo r: onlineRegions %>
<%for RegionInfo r: onlineRegions %>
<tr>
<%java>
RegionLoad load = regionServer.createRegionLoad(r.getEncodedName());
String displayName = HRegionInfo.getRegionNameAsStringForDisplay(r,
String displayName = RegionInfoDisplay.getRegionNameAsStringForDisplay(r,
regionServer.getConfiguration());
</%java>
<td><a href="region.jsp?name=<% r.getEncodedName() %>"><% displayName %></a></td>
@ -146,7 +147,7 @@
<%def storeStats>
<%args>
List<HRegionInfo> onlineRegions;
List<RegionInfo> onlineRegions;
</%args>
<table class="table table-striped">
<tr>
@ -160,12 +161,12 @@
<th>Data Locality</th>
</tr>
<%for HRegionInfo r: onlineRegions %>
<%for RegionInfo r: onlineRegions %>
<tr>
<%java>
RegionLoad load = regionServer.createRegionLoad(r.getEncodedName());
String displayName = HRegionInfo.getRegionNameAsStringForDisplay(r,
String displayName = RegionInfoDisplay.getRegionNameAsStringForDisplay(r,
regionServer.getConfiguration());
</%java>
<td><a href="region.jsp?name=<% r.getEncodedName() %>"><% displayName %></a></td>
@ -190,7 +191,7 @@
<%def compactStats>
<%args>
List<HRegionInfo> onlineRegions;
List<RegionInfo> onlineRegions;
</%args>
<table class="table table-striped">
<tr>
@ -201,7 +202,7 @@
<th>Last Major Compaction</th>
</tr>
<%for HRegionInfo r: onlineRegions %>
<%for RegionInfo r: onlineRegions %>
<tr>
<%java>
@ -218,7 +219,7 @@
compactTime = fdf.format(load.getLastMajorCompactionTs());
}
}
String displayName = HRegionInfo.getRegionNameAsStringForDisplay(r,
String displayName = RegionInfoDisplay.getRegionNameAsStringForDisplay(r,
regionServer.getConfiguration());
</%java>
<td><a href="region.jsp?name=<% r.getEncodedName() %>"><% displayName %></a></td>
@ -235,7 +236,7 @@
<%def memstoreStats>
<%args>
List<HRegionInfo> onlineRegions;
List<RegionInfo> onlineRegions;
</%args>
<table class="table table-striped">
<tr>
@ -243,12 +244,12 @@
<th>Memstore Size</th>
</tr>
<%for HRegionInfo r: onlineRegions %>
<%for RegionInfo r: onlineRegions %>
<tr>
<%java>
RegionLoad load = regionServer.createRegionLoad(r.getEncodedName());
String displayName = HRegionInfo.getRegionNameAsStringForDisplay(r,
String displayName = RegionInfoDisplay.getRegionNameAsStringForDisplay(r,
regionServer.getConfiguration());
</%java>
<td><a href="region.jsp?name=<% r.getEncodedName() %>"><% displayName %></a></td>

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
/**
@ -31,22 +32,22 @@ public interface RegionStateListener {
/**
* Process region split event.
*
* @param hri An instance of HRegionInfo
* @param hri An instance of RegionInfo
* @throws IOException
*/
void onRegionSplit(HRegionInfo hri) throws IOException;
void onRegionSplit(RegionInfo hri) throws IOException;
/**
* Process region split reverted event.
*
* @param hri An instance of HRegionInfo
* @param hri An instance of RegionInfo
* @throws IOException Signals that an I/O exception has occurred.
*/
void onRegionSplitReverted(HRegionInfo hri) throws IOException;
void onRegionSplitReverted(RegionInfo hri) throws IOException;
/**
* Process region merge event.
* @throws IOException
*/
void onRegionMerged(HRegionInfo mergedRegion) throws IOException;
void onRegionMerged(RegionInfo mergedRegion) throws IOException;
}

View File

@ -32,7 +32,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.util.Bytes;
@ -75,7 +75,7 @@ public class HFileArchiver {
/**
* @return True if the Region exits in the filesystem.
*/
public static boolean exists(Configuration conf, FileSystem fs, HRegionInfo info)
public static boolean exists(Configuration conf, FileSystem fs, RegionInfo info)
throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
Path regionDir = HRegion.getRegionDir(rootDir, info);
@ -87,10 +87,10 @@ public class HFileArchiver {
* archive directory
* @param conf the configuration to use
* @param fs the file system object
* @param info HRegionInfo for region to be deleted
* @param info RegionInfo for region to be deleted
* @throws IOException
*/
public static void archiveRegion(Configuration conf, FileSystem fs, HRegionInfo info)
public static void archiveRegion(Configuration conf, FileSystem fs, RegionInfo info)
throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
archiveRegion(fs, rootDir, FSUtils.getTableDir(rootDir, info.getTable()),
@ -176,7 +176,7 @@ public class HFileArchiver {
* @throws IOException if the files could not be correctly disposed.
*/
public static void archiveFamily(FileSystem fs, Configuration conf,
HRegionInfo parent, Path tableDir, byte[] family) throws IOException {
RegionInfo parent, Path tableDir, byte[] family) throws IOException {
Path familyDir = new Path(tableDir, new Path(parent.getEncodedName(), Bytes.toString(family)));
archiveFamilyByFamilyDir(fs, conf, parent, familyDir, family);
}
@ -192,7 +192,7 @@ public class HFileArchiver {
* @throws IOException if the files could not be correctly disposed.
*/
public static void archiveFamilyByFamilyDir(FileSystem fs, Configuration conf,
HRegionInfo parent, Path familyDir, byte[] family) throws IOException {
RegionInfo parent, Path familyDir, byte[] family) throws IOException {
FileStatus[] storeFiles = FSUtils.listStatus(fs, familyDir);
if (storeFiles == null) {
LOG.debug("No store files to dispose for region=" + parent.getRegionNameAsString() +
@ -219,13 +219,13 @@ public class HFileArchiver {
* Remove the store files, either by archiving them or outright deletion
* @param conf {@link Configuration} to examine to determine the archive directory
* @param fs the filesystem where the store files live
* @param regionInfo {@link HRegionInfo} of the region hosting the store files
* @param regionInfo {@link RegionInfo} of the region hosting the store files
* @param family the family hosting the store files
* @param compactedFiles files to be disposed of. No further reading of these files should be
* attempted; otherwise likely to cause an {@link IOException}
* @throws IOException if the files could not be correctly disposed.
*/
public static void archiveStoreFiles(Configuration conf, FileSystem fs, HRegionInfo regionInfo,
public static void archiveStoreFiles(Configuration conf, FileSystem fs, RegionInfo regionInfo,
Path tableDir, byte[] family, Collection<HStoreFile> compactedFiles)
throws IOException, FailedArchiveException {
@ -284,7 +284,7 @@ public class HFileArchiver {
* @param storeFile file to be archived
* @throws IOException if the files could not be correctly disposed.
*/
public static void archiveStoreFile(Configuration conf, FileSystem fs, HRegionInfo regionInfo,
public static void archiveStoreFile(Configuration conf, FileSystem fs, RegionInfo regionInfo,
Path tableDir, byte[] family, Path storeFile) throws IOException {
Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, regionInfo, tableDir, family);
// make sure we don't archive if we can't and that the archive dir exists

View File

@ -29,11 +29,10 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A client scanner for a region opened for read-only on the client side. Assumes region data
@ -49,7 +48,7 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
List<Cell> values;
public ClientSideRegionScanner(Configuration conf, FileSystem fs,
Path rootDir, TableDescriptor htd, HRegionInfo hri, Scan scan, ScanMetrics scanMetrics)
Path rootDir, TableDescriptor htd, RegionInfo hri, Scan scan, ScanMetrics scanMetrics)
throws IOException {
// region is immutable, set isolation level
scan.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);

View File

@ -30,10 +30,9 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A Scanner which performs a scan over snapshot files. Using this class requires copying the
@ -73,7 +72,7 @@ public class TableSnapshotScanner extends AbstractClientScanner {
private Path rootDir;
private Path restoreDir;
private Scan scan;
private ArrayList<HRegionInfo> regions;
private ArrayList<RegionInfo> regions;
private TableDescriptor htd;
private ClientSideRegionScanner currentRegionScanner = null;
@ -121,11 +120,11 @@ public class TableSnapshotScanner extends AbstractClientScanner {
final RestoreSnapshotHelper.RestoreMetaChanges meta =
RestoreSnapshotHelper.copySnapshotForScanner(
conf, fs, rootDir, restoreDir, snapshotName);
final List<HRegionInfo> restoredRegions = meta.getRegionsToAdd();
final List<RegionInfo> restoredRegions = meta.getRegionsToAdd();
htd = meta.getTableDescriptor();
regions = new ArrayList<>(restoredRegions.size());
for (HRegionInfo hri : restoredRegions) {
for (RegionInfo hri : restoredRegions) {
if (hri.isOffline() && (hri.isSplit() || hri.isSplitParent())) {
continue;
}
@ -136,7 +135,7 @@ public class TableSnapshotScanner extends AbstractClientScanner {
}
// sort for regions according to startKey.
Collections.sort(regions);
Collections.sort(regions, RegionInfo.COMPARATOR);
initScanMetrics(scan);
}
@ -150,7 +149,7 @@ public class TableSnapshotScanner extends AbstractClientScanner {
return null;
}
HRegionInfo hri = regions.get(currentRegion);
RegionInfo hri = regions.get(currentRegion);
currentRegionScanner = new ClientSideRegionScanner(conf, fs,
restoreDir, htd, hri, scan, scanMetrics);
if (this.scanMetrics != null) {

View File

@ -19,22 +19,22 @@
package org.apache.hadoop.hbase.client.locking;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.NonceGenerator;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.client.NonceGenerator;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType;
/**
* Helper class to create "master locks" for namespaces, tables and regions.
@ -83,7 +83,7 @@ public class LockServiceClient {
* Create a new EntityLock object to acquire exclusive lock on multiple regions of same tables.
* Internally, the table and its namespace will also be locked in shared mode.
*/
public EntityLock regionLock(List<HRegionInfo> regionInfos, String description, Abortable abort) {
public EntityLock regionLock(List<RegionInfo> regionInfos, String description, Abortable abort) {
LockRequest lockRequest = buildLockRequest(LockType.EXCLUSIVE,
null, null, regionInfos, description, ng.getNonceGroup(), ng.newNonce());
return new EntityLock(conf, stub, lockRequest, abort);
@ -91,15 +91,15 @@ public class LockServiceClient {
@VisibleForTesting
public static LockRequest buildLockRequest(final LockType type,
final String namespace, final TableName tableName, final List<HRegionInfo> regionInfos,
final String namespace, final TableName tableName, final List<RegionInfo> regionInfos,
final String description, final long nonceGroup, final long nonce) {
final LockRequest.Builder builder = LockRequest.newBuilder()
.setLockType(type)
.setNonceGroup(nonceGroup)
.setNonce(nonce);
if (regionInfos != null) {
for (HRegionInfo hri: regionInfos) {
builder.addRegionInfo(HRegionInfo.convert(hri));
for (RegionInfo hri: regionInfos) {
builder.addRegionInfo(ProtobufUtil.toRegionInfo(hri));
}
} else if (namespace != null) {
builder.setNamespace(namespace);

View File

@ -24,15 +24,15 @@ import java.io.InterruptedIOException;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective;
import org.apache.hadoop.hbase.master.SplitLogManager.Task;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
/**
* Coordination for SplitLogManager. It creates and works with tasks for split log operations<BR>
@ -123,7 +123,7 @@ public interface SplitLogManagerCoordination {
* @throws IOException in case of failure
* @throws InterruptedIOException
*/
void markRegionsRecovering(final ServerName serverName, Set<HRegionInfo> userRegions)
void markRegionsRecovering(final ServerName serverName, Set<RegionInfo> userRegions)
throws IOException, InterruptedIOException;
/**

View File

@ -18,13 +18,13 @@
package org.apache.hadoop.hbase.coordination;
import static org.apache.hadoop.hbase.util.CollectionUtils.*;
import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.CHECK;
import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.FORCE;
import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.DELETED;
import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.FAILURE;
import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.IN_PROGRESS;
import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.SUCCESS;
import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
import java.io.IOException;
import java.io.InterruptedIOException;
@ -39,17 +39,16 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SplitLogCounters;
import org.apache.hadoop.hbase.SplitLogTask;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination.TaskFinisher.Status;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective;
import org.apache.hadoop.hbase.master.SplitLogManager.Task;
import org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WALSplitter;
@ -59,6 +58,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@ -66,6 +66,8 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.Stat;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
/**
* ZooKeeper based implementation of
* {@link SplitLogManagerCoordination}
@ -287,7 +289,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
public void removeRecoveringRegions(final Set<String> recoveredServerNameSet,
Boolean isMetaRecovery)
throws IOException {
final String metaEncodeRegionName = HRegionInfo.FIRST_META_REGIONINFO.getEncodedName();
final String metaEncodeRegionName = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName();
int count = 0;
try {
List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.znodePaths.splitLogZNode);
@ -594,10 +596,10 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
* @param userRegions user regiones assigned on the region server
*/
@Override
public void markRegionsRecovering(final ServerName serverName, Set<HRegionInfo> userRegions)
public void markRegionsRecovering(final ServerName serverName, Set<RegionInfo> userRegions)
throws IOException, InterruptedIOException {
this.lastRecoveringNodeCreationTime = EnvironmentEdgeManager.currentTime();
for (HRegionInfo region : userRegions) {
for (RegionInfo region : userRegions) {
String regionEncodeName = region.getEncodedName();
long retries = this.zkretries;

View File

@ -24,22 +24,22 @@ import java.util.Optional;
import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.WrongRegionException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
@ -92,7 +92,7 @@ public class MultiRowMutationEndpoint extends MultiRowMutationService implements
mutations.add(ProtobufUtil.toMutation(m));
}
HRegionInfo regionInfo = env.getRegion().getRegionInfo();
RegionInfo regionInfo = env.getRegion().getRegionInfo();
for (Mutation m : mutations) {
// check whether rows are in range for this region
if (!HRegion.rowIsInRange(regionInfo, m.getRow())) {

View File

@ -37,29 +37,29 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.RackManager;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes;
/**
* Helper class for {@link FavoredNodeLoadBalancer} that has all the intelligence for racks,
* meta scans, etc. Instantiated by the {@link FavoredNodeLoadBalancer} when needed (from
* within calls like {@link FavoredNodeLoadBalancer#randomAssignment(HRegionInfo, List)}).
* within calls like {@link FavoredNodeLoadBalancer#randomAssignment(RegionInfo, List)}).
* All updates to favored nodes should only be done from {@link FavoredNodesManager} and not
* through this helper class (except for tests).
*/
@ -116,15 +116,15 @@ public class FavoredNodeAssignmentHelper {
/**
* Update meta table with favored nodes info
* @param regionToFavoredNodes map of HRegionInfo's to their favored nodes
* @param regionToFavoredNodes map of RegionInfo's to their favored nodes
* @param connection connection to be used
* @throws IOException
*/
public static void updateMetaWithFavoredNodesInfo(
Map<HRegionInfo, List<ServerName>> regionToFavoredNodes,
Map<RegionInfo, List<ServerName>> regionToFavoredNodes,
Connection connection) throws IOException {
List<Put> puts = new ArrayList<>();
for (Map.Entry<HRegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
for (Map.Entry<RegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
Put put = makePutFromRegionInfo(entry.getKey(), entry.getValue());
if (put != null) {
puts.add(put);
@ -141,10 +141,10 @@ public class FavoredNodeAssignmentHelper {
* @throws IOException
*/
public static void updateMetaWithFavoredNodesInfo(
Map<HRegionInfo, List<ServerName>> regionToFavoredNodes,
Map<RegionInfo, List<ServerName>> regionToFavoredNodes,
Configuration conf) throws IOException {
List<Put> puts = new ArrayList<>();
for (Map.Entry<HRegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
for (Map.Entry<RegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
Put put = makePutFromRegionInfo(entry.getKey(), entry.getValue());
if (put != null) {
puts.add(put);
@ -170,7 +170,7 @@ public class FavoredNodeAssignmentHelper {
* @param favoredNodeList
* @return Put object
*/
static Put makePutFromRegionInfo(HRegionInfo regionInfo, List<ServerName>favoredNodeList)
static Put makePutFromRegionInfo(RegionInfo regionInfo, List<ServerName>favoredNodeList)
throws IOException {
Put put = null;
if (favoredNodeList != null) {
@ -225,8 +225,8 @@ public class FavoredNodeAssignmentHelper {
// If there were fewer servers in one rack, say r3, which had 3 servers, one possible
// placement could be r2:s5, <skip-r3>, r4:s5, r1:s5, r2:s6, <skip-r3> ...
// The regions should be distributed proportionately to the racksizes
public void placePrimaryRSAsRoundRobin(Map<ServerName, List<HRegionInfo>> assignmentMap,
Map<HRegionInfo, ServerName> primaryRSMap, List<HRegionInfo> regions) {
public void placePrimaryRSAsRoundRobin(Map<ServerName, List<RegionInfo>> assignmentMap,
Map<RegionInfo, ServerName> primaryRSMap, List<RegionInfo> regions) {
List<String> rackList = new ArrayList<>(rackToRegionServerMap.size());
rackList.addAll(rackToRegionServerMap.keySet());
int rackIndex = random.nextInt(rackList.size());
@ -239,7 +239,7 @@ public class FavoredNodeAssignmentHelper {
int numIterations = 0;
// Initialize the current processing host index.
int serverIndex = random.nextInt(maxRackSize);
for (HRegionInfo regionInfo : regions) {
for (RegionInfo regionInfo : regions) {
List<ServerName> currentServerList;
String rackName;
while (true) {
@ -264,7 +264,7 @@ public class FavoredNodeAssignmentHelper {
// Place the current region with the current primary region server
primaryRSMap.put(regionInfo, currentServer);
if (assignmentMap != null) {
List<HRegionInfo> regionsForServer = assignmentMap.get(currentServer);
List<RegionInfo> regionsForServer = assignmentMap.get(currentServer);
if (regionsForServer == null) {
regionsForServer = new ArrayList<>();
assignmentMap.put(currentServer, regionsForServer);
@ -282,12 +282,12 @@ public class FavoredNodeAssignmentHelper {
}
}
public Map<HRegionInfo, ServerName[]> placeSecondaryAndTertiaryRS(
Map<HRegionInfo, ServerName> primaryRSMap) {
Map<HRegionInfo, ServerName[]> secondaryAndTertiaryMap = new HashMap<>();
for (Map.Entry<HRegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
public Map<RegionInfo, ServerName[]> placeSecondaryAndTertiaryRS(
Map<RegionInfo, ServerName> primaryRSMap) {
Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap = new HashMap<>();
for (Map.Entry<RegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
// Get the target region and its primary region server rack
HRegionInfo regionInfo = entry.getKey();
RegionInfo regionInfo = entry.getKey();
ServerName primaryRS = entry.getValue();
try {
// Create the secondary and tertiary region server pair object.
@ -306,7 +306,7 @@ public class FavoredNodeAssignmentHelper {
return secondaryAndTertiaryMap;
}
public ServerName[] getSecondaryAndTertiary(HRegionInfo regionInfo, ServerName primaryRS)
public ServerName[] getSecondaryAndTertiary(RegionInfo regionInfo, ServerName primaryRS)
throws IOException {
ServerName[] favoredNodes;// Get the rack for the primary region server
@ -320,11 +320,11 @@ public class FavoredNodeAssignmentHelper {
return favoredNodes;
}
private Map<ServerName, Set<HRegionInfo>> mapRSToPrimaries(
Map<HRegionInfo, ServerName> primaryRSMap) {
Map<ServerName, Set<HRegionInfo>> primaryServerMap = new HashMap<>();
for (Entry<HRegionInfo, ServerName> e : primaryRSMap.entrySet()) {
Set<HRegionInfo> currentSet = primaryServerMap.get(e.getValue());
private Map<ServerName, Set<RegionInfo>> mapRSToPrimaries(
Map<RegionInfo, ServerName> primaryRSMap) {
Map<ServerName, Set<RegionInfo>> primaryServerMap = new HashMap<>();
for (Entry<RegionInfo, ServerName> e : primaryRSMap.entrySet()) {
Set<RegionInfo> currentSet = primaryServerMap.get(e.getValue());
if (currentSet == null) {
currentSet = new HashSet<>();
}
@ -341,15 +341,15 @@ public class FavoredNodeAssignmentHelper {
* @param primaryRSMap
* @return the map of regions to the servers the region-files should be hosted on
*/
public Map<HRegionInfo, ServerName[]> placeSecondaryAndTertiaryWithRestrictions(
Map<HRegionInfo, ServerName> primaryRSMap) {
Map<ServerName, Set<HRegionInfo>> serverToPrimaries =
public Map<RegionInfo, ServerName[]> placeSecondaryAndTertiaryWithRestrictions(
Map<RegionInfo, ServerName> primaryRSMap) {
Map<ServerName, Set<RegionInfo>> serverToPrimaries =
mapRSToPrimaries(primaryRSMap);
Map<HRegionInfo, ServerName[]> secondaryAndTertiaryMap = new HashMap<>();
Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap = new HashMap<>();
for (Entry<HRegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
for (Entry<RegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
// Get the target region and its primary region server rack
HRegionInfo regionInfo = entry.getKey();
RegionInfo regionInfo = entry.getKey();
ServerName primaryRS = entry.getValue();
try {
// Get the rack for the primary region server
@ -378,9 +378,9 @@ public class FavoredNodeAssignmentHelper {
}
private ServerName[] multiRackCaseWithRestrictions(
Map<ServerName, Set<HRegionInfo>> serverToPrimaries,
Map<HRegionInfo, ServerName[]> secondaryAndTertiaryMap,
String primaryRack, ServerName primaryRS, HRegionInfo regionInfo) throws IOException {
Map<ServerName, Set<RegionInfo>> serverToPrimaries,
Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap,
String primaryRack, ServerName primaryRS, RegionInfo regionInfo) throws IOException {
// Random to choose the secondary and tertiary region server
// from another rack to place the secondary and tertiary
// Random to choose one rack except for the current rack
@ -395,13 +395,13 @@ public class FavoredNodeAssignmentHelper {
// Randomly pick up two servers from this secondary rack
// Skip the secondary for the tertiary placement
// skip the servers which share the primary already
Set<HRegionInfo> primaries = serverToPrimaries.get(primaryRS);
Set<RegionInfo> primaries = serverToPrimaries.get(primaryRS);
Set<ServerName> skipServerSet = new HashSet<>();
while (true) {
ServerName[] secondaryAndTertiary = null;
if (primaries.size() > 1) {
// check where his tertiary and secondary are
for (HRegionInfo primary : primaries) {
for (RegionInfo primary : primaries) {
secondaryAndTertiary = secondaryAndTertiaryMap.get(primary);
if (secondaryAndTertiary != null) {
if (getRackOfServer(secondaryAndTertiary[0]).equals(secondaryRack)) {
@ -468,7 +468,7 @@ public class FavoredNodeAssignmentHelper {
return favoredNodes;
}
private ServerName[] singleRackCase(HRegionInfo regionInfo,
private ServerName[] singleRackCase(RegionInfo regionInfo,
ServerName primaryRS,
String primaryRack) throws IOException {
// Single rack case: have to pick the secondary and tertiary
@ -516,7 +516,7 @@ public class FavoredNodeAssignmentHelper {
* @return Array containing secondary and tertiary favored nodes.
* @throws IOException Signals that an I/O exception has occurred.
*/
private ServerName[] multiRackCase(HRegionInfo regionInfo, ServerName primaryRS,
private ServerName[] multiRackCase(RegionInfo regionInfo, ServerName primaryRS,
String primaryRack) throws IOException {
List<ServerName>favoredNodes = Lists.newArrayList(primaryRS);
@ -764,15 +764,15 @@ public class FavoredNodeAssignmentHelper {
* Choose a random server as primary and then choose secondary and tertiary FN so its spread
* across two racks.
*/
public List<ServerName> generateFavoredNodes(HRegionInfo hri) throws IOException {
public List<ServerName> generateFavoredNodes(RegionInfo hri) throws IOException {
List<ServerName> favoredNodesForRegion = new ArrayList<>(FAVORED_NODES_NUM);
ServerName primary = servers.get(random.nextInt(servers.size()));
favoredNodesForRegion.add(ServerName.valueOf(primary.getHostAndPort(), ServerName.NON_STARTCODE));
Map<HRegionInfo, ServerName> primaryRSMap = new HashMap<>(1);
Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>(1);
primaryRSMap.put(hri, primary);
Map<HRegionInfo, ServerName[]> secondaryAndTertiaryRSMap =
Map<RegionInfo, ServerName[]> secondaryAndTertiaryRSMap =
placeSecondaryAndTertiaryRS(primaryRSMap);
ServerName[] secondaryAndTertiaryNodes = secondaryAndTertiaryRSMap.get(hri);
if (secondaryAndTertiaryNodes != null && secondaryAndTertiaryNodes.length == 2) {
@ -785,13 +785,13 @@ public class FavoredNodeAssignmentHelper {
}
}
public Map<HRegionInfo, List<ServerName>> generateFavoredNodesRoundRobin(
Map<ServerName, List<HRegionInfo>> assignmentMap, List<HRegionInfo> regions)
public Map<RegionInfo, List<ServerName>> generateFavoredNodesRoundRobin(
Map<ServerName, List<RegionInfo>> assignmentMap, List<RegionInfo> regions)
throws IOException {
if (regions.size() > 0) {
if (canPlaceFavoredNodes()) {
Map<HRegionInfo, ServerName> primaryRSMap = new HashMap<>();
Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>();
// Lets try to have an equal distribution for primary favored node
placePrimaryRSAsRoundRobin(assignmentMap, primaryRSMap, regions);
return generateFavoredNodes(primaryRSMap);
@ -806,16 +806,16 @@ public class FavoredNodeAssignmentHelper {
/*
* Generate favored nodes for a set of regions when we know where they are currently hosted.
*/
private Map<HRegionInfo, List<ServerName>> generateFavoredNodes(
Map<HRegionInfo, ServerName> primaryRSMap) {
private Map<RegionInfo, List<ServerName>> generateFavoredNodes(
Map<RegionInfo, ServerName> primaryRSMap) {
Map<HRegionInfo, List<ServerName>> generatedFavNodes = new HashMap<>();
Map<HRegionInfo, ServerName[]> secondaryAndTertiaryRSMap =
Map<RegionInfo, List<ServerName>> generatedFavNodes = new HashMap<>();
Map<RegionInfo, ServerName[]> secondaryAndTertiaryRSMap =
placeSecondaryAndTertiaryRS(primaryRSMap);
for (Entry<HRegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
for (Entry<RegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
List<ServerName> favoredNodesForRegion = new ArrayList<>(FAVORED_NODES_NUM);
HRegionInfo region = entry.getKey();
RegionInfo region = entry.getKey();
ServerName primarySN = entry.getValue();
favoredNodesForRegion.add(ServerName.valueOf(primarySN.getHostname(), primarySN.getPort(),
NON_STARTCODE));

View File

@ -29,23 +29,26 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.*;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position;
import org.apache.hadoop.hbase.master.RackManager;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.master.SnapshotOfRegionAssignmentFromMeta;
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
/**
@ -85,7 +88,7 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
}
@Override
public List<RegionPlan> balanceCluster(Map<ServerName, List<HRegionInfo>> clusterState) {
public List<RegionPlan> balanceCluster(Map<ServerName, List<RegionInfo>> clusterState) {
//TODO. Look at is whether Stochastic loadbalancer can be integrated with this
List<RegionPlan> plans = new ArrayList<>();
//perform a scan of the meta to get the latest updates (if any)
@ -105,13 +108,13 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
// FindBugs complains about useless store! serverNameToServerNameWithoutCode.put(sn, s);
serverNameWithoutCodeToServerName.put(s, sn);
}
for (Map.Entry<ServerName, List<HRegionInfo>> entry : clusterState.entrySet()) {
for (Map.Entry<ServerName, List<RegionInfo>> entry : clusterState.entrySet()) {
ServerName currentServer = entry.getKey();
//get a server without the startcode for the currentServer
ServerName currentServerWithoutStartCode = ServerName.valueOf(currentServer.getHostname(),
currentServer.getPort(), ServerName.NON_STARTCODE);
List<HRegionInfo> list = entry.getValue();
for (HRegionInfo region : list) {
List<RegionInfo> list = entry.getValue();
for (RegionInfo region : list) {
if(!FavoredNodesManager.isFavoredNodeApplicable(region)) {
continue;
}
@ -157,9 +160,9 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
}
@Override
public Map<ServerName, List<HRegionInfo>> roundRobinAssignment(List<HRegionInfo> regions,
public Map<ServerName, List<RegionInfo>> roundRobinAssignment(List<RegionInfo> regions,
List<ServerName> servers) throws HBaseIOException {
Map<ServerName, List<HRegionInfo>> assignmentMap;
Map<ServerName, List<RegionInfo>> assignmentMap;
try {
FavoredNodeAssignmentHelper assignmentHelper =
new FavoredNodeAssignmentHelper(servers, rackManager);
@ -183,10 +186,10 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
// need to come up with favored nodes assignments for them. The corner case
// in (1) above is that all the nodes are unavailable and in that case, we
// will note that this region doesn't have favored nodes.
Pair<Map<ServerName,List<HRegionInfo>>, List<HRegionInfo>> segregatedRegions =
Pair<Map<ServerName,List<RegionInfo>>, List<RegionInfo>> segregatedRegions =
segregateRegionsAndAssignRegionsWithFavoredNodes(regions, servers);
Map<ServerName,List<HRegionInfo>> regionsWithFavoredNodesMap = segregatedRegions.getFirst();
List<HRegionInfo> regionsWithNoFavoredNodes = segregatedRegions.getSecond();
Map<ServerName,List<RegionInfo>> regionsWithFavoredNodesMap = segregatedRegions.getFirst();
List<RegionInfo> regionsWithNoFavoredNodes = segregatedRegions.getSecond();
assignmentMap = new HashMap<>();
roundRobinAssignmentImpl(assignmentHelper, assignmentMap, regionsWithNoFavoredNodes,
servers);
@ -201,7 +204,7 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
}
@Override
public ServerName randomAssignment(HRegionInfo regionInfo, List<ServerName> servers)
public ServerName randomAssignment(RegionInfo regionInfo, List<ServerName> servers)
throws HBaseIOException {
try {
FavoredNodeAssignmentHelper assignmentHelper =
@ -224,9 +227,9 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
}
}
}
List<HRegionInfo> regions = new ArrayList<>(1);
List<RegionInfo> regions = new ArrayList<>(1);
regions.add(regionInfo);
Map<HRegionInfo, ServerName> primaryRSMap = new HashMap<>(1);
Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>(1);
primaryRSMap.put(regionInfo, primary);
assignSecondaryAndTertiaryNodesForRegion(assignmentHelper, regions, primaryRSMap);
return primary;
@ -237,12 +240,12 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
}
}
private Pair<Map<ServerName, List<HRegionInfo>>, List<HRegionInfo>>
segregateRegionsAndAssignRegionsWithFavoredNodes(List<HRegionInfo> regions,
private Pair<Map<ServerName, List<RegionInfo>>, List<RegionInfo>>
segregateRegionsAndAssignRegionsWithFavoredNodes(List<RegionInfo> regions,
List<ServerName> availableServers) {
Map<ServerName, List<HRegionInfo>> assignmentMapForFavoredNodes = new HashMap<>(regions.size() / 2);
List<HRegionInfo> regionsWithNoFavoredNodes = new ArrayList<>(regions.size()/2);
for (HRegionInfo region : regions) {
Map<ServerName, List<RegionInfo>> assignmentMapForFavoredNodes = new HashMap<>(regions.size() / 2);
List<RegionInfo> regionsWithNoFavoredNodes = new ArrayList<>(regions.size()/2);
for (RegionInfo region : regions) {
List<ServerName> favoredNodes = fnm.getFavoredNodes(region);
ServerName primaryHost = null;
ServerName secondaryHost = null;
@ -286,7 +289,7 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
}
private void assignRegionToAvailableFavoredNode(Map<ServerName,
List<HRegionInfo>> assignmentMapForFavoredNodes, HRegionInfo region, ServerName primaryHost,
List<RegionInfo>> assignmentMapForFavoredNodes, RegionInfo region, ServerName primaryHost,
ServerName secondaryHost, ServerName tertiaryHost) {
if (primaryHost != null) {
addRegionToMap(assignmentMapForFavoredNodes, region, primaryHost);
@ -309,9 +312,9 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
}
}
private void addRegionToMap(Map<ServerName, List<HRegionInfo>> assignmentMapForFavoredNodes,
HRegionInfo region, ServerName host) {
List<HRegionInfo> regionsOnServer = null;
private void addRegionToMap(Map<ServerName, List<RegionInfo>> assignmentMapForFavoredNodes,
RegionInfo region, ServerName host) {
List<RegionInfo> regionsOnServer = null;
if ((regionsOnServer = assignmentMapForFavoredNodes.get(host)) == null) {
regionsOnServer = new ArrayList<>();
assignmentMapForFavoredNodes.put(host, regionsOnServer);
@ -319,14 +322,14 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
regionsOnServer.add(region);
}
public synchronized List<ServerName> getFavoredNodes(HRegionInfo regionInfo) {
public synchronized List<ServerName> getFavoredNodes(RegionInfo regionInfo) {
return this.fnm.getFavoredNodes(regionInfo);
}
private void roundRobinAssignmentImpl(FavoredNodeAssignmentHelper assignmentHelper,
Map<ServerName, List<HRegionInfo>> assignmentMap,
List<HRegionInfo> regions, List<ServerName> servers) throws IOException {
Map<HRegionInfo, ServerName> primaryRSMap = new HashMap<>();
Map<ServerName, List<RegionInfo>> assignmentMap,
List<RegionInfo> regions, List<ServerName> servers) throws IOException {
Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>();
// figure the primary RSs
assignmentHelper.placePrimaryRSAsRoundRobin(assignmentMap, primaryRSMap, regions);
assignSecondaryAndTertiaryNodesForRegion(assignmentHelper, regions, primaryRSMap);
@ -334,14 +337,14 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
private void assignSecondaryAndTertiaryNodesForRegion(
FavoredNodeAssignmentHelper assignmentHelper,
List<HRegionInfo> regions, Map<HRegionInfo, ServerName> primaryRSMap) throws IOException {
List<RegionInfo> regions, Map<RegionInfo, ServerName> primaryRSMap) throws IOException {
// figure the secondary and tertiary RSs
Map<HRegionInfo, ServerName[]> secondaryAndTertiaryRSMap =
Map<RegionInfo, ServerName[]> secondaryAndTertiaryRSMap =
assignmentHelper.placeSecondaryAndTertiaryRS(primaryRSMap);
Map<HRegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
Map<RegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
// now record all the assignments so that we can serve queries later
for (HRegionInfo region : regions) {
for (RegionInfo region : regions) {
// Store the favored nodes without startCode for the ServerName objects
// We don't care about the startcode; but only the hostname really
List<ServerName> favoredNodesForRegion = new ArrayList<>(3);
@ -371,10 +374,10 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
* randomly. This would give us better distribution over a period of time after enough splits.
*/
@Override
public void generateFavoredNodesForDaughter(List<ServerName> servers, HRegionInfo parent,
HRegionInfo regionA, HRegionInfo regionB) throws IOException {
public void generateFavoredNodesForDaughter(List<ServerName> servers, RegionInfo parent,
RegionInfo regionA, RegionInfo regionB) throws IOException {
Map<HRegionInfo, List<ServerName>> result = new HashMap<>();
Map<RegionInfo, List<ServerName>> result = new HashMap<>();
FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper(servers, rackManager);
helper.initialize();
@ -426,16 +429,16 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored
* keep it simple.
*/
@Override
public void generateFavoredNodesForMergedRegion(HRegionInfo merged, HRegionInfo regionA,
HRegionInfo regionB) throws IOException {
Map<HRegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
public void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo regionA,
RegionInfo regionB) throws IOException {
Map<RegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
regionFNMap.put(merged, getFavoredNodes(regionA));
fnm.updateFavoredNodes(regionFNMap);
}
@Override
public List<RegionPlan> balanceCluster(TableName tableName,
Map<ServerName, List<HRegionInfo>> clusterState) throws HBaseIOException {
Map<ServerName, List<RegionInfo>> clusterState) throws HBaseIOException {
return balanceCluster(clusterState);
}
}

View File

@ -18,11 +18,11 @@
*/
package org.apache.hadoop.hbase.favored;
import static org.apache.hadoop.hbase.ServerName.NON_STARTCODE;
import static org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper.FAVORED_NODES_NUM;
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.PRIMARY;
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.SECONDARY;
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.TERTIARY;
import static org.apache.hadoop.hbase.ServerName.NON_STARTCODE;
import java.io.IOException;
import java.util.ArrayList;
@ -34,17 +34,17 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RackManager;
import org.apache.hadoop.hbase.master.SnapshotOfRegionAssignmentFromMeta;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.net.NetUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
@ -66,9 +66,9 @@ public class FavoredNodesManager {
private static final Log LOG = LogFactory.getLog(FavoredNodesManager.class);
private FavoredNodesPlan globalFavoredNodesAssignmentPlan;
private Map<ServerName, List<HRegionInfo>> primaryRSToRegionMap;
private Map<ServerName, List<HRegionInfo>> secondaryRSToRegionMap;
private Map<ServerName, List<HRegionInfo>> teritiaryRSToRegionMap;
private Map<ServerName, List<RegionInfo>> primaryRSToRegionMap;
private Map<ServerName, List<RegionInfo>> secondaryRSToRegionMap;
private Map<ServerName, List<RegionInfo>> teritiaryRSToRegionMap;
private MasterServices masterServices;
private RackManager rackManager;
@ -108,7 +108,7 @@ public class FavoredNodesManager {
return dnPort;
}
public synchronized List<ServerName> getFavoredNodes(HRegionInfo regionInfo) {
public synchronized List<ServerName> getFavoredNodes(RegionInfo regionInfo) {
return this.globalFavoredNodesAssignmentPlan.getFavoredNodes(regionInfo);
}
@ -116,7 +116,7 @@ public class FavoredNodesManager {
* Favored nodes are not applicable for system tables. We will use this to check before
* we apply any favored nodes logic on a region.
*/
public static boolean isFavoredNodeApplicable(HRegionInfo regionInfo) {
public static boolean isFavoredNodeApplicable(RegionInfo regionInfo) {
return !regionInfo.isSystemTable();
}
@ -126,9 +126,9 @@ public class FavoredNodesManager {
* @param regions - collection of regions
* @return set of regions for which favored nodes is not applicable
*/
public static Set<HRegionInfo> filterNonFNApplicableRegions(Collection<HRegionInfo> regions) {
Set<HRegionInfo> fnRegions = Sets.newHashSet();
for (HRegionInfo regionInfo : regions) {
public static Set<RegionInfo> filterNonFNApplicableRegions(Collection<RegionInfo> regions) {
Set<RegionInfo> fnRegions = Sets.newHashSet();
for (RegionInfo regionInfo : regions) {
if (!isFavoredNodeApplicable(regionInfo)) {
fnRegions.add(regionInfo);
}
@ -141,7 +141,7 @@ public class FavoredNodesManager {
* sending the region server port, we use the datanode port. This helps in centralizing the DN
* port logic in Master. The RS uses the port from the favored node list as hints.
*/
public synchronized List<ServerName> getFavoredNodesWithDNPort(HRegionInfo regionInfo) {
public synchronized List<ServerName> getFavoredNodesWithDNPort(RegionInfo regionInfo) {
if (getFavoredNodes(regionInfo) == null) {
return null;
}
@ -154,12 +154,12 @@ public class FavoredNodesManager {
return fnWithDNPort;
}
public synchronized void updateFavoredNodes(Map<HRegionInfo, List<ServerName>> regionFNMap)
public synchronized void updateFavoredNodes(Map<RegionInfo, List<ServerName>> regionFNMap)
throws IOException {
Map<HRegionInfo, List<ServerName>> regionToFavoredNodes = new HashMap<>();
for (Map.Entry<HRegionInfo, List<ServerName>> entry : regionFNMap.entrySet()) {
HRegionInfo regionInfo = entry.getKey();
Map<RegionInfo, List<ServerName>> regionToFavoredNodes = new HashMap<>();
for (Map.Entry<RegionInfo, List<ServerName>> entry : regionFNMap.entrySet()) {
RegionInfo regionInfo = entry.getKey();
List<ServerName> servers = entry.getValue();
/*
@ -199,18 +199,18 @@ public class FavoredNodesManager {
masterServices.getConnection());
deleteFavoredNodesForRegions(regionToFavoredNodes.keySet());
for (Map.Entry<HRegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
HRegionInfo regionInfo = entry.getKey();
for (Map.Entry<RegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
RegionInfo regionInfo = entry.getKey();
List<ServerName> serversWithNoStartCodes = entry.getValue();
globalFavoredNodesAssignmentPlan.updateFavoredNodesMap(regionInfo, serversWithNoStartCodes);
addToReplicaLoad(regionInfo, serversWithNoStartCodes);
}
}
private synchronized void addToReplicaLoad(HRegionInfo hri, List<ServerName> servers) {
private synchronized void addToReplicaLoad(RegionInfo hri, List<ServerName> servers) {
ServerName serverToUse = ServerName.valueOf(servers.get(PRIMARY.ordinal()).getHostAndPort(),
NON_STARTCODE);
List<HRegionInfo> regionList = primaryRSToRegionMap.get(serverToUse);
List<RegionInfo> regionList = primaryRSToRegionMap.get(serverToUse);
if (regionList == null) {
regionList = new ArrayList<>();
}
@ -269,8 +269,8 @@ public class FavoredNodesManager {
return result;
}
public synchronized void deleteFavoredNodesForRegions(Collection<HRegionInfo> regionInfoList) {
for (HRegionInfo hri : regionInfoList) {
public synchronized void deleteFavoredNodesForRegions(Collection<RegionInfo> regionInfoList) {
for (RegionInfo hri : regionInfoList) {
List<ServerName> favNodes = getFavoredNodes(hri);
if (favNodes != null) {
if (primaryRSToRegionMap.containsKey(favNodes.get(PRIMARY.ordinal()))) {

View File

@ -22,9 +22,9 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
/**
* This class contains the mapping information between each region name and
@ -54,7 +54,7 @@ public class FavoredNodesPlan {
* @param region
* @param servers
*/
public void updateFavoredNodesMap(HRegionInfo region, List<ServerName> servers) {
public void updateFavoredNodesMap(RegionInfo region, List<ServerName> servers) {
if (region == null || servers == null || servers.isEmpty()) {
return;
}
@ -66,7 +66,7 @@ public class FavoredNodesPlan {
* @param region region
* @return the list of favored region server for this region based on the plan
*/
public List<ServerName> removeFavoredNodes(HRegionInfo region) {
public List<ServerName> removeFavoredNodes(RegionInfo region) {
return favoredNodesMap.remove(region.getRegionNameAsString());
}
@ -74,7 +74,7 @@ public class FavoredNodesPlan {
* @param region
* @return the list of favored region server for this region based on the plan
*/
public List<ServerName> getFavoredNodes(HRegionInfo region) {
public List<ServerName> getFavoredNodes(RegionInfo region) {
return favoredNodesMap.get(region.getRegionNameAsString());
}

View File

@ -21,7 +21,7 @@ import java.io.IOException;
import java.util.List;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.ServerName;
@InterfaceAudience.Private
@ -31,8 +31,8 @@ public interface FavoredNodesPromoter {
String FAVORED_ALWAYS_ASSIGN_REGIONS = "hbase.favored.assignment.always.assign";
void generateFavoredNodesForDaughter(List<ServerName> servers,
HRegionInfo parent, HRegionInfo hriA, HRegionInfo hriB) throws IOException;
RegionInfo parent, RegionInfo hriA, RegionInfo hriB) throws IOException;
void generateFavoredNodesForMergedRegion(HRegionInfo merged, HRegionInfo hriA,
HRegionInfo hriB) throws IOException;
void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo hriA,
RegionInfo hriB) throws IOException;
}

View File

@ -24,19 +24,20 @@ import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
/**
* HFileLink describes a link to an hfile.
@ -73,14 +74,14 @@ public class HFileLink extends FileLink {
public static final String LINK_NAME_REGEX =
String.format("(?:(?:%s=)?)%s=%s-%s",
TableName.VALID_NAMESPACE_REGEX, TableName.VALID_TABLE_QUALIFIER_REGEX,
HRegionInfo.ENCODED_REGION_NAME_REGEX, StoreFileInfo.HFILE_NAME_REGEX);
RegionInfoBuilder.ENCODED_REGION_NAME_REGEX, StoreFileInfo.HFILE_NAME_REGEX);
/** Define the HFile Link name parser in the form of: table=region-hfile */
//made package private for testing
static final Pattern LINK_NAME_PATTERN =
Pattern.compile(String.format("^(?:(%s)(?:\\=))?(%s)=(%s)-(%s)$",
TableName.VALID_NAMESPACE_REGEX, TableName.VALID_TABLE_QUALIFIER_REGEX,
HRegionInfo.ENCODED_REGION_NAME_REGEX, StoreFileInfo.HFILE_NAME_REGEX));
RegionInfoBuilder.ENCODED_REGION_NAME_REGEX, StoreFileInfo.HFILE_NAME_REGEX));
/**
* The pattern should be used for hfile and reference links
@ -89,7 +90,7 @@ public class HFileLink extends FileLink {
private static final Pattern REF_OR_HFILE_LINK_PATTERN =
Pattern.compile(String.format("^(?:(%s)(?:=))?(%s)=(%s)-(.+)$",
TableName.VALID_NAMESPACE_REGEX, TableName.VALID_TABLE_QUALIFIER_REGEX,
HRegionInfo.ENCODED_REGION_NAME_REGEX));
RegionInfoBuilder.ENCODED_REGION_NAME_REGEX));
private final Path archivePath;
private final Path originPath;
@ -285,7 +286,7 @@ public class HFileLink extends FileLink {
* @param hfileName - Linked HFile name
* @return file name of the HFile Link
*/
public static String createHFileLinkName(final HRegionInfo hfileRegionInfo,
public static String createHFileLinkName(final RegionInfo hfileRegionInfo,
final String hfileName) {
return createHFileLinkName(hfileRegionInfo.getTable(),
hfileRegionInfo.getEncodedName(), hfileName);
@ -322,7 +323,7 @@ public class HFileLink extends FileLink {
* @throws IOException on file or parent directory creation failure
*/
public static boolean create(final Configuration conf, final FileSystem fs,
final Path dstFamilyPath, final HRegionInfo hfileRegionInfo,
final Path dstFamilyPath, final RegionInfo hfileRegionInfo,
final String hfileName) throws IOException {
return create(conf, fs, dstFamilyPath, hfileRegionInfo, hfileName, true);
}
@ -343,7 +344,7 @@ public class HFileLink extends FileLink {
* @throws IOException on file or parent directory creation failure
*/
public static boolean create(final Configuration conf, final FileSystem fs,
final Path dstFamilyPath, final HRegionInfo hfileRegionInfo,
final Path dstFamilyPath, final RegionInfo hfileRegionInfo,
final String hfileName, final boolean createBackRef) throws IOException {
TableName linkedTable = hfileRegionInfo.getTable();
String linkedRegion = hfileRegionInfo.getEncodedName();

View File

@ -17,10 +17,9 @@
*/
package org.apache.hadoop.hbase.master;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Get notification of assignment events. The invocations are inline
@ -33,11 +32,11 @@ public interface AssignmentListener {
* @param regionInfo The opened region.
* @param serverName The remote servers name.
*/
void regionOpened(final HRegionInfo regionInfo, final ServerName serverName);
void regionOpened(final RegionInfo regionInfo, final ServerName serverName);
/**
* The region was closed on the region server.
* @param regionInfo The closed region.
*/
void regionClosed(final HRegionInfo regionInfo);
void regionClosed(final RegionInfo regionInfo);
}

View File

@ -28,12 +28,12 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Helper class that is used by {@link RegionPlacementMaintainer} to print
* information for favored nodes
@ -51,13 +51,13 @@ public class AssignmentVerificationReport {
private int totalRegions = 0;
private int totalRegionServers = 0;
// for unassigned regions
private List<HRegionInfo> unAssignedRegionsList = new ArrayList<>();
private List<RegionInfo> unAssignedRegionsList = new ArrayList<>();
// For regions without valid favored nodes
private List<HRegionInfo> regionsWithoutValidFavoredNodes = new ArrayList<>();
private List<RegionInfo> regionsWithoutValidFavoredNodes = new ArrayList<>();
// For regions not running on the favored nodes
private List<HRegionInfo> nonFavoredAssignedRegionList = new ArrayList<>();
private List<RegionInfo> nonFavoredAssignedRegionList = new ArrayList<>();
// For regions running on the favored nodes
private int totalFavoredAssignments = 0;
@ -91,7 +91,7 @@ public class AssignmentVerificationReport {
this.tableName = tableName;
// Get all the regions for this table
List<HRegionInfo> regionInfoList =
List<RegionInfo> regionInfoList =
snapshot.getTableToRegionMap().get(tableName);
// Get the total region num for the current table
this.totalRegions = regionInfoList.size();
@ -99,7 +99,7 @@ public class AssignmentVerificationReport {
// Get the existing assignment plan
FavoredNodesPlan favoredNodesAssignment = snapshot.getExistingAssignmentPlan();
// Get the region to region server mapping
Map<HRegionInfo, ServerName> currentAssignment =
Map<RegionInfo, ServerName> currentAssignment =
snapshot.getRegionToRegionServerMap();
// Initialize the server to its hosing region counter map
Map<ServerName, Integer> serverToHostingRegionCounterMap = new HashMap<>();
@ -109,7 +109,7 @@ public class AssignmentVerificationReport {
// Check the favored nodes and its locality information
// Also keep tracker of the most loaded and least loaded region servers
for (HRegionInfo region : regionInfoList) {
for (RegionInfo region : regionInfoList) {
try {
ServerName currentRS = currentAssignment.get(region);
// Handle unassigned regions
@ -317,7 +317,7 @@ public class AssignmentVerificationReport {
// Set the table name
this.tableName = tableName;
// Get all the regions for this table
List<HRegionInfo> regionInfoList = snapshot.getTableToRegionMap().get(
List<RegionInfo> regionInfoList = snapshot.getTableToRegionMap().get(
tableName);
// Get the total region num for the current table
this.totalRegions = regionInfoList.size();
@ -333,7 +333,7 @@ public class AssignmentVerificationReport {
// Check the favored nodes and its locality information
// Also keep tracker of the most loaded and least loaded region servers
for (HRegionInfo region : regionInfoList) {
for (RegionInfo region : regionInfoList) {
try {
// Get the favored nodes from the assignment plan and verify it.
List<ServerName> favoredNodes = plan.getFavoredNodes(region);
@ -466,7 +466,7 @@ public class AssignmentVerificationReport {
System.out.println("\tTotal unassigned regions: " +
unAssignedRegionsList.size());
if (isDetailMode) {
for (HRegionInfo region : unAssignedRegionsList) {
for (RegionInfo region : unAssignedRegionsList) {
System.out.println("\t\t" + region.getRegionNameAsString());
}
}
@ -474,7 +474,7 @@ public class AssignmentVerificationReport {
System.out.println("\tTotal regions NOT on favored nodes: " +
nonFavoredAssignedRegionList.size());
if (isDetailMode) {
for (HRegionInfo region : nonFavoredAssignedRegionList) {
for (RegionInfo region : nonFavoredAssignedRegionList) {
System.out.println("\t\t" + region.getRegionNameAsString());
}
}
@ -482,7 +482,7 @@ public class AssignmentVerificationReport {
System.out.println("\tTotal regions without favored nodes: " +
regionsWithoutValidFavoredNodes.size());
if (isDetailMode) {
for (HRegionInfo region : regionsWithoutValidFavoredNodes) {
for (RegionInfo region : regionsWithoutValidFavoredNodes) {
System.out.println("\t\t" + region.getRegionNameAsString());
}
}
@ -572,7 +572,7 @@ public class AssignmentVerificationReport {
* Return the unassigned regions
* @return unassigned regions
*/
List<HRegionInfo> getUnassignedRegions() {
List<RegionInfo> getUnassignedRegions() {
return unAssignedRegionsList;
}
@ -580,7 +580,7 @@ public class AssignmentVerificationReport {
* Return the regions without favored nodes
* @return regions without favored nodes
*/
List<HRegionInfo> getRegionsWithoutValidFavoredNodes() {
List<RegionInfo> getRegionsWithoutValidFavoredNodes() {
return regionsWithoutValidFavoredNodes;
}
@ -588,10 +588,10 @@ public class AssignmentVerificationReport {
* Return the regions not assigned to its favored nodes
* @return regions not assigned to its favored nodes
*/
List<HRegionInfo> getNonFavoredAssignedRegions() {
List<RegionInfo> getNonFavoredAssignedRegions() {
return nonFavoredAssignedRegionList;
}
/**
* Return the number of regions assigned to their favored nodes
* @return number of regions assigned to their favored nodes

View File

@ -32,13 +32,12 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
@ -53,8 +52,7 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.Triple;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A janitor for the catalog tables. Scans the <code>hbase:meta</code> catalog
@ -137,7 +135,7 @@ public class CatalogJanitor extends ScheduledChore {
* parent regioninfos
* @throws IOException
*/
Triple<Integer, Map<HRegionInfo, Result>, Map<HRegionInfo, Result>>
Triple<Integer, Map<RegionInfo, Result>, Map<RegionInfo, Result>>
getMergedRegionsAndSplitParents() throws IOException {
return getMergedRegionsAndSplitParents(null);
}
@ -152,15 +150,15 @@ public class CatalogJanitor extends ScheduledChore {
* parent regioninfos
* @throws IOException
*/
Triple<Integer, Map<HRegionInfo, Result>, Map<HRegionInfo, Result>>
Triple<Integer, Map<RegionInfo, Result>, Map<RegionInfo, Result>>
getMergedRegionsAndSplitParents(final TableName tableName) throws IOException {
final boolean isTableSpecified = (tableName != null);
// TODO: Only works with single hbase:meta region currently. Fix.
final AtomicInteger count = new AtomicInteger(0);
// Keep Map of found split parents. There are candidates for cleanup.
// Use a comparator that has split parents come before its daughters.
final Map<HRegionInfo, Result> splitParents = new TreeMap<>(new SplitParentFirstComparator());
final Map<HRegionInfo, Result> mergedRegions = new TreeMap<>();
final Map<RegionInfo, Result> splitParents = new TreeMap<>(new SplitParentFirstComparator());
final Map<RegionInfo, Result> mergedRegions = new TreeMap<>(RegionInfo.COMPARATOR);
// This visitor collects split parents and counts rows in the hbase:meta table
MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
@ -168,7 +166,7 @@ public class CatalogJanitor extends ScheduledChore {
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
count.incrementAndGet();
HRegionInfo info = MetaTableAccessor.getHRegionInfo(r);
RegionInfo info = MetaTableAccessor.getRegionInfo(r);
if (info == null) return true; // Keep scanning
if (isTableSpecified
&& info.getTable().compareTo(tableName) > 0) {
@ -200,8 +198,8 @@ public class CatalogJanitor extends ScheduledChore {
* the files on the file system
* @throws IOException
*/
boolean cleanMergeRegion(final HRegionInfo mergedRegion,
final HRegionInfo regionA, final HRegionInfo regionB) throws IOException {
boolean cleanMergeRegion(final RegionInfo mergedRegion,
final RegionInfo regionA, final RegionInfo regionB) throws IOException {
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
Path rootdir = this.services.getMasterFileSystem().getRootDir();
Path tabledir = FSUtils.getTableDir(rootdir, mergedRegion.getTable());
@ -244,21 +242,21 @@ public class CatalogJanitor extends ScheduledChore {
LOG.debug("CatalogJanitor already running");
return result;
}
Triple<Integer, Map<HRegionInfo, Result>, Map<HRegionInfo, Result>> scanTriple =
Triple<Integer, Map<RegionInfo, Result>, Map<RegionInfo, Result>> scanTriple =
getMergedRegionsAndSplitParents();
/**
* clean merge regions first
*/
Map<HRegionInfo, Result> mergedRegions = scanTriple.getSecond();
for (Map.Entry<HRegionInfo, Result> e : mergedRegions.entrySet()) {
Map<RegionInfo, Result> mergedRegions = scanTriple.getSecond();
for (Map.Entry<RegionInfo, Result> e : mergedRegions.entrySet()) {
if (this.services.isInMaintenanceMode()) {
// Stop cleaning if the master is in maintenance mode
break;
}
PairOfSameType<HRegionInfo> p = MetaTableAccessor.getMergeRegions(e.getValue());
HRegionInfo regionA = p.getFirst();
HRegionInfo regionB = p.getSecond();
PairOfSameType<RegionInfo> p = MetaTableAccessor.getMergeRegions(e.getValue());
RegionInfo regionA = p.getFirst();
RegionInfo regionB = p.getSecond();
if (regionA == null || regionB == null) {
LOG.warn("Unexpected references regionA="
+ (regionA == null ? "null" : regionA.getShortNameToLog())
@ -274,12 +272,12 @@ public class CatalogJanitor extends ScheduledChore {
/**
* clean split parents
*/
Map<HRegionInfo, Result> splitParents = scanTriple.getThird();
Map<RegionInfo, Result> splitParents = scanTriple.getThird();
// Now work on our list of found parents. See if any we can clean up.
// regions whose parents are still around
HashSet<String> parentNotCleaned = new HashSet<>();
for (Map.Entry<HRegionInfo, Result> e : splitParents.entrySet()) {
for (Map.Entry<RegionInfo, Result> e : splitParents.entrySet()) {
if (this.services.isInMaintenanceMode()) {
// Stop cleaning if the master is in maintenance mode
break;
@ -291,7 +289,7 @@ public class CatalogJanitor extends ScheduledChore {
} else {
// We could not clean the parent, so it's daughters should not be
// cleaned either (HBASE-6160)
PairOfSameType<HRegionInfo> daughters =
PairOfSameType<RegionInfo> daughters =
MetaTableAccessor.getDaughterRegions(e.getValue());
parentNotCleaned.add(daughters.getFirst().getEncodedName());
parentNotCleaned.add(daughters.getSecond().getEncodedName());
@ -307,11 +305,11 @@ public class CatalogJanitor extends ScheduledChore {
* Compare HRegionInfos in a way that has split parents sort BEFORE their
* daughters.
*/
static class SplitParentFirstComparator implements Comparator<HRegionInfo> {
static class SplitParentFirstComparator implements Comparator<RegionInfo> {
Comparator<byte[]> rowEndKeyComparator = new Bytes.RowEndKeyComparator();
@Override
public int compare(HRegionInfo left, HRegionInfo right) {
// This comparator differs from the one HRegionInfo in that it sorts
public int compare(RegionInfo left, RegionInfo right) {
// This comparator differs from the one RegionInfo in that it sorts
// parent before daughters.
if (left == null) return -1;
if (right == null) return 1;
@ -330,14 +328,14 @@ public class CatalogJanitor extends ScheduledChore {
/**
* If daughters no longer hold reference to the parents, delete the parent.
* @param parent HRegionInfo of split offlined parent
* @param parent RegionInfo of split offlined parent
* @param rowContent Content of <code>parent</code> row in
* <code>metaRegionName</code>
* @return True if we removed <code>parent</code> from meta table and from
* the filesystem.
* @throws IOException
*/
boolean cleanParent(final HRegionInfo parent, Result rowContent)
boolean cleanParent(final RegionInfo parent, Result rowContent)
throws IOException {
// Check whether it is a merged region and not clean reference
// No necessary to check MERGEB_QUALIFIER because these two qualifiers will
@ -347,7 +345,7 @@ public class CatalogJanitor extends ScheduledChore {
return false;
}
// Run checks on each daughter split.
PairOfSameType<HRegionInfo> daughters = MetaTableAccessor.getDaughterRegions(rowContent);
PairOfSameType<RegionInfo> daughters = MetaTableAccessor.getDaughterRegions(rowContent);
Pair<Boolean, Boolean> a = checkDaughterInFs(parent, daughters.getFirst());
Pair<Boolean, Boolean> b = checkDaughterInFs(parent, daughters.getSecond());
if (hasNoReferences(a) && hasNoReferences(b)) {
@ -388,7 +386,7 @@ public class CatalogJanitor extends ScheduledChore {
* whether the daughter has references to the parent.
* @throws IOException
*/
Pair<Boolean, Boolean> checkDaughterInFs(final HRegionInfo parent, final HRegionInfo daughter)
Pair<Boolean, Boolean> checkDaughterInFs(final RegionInfo parent, final RegionInfo daughter)
throws IOException {
if (daughter == null) {
return new Pair<>(Boolean.FALSE, Boolean.FALSE);
@ -443,11 +441,11 @@ public class CatalogJanitor extends ScheduledChore {
* @return true if the specified region doesn't have merge qualifier now
* @throws IOException
*/
public boolean cleanMergeQualifier(final HRegionInfo region)
public boolean cleanMergeQualifier(final RegionInfo region)
throws IOException {
// Get merge regions if it is a merged region and already has merge
// qualifier
Pair<HRegionInfo, HRegionInfo> mergeRegions = MetaTableAccessor
Pair<RegionInfo, RegionInfo> mergeRegions = MetaTableAccessor
.getRegionsFromMergeQualifier(this.services.getConnection(),
region.getRegionName());
if (mergeRegions == null

View File

@ -18,6 +18,10 @@
*/
package org.apache.hadoop.hbase.master;
import javax.servlet.ServletException;
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.reflect.Constructor;
@ -46,11 +50,6 @@ import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.regex.Pattern;
import javax.servlet.ServletException;
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -64,7 +63,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
@ -76,10 +74,10 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
@ -167,16 +165,6 @@ import org.apache.hadoop.hbase.replication.master.TableCFsUpdater;
import org.apache.hadoop.hbase.replication.regionserver.Replication;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CompressionTest;
@ -200,12 +188,24 @@ import org.apache.hadoop.hbase.zookeeper.SplitOrMergeTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.server.ServerConnector;
import org.eclipse.jetty.servlet.ServletHolder;
import org.eclipse.jetty.webapp.WebAppContext;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Service;
@ -1416,7 +1416,7 @@ public class HMaster extends HRegionServer implements MasterServices {
}
}
Map<TableName, Map<ServerName, List<HRegionInfo>>> assignmentsByTable =
Map<TableName, Map<ServerName, List<RegionInfo>>> assignmentsByTable =
this.assignmentManager.getRegionStates().getAssignmentsByTable();
List<RegionPlan> plans = new ArrayList<>();
@ -1426,7 +1426,7 @@ public class HMaster extends HRegionServer implements MasterServices {
this.balancer.setClusterLoad(
this.assignmentManager.getRegionStates().getAssignmentsByTable());
for (Entry<TableName, Map<ServerName, List<HRegionInfo>>> e : assignmentsByTable.entrySet()) {
for (Entry<TableName, Map<ServerName, List<RegionInfo>>> e : assignmentsByTable.entrySet()) {
List<RegionPlan> partialPlans = this.balancer.balanceCluster(e.getKey(), e.getValue());
if (partialPlans != null) plans.addAll(partialPlans);
}
@ -1560,7 +1560,7 @@ public class HMaster extends HRegionServer implements MasterServices {
@Override
public long mergeRegions(
final HRegionInfo[] regionsToMerge,
final RegionInfo[] regionsToMerge,
final boolean forcible,
final long nonceGroup,
final long nonce) throws IOException {
@ -1579,7 +1579,7 @@ public class HMaster extends HRegionServer implements MasterServices {
+ " and " + regionsToMerge[1].getTable());
}
if (regionsToMerge[0].compareTo(regionsToMerge[1]) == 0) {
if (RegionInfo.COMPARATOR.compare(regionsToMerge[0], regionsToMerge[1]) == 0) {
throw new MergeRegionException(
"Cannot merge a region to itself " + regionsToMerge[0] + ", " + regionsToMerge[1]);
}
@ -1607,7 +1607,7 @@ public class HMaster extends HRegionServer implements MasterServices {
}
@Override
public long splitRegion(final HRegionInfo regionInfo, final byte[] splitRow,
public long splitRegion(final RegionInfo regionInfo, final byte[] splitRow,
final long nonceGroup, final long nonce)
throws IOException {
checkInitialized();
@ -1637,7 +1637,7 @@ public class HMaster extends HRegionServer implements MasterServices {
RegionState regionState = assignmentManager.getRegionStates().
getRegionState(Bytes.toString(encodedRegionName));
HRegionInfo hri;
RegionInfo hri;
if (regionState != null) {
hri = regionState.getRegion();
} else {
@ -1736,7 +1736,7 @@ public class HMaster extends HRegionServer implements MasterServices {
String namespace = tableDescriptor.getTableName().getNamespaceAsString();
this.clusterSchemaService.getNamespace(namespace);
HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(tableDescriptor, splitKeys);
RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(tableDescriptor, splitKeys);
sanityCheckTableDescriptor(tableDescriptor);
return MasterProcedureUtil.submitProcedure(
@ -1776,7 +1776,7 @@ public class HMaster extends HRegionServer implements MasterServices {
"Only system table creation can use this createSystemTable API");
}
HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(tableDescriptor, null);
RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(tableDescriptor, null);
LOG.info(getClientIdAuditPrefix() + " create " + tableDescriptor);
@ -2357,10 +2357,10 @@ public class HMaster extends HRegionServer implements MasterServices {
* may be null.
*/
@VisibleForTesting // Used by TestMaster.
Pair<HRegionInfo, ServerName> getTableRegionForRow(
Pair<RegionInfo, ServerName> getTableRegionForRow(
final TableName tableName, final byte [] rowKey)
throws IOException {
final AtomicReference<Pair<HRegionInfo, ServerName>> result = new AtomicReference<>(null);
final AtomicReference<Pair<RegionInfo, ServerName>> result = new AtomicReference<>(null);
MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
@Override
@ -2368,8 +2368,8 @@ public class HMaster extends HRegionServer implements MasterServices {
if (data == null || data.size() <= 0) {
return true;
}
Pair<HRegionInfo, ServerName> pair =
new Pair(MetaTableAccessor.getHRegionInfo(data),
Pair<RegionInfo, ServerName> pair =
new Pair(MetaTableAccessor.getRegionInfo(data),
MetaTableAccessor.getServerName(data,0));
if (pair == null) {
return false;

View File

@ -18,25 +18,21 @@
*/
package org.apache.hadoop.hbase.master;
import java.util.*;
import java.util.List;
import java.util.Map;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.UnmodifiableIterator;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.conf.ConfigurationObserver;
import org.apache.yetus.audience.InterfaceAudience;
import edu.umd.cs.findbugs.annotations.Nullable;
import org.apache.hadoop.hbase.security.access.AccessControlLists;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.util.StringUtils;
/**
* Makes decisions about the placement and movement of Regions across
@ -81,7 +77,7 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* Pass RegionStates and allow balancer to set the current cluster load.
* @param ClusterLoad
*/
void setClusterLoad(Map<TableName, Map<ServerName, List<HRegionInfo>>> ClusterLoad);
void setClusterLoad(Map<TableName, Map<ServerName, List<RegionInfo>>> ClusterLoad);
/**
* Set the master service.
@ -96,7 +92,7 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* @return List of plans
*/
List<RegionPlan> balanceCluster(TableName tableName, Map<ServerName,
List<HRegionInfo>> clusterState) throws HBaseIOException;
List<RegionInfo>> clusterState) throws HBaseIOException;
/**
* Perform the major balance operation
@ -104,7 +100,7 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* @return List of plans
*/
List<RegionPlan> balanceCluster(Map<ServerName,
List<HRegionInfo>> clusterState) throws HBaseIOException;
List<RegionInfo>> clusterState) throws HBaseIOException;
/**
* Perform a Round Robin assignment of regions.
@ -112,8 +108,8 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* @param servers
* @return Map of servername to regioninfos
*/
Map<ServerName, List<HRegionInfo>> roundRobinAssignment(
List<HRegionInfo> regions,
Map<ServerName, List<RegionInfo>> roundRobinAssignment(
List<RegionInfo> regions,
List<ServerName> servers
) throws HBaseIOException;
@ -124,8 +120,8 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* @return List of plans
*/
@Nullable
Map<ServerName, List<HRegionInfo>> retainAssignment(
Map<HRegionInfo, ServerName> regions,
Map<ServerName, List<RegionInfo>> retainAssignment(
Map<RegionInfo, ServerName> regions,
List<ServerName> servers
) throws HBaseIOException;
@ -136,7 +132,7 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* @return Servername
*/
ServerName randomAssignment(
HRegionInfo regionInfo, List<ServerName> servers
RegionInfo regionInfo, List<ServerName> servers
) throws HBaseIOException;
/**
@ -150,13 +146,13 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* @param regionInfo
* @param sn
*/
void regionOnline(HRegionInfo regionInfo, ServerName sn);
void regionOnline(RegionInfo regionInfo, ServerName sn);
/**
* Marks the region as offline at balancer.
* @param regionInfo
*/
void regionOffline(HRegionInfo regionInfo);
void regionOffline(RegionInfo regionInfo);
/*
* Notification that config has changed

View File

@ -29,12 +29,12 @@ import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.ClusterId;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.yetus.audience.InterfaceAudience;
/**
* This class abstracts a bunch of operations the HMaster needs to interact with
@ -388,9 +389,9 @@ public class MasterFileSystem {
// created here in bootstrap and it'll need to be cleaned up. Better to
// not make it in first place. Turn off block caching for bootstrap.
// Enable after.
HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
TableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
HRegion meta = HRegion.createHRegion(metaHRI, rd, c, setInfoFamilyCachingForMeta(metaDescriptor, false), null);
HRegion meta = HRegion.createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, rd,
c, setInfoFamilyCachingForMeta(metaDescriptor, false), null);
meta.close();
} catch (IOException e) {
e = e instanceof RemoteException ?
@ -416,12 +417,12 @@ public class MasterFileSystem {
return builder.build();
}
public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName)
public void deleteFamilyFromFS(RegionInfo region, byte[] familyName)
throws IOException {
deleteFamilyFromFS(rootdir, region, familyName);
}
public void deleteFamilyFromFS(Path rootDir, HRegionInfo region, byte[] familyName)
public void deleteFamilyFromFS(Path rootDir, RegionInfo region, byte[] familyName)
throws IOException {
// archive family store files
Path tableDir = FSUtils.getTableDir(rootDir, region.getTable());

View File

@ -25,10 +25,10 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
/**
@ -118,7 +119,7 @@ public class MasterMetaBootstrap {
// TODO: Unimplemented
// long timeout =
// master.getConfiguration().getLong("hbase.catalog.verification.timeout", 1000);
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
status.setStatus("Assigning hbase:meta region");
} else {
status.setStatus("Assigning hbase:meta region, replicaId " + replicaId);
@ -127,11 +128,11 @@ public class MasterMetaBootstrap {
// Get current meta state from zk.
RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper(), replicaId);
LOG.debug("meta state from zookeeper: " + metaState);
HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
HRegionInfo.FIRST_META_REGIONINFO, replicaId);
RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId);
assignmentManager.assignMeta(hri, metaState.getServerName());
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
// TODO: should we prevent from using state manager before meta was initialized?
// tableStateManager.start();
master.getTableStateManager()
@ -144,7 +145,7 @@ public class MasterMetaBootstrap {
// if the meta region server is died at this time, we need it to be re-assigned
// by SSH so that system tables can be assigned.
// No need to wait for meta is assigned = 0 when meta is just verified.
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) enableCrashedServerProcessing(false);
if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) enableCrashedServerProcessing(false);
LOG.info("hbase:meta with replicaId " + replicaId + ", location="
+ master.getMetaTableLocator().getMetaRegionLocation(master.getZooKeeper(), replicaId));
status.setStatus("META assigned.");

View File

@ -32,16 +32,16 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.client.VersionInfoUtil;
@ -74,6 +74,15 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.access.AccessController;
import org.apache.hadoop.hbase.security.visibility.VisibilityController;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
@ -262,13 +271,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Remov
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.zookeeper.KeeperException;
/**
* Implements the master RPC services.
@ -501,7 +503,7 @@ public class MasterRpcServices extends RSRpcServices
}
final byte[] regionName = req.getRegion().getValue().toByteArray();
final HRegionInfo regionInfo = master.getAssignmentManager().getRegionInfo(regionName);
final RegionInfo regionInfo = master.getAssignmentManager().getRegionInfo(regionName);
if (regionInfo == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
final AssignRegionResponse arr = AssignRegionResponse.newBuilder().build();
@ -712,7 +714,7 @@ public class MasterRpcServices extends RSRpcServices
RegionStates regionStates = master.getAssignmentManager().getRegionStates();
assert(request.getRegionCount() == 2);
HRegionInfo[] regionsToMerge = new HRegionInfo[request.getRegionCount()];
RegionInfo[] regionsToMerge = new RegionInfo[request.getRegionCount()];
for (int i = 0; i < request.getRegionCount(); i++) {
final byte[] encodedNameOfRegion = request.getRegion(i).getValue().toByteArray();
if (request.getRegion(i).getType() != RegionSpecifierType.ENCODED_REGION_NAME) {
@ -745,7 +747,7 @@ public class MasterRpcServices extends RSRpcServices
final SplitTableRegionRequest request) throws ServiceException {
try {
long procId = master.splitRegion(
HRegionInfo.convert(request.getRegionInfo()),
ProtobufUtil.toRegionInfo(request.getRegionInfo()),
request.hasSplitRow() ? request.getSplitRow().toByteArray() : null,
request.getNonceGroup(),
request.getNonce());
@ -1344,7 +1346,7 @@ public class MasterRpcServices extends RSRpcServices
}
final byte[] regionName = request.getRegion().getValue().toByteArray();
final HRegionInfo hri = master.getAssignmentManager().getRegionInfo(regionName);
final RegionInfo hri = master.getAssignmentManager().getRegionInfo(regionName);
if (hri == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
if (master.cpHost != null) {
@ -1502,17 +1504,17 @@ public class MasterRpcServices extends RSRpcServices
LOG.warn("unassignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
+ " actual: " + type);
}
Pair<HRegionInfo, ServerName> pair =
Pair<RegionInfo, ServerName> pair =
MetaTableAccessor.getRegion(master.getConnection(), regionName);
if (Bytes.equals(HRegionInfo.FIRST_META_REGIONINFO.getRegionName(),regionName)) {
pair = new Pair<>(HRegionInfo.FIRST_META_REGIONINFO,
if (Bytes.equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(),regionName)) {
pair = new Pair<>(RegionInfoBuilder.FIRST_META_REGIONINFO,
master.getMetaTableLocator().getMetaRegionLocation(master.getZooKeeper()));
}
if (pair == null) {
throw new UnknownRegionException(Bytes.toString(regionName));
}
HRegionInfo hri = pair.getFirst();
RegionInfo hri = pair.getFirst();
if (master.cpHost != null) {
if (master.cpHost.preUnassign(hri, force)) {
return urr;
@ -1598,7 +1600,7 @@ public class MasterRpcServices extends RSRpcServices
try {
master.checkInitialized();
byte[] regionName = request.getRegion().getValue().toByteArray();
TableName tableName = HRegionInfo.getTable(regionName);
TableName tableName = RegionInfo.getTable(regionName);
// if the region is a mob region, do the mob file compaction.
if (MobUtils.isMobRegionName(tableName, regionName)) {
return compactMob(request, tableName);
@ -1615,12 +1617,12 @@ public class MasterRpcServices extends RSRpcServices
public GetRegionInfoResponse getRegionInfo(final RpcController controller,
final GetRegionInfoRequest request) throws ServiceException {
byte[] regionName = request.getRegion().getValue().toByteArray();
TableName tableName = HRegionInfo.getTable(regionName);
TableName tableName = RegionInfo.getTable(regionName);
if (MobUtils.isMobRegionName(tableName, regionName)) {
// a dummy region info contains the compaction state.
HRegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(tableName);
RegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(tableName);
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
builder.setRegionInfo(HRegionInfo.convert(mobRegionInfo));
builder.setRegionInfo(ProtobufUtil.toRegionInfo(mobRegionInfo));
if (request.hasCompactionState() && request.getCompactionState()) {
builder.setCompactionState(master.getMobCompactionState(tableName));
}
@ -1960,9 +1962,9 @@ public class MasterRpcServices extends RSRpcServices
NonceProcedureRunnable npr;
LockType type = LockType.valueOf(request.getLockType().name());
if (request.getRegionInfoCount() > 0) {
final HRegionInfo[] regionInfos = new HRegionInfo[request.getRegionInfoCount()];
final RegionInfo[] regionInfos = new RegionInfo[request.getRegionInfoCount()];
for (int i = 0; i < request.getRegionInfoCount(); ++i) {
regionInfos[i] = HRegionInfo.convert(request.getRegionInfo(i));
regionInfos[i] = ProtobufUtil.toRegionInfo(request.getRegionInfo(i));
}
npr = new NonceProcedureRunnable(master, request.getNonceGroup(), request.getNonce()) {
@Override
@ -2051,7 +2053,7 @@ public class MasterRpcServices extends RSRpcServices
MasterQuotaManager quotaManager = this.master.getMasterQuotaManager();
final long now = EnvironmentEdgeManager.currentTime();
for (RegionSpaceUse report : request.getSpaceUseList()) {
quotaManager.addRegionSize(HRegionInfo.convert(
quotaManager.addRegionSize(ProtobufUtil.toRegionInfo(
report.getRegionInfo()), report.getRegionSize(), now);
}
return RegionSpaceUseReportResponse.newBuilder().build();
@ -2069,10 +2071,10 @@ public class MasterRpcServices extends RSRpcServices
GetSpaceQuotaRegionSizesResponse.Builder builder =
GetSpaceQuotaRegionSizesResponse.newBuilder();
if (quotaManager != null) {
Map<HRegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes();
Map<RegionInfo,Long> regionSizes = quotaManager.snapshotRegionSizes();
Map<TableName,Long> regionSizesByTable = new HashMap<>();
// Translate hregioninfo+long -> tablename+long
for (Entry<HRegionInfo,Long> entry : regionSizes.entrySet()) {
for (Entry<RegionInfo,Long> entry : regionSizes.entrySet()) {
final TableName tableName = entry.getKey().getTable();
Long prevSize = regionSizesByTable.get(tableName);
if (prevSize == null) {

View File

@ -21,21 +21,20 @@ package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.locking.LockManager;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
@ -48,8 +47,10 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.Service;
/**
@ -291,7 +292,7 @@ public interface MasterServices extends Server {
* @throws IOException
*/
long mergeRegions(
final HRegionInfo[] regionsToMerge,
final RegionInfo[] regionsToMerge,
final boolean forcible,
final long nonceGroup,
final long nonce) throws IOException;
@ -306,7 +307,7 @@ public interface MasterServices extends Server {
* @throws IOException
*/
long splitRegion(
final HRegionInfo regionInfo,
final RegionInfo regionInfo,
final byte [] splitRow,
final long nonceGroup,
final long nonce) throws IOException;

View File

@ -36,13 +36,13 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
@ -284,7 +284,7 @@ public class MasterWalManager {
* @param serverName Failed region server whose wals to be replayed
* @param regions Set of regions to be recovered
*/
public void prepareLogReplay(ServerName serverName, Set<HRegionInfo> regions) throws IOException {
public void prepareLogReplay(ServerName serverName, Set<RegionInfo> regions) throws IOException {
if (!this.distributedLogReplay) {
return;
}

View File

@ -40,31 +40,32 @@ import org.apache.commons.cli.ParseException;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.MunkresAssignment;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
/**
* A tool that is used for manipulating and viewing favored nodes information
@ -197,12 +198,12 @@ public class RegionPlacementMaintainer {
Map<String, Map<String, Float>> regionLocalityMap, FavoredNodesPlan plan,
boolean munkresForSecondaryAndTertiary) throws IOException {
// Get the all the regions for the current table
List<HRegionInfo> regions =
List<RegionInfo> regions =
assignmentSnapshot.getTableToRegionMap().get(tableName);
int numRegions = regions.size();
// Get the current assignment map
Map<HRegionInfo, ServerName> currentAssignmentMap =
Map<RegionInfo, ServerName> currentAssignmentMap =
assignmentSnapshot.getRegionToRegionServerMap();
// Get the all the region servers
@ -257,12 +258,12 @@ public class RegionPlacementMaintainer {
// Compute the total rack locality for each region in each rack. The total
// rack locality is the sum of the localities of a region on all servers in
// a rack.
Map<String, Map<HRegionInfo, Float>> rackRegionLocality = new HashMap<>();
Map<String, Map<RegionInfo, Float>> rackRegionLocality = new HashMap<>();
for (int i = 0; i < numRegions; i++) {
HRegionInfo region = regions.get(i);
RegionInfo region = regions.get(i);
for (int j = 0; j < regionSlots; j += slotsPerServer) {
String rack = rackManager.getRack(servers.get(j / slotsPerServer));
Map<HRegionInfo, Float> rackLocality = rackRegionLocality.get(rack);
Map<RegionInfo, Float> rackLocality = rackRegionLocality.get(rack);
if (rackLocality == null) {
rackLocality = new HashMap<>();
rackRegionLocality.put(rack, rackLocality);
@ -417,18 +418,18 @@ public class RegionPlacementMaintainer {
LOG.info("Assignment plan for secondary and tertiary generated " +
"using MunkresAssignment");
} else {
Map<HRegionInfo, ServerName> primaryRSMap = new HashMap<>();
Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>();
for (int i = 0; i < numRegions; i++) {
primaryRSMap.put(regions.get(i), servers.get(primaryAssignment[i] / slotsPerServer));
}
FavoredNodeAssignmentHelper favoredNodeHelper =
new FavoredNodeAssignmentHelper(servers, conf);
favoredNodeHelper.initialize();
Map<HRegionInfo, ServerName[]> secondaryAndTertiaryMap =
Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap =
favoredNodeHelper.placeSecondaryAndTertiaryWithRestrictions(primaryRSMap);
for (int i = 0; i < numRegions; i++) {
List<ServerName> favoredServers = new ArrayList<>(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM);
HRegionInfo currentRegion = regions.get(i);
RegionInfo currentRegion = regions.get(i);
ServerName s = primaryRSMap.get(currentRegion);
favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
ServerName.NON_STARTCODE));
@ -467,7 +468,7 @@ public class RegionPlacementMaintainer {
FavoredNodesPlan plan = new FavoredNodesPlan();
// Get the table to region mapping
Map<TableName, List<HRegionInfo>> tableToRegionMap =
Map<TableName, List<RegionInfo>> tableToRegionMap =
assignmentSnapshot.getTableToRegionMap();
LOG.info("Start to generate the new assignment plan for the " +
+ tableToRegionMap.keySet().size() + " tables" );
@ -635,8 +636,8 @@ public class RegionPlacementMaintainer {
try {
LOG.info("Start to update the hbase:meta with the new assignment plan");
Map<String, List<ServerName>> assignmentMap = plan.getAssignmentMap();
Map<HRegionInfo, List<ServerName>> planToUpdate = new HashMap<>(assignmentMap.size());
Map<String, HRegionInfo> regionToRegionInfoMap =
Map<RegionInfo, List<ServerName>> planToUpdate = new HashMap<>(assignmentMap.size());
Map<String, RegionInfo> regionToRegionInfoMap =
getRegionAssignmentSnapshot().getRegionNameToRegionInfoMap();
for (Map.Entry<String, List<ServerName>> entry : assignmentMap.entrySet()) {
planToUpdate.put(regionToRegionInfoMap.get(entry.getKey()), entry.getValue());
@ -659,21 +660,21 @@ public class RegionPlacementMaintainer {
throws IOException{
LOG.info("Start to update the region servers with the new assignment plan");
// Get the region to region server map
Map<ServerName, List<HRegionInfo>> currentAssignment =
Map<ServerName, List<RegionInfo>> currentAssignment =
this.getRegionAssignmentSnapshot().getRegionServerToRegionMap();
// track of the failed and succeeded updates
int succeededNum = 0;
Map<ServerName, Exception> failedUpdateMap = new HashMap<>();
for (Map.Entry<ServerName, List<HRegionInfo>> entry :
for (Map.Entry<ServerName, List<RegionInfo>> entry :
currentAssignment.entrySet()) {
List<Pair<HRegionInfo, List<ServerName>>> regionUpdateInfos = new ArrayList<>();
List<Pair<RegionInfo, List<ServerName>>> regionUpdateInfos = new ArrayList<>();
try {
// Keep track of the favored updates for the current region server
FavoredNodesPlan singleServerPlan = null;
// Find out all the updates for the current region server
for (HRegionInfo region : entry.getValue()) {
for (RegionInfo region : entry.getValue()) {
List<ServerName> favoredServerList = plan.getFavoredNodes(region);
if (favoredServerList != null &&
favoredServerList.size() == FavoredNodeAssignmentHelper.FAVORED_NODES_NUM) {
@ -746,7 +747,7 @@ public class RegionPlacementMaintainer {
throws IOException {
Map<TableName, Integer> movesPerTable = new HashMap<>();
SnapshotOfRegionAssignmentFromMeta snapshot = this.getRegionAssignmentSnapshot();
Map<TableName, List<HRegionInfo>> tableToRegions = snapshot
Map<TableName, List<RegionInfo>> tableToRegions = snapshot
.getTableToRegionMap();
FavoredNodesPlan oldPlan = snapshot.getExistingAssignmentPlan();
Set<TableName> tables = snapshot.getTableSet();
@ -756,8 +757,8 @@ public class RegionPlacementMaintainer {
&& !this.targetTableSet.contains(table)) {
continue;
}
List<HRegionInfo> regions = tableToRegions.get(table);
for (HRegionInfo region : regions) {
List<RegionInfo> regions = tableToRegions.get(table);
for (RegionInfo region : regions) {
List<ServerName> oldServers = oldPlan.getFavoredNodes(region);
List<ServerName> newServers = newPlan.getFavoredNodes(region);
if (oldServers != null && newServers != null) {
@ -789,7 +790,7 @@ public class RegionPlacementMaintainer {
SnapshotOfRegionAssignmentFromMeta snapshot = this.getRegionAssignmentSnapshot();
FavoredNodesPlan oldPlan = snapshot.getExistingAssignmentPlan();
Set<TableName> tables = snapshot.getTableSet();
Map<TableName, List<HRegionInfo>> tableToRegionsMap = snapshot.getTableToRegionMap();
Map<TableName, List<RegionInfo>> tableToRegionsMap = snapshot.getTableToRegionMap();
for (TableName table : tables) {
float[] deltaLocality = new float[3];
float[] locality = new float[3];
@ -797,13 +798,13 @@ public class RegionPlacementMaintainer {
&& !this.targetTableSet.contains(table)) {
continue;
}
List<HRegionInfo> regions = tableToRegionsMap.get(table);
List<RegionInfo> regions = tableToRegionsMap.get(table);
System.out.println("==================================================");
System.out.println("Assignment Plan Projection Report For Table: " + table);
System.out.println("\t Total regions: " + regions.size());
System.out.println("\t" + movesPerTable.get(table)
+ " primaries will move due to their primary has changed");
for (HRegionInfo currentRegion : regions) {
for (RegionInfo currentRegion : regions) {
Map<String, Float> regionLocality = regionLocalityMap.get(currentRegion
.getEncodedName());
if (regionLocality == null) {
@ -881,7 +882,7 @@ public class RegionPlacementMaintainer {
SnapshotOfRegionAssignmentFromMeta snapshot = this.getRegionAssignmentSnapshot();
FavoredNodesPlan assignmentPlan = snapshot.getExistingAssignmentPlan();
Set<TableName> tables = snapshot.getTableSet();
Map<TableName, List<HRegionInfo>> tableToRegionsMap = snapshot
Map<TableName, List<RegionInfo>> tableToRegionsMap = snapshot
.getTableToRegionMap();
for (TableName table : tables) {
float[] locality = new float[3];
@ -889,8 +890,8 @@ public class RegionPlacementMaintainer {
&& !this.targetTableSet.contains(table)) {
continue;
}
List<HRegionInfo> regions = tableToRegionsMap.get(table);
for (HRegionInfo currentRegion : regions) {
List<RegionInfo> regions = tableToRegionsMap.get(table);
for (RegionInfo currentRegion : regions) {
Map<String, Float> regionLocality = regionLocalityMap.get(currentRegion
.getEncodedName());
if (regionLocality == null) {
@ -1094,7 +1095,7 @@ public class RegionPlacementMaintainer {
LOG.info("Going to update the region " + regionName + " with the new favored nodes " +
favoredNodesStr);
List<ServerName> favoredNodes = null;
HRegionInfo regionInfo =
RegionInfo regionInfo =
rp.getRegionAssignmentSnapshot().getRegionNameToRegionInfoMap().get(regionName);
if (regionInfo == null) {
LOG.error("Cannot find the region " + regionName + " from the META");

View File

@ -20,10 +20,10 @@ package org.apache.hadoop.hbase.master;
import java.io.Serializable;
import java.util.Comparator;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
/**
* Stores the plan for the move of an individual region.
@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.ServerName;
@InterfaceAudience.LimitedPrivate("Coprocessors")
@InterfaceStability.Evolving
public class RegionPlan implements Comparable<RegionPlan> {
private final HRegionInfo hri;
private final RegionInfo hri;
private final ServerName source;
private ServerName dest;
@ -66,7 +66,7 @@ public class RegionPlan implements Comparable<RegionPlan> {
* @param source regionserver region should be moved from
* @param dest regionserver region should be moved to
*/
public RegionPlan(final HRegionInfo hri, ServerName source, ServerName dest) {
public RegionPlan(final RegionInfo hri, ServerName source, ServerName dest) {
this.hri = hri;
this.source = source;
this.dest = dest;
@ -103,7 +103,7 @@ public class RegionPlan implements Comparable<RegionPlan> {
return this.hri.getEncodedName();
}
public HRegionInfo getRegionInfo() {
public RegionInfo getRegionInfo() {
return this.hri;
}

View File

@ -43,32 +43,32 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClockOutOfSyncException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.YouAreDeadException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* The ServerManager class manages info about region servers.
@ -257,7 +257,7 @@ public class ServerManager {
private void updateLastFlushedSequenceIds(ServerName sn, ServerLoad hsl) {
Map<byte[], RegionLoad> regionsLoad = hsl.getRegionsLoad();
for (Entry<byte[], RegionLoad> entry : regionsLoad.entrySet()) {
byte[] encodedRegionName = Bytes.toBytes(HRegionInfo.encodeRegionName(entry.getKey()));
byte[] encodedRegionName = Bytes.toBytes(RegionInfo.encodeRegionName(entry.getKey()));
Long existingValue = flushedSequenceIdByRegion.get(encodedRegionName);
long l = entry.getValue().getCompleteSequenceId();
// Don't let smaller sequence ids override greater sequence ids.
@ -714,7 +714,7 @@ public class ServerManager {
* @param region region to warmup
*/
public void sendRegionWarmup(ServerName server,
HRegionInfo region) {
RegionInfo region) {
if (server == null) return;
try {
AdminService.BlockingInterface admin = getRsAdmin(server);
@ -732,7 +732,7 @@ public class ServerManager {
* to close the region. This bypasses the active hmaster.
*/
public static void closeRegionSilentlyAndWait(ClusterConnection connection,
ServerName server, HRegionInfo region, long timeout) throws IOException, InterruptedException {
ServerName server, RegionInfo region, long timeout) throws IOException, InterruptedException {
AdminService.BlockingInterface rs = connection.getAdmin(server);
HBaseRpcController controller = connection.getRpcControllerFactory().newController();
try {
@ -744,7 +744,7 @@ public class ServerManager {
while (System.currentTimeMillis() < expiration) {
controller.reset();
try {
HRegionInfo rsRegion =
RegionInfo rsRegion =
ProtobufUtil.getRegionInfo(controller, rs, region.getRegionName());
if (rsRegion == null) return;
} catch (IOException ioe) {
@ -1023,14 +1023,14 @@ public class ServerManager {
/**
* Called by delete table and similar to notify the ServerManager that a region was removed.
*/
public void removeRegion(final HRegionInfo regionInfo) {
public void removeRegion(final RegionInfo regionInfo) {
final byte[] encodedName = regionInfo.getEncodedNameAsBytes();
storeFlushedSequenceIdsByRegion.remove(encodedName);
flushedSequenceIdByRegion.remove(encodedName);
}
@VisibleForTesting
public boolean isRegionInServerManagerStates(final HRegionInfo hri) {
public boolean isRegionInServerManagerStates(final RegionInfo hri) {
final byte[] encodedName = hri.getEncodedNameAsBytes();
return (storeFlushedSequenceIdsByRegion.containsKey(encodedName)
|| flushedSequenceIdByRegion.containsKey(encodedName));
@ -1039,8 +1039,8 @@ public class ServerManager {
/**
* Called by delete table and similar to notify the ServerManager that a region was removed.
*/
public void removeRegions(final List<HRegionInfo> regions) {
for (HRegionInfo hri: regions) {
public void removeRegions(final List<RegionInfo> regions) {
for (RegionInfo hri: regions) {
removeRegion(hri);
}
}

View File

@ -34,19 +34,19 @@ import java.util.TreeMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
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.MetaTableAccessor.Visitor;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Used internally for reading meta and constructing datastructures that are
@ -62,18 +62,18 @@ public class SnapshotOfRegionAssignmentFromMeta {
private final Connection connection;
/** the table name to region map */
private final Map<TableName, List<HRegionInfo>> tableToRegionMap;
private final Map<TableName, List<RegionInfo>> tableToRegionMap;
/** the region to region server map */
//private final Map<HRegionInfo, ServerName> regionToRegionServerMap;
private Map<HRegionInfo, ServerName> regionToRegionServerMap;
//private final Map<RegionInfo, ServerName> regionToRegionServerMap;
private Map<RegionInfo, ServerName> regionToRegionServerMap;
/** the region name to region info map */
private final Map<String, HRegionInfo> regionNameToRegionInfoMap;
private final Map<String, RegionInfo> regionNameToRegionInfoMap;
/** the regionServer to region map */
private final Map<ServerName, List<HRegionInfo>> currentRSToRegionMap;
private final Map<ServerName, List<HRegionInfo>> secondaryRSToRegionMap;
private final Map<ServerName, List<HRegionInfo>> teritiaryRSToRegionMap;
private final Map<ServerName, List<HRegionInfo>> primaryRSToRegionMap;
private final Map<ServerName, List<RegionInfo>> currentRSToRegionMap;
private final Map<ServerName, List<RegionInfo>> secondaryRSToRegionMap;
private final Map<ServerName, List<RegionInfo>> teritiaryRSToRegionMap;
private final Map<ServerName, List<RegionInfo>> primaryRSToRegionMap;
/** the existing assignment plan in the hbase:meta region */
private final FavoredNodesPlan existingAssignmentPlan;
private final Set<TableName> disabledTables;
@ -113,7 +113,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
if (result == null || result.isEmpty()) return true;
RegionLocations rl = MetaTableAccessor.getRegionLocations(result);
if (rl == null) return true;
HRegionInfo hri = rl.getRegionLocation(0).getRegionInfo();
RegionInfo hri = rl.getRegionLocation(0).getRegionInfo();
if (hri == null) return true;
if (hri.getTable() == null) return true;
if (disabledTables.contains(hri.getTable())) {
@ -172,13 +172,13 @@ public class SnapshotOfRegionAssignmentFromMeta {
"snapshot");
}
private void addRegion(HRegionInfo regionInfo) {
private void addRegion(RegionInfo regionInfo) {
// Process the region name to region info map
regionNameToRegionInfoMap.put(regionInfo.getRegionNameAsString(), regionInfo);
// Process the table to region map
TableName tableName = regionInfo.getTable();
List<HRegionInfo> regionList = tableToRegionMap.get(tableName);
List<RegionInfo> regionList = tableToRegionMap.get(tableName);
if (regionList == null) {
regionList = new ArrayList<>();
}
@ -187,14 +187,14 @@ public class SnapshotOfRegionAssignmentFromMeta {
tableToRegionMap.put(tableName, regionList);
}
private void addAssignment(HRegionInfo regionInfo, ServerName server) {
private void addAssignment(RegionInfo regionInfo, ServerName server) {
// Process the region to region server map
regionToRegionServerMap.put(regionInfo, server);
if (server == null) return;
// Process the region server to region map
List<HRegionInfo> regionList = currentRSToRegionMap.get(server);
List<RegionInfo> regionList = currentRSToRegionMap.get(server);
if (regionList == null) {
regionList = new ArrayList<>();
}
@ -202,9 +202,9 @@ public class SnapshotOfRegionAssignmentFromMeta {
currentRSToRegionMap.put(server, regionList);
}
private void addPrimaryAssignment(HRegionInfo regionInfo, ServerName server) {
private void addPrimaryAssignment(RegionInfo regionInfo, ServerName server) {
// Process the region server to region map
List<HRegionInfo> regionList = primaryRSToRegionMap.get(server);
List<RegionInfo> regionList = primaryRSToRegionMap.get(server);
if (regionList == null) {
regionList = new ArrayList<>();
}
@ -212,9 +212,9 @@ public class SnapshotOfRegionAssignmentFromMeta {
primaryRSToRegionMap.put(server, regionList);
}
private void addSecondaryAssignment(HRegionInfo regionInfo, ServerName server) {
private void addSecondaryAssignment(RegionInfo regionInfo, ServerName server) {
// Process the region server to region map
List<HRegionInfo> regionList = secondaryRSToRegionMap.get(server);
List<RegionInfo> regionList = secondaryRSToRegionMap.get(server);
if (regionList == null) {
regionList = new ArrayList<>();
}
@ -222,9 +222,9 @@ public class SnapshotOfRegionAssignmentFromMeta {
secondaryRSToRegionMap.put(server, regionList);
}
private void addTeritiaryAssignment(HRegionInfo regionInfo, ServerName server) {
private void addTeritiaryAssignment(RegionInfo regionInfo, ServerName server) {
// Process the region server to region map
List<HRegionInfo> regionList = teritiaryRSToRegionMap.get(server);
List<RegionInfo> regionList = teritiaryRSToRegionMap.get(server);
if (regionList == null) {
regionList = new ArrayList<>();
}
@ -236,7 +236,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
* Get the regioninfo for a region
* @return the regioninfo
*/
public Map<String, HRegionInfo> getRegionNameToRegionInfoMap() {
public Map<String, RegionInfo> getRegionNameToRegionInfoMap() {
return this.regionNameToRegionInfoMap;
}
@ -244,7 +244,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
* Get regions for tables
* @return a mapping from table to regions
*/
public Map<TableName, List<HRegionInfo>> getTableToRegionMap() {
public Map<TableName, List<RegionInfo>> getTableToRegionMap() {
return tableToRegionMap;
}
@ -252,7 +252,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
* Get region to region server map
* @return region to region server map
*/
public Map<HRegionInfo, ServerName> getRegionToRegionServerMap() {
public Map<RegionInfo, ServerName> getRegionToRegionServerMap() {
return regionToRegionServerMap;
}
@ -260,7 +260,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
* Get regionserver to region map
* @return regionserver to region map
*/
public Map<ServerName, List<HRegionInfo>> getRegionServerToRegionMap() {
public Map<ServerName, List<RegionInfo>> getRegionServerToRegionMap() {
return currentRSToRegionMap;
}
@ -280,15 +280,15 @@ public class SnapshotOfRegionAssignmentFromMeta {
return this.tableToRegionMap.keySet();
}
public Map<ServerName, List<HRegionInfo>> getSecondaryToRegionInfoMap() {
public Map<ServerName, List<RegionInfo>> getSecondaryToRegionInfoMap() {
return this.secondaryRSToRegionMap;
}
public Map<ServerName, List<HRegionInfo>> getTertiaryToRegionInfoMap() {
public Map<ServerName, List<RegionInfo>> getTertiaryToRegionInfoMap() {
return this.teritiaryRSToRegionMap;
}
public Map<ServerName, List<HRegionInfo>> getPrimaryToRegionInfoMap() {
public Map<ServerName, List<RegionInfo>> getPrimaryToRegionInfoMap() {
return this.primaryRSToRegionMap;
}
}

View File

@ -24,8 +24,6 @@ import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.F
import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.IN_PROGRESS;
import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.SUCCESS;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
@ -48,22 +46,24 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SplitLogCounters;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination;
import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLogManagerDetails;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
/**
* Distributes the task of log splitting to the available region servers.
@ -545,7 +545,7 @@ public class SplitLogManager {
getSplitLogManagerCoordination().setRecoveryMode(isForInitialization);
}
public void markRegionsRecovering(ServerName server, Set<HRegionInfo> userRegions)
public void markRegionsRecovering(ServerName server, Set<RegionInfo> userRegions)
throws InterruptedIOException, IOException {
if (userRegions == null || (!isLogReplaying())) {
return;

View File

@ -20,12 +20,12 @@
package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
@ -38,6 +38,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AssignRegionStateData;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
@ -87,17 +89,17 @@ public class AssignProcedure extends RegionTransitionProcedure {
super();
}
public AssignProcedure(final HRegionInfo regionInfo) {
public AssignProcedure(final RegionInfo regionInfo) {
this(regionInfo, false);
}
public AssignProcedure(final HRegionInfo regionInfo, final boolean forceNewPlan) {
public AssignProcedure(final RegionInfo regionInfo, final boolean forceNewPlan) {
super(regionInfo);
this.forceNewPlan = forceNewPlan;
this.targetServer = null;
}
public AssignProcedure(final HRegionInfo regionInfo, final ServerName destinationServer) {
public AssignProcedure(final RegionInfo regionInfo, final ServerName destinationServer) {
super(regionInfo);
this.forceNewPlan = false;
this.targetServer = destinationServer;
@ -123,7 +125,7 @@ public class AssignProcedure extends RegionTransitionProcedure {
throws IOException {
final AssignRegionStateData.Builder state = AssignRegionStateData.newBuilder()
.setTransitionState(getTransitionState())
.setRegionInfo(HRegionInfo.convert(getRegionInfo()));
.setRegionInfo(ProtobufUtil.toRegionInfo(getRegionInfo()));
if (forceNewPlan) {
state.setForceNewPlan(true);
}
@ -138,7 +140,7 @@ public class AssignProcedure extends RegionTransitionProcedure {
throws IOException {
final AssignRegionStateData state = serializer.deserialize(AssignRegionStateData.class);
setTransitionState(state.getTransitionState());
setRegionInfo(HRegionInfo.convert(state.getRegionInfo()));
setRegionInfo(ProtobufUtil.toRegionInfo(state.getRegionInfo()));
forceNewPlan = state.getForceNewPlan();
if (state.hasTargetServer()) {
this.targetServer = ProtobufUtil.toServerName(state.getTargetServer());

View File

@ -40,17 +40,16 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.PleaseHoldException;
import org.apache.hadoop.hbase.RegionException;
import org.apache.hadoop.hbase.RegionStateListener;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.YouAreDeadException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
import org.apache.hadoop.hbase.master.balancer.FavoredStochasticBalancer;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
import org.apache.hadoop.hbase.master.AssignmentListener;
@ -66,7 +65,7 @@ import org.apache.hadoop.hbase.master.TableStateManager;
import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
import org.apache.hadoop.hbase.master.assignment.RegionStates.ServerState;
import org.apache.hadoop.hbase.master.assignment.RegionStates.ServerStateNode;
// TODO: why are they here?
import org.apache.hadoop.hbase.master.balancer.FavoredStochasticBalancer;
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
@ -77,19 +76,21 @@ import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureInMemoryChore;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
// TODO: why are they here?
/**
* The AssignmentManager is the coordinator for region assign/unassign operations.
@ -256,7 +257,7 @@ public class AssignmentManager implements ServerListener {
if (hasProcExecutor) {
getProcedureScheduler().suspendEvent(metaLoadEvent);
setFailoverCleanupDone(false);
for (HRegionInfo hri: getMetaRegionSet()) {
for (RegionInfo hri: getMetaRegionSet()) {
setMetaInitialized(hri, false);
}
}
@ -326,7 +327,7 @@ public class AssignmentManager implements ServerListener {
return regionStateStore;
}
public List<ServerName> getFavoredNodes(final HRegionInfo regionInfo) {
public List<ServerName> getFavoredNodes(final RegionInfo regionInfo) {
return this.shouldAssignRegionsWithFavoredNodes?
((FavoredStochasticBalancer)getBalancer()).getFavoredNodes(regionInfo):
ServerName.EMPTY_SERVER_LIST;
@ -351,7 +352,7 @@ public class AssignmentManager implements ServerListener {
// ============================================================================================
// META Helpers
// ============================================================================================
private boolean isMetaRegion(final HRegionInfo regionInfo) {
private boolean isMetaRegion(final RegionInfo regionInfo) {
return regionInfo.isMetaRegion();
}
@ -359,8 +360,8 @@ public class AssignmentManager implements ServerListener {
return getMetaRegionFromName(regionName) != null;
}
public HRegionInfo getMetaRegionFromName(final byte[] regionName) {
for (HRegionInfo hri: getMetaRegionSet()) {
public RegionInfo getMetaRegionFromName(final byte[] regionName) {
for (RegionInfo hri: getMetaRegionSet()) {
if (Bytes.equals(hri.getRegionName(), regionName)) {
return hri;
}
@ -369,7 +370,7 @@ public class AssignmentManager implements ServerListener {
}
public boolean isCarryingMeta(final ServerName serverName) {
for (HRegionInfo hri: getMetaRegionSet()) {
for (RegionInfo hri: getMetaRegionSet()) {
if (isCarryingRegion(serverName, hri)) {
return true;
}
@ -377,23 +378,23 @@ public class AssignmentManager implements ServerListener {
return false;
}
private boolean isCarryingRegion(final ServerName serverName, final HRegionInfo regionInfo) {
private boolean isCarryingRegion(final ServerName serverName, final RegionInfo regionInfo) {
// TODO: check for state?
final RegionStateNode node = regionStates.getRegionNode(regionInfo);
return(node != null && serverName.equals(node.getRegionLocation()));
}
private HRegionInfo getMetaForRegion(final HRegionInfo regionInfo) {
private RegionInfo getMetaForRegion(final RegionInfo regionInfo) {
//if (regionInfo.isMetaRegion()) return regionInfo;
// TODO: handle multiple meta. if the region provided is not meta lookup
// which meta the region belongs to.
return HRegionInfo.FIRST_META_REGIONINFO;
return RegionInfoBuilder.FIRST_META_REGIONINFO;
}
// TODO: handle multiple meta.
private static final Set<HRegionInfo> META_REGION_SET =
Collections.singleton(HRegionInfo.FIRST_META_REGIONINFO);
public Set<HRegionInfo> getMetaRegionSet() {
private static final Set<RegionInfo> META_REGION_SET =
Collections.singleton(RegionInfoBuilder.FIRST_META_REGIONINFO);
public Set<RegionInfo> getMetaRegionSet() {
return META_REGION_SET;
}
@ -411,15 +412,15 @@ public class AssignmentManager implements ServerListener {
public boolean waitMetaInitialized(final Procedure proc) {
// TODO: handle multiple meta. should this wait on all meta?
// this is used by the ServerCrashProcedure...
return waitMetaInitialized(proc, HRegionInfo.FIRST_META_REGIONINFO);
return waitMetaInitialized(proc, RegionInfoBuilder.FIRST_META_REGIONINFO);
}
public boolean waitMetaInitialized(final Procedure proc, final HRegionInfo regionInfo) {
public boolean waitMetaInitialized(final Procedure proc, final RegionInfo regionInfo) {
return getProcedureScheduler().waitEvent(
getMetaInitializedEvent(getMetaForRegion(regionInfo)), proc);
}
private void setMetaInitialized(final HRegionInfo metaRegionInfo, final boolean isInitialized) {
private void setMetaInitialized(final RegionInfo metaRegionInfo, final boolean isInitialized) {
assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
final ProcedureEvent metaInitEvent = getMetaInitializedEvent(metaRegionInfo);
if (isInitialized) {
@ -429,7 +430,7 @@ public class AssignmentManager implements ServerListener {
}
}
private ProcedureEvent getMetaInitializedEvent(final HRegionInfo metaRegionInfo) {
private ProcedureEvent getMetaInitializedEvent(final RegionInfo metaRegionInfo) {
assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
// TODO: handle multiple meta.
return metaInitializedEvent;
@ -451,11 +452,11 @@ public class AssignmentManager implements ServerListener {
// ============================================================================================
// TODO: Sync helpers
// ============================================================================================
public void assignMeta(final HRegionInfo metaRegionInfo) throws IOException {
public void assignMeta(final RegionInfo metaRegionInfo) throws IOException {
assignMeta(metaRegionInfo, null);
}
public void assignMeta(final HRegionInfo metaRegionInfo, final ServerName serverName)
public void assignMeta(final RegionInfo metaRegionInfo, final ServerName serverName)
throws IOException {
assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
AssignProcedure proc;
@ -490,9 +491,9 @@ public class AssignmentManager implements ServerListener {
// handling.
continue;
}
List<HRegionInfo> regionsShouldMove = getCarryingSystemTables(server);
List<RegionInfo> regionsShouldMove = getCarryingSystemTables(server);
if (!regionsShouldMove.isEmpty()) {
for (HRegionInfo regionInfo : regionsShouldMove) {
for (RegionInfo regionInfo : regionsShouldMove) {
// null value for dest forces destination server to be selected by balancer
RegionPlan plan = new RegionPlan(regionInfo, server, null);
if (regionInfo.isMetaRegion()) {
@ -514,31 +515,31 @@ public class AssignmentManager implements ServerListener {
}).start();
}
private List<HRegionInfo> getCarryingSystemTables(ServerName serverName) {
private List<RegionInfo> getCarryingSystemTables(ServerName serverName) {
Set<RegionStateNode> regions = this.getRegionStates().getServerNode(serverName).getRegions();
if (regions == null) {
return new ArrayList<>();
}
return regions.stream()
.map(RegionStateNode::getRegionInfo)
.filter(HRegionInfo::isSystemTable)
.filter(RegionInfo::isSystemTable)
.collect(Collectors.toList());
}
public void assign(final HRegionInfo regionInfo) throws IOException {
public void assign(final RegionInfo regionInfo) throws IOException {
assign(regionInfo, true);
}
public void assign(final HRegionInfo regionInfo, final boolean forceNewPlan) throws IOException {
public void assign(final RegionInfo regionInfo, final boolean forceNewPlan) throws IOException {
AssignProcedure proc = createAssignProcedure(regionInfo, forceNewPlan);
ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc);
}
public void unassign(final HRegionInfo regionInfo) throws IOException {
public void unassign(final RegionInfo regionInfo) throws IOException {
unassign(regionInfo, false);
}
public void unassign(final HRegionInfo regionInfo, final boolean forceNewPlan)
public void unassign(final RegionInfo regionInfo, final boolean forceNewPlan)
throws IOException {
// TODO: rename this reassign
RegionStateNode node = this.regionStates.getRegionNode(regionInfo);
@ -557,13 +558,13 @@ public class AssignmentManager implements ServerListener {
}
@VisibleForTesting
public boolean waitForAssignment(final HRegionInfo regionInfo) throws IOException {
public boolean waitForAssignment(final RegionInfo regionInfo) throws IOException {
return waitForAssignment(regionInfo, Long.MAX_VALUE);
}
@VisibleForTesting
// TODO: Remove this?
public boolean waitForAssignment(final HRegionInfo regionInfo, final long timeout)
public boolean waitForAssignment(final RegionInfo regionInfo, final long timeout)
throws IOException {
RegionStateNode node = null;
// This method can be called before the regionInfo has made it into the regionStateMap
@ -597,16 +598,16 @@ public class AssignmentManager implements ServerListener {
// RegionTransition procedures helpers
// ============================================================================================
public AssignProcedure[] createAssignProcedures(final Collection<HRegionInfo> regionInfo) {
public AssignProcedure[] createAssignProcedures(final Collection<RegionInfo> regionInfo) {
return createAssignProcedures(regionInfo, false);
}
public AssignProcedure[] createAssignProcedures(final Collection<HRegionInfo> regionInfo,
public AssignProcedure[] createAssignProcedures(final Collection<RegionInfo> regionInfo,
final boolean forceNewPlan) {
if (regionInfo.isEmpty()) return null;
final AssignProcedure[] procs = new AssignProcedure[regionInfo.size()];
int index = 0;
for (HRegionInfo hri: regionInfo) {
for (RegionInfo hri: regionInfo) {
procs[index++] = createAssignProcedure(hri, forceNewPlan);
}
return procs;
@ -629,10 +630,10 @@ public class AssignmentManager implements ServerListener {
return procs.toArray(UNASSIGNED_PROCEDURE_FOR_TYPE_INFO);
}
public MoveRegionProcedure[] createReopenProcedures(final Collection<HRegionInfo> regionInfo) {
public MoveRegionProcedure[] createReopenProcedures(final Collection<RegionInfo> regionInfo) {
final MoveRegionProcedure[] procs = new MoveRegionProcedure[regionInfo.size()];
int index = 0;
for (HRegionInfo hri: regionInfo) {
for (RegionInfo hri: regionInfo) {
final ServerName serverName = regionStates.getRegionServerOfRegion(hri);
final RegionPlan plan = new RegionPlan(hri, serverName, serverName);
procs[index++] = createMoveRegionProcedure(plan);
@ -664,21 +665,21 @@ public class AssignmentManager implements ServerListener {
return createReopenProcedures(regionStates.getRegionsOfTable(tableName));
}
public AssignProcedure createAssignProcedure(final HRegionInfo regionInfo,
public AssignProcedure createAssignProcedure(final RegionInfo regionInfo,
final boolean forceNewPlan) {
AssignProcedure proc = new AssignProcedure(regionInfo, forceNewPlan);
proc.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
return proc;
}
public AssignProcedure createAssignProcedure(final HRegionInfo regionInfo,
public AssignProcedure createAssignProcedure(final RegionInfo regionInfo,
final ServerName targetServer) {
AssignProcedure proc = new AssignProcedure(regionInfo, targetServer);
proc.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
return proc;
}
public UnassignProcedure createUnassignProcedure(final HRegionInfo regionInfo,
public UnassignProcedure createUnassignProcedure(final RegionInfo regionInfo,
final ServerName destinationServer, final boolean force) {
// If destinationServer is null, figure it.
ServerName sn = destinationServer != null? destinationServer:
@ -707,13 +708,13 @@ public class AssignmentManager implements ServerListener {
}
public SplitTableRegionProcedure createSplitProcedure(final HRegionInfo regionToSplit,
public SplitTableRegionProcedure createSplitProcedure(final RegionInfo regionToSplit,
final byte[] splitKey) throws IOException {
return new SplitTableRegionProcedure(getProcedureEnvironment(), regionToSplit, splitKey);
}
public MergeTableRegionsProcedure createMergeProcedure(final HRegionInfo regionToMergeA,
final HRegionInfo regionToMergeB) throws IOException {
public MergeTableRegionsProcedure createMergeProcedure(final RegionInfo regionToMergeA,
final RegionInfo regionToMergeB) throws IOException {
return new MergeTableRegionsProcedure(getProcedureEnvironment(), regionToMergeA,regionToMergeB);
}
@ -721,10 +722,10 @@ public class AssignmentManager implements ServerListener {
* Delete the region states. This is called by "DeleteTable"
*/
public void deleteTable(final TableName tableName) throws IOException {
final ArrayList<HRegionInfo> regions = regionStates.getTableRegionsInfo(tableName);
final ArrayList<RegionInfo> regions = regionStates.getTableRegionsInfo(tableName);
regionStateStore.deleteRegions(regions);
for (int i = 0; i < regions.size(); ++i) {
final HRegionInfo regionInfo = regions.get(i);
final RegionInfo regionInfo = regions.get(i);
// we expect the region to be offline
regionStates.removeFromOfflineRegions(regionInfo);
regionStates.deleteRegion(regionInfo);
@ -748,7 +749,7 @@ public class AssignmentManager implements ServerListener {
case FAILED_OPEN:
case CLOSED:
assert transition.getRegionInfoCount() == 1 : transition;
final HRegionInfo hri = HRegionInfo.convert(transition.getRegionInfo(0));
final RegionInfo hri = ProtobufUtil.toRegionInfo(transition.getRegionInfo(0));
updateRegionTransition(serverName, transition.getTransitionCode(), hri,
transition.hasOpenSeqNum() ? transition.getOpenSeqNum() : HConstants.NO_SEQNUM);
break;
@ -757,9 +758,9 @@ public class AssignmentManager implements ServerListener {
case SPLIT:
case SPLIT_REVERTED:
assert transition.getRegionInfoCount() == 3 : transition;
final HRegionInfo parent = HRegionInfo.convert(transition.getRegionInfo(0));
final HRegionInfo splitA = HRegionInfo.convert(transition.getRegionInfo(1));
final HRegionInfo splitB = HRegionInfo.convert(transition.getRegionInfo(2));
final RegionInfo parent = ProtobufUtil.toRegionInfo(transition.getRegionInfo(0));
final RegionInfo splitA = ProtobufUtil.toRegionInfo(transition.getRegionInfo(1));
final RegionInfo splitB = ProtobufUtil.toRegionInfo(transition.getRegionInfo(2));
updateRegionSplitTransition(serverName, transition.getTransitionCode(),
parent, splitA, splitB);
break;
@ -768,9 +769,9 @@ public class AssignmentManager implements ServerListener {
case MERGED:
case MERGE_REVERTED:
assert transition.getRegionInfoCount() == 3 : transition;
final HRegionInfo merged = HRegionInfo.convert(transition.getRegionInfo(0));
final HRegionInfo mergeA = HRegionInfo.convert(transition.getRegionInfo(1));
final HRegionInfo mergeB = HRegionInfo.convert(transition.getRegionInfo(2));
final RegionInfo merged = ProtobufUtil.toRegionInfo(transition.getRegionInfo(0));
final RegionInfo mergeA = ProtobufUtil.toRegionInfo(transition.getRegionInfo(1));
final RegionInfo mergeB = ProtobufUtil.toRegionInfo(transition.getRegionInfo(2));
updateRegionMergeTransition(serverName, transition.getTransitionCode(),
merged, mergeA, mergeB);
break;
@ -789,7 +790,7 @@ public class AssignmentManager implements ServerListener {
}
private void updateRegionTransition(final ServerName serverName, final TransitionCode state,
final HRegionInfo regionInfo, final long seqId)
final RegionInfo regionInfo, final long seqId)
throws PleaseHoldException, UnexpectedStateException {
checkFailoverCleanupCompleted(regionInfo);
@ -830,7 +831,7 @@ public class AssignmentManager implements ServerListener {
}
private void updateRegionSplitTransition(final ServerName serverName, final TransitionCode state,
final HRegionInfo parent, final HRegionInfo hriA, final HRegionInfo hriB)
final RegionInfo parent, final RegionInfo hriA, final RegionInfo hriB)
throws IOException {
checkFailoverCleanupCompleted(parent);
@ -863,7 +864,7 @@ public class AssignmentManager implements ServerListener {
}
private void updateRegionMergeTransition(final ServerName serverName, final TransitionCode state,
final HRegionInfo merged, final HRegionInfo hriA, final HRegionInfo hriB) throws IOException {
final RegionInfo merged, final RegionInfo hriA, final RegionInfo hriB) throws IOException {
checkFailoverCleanupCompleted(merged);
if (state != TransitionCode.READY_TO_MERGE) {
@ -935,7 +936,7 @@ public class AssignmentManager implements ServerListener {
final Set<byte[]> regionNames) {
try {
for (byte[] regionName: regionNames) {
final HRegionInfo hri = getMetaRegionFromName(regionName);
final RegionInfo hri = getMetaRegionFromName(regionName);
if (hri == null) {
if (LOG.isTraceEnabled()) {
LOG.trace("Skip online report for region=" + Bytes.toStringBinary(regionName) +
@ -1094,12 +1095,12 @@ public class AssignmentManager implements ServerListener {
return m != null? m.values(): Collections.EMPTY_SET;
}
public boolean isRegionOverThreshold(final HRegionInfo regionInfo) {
public boolean isRegionOverThreshold(final RegionInfo regionInfo) {
Map<String, RegionState> m = this.ritsOverThreshold;
return m != null && m.containsKey(regionInfo.getEncodedName());
}
public boolean isRegionTwiceOverThreshold(final HRegionInfo regionInfo) {
public boolean isRegionTwiceOverThreshold(final RegionInfo regionInfo) {
Map<String, RegionState> m = this.ritsOverThreshold;
if (m == null) return false;
final RegionState state = m.get(regionInfo.getEncodedName());
@ -1138,7 +1139,7 @@ public class AssignmentManager implements ServerListener {
metrics.updateRITCountOverThreshold(ritStat.getTotalRITsOverThreshold());
}
private void handleRegionOverStuckWarningThreshold(final HRegionInfo regionInfo) {
private void handleRegionOverStuckWarningThreshold(final RegionInfo regionInfo) {
final RegionStateNode regionNode = regionStates.getRegionNode(regionInfo);
//if (regionNode.isStuck()) {
LOG.warn("TODO Handle stuck in transition: " + regionNode);
@ -1176,7 +1177,7 @@ public class AssignmentManager implements ServerListener {
// TODO: use a thread pool
regionStateStore.visitMeta(new RegionStateStore.RegionStateVisitor() {
@Override
public void visitRegionState(final HRegionInfo regionInfo, final State state,
public void visitRegionState(final RegionInfo regionInfo, final State state,
final ServerName regionLocation, final ServerName lastHost, final long openSeqNum) {
final RegionStateNode regionNode = regionStates.getOrCreateRegionNode(regionInfo);
synchronized (regionNode) {
@ -1211,7 +1212,7 @@ public class AssignmentManager implements ServerListener {
boolean failover = !master.getServerManager().getDeadServers().isEmpty();
final Set<ServerName> offlineServersWithOnlineRegions = new HashSet<ServerName>();
final ArrayList<HRegionInfo> regionsToAssign = new ArrayList<HRegionInfo>();
final ArrayList<RegionInfo> regionsToAssign = new ArrayList<RegionInfo>();
long st, et;
st = System.currentTimeMillis();
@ -1246,7 +1247,7 @@ public class AssignmentManager implements ServerListener {
// assign offline regions
st = System.currentTimeMillis();
for (HRegionInfo regionInfo: getOrderedRegions(regionsToAssign)) {
for (RegionInfo regionInfo: getOrderedRegions(regionsToAssign)) {
master.getMasterProcedureExecutor().submitProcedure(
createAssignProcedure(regionInfo, false));
}
@ -1285,7 +1286,7 @@ public class AssignmentManager implements ServerListener {
* @param hri region to check if it is already rebuild
* @throws PleaseHoldException if the failover cleanup is not completed
*/
private void checkFailoverCleanupCompleted(final HRegionInfo hri) throws PleaseHoldException {
private void checkFailoverCleanupCompleted(final RegionInfo hri) throws PleaseHoldException {
if (!isRunning()) {
throw new PleaseHoldException("AssignmentManager not running");
}
@ -1317,18 +1318,18 @@ public class AssignmentManager implements ServerListener {
" to dead servers, submitted shutdown handler to be executed meta=" + carryingMeta);
}
public void offlineRegion(final HRegionInfo regionInfo) {
public void offlineRegion(final RegionInfo regionInfo) {
// TODO used by MasterRpcServices ServerCrashProcedure
final RegionStateNode node = regionStates.getRegionNode(regionInfo);
if (node != null) node.offline();
}
public void onlineRegion(final HRegionInfo regionInfo, final ServerName serverName) {
public void onlineRegion(final RegionInfo regionInfo, final ServerName serverName) {
// TODO used by TestSplitTransactionOnCluster.java
}
public Map<ServerName, List<HRegionInfo>> getSnapShotOfAssignment(
final Collection<HRegionInfo> regions) {
public Map<ServerName, List<RegionInfo>> getSnapShotOfAssignment(
final Collection<RegionInfo> regions) {
return regionStates.getSnapShotOfAssignment(regions);
}
@ -1359,13 +1360,13 @@ public class AssignmentManager implements ServerListener {
* @param regions
* @return A list of regions with system regions at front
*/
public List<HRegionInfo> getOrderedRegions(
final List<HRegionInfo> regions) {
public List<RegionInfo> getOrderedRegions(
final List<RegionInfo> regions) {
if (regions == null) return Collections.emptyList();
List<HRegionInfo> systemList = new ArrayList<>();
List<HRegionInfo> userList = new ArrayList<>();
for (HRegionInfo hri : regions) {
List<RegionInfo> systemList = new ArrayList<>();
List<RegionInfo> userList = new ArrayList<>();
for (RegionInfo hri : regions) {
if (hri.isSystemTable()) systemList.add(hri);
else userList.add(hri);
}
@ -1395,11 +1396,11 @@ public class AssignmentManager implements ServerListener {
return regionStates.getRegionsInTransition();
}
public List<HRegionInfo> getAssignedRegions() {
public List<RegionInfo> getAssignedRegions() {
return regionStates.getAssignedRegions();
}
public HRegionInfo getRegionInfo(final byte[] regionName) {
public RegionInfo getRegionInfo(final byte[] regionName) {
final RegionStateNode regionState = regionStates.getRegionNodeFromName(regionName);
return regionState != null ? regionState.getRegionInfo() : null;
}
@ -1407,7 +1408,7 @@ public class AssignmentManager implements ServerListener {
// ============================================================================================
// TODO: Region Status update
// ============================================================================================
private void sendRegionOpenedNotification(final HRegionInfo regionInfo,
private void sendRegionOpenedNotification(final RegionInfo regionInfo,
final ServerName serverName) {
getBalancer().regionOnline(regionInfo, serverName);
if (!this.listeners.isEmpty()) {
@ -1417,7 +1418,7 @@ public class AssignmentManager implements ServerListener {
}
}
private void sendRegionClosedNotification(final HRegionInfo regionInfo) {
private void sendRegionClosedNotification(final RegionInfo regionInfo) {
getBalancer().regionOffline(regionInfo);
if (!this.listeners.isEmpty()) {
for (AssignmentListener listener : this.listeners) {
@ -1454,7 +1455,7 @@ public class AssignmentManager implements ServerListener {
}
public void markRegionAsOpened(final RegionStateNode regionNode) throws IOException {
final HRegionInfo hri = regionNode.getRegionInfo();
final RegionInfo hri = regionNode.getRegionInfo();
synchronized (regionNode) {
State state = regionNode.transitionState(State.OPEN, RegionStates.STATES_EXPECTED_ON_OPEN);
if (isMetaRegion(hri)) {
@ -1473,7 +1474,7 @@ public class AssignmentManager implements ServerListener {
}
public void markRegionAsClosing(final RegionStateNode regionNode) throws IOException {
final HRegionInfo hri = regionNode.getRegionInfo();
final RegionInfo hri = regionNode.getRegionInfo();
synchronized (regionNode) {
State state = regionNode.transitionState(State.CLOSING, RegionStates.STATES_EXPECTED_ON_CLOSE);
// Set meta has not initialized early. so people trying to create/edit tables will wait
@ -1496,7 +1497,7 @@ public class AssignmentManager implements ServerListener {
}
public void markRegionAsClosed(final RegionStateNode regionNode) throws IOException {
final HRegionInfo hri = regionNode.getRegionInfo();
final RegionInfo hri = regionNode.getRegionInfo();
synchronized (regionNode) {
State state = regionNode.transitionState(State.CLOSED, RegionStates.STATES_EXPECTED_ON_CLOSE);
regionStates.removeRegionFromServer(regionNode.getRegionLocation(), regionNode);
@ -1509,8 +1510,8 @@ public class AssignmentManager implements ServerListener {
}
}
public void markRegionAsSplit(final HRegionInfo parent, final ServerName serverName,
final HRegionInfo daughterA, final HRegionInfo daughterB)
public void markRegionAsSplit(final RegionInfo parent, final ServerName serverName,
final RegionInfo daughterA, final RegionInfo daughterB)
throws IOException {
// Update hbase:meta. Parent will be marked offline and split up in hbase:meta.
// The parent stays in regionStates until cleared when removed by CatalogJanitor.
@ -1535,8 +1536,8 @@ public class AssignmentManager implements ServerListener {
* by the catalog janitor running against hbase:meta. It notices when the merged region no
* longer holds references to the old regions.
*/
public void markRegionAsMerged(final HRegionInfo child, final ServerName serverName,
final HRegionInfo mother, final HRegionInfo father) throws IOException {
public void markRegionAsMerged(final RegionInfo child, final ServerName serverName,
final RegionInfo mother, final RegionInfo father) throws IOException {
final RegionStateNode node = regionStates.getOrCreateRegionNode(child);
node.setState(State.MERGED);
regionStates.deleteRegion(mother);
@ -1552,7 +1553,7 @@ public class AssignmentManager implements ServerListener {
* Favored nodes should be applied only when FavoredNodes balancer is configured and the region
* belongs to a non-system table.
*/
private boolean shouldAssignFavoredNodes(HRegionInfo region) {
private boolean shouldAssignFavoredNodes(RegionInfo region) {
return this.shouldAssignRegionsWithFavoredNodes &&
FavoredNodesManager.isFavoredNodeApplicable(region);
}
@ -1622,8 +1623,8 @@ public class AssignmentManager implements ServerListener {
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings("WA_AWAIT_NOT_IN_LOOP")
private HashMap<HRegionInfo, RegionStateNode> waitOnAssignQueue() {
HashMap<HRegionInfo, RegionStateNode> regions = null;
private HashMap<RegionInfo, RegionStateNode> waitOnAssignQueue() {
HashMap<RegionInfo, RegionStateNode> regions = null;
assignQueueLock.lock();
try {
@ -1633,7 +1634,7 @@ public class AssignmentManager implements ServerListener {
if (!isRunning()) return null;
assignQueueFullCond.await(assignDispatchWaitMillis, TimeUnit.MILLISECONDS);
regions = new HashMap<HRegionInfo, RegionStateNode>(pendingAssignQueue.size());
regions = new HashMap<RegionInfo, RegionStateNode>(pendingAssignQueue.size());
for (RegionStateNode regionNode: pendingAssignQueue) {
regions.put(regionNode.getRegionInfo(), regionNode);
}
@ -1648,7 +1649,7 @@ public class AssignmentManager implements ServerListener {
}
private void processAssignQueue() {
final HashMap<HRegionInfo, RegionStateNode> regions = waitOnAssignQueue();
final HashMap<RegionInfo, RegionStateNode> regions = waitOnAssignQueue();
if (regions == null || regions.size() == 0 || !isRunning()) {
return;
}
@ -1658,13 +1659,13 @@ public class AssignmentManager implements ServerListener {
}
// TODO: Optimize balancer. pass a RegionPlan?
final HashMap<HRegionInfo, ServerName> retainMap = new HashMap<>();
final List<HRegionInfo> userRRList = new ArrayList<>();
final HashMap<RegionInfo, ServerName> retainMap = new HashMap<>();
final List<RegionInfo> userRRList = new ArrayList<>();
// regions for system tables requiring reassignment
final List<HRegionInfo> sysRRList = new ArrayList<>();
final List<RegionInfo> sysRRList = new ArrayList<>();
for (RegionStateNode regionNode : regions.values()) {
boolean sysTable = regionNode.isSystemTable();
final List<HRegionInfo> rrList = sysTable ? sysRRList : userRRList;
final List<RegionInfo> rrList = sysTable ? sysRRList : userRRList;
if (regionNode.getRegionLocation() != null) {
retainMap.put(regionNode.getRegionInfo(), regionNode.getRegionLocation());
@ -1711,8 +1712,8 @@ public class AssignmentManager implements ServerListener {
processAssignmentPlans(regions, retainMap, userRRList, servers);
}
private void processAssignmentPlans(final HashMap<HRegionInfo, RegionStateNode> regions,
final HashMap<HRegionInfo, ServerName> retainMap, final List<HRegionInfo> rrList,
private void processAssignmentPlans(final HashMap<RegionInfo, RegionStateNode> regions,
final HashMap<RegionInfo, ServerName> retainMap, final List<RegionInfo> rrList,
final List<ServerName> servers) {
boolean isTraceEnabled = LOG.isTraceEnabled();
if (isTraceEnabled) {
@ -1736,7 +1737,7 @@ public class AssignmentManager implements ServerListener {
// TODO: Do we need to split retain and round-robin?
// the retain seems to fallback to round-robin/random if the region is not in the map.
if (!rrList.isEmpty()) {
Collections.sort(rrList);
Collections.sort(rrList, RegionInfo.COMPARATOR);
if (isTraceEnabled) {
LOG.trace("round robin regions=" + rrList);
}
@ -1749,8 +1750,8 @@ public class AssignmentManager implements ServerListener {
}
}
private void acceptPlan(final HashMap<HRegionInfo, RegionStateNode> regions,
final Map<ServerName, List<HRegionInfo>> plan) throws HBaseIOException {
private void acceptPlan(final HashMap<RegionInfo, RegionStateNode> regions,
final Map<ServerName, List<RegionInfo>> plan) throws HBaseIOException {
final ProcedureEvent[] events = new ProcedureEvent[regions.size()];
final long st = System.currentTimeMillis();
@ -1761,9 +1762,9 @@ public class AssignmentManager implements ServerListener {
if (plan.isEmpty()) return;
int evcount = 0;
for (Map.Entry<ServerName, List<HRegionInfo>> entry: plan.entrySet()) {
for (Map.Entry<ServerName, List<RegionInfo>> entry: plan.entrySet()) {
final ServerName server = entry.getKey();
for (HRegionInfo hri: entry.getValue()) {
for (RegionInfo hri: entry.getValue()) {
final RegionStateNode regionNode = regions.get(hri);
regionNode.setRegionLocation(server);
events[evcount++] = regionNode.getProcedureEvent();
@ -1778,11 +1779,11 @@ public class AssignmentManager implements ServerListener {
}
}
private void addToPendingAssignment(final HashMap<HRegionInfo, RegionStateNode> regions,
final Collection<HRegionInfo> pendingRegions) {
private void addToPendingAssignment(final HashMap<RegionInfo, RegionStateNode> regions,
final Collection<RegionInfo> pendingRegions) {
assignQueueLock.lock();
try {
for (HRegionInfo hri: pendingRegions) {
for (RegionInfo hri: pendingRegions) {
pendingAssignQueue.add(regions.get(hri));
}
} finally {

View File

@ -18,17 +18,20 @@
package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsState;
@ -43,14 +46,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.G
public class GCMergedRegionsProcedure
extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
private static final Log LOG = LogFactory.getLog(GCMergedRegionsProcedure.class);
private HRegionInfo father;
private HRegionInfo mother;
private HRegionInfo mergedChild;
private RegionInfo father;
private RegionInfo mother;
private RegionInfo mergedChild;
public GCMergedRegionsProcedure(final MasterProcedureEnv env,
final HRegionInfo mergedChild,
final HRegionInfo father,
final HRegionInfo mother) {
final RegionInfo mergedChild,
final RegionInfo father,
final RegionInfo mother) {
super(env);
this.father = father;
this.mother = mother;
@ -100,7 +103,7 @@ extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
private GCRegionProcedure[] createGCRegionProcedures(final MasterProcedureEnv env) {
GCRegionProcedure [] procs = new GCRegionProcedure[2];
int index = 0;
for (HRegionInfo hri: new HRegionInfo [] {this.father, this.mother}) {
for (RegionInfo hri: new RegionInfo [] {this.father, this.mother}) {
GCRegionProcedure proc = new GCRegionProcedure(env, hri);
proc.setOwner(env.getRequestUser().getShortName());
procs[index++] = proc;
@ -135,9 +138,9 @@ extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
super.serializeStateData(serializer);
final MasterProcedureProtos.GCMergedRegionsStateData.Builder msg =
MasterProcedureProtos.GCMergedRegionsStateData.newBuilder().
setParentA(HRegionInfo.convert(this.father)).
setParentB(HRegionInfo.convert(this.mother)).
setMergedChild(HRegionInfo.convert(this.mergedChild));
setParentA(ProtobufUtil.toRegionInfo(this.father)).
setParentB(ProtobufUtil.toRegionInfo(this.mother)).
setMergedChild(ProtobufUtil.toRegionInfo(this.mergedChild));
serializer.serialize(msg.build());
}
@ -147,9 +150,9 @@ extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
super.deserializeStateData(serializer);
final MasterProcedureProtos.GCMergedRegionsStateData msg =
serializer.deserialize(MasterProcedureProtos.GCMergedRegionsStateData.class);
this.father = HRegionInfo.convert(msg.getParentA());
this.mother = HRegionInfo.convert(msg.getParentB());
this.mergedChild = HRegionInfo.convert(msg.getMergedChild());
this.father = ProtobufUtil.toRegionInfo(msg.getParentA());
this.mother = ProtobufUtil.toRegionInfo(msg.getParentB());
this.mergedChild = ProtobufUtil.toRegionInfo(msg.getMergedChild());
}
@Override

View File

@ -18,13 +18,13 @@
package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
@ -32,10 +32,12 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionState;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionState;
/**
* GC a Region that is no longer in use. It has been split or merged away.
@ -47,7 +49,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
public class GCRegionProcedure extends AbstractStateMachineRegionProcedure<GCRegionState> {
private static final Log LOG = LogFactory.getLog(GCRegionProcedure.class);
public GCRegionProcedure(final MasterProcedureEnv env, final HRegionInfo hri) {
public GCRegionProcedure(final MasterProcedureEnv env, final RegionInfo hri) {
super(env, hri);
}
@ -135,7 +137,7 @@ public class GCRegionProcedure extends AbstractStateMachineRegionProcedure<GCReg
// Double serialization of regionname. Superclass is also serializing. Fix.
final MasterProcedureProtos.GCRegionStateData.Builder msg =
MasterProcedureProtos.GCRegionStateData.newBuilder()
.setRegionInfo(HRegionInfo.convert(getRegion()));
.setRegionInfo(ProtobufUtil.toRegionInfo(getRegion()));
serializer.serialize(msg.build());
}
@ -145,7 +147,7 @@ public class GCRegionProcedure extends AbstractStateMachineRegionProcedure<GCReg
super.deserializeStateData(serializer);
final MasterProcedureProtos.GCRegionStateData msg =
serializer.deserialize(MasterProcedureProtos.GCRegionStateData.class);
setRegion(HRegionInfo.convert(msg.getRegionInfo()));
setRegion(ProtobufUtil.toRegionInfo(msg.getRegionInfo()));
}
@Override

View File

@ -30,15 +30,15 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaMutationAnnotation;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.exceptions.MergeRegionException;
@ -59,13 +59,16 @@ import org.apache.hadoop.hbase.quotas.QuotaExceededException;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState;
/**
* The procedure to Merge a region in a table.
@ -79,8 +82,8 @@ public class MergeTableRegionsProcedure
private Boolean traceEnabled;
private volatile boolean lock = false;
private ServerName regionLocation;
private HRegionInfo[] regionsToMerge;
private HRegionInfo mergedRegion;
private RegionInfo[] regionsToMerge;
private RegionInfo mergedRegion;
private boolean forcible;
public MergeTableRegionsProcedure() {
@ -88,18 +91,18 @@ public class MergeTableRegionsProcedure
}
public MergeTableRegionsProcedure(final MasterProcedureEnv env,
final HRegionInfo regionToMergeA, final HRegionInfo regionToMergeB) throws IOException {
final RegionInfo regionToMergeA, final RegionInfo regionToMergeB) throws IOException {
this(env, regionToMergeA, regionToMergeB, false);
}
public MergeTableRegionsProcedure(final MasterProcedureEnv env,
final HRegionInfo regionToMergeA, final HRegionInfo regionToMergeB,
final RegionInfo regionToMergeA, final RegionInfo regionToMergeB,
final boolean forcible) throws MergeRegionException {
this(env, new HRegionInfo[] {regionToMergeA, regionToMergeB}, forcible);
this(env, new RegionInfo[] {regionToMergeA, regionToMergeB}, forcible);
}
public MergeTableRegionsProcedure(final MasterProcedureEnv env,
final HRegionInfo[] regionsToMerge, final boolean forcible)
final RegionInfo[] regionsToMerge, final boolean forcible)
throws MergeRegionException {
super(env);
@ -117,7 +120,7 @@ public class MergeTableRegionsProcedure
this.forcible = forcible;
}
private static void checkRegionsToMerge(final HRegionInfo[] regionsToMerge,
private static void checkRegionsToMerge(final RegionInfo[] regionsToMerge,
final boolean forcible) throws MergeRegionException {
// For now, we only merge 2 regions.
// It could be extended to more than 2 regions in the future.
@ -129,19 +132,19 @@ public class MergeTableRegionsProcedure
checkRegionsToMerge(regionsToMerge[0], regionsToMerge[1], forcible);
}
private static void checkRegionsToMerge(final HRegionInfo regionToMergeA,
final HRegionInfo regionToMergeB, final boolean forcible) throws MergeRegionException {
private static void checkRegionsToMerge(final RegionInfo regionToMergeA,
final RegionInfo regionToMergeB, final boolean forcible) throws MergeRegionException {
if (!regionToMergeA.getTable().equals(regionToMergeB.getTable())) {
throw new MergeRegionException("Can't merge regions from two different tables: " +
regionToMergeA + ", " + regionToMergeB);
}
if (regionToMergeA.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID ||
regionToMergeB.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
if (regionToMergeA.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID ||
regionToMergeB.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
throw new MergeRegionException("Can't merge non-default replicas");
}
if (!HRegionInfo.areAdjacent(regionToMergeA, regionToMergeB)) {
if (!RegionInfo.areAdjacent(regionToMergeA, regionToMergeB)) {
String msg = "Unable to merge not adjacent regions " + regionToMergeA.getShortNameToLog() +
", " + regionToMergeB.getShortNameToLog() + " where forcible = " + forcible;
LOG.warn(msg);
@ -151,18 +154,18 @@ public class MergeTableRegionsProcedure
}
}
private static HRegionInfo createMergedRegionInfo(final HRegionInfo[] regionsToMerge) {
private static RegionInfo createMergedRegionInfo(final RegionInfo[] regionsToMerge) {
return createMergedRegionInfo(regionsToMerge[0], regionsToMerge[1]);
}
/**
* Create merged region info through the specified two regions
*/
private static HRegionInfo createMergedRegionInfo(final HRegionInfo regionToMergeA,
final HRegionInfo regionToMergeB) {
private static RegionInfo createMergedRegionInfo(final RegionInfo regionToMergeA,
final RegionInfo regionToMergeB) {
// Choose the smaller as start key
final byte[] startKey;
if (regionToMergeA.compareTo(regionToMergeB) <= 0) {
if (RegionInfo.COMPARATOR.compare(regionToMergeA, regionToMergeB) <= 0) {
startKey = regionToMergeA.getStartKey();
} else {
startKey = regionToMergeB.getStartKey();
@ -179,12 +182,16 @@ public class MergeTableRegionsProcedure
}
// Merged region is sorted between two merging regions in META
final long rid = getMergedRegionIdTimestamp(regionToMergeA, regionToMergeB);
return new HRegionInfo(regionToMergeA.getTable(), startKey, endKey, false, rid);
return RegionInfoBuilder.newBuilder(regionToMergeA.getTable())
.setStartKey(startKey)
.setEndKey(endKey)
.setSplit(false)
.setRegionId(getMergedRegionIdTimestamp(regionToMergeA, regionToMergeB))
.build();
}
private static long getMergedRegionIdTimestamp(final HRegionInfo regionToMergeA,
final HRegionInfo regionToMergeB) {
private static long getMergedRegionIdTimestamp(final RegionInfo regionToMergeA,
final RegionInfo regionToMergeB) {
long rid = EnvironmentEdgeManager.currentTime();
// Regionid is timestamp. Merged region's id can't be less than that of
// merging regions else will insert at wrong location in hbase:meta (See HBASE-710).
@ -252,7 +259,7 @@ public class MergeTableRegionsProcedure
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (IOException e) {
LOG.warn("Error trying to merge regions " + HRegionInfo.getShortNameToLog(regionsToMerge) +
LOG.warn("Error trying to merge regions " + RegionInfo.getShortNameToLog(regionsToMerge) +
" in the table " + getTableName() + " (in state=" + state + ")", e);
setFailure("master-merge-regions", e);
@ -305,7 +312,7 @@ public class MergeTableRegionsProcedure
// This will be retried. Unless there is a bug in the code,
// this should be just a "temporary error" (e.g. network down)
LOG.warn("Failed rollback attempt step " + state + " for merging the regions "
+ HRegionInfo.getShortNameToLog(regionsToMerge) + " in table " + getTableName(), e);
+ RegionInfo.getShortNameToLog(regionsToMerge) + " in table " + getTableName(), e);
throw e;
}
}
@ -351,10 +358,10 @@ public class MergeTableRegionsProcedure
final MasterProcedureProtos.MergeTableRegionsStateData.Builder mergeTableRegionsMsg =
MasterProcedureProtos.MergeTableRegionsStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
.setMergedRegionInfo(HRegionInfo.convert(mergedRegion))
.setMergedRegionInfo(ProtobufUtil.toRegionInfo(mergedRegion))
.setForcible(forcible);
for (int i = 0; i < regionsToMerge.length; ++i) {
mergeTableRegionsMsg.addRegionInfo(HRegionInfo.convert(regionsToMerge[i]));
mergeTableRegionsMsg.addRegionInfo(ProtobufUtil.toRegionInfo(regionsToMerge[i]));
}
serializer.serialize(mergeTableRegionsMsg.build());
}
@ -369,12 +376,12 @@ public class MergeTableRegionsProcedure
setUser(MasterProcedureUtil.toUserInfo(mergeTableRegionsMsg.getUserInfo()));
assert(mergeTableRegionsMsg.getRegionInfoCount() == 2);
regionsToMerge = new HRegionInfo[mergeTableRegionsMsg.getRegionInfoCount()];
regionsToMerge = new RegionInfo[mergeTableRegionsMsg.getRegionInfoCount()];
for (int i = 0; i < regionsToMerge.length; i++) {
regionsToMerge[i] = HRegionInfo.convert(mergeTableRegionsMsg.getRegionInfo(i));
regionsToMerge[i] = ProtobufUtil.toRegionInfo(mergeTableRegionsMsg.getRegionInfo(i));
}
mergedRegion = HRegionInfo.convert(mergeTableRegionsMsg.getMergedRegionInfo());
mergedRegion = ProtobufUtil.toRegionInfo(mergeTableRegionsMsg.getMergedRegionInfo());
}
@Override
@ -383,7 +390,7 @@ public class MergeTableRegionsProcedure
sb.append(" table=");
sb.append(getTableName());
sb.append(", regions=");
sb.append(HRegionInfo.getShortNameToLog(regionsToMerge));
sb.append(RegionInfo.getShortNameToLog(regionsToMerge));
sb.append(", forcibly=");
sb.append(forcible);
}
@ -450,7 +457,7 @@ public class MergeTableRegionsProcedure
boolean regionAHasMergeQualifier = !catalogJanitor.cleanMergeQualifier(regionsToMerge[0]);
if (regionAHasMergeQualifier
|| !catalogJanitor.cleanMergeQualifier(regionsToMerge[1])) {
String msg = "Skip merging regions " + HRegionInfo.getShortNameToLog(regionsToMerge) +
String msg = "Skip merging regions " + RegionInfo.getShortNameToLog(regionsToMerge) +
", because region "
+ (regionAHasMergeQualifier ? regionsToMerge[0].getEncodedName() : regionsToMerge[1]
.getEncodedName()) + " has merge qualifier";
@ -526,7 +533,7 @@ public class MergeTableRegionsProcedure
boolean ret = cpHost.preMergeRegionsAction(regionsToMerge, getUser());
if (ret) {
throw new IOException(
"Coprocessor bypassing regions " + HRegionInfo.getShortNameToLog(regionsToMerge) +
"Coprocessor bypassing regions " + RegionInfo.getShortNameToLog(regionsToMerge) +
" merge.");
}
}
@ -649,7 +656,7 @@ public class MergeTableRegionsProcedure
int procsIdx = 0;
for (int i = 0; i < regionsToMerge.length; ++i) {
for (int j = 0; j < regionReplication; ++j) {
final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(regionsToMerge[i], j);
final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(regionsToMerge[i], j);
procs[procsIdx++] = env.getAssignmentManager().createAssignProcedure(hri, serverName);
}
}
@ -663,7 +670,7 @@ public class MergeTableRegionsProcedure
int procsIdx = 0;
for (int i = 0; i < regionsToMerge.length; ++i) {
for (int j = 0; j < regionReplication; ++j) {
final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(regionsToMerge[i], j);
final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(regionsToMerge[i], j);
procs[procsIdx++] = env.getAssignmentManager().createUnassignProcedure(hri,null,true);
}
}
@ -675,7 +682,7 @@ public class MergeTableRegionsProcedure
final ServerName targetServer = getServerName(env);
final AssignProcedure[] procs = new AssignProcedure[regionReplication];
for (int i = 0; i < procs.length; ++i) {
final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(mergedRegion, i);
final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(mergedRegion, i);
procs[i] = env.getAssignmentManager().createAssignProcedure(hri, targetServer);
}
return procs;
@ -699,12 +706,12 @@ public class MergeTableRegionsProcedure
if (ret) {
throw new IOException(
"Coprocessor bypassing regions " + HRegionInfo.getShortNameToLog(regionsToMerge) +
"Coprocessor bypassing regions " + RegionInfo.getShortNameToLog(regionsToMerge) +
" merge.");
}
try {
for (Mutation p : metaEntries) {
HRegionInfo.parseRegionName(p.getRow());
RegionInfo.parseRegionName(p.getRow());
}
} catch (IOException e) {
LOG.error("Row key of mutation from coprocessor is not parsable as region name."
@ -780,7 +787,7 @@ public class MergeTableRegionsProcedure
* @return The merged region. Maybe be null if called to early or we failed.
*/
@VisibleForTesting
public HRegionInfo getMergedRegion() {
public RegionInfo getMergedRegion() {
return this.mergedRegion;
}
}

View File

@ -23,14 +23,15 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MoveRegionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MoveRegionStateData;
@ -128,7 +129,7 @@ public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<Mov
super.serializeStateData(serializer);
final MoveRegionStateData.Builder state = MoveRegionStateData.newBuilder()
// No need to serialize the HRegionInfo. The super class has the region.
// No need to serialize the RegionInfo. The super class has the region.
.setSourceServer(ProtobufUtil.toServerName(plan.getSource()));
if (plan.getDestination() != null) {
state.setDestinationServer(ProtobufUtil.toServerName(plan.getDestination()));
@ -143,7 +144,7 @@ public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<Mov
super.deserializeStateData(serializer);
final MoveRegionStateData state = serializer.deserialize(MoveRegionStateData.class);
final HRegionInfo regionInfo = getRegion(); // Get it from super class deserialization.
final RegionInfo regionInfo = getRegion(); // Get it from super class deserialization.
final ServerName sourceServer = ProtobufUtil.toServerName(state.getSourceServer());
final ServerName destinationServer = state.hasDestinationServer() ?
ProtobufUtil.toServerName(state.getDestinationServer()) : null;

View File

@ -28,24 +28,24 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
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.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
import org.apache.hadoop.hbase.util.MultiHConnection;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.MultiHConnection;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
@ -79,7 +79,7 @@ public class RegionStateStore {
}
public interface RegionStateVisitor {
void visitRegionState(HRegionInfo regionInfo, State state,
void visitRegionState(RegionInfo regionInfo, State state,
ServerName regionLocation, ServerName lastHost, long openSeqNum);
}
@ -114,7 +114,7 @@ public class RegionStateStore {
final HRegionLocation hrl = locations[i];
if (hrl == null) continue;
final HRegionInfo regionInfo = hrl.getRegionInfo();
final RegionInfo regionInfo = hrl.getRegionInfo();
if (regionInfo == null) continue;
final int replicaId = regionInfo.getReplicaId();
@ -132,7 +132,7 @@ public class RegionStateStore {
}
}
public void updateRegionLocation(final HRegionInfo regionInfo, final State state,
public void updateRegionLocation(final RegionInfo regionInfo, final State state,
final ServerName regionLocation, final ServerName lastHost, final long openSeqNum,
final long pid)
throws IOException {
@ -149,7 +149,7 @@ public class RegionStateStore {
oldState != null ? oldState.getServerName() : null, openSeqNum, pid);
}
protected void updateMetaLocation(final HRegionInfo regionInfo, final ServerName serverName)
protected void updateMetaLocation(final RegionInfo regionInfo, final ServerName serverName)
throws IOException {
try {
MetaTableLocator.setMetaLocation(master.getZooKeeper(), serverName,
@ -159,7 +159,7 @@ public class RegionStateStore {
}
}
protected void updateUserRegionLocation(final HRegionInfo regionInfo, final State state,
protected void updateUserRegionLocation(final RegionInfo regionInfo, final State state,
final ServerName regionLocation, final ServerName lastHost, final long openSeqNum,
final long pid)
throws IOException {
@ -195,7 +195,7 @@ public class RegionStateStore {
}
}
protected void updateRegionLocation(final HRegionInfo regionInfo, final State state,
protected void updateRegionLocation(final RegionInfo regionInfo, final State state,
final Put... put) throws IOException {
synchronized (this) {
if (multiHConnection == null) {
@ -219,8 +219,8 @@ public class RegionStateStore {
// ============================================================================================
// Update Region Splitting State helpers
// ============================================================================================
public void splitRegion(final HRegionInfo parent, final HRegionInfo hriA,
final HRegionInfo hriB, final ServerName serverName) throws IOException {
public void splitRegion(final RegionInfo parent, final RegionInfo hriA,
final RegionInfo hriB, final ServerName serverName) throws IOException {
final TableDescriptor htd = getTableDescriptor(parent.getTable());
MetaTableAccessor.splitRegion(master.getConnection(), parent, hriA, hriB, serverName,
getRegionReplication(htd), hasSerialReplicationScope(htd));
@ -229,8 +229,8 @@ public class RegionStateStore {
// ============================================================================================
// Update Region Merging State helpers
// ============================================================================================
public void mergeRegions(final HRegionInfo parent, final HRegionInfo hriA,
final HRegionInfo hriB, final ServerName serverName) throws IOException {
public void mergeRegions(final RegionInfo parent, final RegionInfo hriA,
final RegionInfo hriB, final ServerName serverName) throws IOException {
final TableDescriptor htd = getTableDescriptor(parent.getTable());
MetaTableAccessor.mergeRegions(master.getConnection(), parent, hriA, hriB, serverName,
getRegionReplication(htd), EnvironmentEdgeManager.currentTime(),
@ -240,11 +240,11 @@ public class RegionStateStore {
// ============================================================================================
// Delete Region State helpers
// ============================================================================================
public void deleteRegion(final HRegionInfo regionInfo) throws IOException {
public void deleteRegion(final RegionInfo regionInfo) throws IOException {
deleteRegions(Collections.singletonList(regionInfo));
}
public void deleteRegions(final List<HRegionInfo> regions) throws IOException {
public void deleteRegions(final List<RegionInfo> regions) throws IOException {
MetaTableAccessor.deleteRegions(master.getConnection(), regions);
}
@ -300,7 +300,7 @@ public class RegionStateStore {
return replicaId == 0
? HConstants.SERVERNAME_QUALIFIER
: Bytes.toBytes(HConstants.SERVERNAME_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
// ==========================================================================
@ -322,6 +322,6 @@ public class RegionStateStore {
return replicaId == 0
? HConstants.STATE_QUALIFIER
: Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+ String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
}

View File

@ -19,21 +19,6 @@
package org.apache.hadoop.hbase.master.assignment;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@ -50,6 +35,22 @@ import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentSkipListMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* RegionStates contains a set of Maps that describes the in-memory state of the AM, with
* the regions available in the system, the region in transition, the offline regions and
@ -71,8 +72,8 @@ public class RegionStates {
State.CLOSING // already in-progress (retrying)
};
private static class AssignmentProcedureEvent extends ProcedureEvent<HRegionInfo> {
public AssignmentProcedureEvent(final HRegionInfo regionInfo) {
private static class AssignmentProcedureEvent extends ProcedureEvent<RegionInfo> {
public AssignmentProcedureEvent(final RegionInfo regionInfo) {
super(regionInfo);
}
}
@ -96,7 +97,7 @@ public class RegionStates {
// so for now. Odd is that elsewhere in this RegionStates, we synchronize on
// the RegionStateNode instance. TODO.
public static class RegionStateNode implements Comparable<RegionStateNode> {
private final HRegionInfo regionInfo;
private final RegionInfo regionInfo;
private final ProcedureEvent<?> event;
private volatile RegionTransitionProcedure procedure = null;
@ -117,7 +118,7 @@ public class RegionStates {
private volatile long openSeqNum = HConstants.NO_SEQNUM;
public RegionStateNode(final HRegionInfo regionInfo) {
public RegionStateNode(final RegionInfo regionInfo) {
this.regionInfo = regionInfo;
this.event = new AssignmentProcedureEvent(regionInfo);
}
@ -184,7 +185,6 @@ public class RegionStates {
this.openSeqNum = seqId;
}
public ServerName setRegionLocation(final ServerName serverName) {
ServerName lastRegionLocation = this.regionLocation;
if (LOG.isTraceEnabled() && serverName == null) {
@ -219,7 +219,7 @@ public class RegionStates {
return event;
}
public HRegionInfo getRegionInfo() {
public RegionInfo getRegionInfo() {
return regionInfo;
}
@ -255,9 +255,9 @@ public class RegionStates {
@Override
public int compareTo(final RegionStateNode other) {
// NOTE: HRegionInfo sort by table first, so we are relying on that.
// NOTE: RegionInfo sort by table first, so we are relying on that.
// we have a TestRegionState#testOrderedByTable() that check for that.
return getRegionInfo().compareTo(other.getRegionInfo());
return RegionInfo.COMPARATOR.compare(getRegionInfo(), other.getRegionInfo());
}
@Override
@ -276,7 +276,7 @@ public class RegionStates {
public String toString() {
return toDescriptiveString();
}
public String toShortString() {
// rit= is the current Region-In-Transition State -- see State enum.
return String.format("rit=%s, location=%s", getState(), getRegionLocation());
@ -295,7 +295,7 @@ public class RegionStates {
@Override
public int compare(final RegionState l, final RegionState r) {
int stampCmp = Long.compare(l.getStamp(), r.getStamp());
return stampCmp != 0 ? stampCmp : l.getRegion().compareTo(r.getRegion());
return stampCmp != 0 ? stampCmp : RegionInfo.COMPARATOR.compare(l.getRegion(), r.getRegion());
}
}
@ -357,8 +357,8 @@ public class RegionStates {
return regions.size();
}
public ArrayList<HRegionInfo> getRegionInfoList() {
ArrayList<HRegionInfo> hris = new ArrayList<HRegionInfo>(regions.size());
public ArrayList<RegionInfo> getRegionInfoList() {
ArrayList<RegionInfo> hris = new ArrayList<RegionInfo>(regions.size());
for (RegionStateNode region: regions) {
hris.add(region.getRegionInfo());
}
@ -401,20 +401,20 @@ public class RegionStates {
// TODO: Replace the ConcurrentSkipListMaps
/**
* RegionName -- i.e. HRegionInfo.getRegionName() -- as bytes to {@link RegionStateNode}
* RegionName -- i.e. RegionInfo.getRegionName() -- as bytes to {@link RegionStateNode}
*/
private final ConcurrentSkipListMap<byte[], RegionStateNode> regionsMap =
new ConcurrentSkipListMap<byte[], RegionStateNode>(Bytes.BYTES_COMPARATOR);
private final ConcurrentSkipListMap<HRegionInfo, RegionStateNode> regionInTransition =
new ConcurrentSkipListMap<HRegionInfo, RegionStateNode>();
private final ConcurrentSkipListMap<RegionInfo, RegionStateNode> regionInTransition =
new ConcurrentSkipListMap<RegionInfo, RegionStateNode>(RegionInfo.COMPARATOR);
/**
* Regions marked as offline on a read of hbase:meta. Unused or at least, once
* offlined, regions have no means of coming on line again. TODO.
*/
private final ConcurrentSkipListMap<HRegionInfo, RegionStateNode> regionOffline =
new ConcurrentSkipListMap<HRegionInfo, RegionStateNode>();
private final ConcurrentSkipListMap<RegionInfo, RegionStateNode> regionOffline =
new ConcurrentSkipListMap<RegionInfo, RegionStateNode>();
private final ConcurrentSkipListMap<byte[], RegionFailedOpen> regionFailedOpen =
new ConcurrentSkipListMap<byte[], RegionFailedOpen>(Bytes.BYTES_COMPARATOR);
@ -432,7 +432,7 @@ public class RegionStates {
}
@VisibleForTesting
public boolean isRegionInRegionStates(final HRegionInfo hri) {
public boolean isRegionInRegionStates(final RegionInfo hri) {
return (regionsMap.containsKey(hri.getRegionName()) || regionInTransition.containsKey(hri)
|| regionOffline.containsKey(hri));
}
@ -440,13 +440,13 @@ public class RegionStates {
// ==========================================================================
// RegionStateNode helpers
// ==========================================================================
protected RegionStateNode createRegionNode(final HRegionInfo regionInfo) {
protected RegionStateNode createRegionNode(final RegionInfo regionInfo) {
RegionStateNode newNode = new RegionStateNode(regionInfo);
RegionStateNode oldNode = regionsMap.putIfAbsent(regionInfo.getRegionName(), newNode);
return oldNode != null ? oldNode : newNode;
}
protected RegionStateNode getOrCreateRegionNode(final HRegionInfo regionInfo) {
protected RegionStateNode getOrCreateRegionNode(final RegionInfo regionInfo) {
RegionStateNode node = regionsMap.get(regionInfo.getRegionName());
return node != null ? node : createRegionNode(regionInfo);
}
@ -455,7 +455,7 @@ public class RegionStates {
return regionsMap.get(regionName);
}
protected RegionStateNode getRegionNode(final HRegionInfo regionInfo) {
protected RegionStateNode getRegionNode(final RegionInfo regionInfo) {
return getRegionNodeFromName(regionInfo.getRegionName());
}
@ -469,7 +469,7 @@ public class RegionStates {
return null;
}
public void deleteRegion(final HRegionInfo regionInfo) {
public void deleteRegion(final RegionInfo regionInfo) {
regionsMap.remove(regionInfo.getRegionName());
// Remove from the offline regions map too if there.
if (this.regionOffline.containsKey(regionInfo)) {
@ -496,8 +496,8 @@ public class RegionStates {
return regions;
}
ArrayList<HRegionInfo> getTableRegionsInfo(final TableName tableName) {
final ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
ArrayList<RegionInfo> getTableRegionsInfo(final TableName tableName) {
final ArrayList<RegionInfo> regions = new ArrayList<RegionInfo>();
for (RegionStateNode node: regionsMap.tailMap(tableName.getName()).values()) {
if (!node.getTable().equals(tableName)) break;
regions.add(node.getRegionInfo());
@ -520,7 +520,7 @@ public class RegionStates {
// ==========================================================================
// RegionState helpers
// ==========================================================================
public RegionState getRegionState(final HRegionInfo regionInfo) {
public RegionState getRegionState(final RegionInfo regionInfo) {
return createRegionState(getRegionNode(regionInfo));
}
@ -542,13 +542,13 @@ public class RegionStates {
return !getTableRegionStates(tableName).isEmpty();
}
public List<HRegionInfo> getRegionsOfTable(final TableName table) {
public List<RegionInfo> getRegionsOfTable(final TableName table) {
return getRegionsOfTable(table, false);
}
List<HRegionInfo> getRegionsOfTable(final TableName table, final boolean offline) {
List<RegionInfo> getRegionsOfTable(final TableName table, final boolean offline) {
final ArrayList<RegionStateNode> nodes = getTableRegionStateNodes(table);
final ArrayList<HRegionInfo> hris = new ArrayList<HRegionInfo>(nodes.size());
final ArrayList<RegionInfo> hris = new ArrayList<RegionInfo>(nodes.size());
for (RegionStateNode node: nodes) {
if (include(node, offline)) hris.add(node.getRegionInfo());
}
@ -567,7 +567,7 @@ public class RegionStates {
}
if (node.isInState(State.SPLIT)) return false;
if (node.isInState(State.OFFLINE) && !offline) return false;
final HRegionInfo hri = node.getRegionInfo();
final RegionInfo hri = node.getRegionInfo();
return (!hri.isOffline() && !hri.isSplit()) ||
((hri.isOffline() || hri.isSplit()) && offline);
}
@ -575,9 +575,9 @@ public class RegionStates {
/**
* Returns the set of regions hosted by the specified server
* @param serverName the server we are interested in
* @return set of HRegionInfo hosted by the specified server
* @return set of RegionInfo hosted by the specified server
*/
public List<HRegionInfo> getServerRegionInfoSet(final ServerName serverName) {
public List<RegionInfo> getServerRegionInfoSet(final ServerName serverName) {
final ServerStateNode serverInfo = getServerNode(serverName);
if (serverInfo == null) return Collections.emptyList();
@ -603,7 +603,7 @@ public class RegionStates {
}
}
public void logSplit(final HRegionInfo regionInfo) {
public void logSplit(final RegionInfo regionInfo) {
final RegionStateNode regionNode = getRegionNode(regionInfo);
synchronized (regionNode) {
regionNode.setState(State.SPLIT);
@ -611,7 +611,7 @@ public class RegionStates {
}
@VisibleForTesting
public void updateRegionState(final HRegionInfo regionInfo, final State state) {
public void updateRegionState(final RegionInfo regionInfo, final State state) {
final RegionStateNode regionNode = getOrCreateRegionNode(regionInfo);
synchronized (regionNode) {
regionNode.setState(state);
@ -621,8 +621,8 @@ public class RegionStates {
// ============================================================================================
// TODO:
// ============================================================================================
public List<HRegionInfo> getAssignedRegions() {
final List<HRegionInfo> result = new ArrayList<HRegionInfo>();
public List<RegionInfo> getAssignedRegions() {
final List<RegionInfo> result = new ArrayList<RegionInfo>();
for (RegionStateNode node: regionsMap.values()) {
if (!node.isInTransition()) {
result.add(node.getRegionInfo());
@ -631,7 +631,7 @@ public class RegionStates {
return result;
}
public boolean isRegionInState(final HRegionInfo regionInfo, final State... state) {
public boolean isRegionInState(final RegionInfo regionInfo, final State... state) {
final RegionStateNode region = getRegionNode(regionInfo);
if (region != null) {
synchronized (region) {
@ -641,21 +641,21 @@ public class RegionStates {
return false;
}
public boolean isRegionOnline(final HRegionInfo regionInfo) {
public boolean isRegionOnline(final RegionInfo regionInfo) {
return isRegionInState(regionInfo, State.OPEN);
}
/**
* @return True if region is offline (In OFFLINE or CLOSED state).
*/
public boolean isRegionOffline(final HRegionInfo regionInfo) {
public boolean isRegionOffline(final RegionInfo regionInfo) {
return isRegionInState(regionInfo, State.OFFLINE, State.CLOSED);
}
public Map<ServerName, List<HRegionInfo>> getSnapShotOfAssignment(
final Collection<HRegionInfo> regions) {
final Map<ServerName, List<HRegionInfo>> result = new HashMap<ServerName, List<HRegionInfo>>();
for (HRegionInfo hri: regions) {
public Map<ServerName, List<RegionInfo>> getSnapShotOfAssignment(
final Collection<RegionInfo> regions) {
final Map<ServerName, List<RegionInfo>> result = new HashMap<ServerName, List<RegionInfo>>();
for (RegionInfo hri: regions) {
final RegionStateNode node = getRegionNode(hri);
if (node == null) continue;
@ -663,9 +663,9 @@ public class RegionStates {
final ServerName serverName = node.getRegionLocation();
if (serverName == null) continue;
List<HRegionInfo> serverRegions = result.get(serverName);
List<RegionInfo> serverRegions = result.get(serverName);
if (serverRegions == null) {
serverRegions = new ArrayList<HRegionInfo>();
serverRegions = new ArrayList<RegionInfo>();
result.put(serverName, serverRegions);
}
@ -674,20 +674,20 @@ public class RegionStates {
return result;
}
public Map<HRegionInfo, ServerName> getRegionAssignments() {
final HashMap<HRegionInfo, ServerName> assignments = new HashMap<HRegionInfo, ServerName>();
public Map<RegionInfo, ServerName> getRegionAssignments() {
final HashMap<RegionInfo, ServerName> assignments = new HashMap<RegionInfo, ServerName>();
for (RegionStateNode node: regionsMap.values()) {
assignments.put(node.getRegionInfo(), node.getRegionLocation());
}
return assignments;
}
public Map<RegionState.State, List<HRegionInfo>> getRegionByStateOfTable(TableName tableName) {
public Map<RegionState.State, List<RegionInfo>> getRegionByStateOfTable(TableName tableName) {
final State[] states = State.values();
final Map<RegionState.State, List<HRegionInfo>> tableRegions =
new HashMap<State, List<HRegionInfo>>(states.length);
final Map<RegionState.State, List<RegionInfo>> tableRegions =
new HashMap<State, List<RegionInfo>>(states.length);
for (int i = 0; i < states.length; ++i) {
tableRegions.put(states[i], new ArrayList<HRegionInfo>());
tableRegions.put(states[i], new ArrayList<RegionInfo>());
}
for (RegionStateNode node: regionsMap.values()) {
@ -698,7 +698,7 @@ public class RegionStates {
return tableRegions;
}
public ServerName getRegionServerOfRegion(final HRegionInfo regionInfo) {
public ServerName getRegionServerOfRegion(final RegionInfo regionInfo) {
final RegionStateNode region = getRegionNode(regionInfo);
if (region != null) {
synchronized (region) {
@ -717,29 +717,29 @@ public class RegionStates {
* @param forceByCluster a flag to force to aggregate the server-load to the cluster level
* @return A clone of current assignments by table.
*/
public Map<TableName, Map<ServerName, List<HRegionInfo>>> getAssignmentsByTable(
public Map<TableName, Map<ServerName, List<RegionInfo>>> getAssignmentsByTable(
final boolean forceByCluster) {
if (!forceByCluster) return getAssignmentsByTable();
final HashMap<ServerName, List<HRegionInfo>> ensemble =
new HashMap<ServerName, List<HRegionInfo>>(serverMap.size());
final HashMap<ServerName, List<RegionInfo>> ensemble =
new HashMap<ServerName, List<RegionInfo>>(serverMap.size());
for (ServerStateNode serverNode: serverMap.values()) {
ensemble.put(serverNode.getServerName(), serverNode.getRegionInfoList());
}
// TODO: can we use Collections.singletonMap(HConstants.ENSEMBLE_TABLE_NAME, ensemble)?
final Map<TableName, Map<ServerName, List<HRegionInfo>>> result =
new HashMap<TableName, Map<ServerName, List<HRegionInfo>>>(1);
final Map<TableName, Map<ServerName, List<RegionInfo>>> result =
new HashMap<TableName, Map<ServerName, List<RegionInfo>>>(1);
result.put(HConstants.ENSEMBLE_TABLE_NAME, ensemble);
return result;
}
public Map<TableName, Map<ServerName, List<HRegionInfo>>> getAssignmentsByTable() {
final Map<TableName, Map<ServerName, List<HRegionInfo>>> result = new HashMap<>();
public Map<TableName, Map<ServerName, List<RegionInfo>>> getAssignmentsByTable() {
final Map<TableName, Map<ServerName, List<RegionInfo>>> result = new HashMap<>();
for (RegionStateNode node: regionsMap.values()) {
Map<ServerName, List<HRegionInfo>> tableResult = result.get(node.getTable());
Map<ServerName, List<RegionInfo>> tableResult = result.get(node.getTable());
if (tableResult == null) {
tableResult = new HashMap<ServerName, List<HRegionInfo>>();
tableResult = new HashMap<ServerName, List<RegionInfo>>();
result.put(node.getTable(), tableResult);
}
@ -748,9 +748,9 @@ public class RegionStates {
LOG.info("Skipping, no server for " + node);
continue;
}
List<HRegionInfo> serverResult = tableResult.get(serverName);
List<RegionInfo> serverResult = tableResult.get(serverName);
if (serverResult == null) {
serverResult = new ArrayList<HRegionInfo>();
serverResult = new ArrayList<RegionInfo>();
tableResult.put(serverName, serverResult);
}
@ -780,7 +780,7 @@ public class RegionStates {
return !regionInTransition.isEmpty();
}
public boolean isRegionInTransition(final HRegionInfo regionInfo) {
public boolean isRegionInTransition(final RegionInfo regionInfo) {
final RegionStateNode node = regionInTransition.get(regionInfo);
return node != null ? node.isInTransition() : false;
}
@ -788,13 +788,13 @@ public class RegionStates {
/**
* @return If a procedure-in-transition for <code>hri</code>, return it else null.
*/
public RegionTransitionProcedure getRegionTransitionProcedure(final HRegionInfo hri) {
public RegionTransitionProcedure getRegionTransitionProcedure(final RegionInfo hri) {
RegionStateNode node = regionInTransition.get(hri);
if (node == null) return null;
return node.getProcedure();
}
public RegionState getRegionTransitionState(final HRegionInfo hri) {
public RegionState getRegionTransitionState(final RegionInfo hri) {
RegionStateNode node = regionInTransition.get(hri);
if (node == null) return null;
@ -840,7 +840,7 @@ public class RegionStates {
}
// TODO: Unused.
public void removeFromOfflineRegions(final HRegionInfo regionInfo) {
public void removeFromOfflineRegions(final RegionInfo regionInfo) {
regionOffline.remove(regionInfo);
}
@ -861,7 +861,7 @@ public class RegionStates {
return regionNode;
}
public HRegionInfo getRegionInfo() {
public RegionInfo getRegionInfo() {
return regionNode.getRegionInfo();
}
@ -893,11 +893,11 @@ public class RegionStates {
return node;
}
public RegionFailedOpen getFailedOpen(final HRegionInfo regionInfo) {
public RegionFailedOpen getFailedOpen(final RegionInfo regionInfo) {
return regionFailedOpen.get(regionInfo.getRegionName());
}
public void removeFromFailedOpen(final HRegionInfo regionInfo) {
public void removeFromFailedOpen(final RegionInfo regionInfo) {
regionFailedOpen.remove(regionInfo.getRegionName());
}

View File

@ -24,10 +24,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@ -36,6 +35,8 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
@ -89,7 +90,7 @@ public abstract class RegionTransitionProcedure
private RegionTransitionState transitionState =
RegionTransitionState.REGION_TRANSITION_QUEUE;
private HRegionInfo regionInfo;
private RegionInfo regionInfo;
private volatile boolean lock = false;
public RegionTransitionProcedure() {
@ -97,22 +98,22 @@ public abstract class RegionTransitionProcedure
super();
}
public RegionTransitionProcedure(final HRegionInfo regionInfo) {
public RegionTransitionProcedure(final RegionInfo regionInfo) {
this.regionInfo = regionInfo;
}
public HRegionInfo getRegionInfo() {
public RegionInfo getRegionInfo() {
return regionInfo;
}
protected void setRegionInfo(final HRegionInfo regionInfo) {
protected void setRegionInfo(final RegionInfo regionInfo) {
// Setter is for deserialization.
this.regionInfo = regionInfo;
}
@Override
public TableName getTableName() {
HRegionInfo hri = getRegionInfo();
RegionInfo hri = getRegionInfo();
return hri != null? hri.getTable(): null;
}

View File

@ -40,12 +40,13 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@ -72,6 +73,7 @@ import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
@ -86,8 +88,8 @@ public class SplitTableRegionProcedure
extends AbstractStateMachineRegionProcedure<SplitTableRegionState> {
private static final Log LOG = LogFactory.getLog(SplitTableRegionProcedure.class);
private Boolean traceEnabled = null;
private HRegionInfo daughter_1_HRI;
private HRegionInfo daughter_2_HRI;
private RegionInfo daughter_1_RI;
private RegionInfo daughter_2_RI;
private byte[] bestSplitRow;
public SplitTableRegionProcedure() {
@ -95,14 +97,24 @@ public class SplitTableRegionProcedure
}
public SplitTableRegionProcedure(final MasterProcedureEnv env,
final HRegionInfo regionToSplit, final byte[] splitRow) throws IOException {
final RegionInfo regionToSplit, final byte[] splitRow) throws IOException {
super(env, regionToSplit);
this.bestSplitRow = splitRow;
checkSplittable(env, regionToSplit, bestSplitRow);
final TableName table = regionToSplit.getTable();
final long rid = getDaughterRegionIdTimestamp(regionToSplit);
this.daughter_1_HRI = new HRegionInfo(table, regionToSplit.getStartKey(), bestSplitRow, false, rid);
this.daughter_2_HRI = new HRegionInfo(table, bestSplitRow, regionToSplit.getEndKey(), false, rid);
this.daughter_1_RI = RegionInfoBuilder.newBuilder(table)
.setStartKey(regionToSplit.getStartKey())
.setEndKey(bestSplitRow)
.setSplit(false)
.setRegionId(rid)
.build();
this.daughter_2_RI = RegionInfoBuilder.newBuilder(table)
.setStartKey(bestSplitRow)
.setEndKey(regionToSplit.getEndKey())
.setSplit(false)
.setRegionId(rid)
.build();
}
/**
@ -113,10 +125,10 @@ public class SplitTableRegionProcedure
* @throws IOException
*/
private void checkSplittable(final MasterProcedureEnv env,
final HRegionInfo regionToSplit, final byte[] splitRow) throws IOException {
final RegionInfo regionToSplit, final byte[] splitRow) throws IOException {
// Ask the remote RS if this region is splittable.
// If we get an IOE, report it along w/ the failure so can see why we are not splittable at this time.
if(regionToSplit.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
if(regionToSplit.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
throw new IllegalArgumentException ("Can't invoke split on non-default regions directly");
}
RegionStateNode node =
@ -169,10 +181,10 @@ public class SplitTableRegionProcedure
/**
* Calculate daughter regionid to use.
* @param hri Parent {@link HRegionInfo}
* @param hri Parent {@link RegionInfo}
* @return Daughter region id (timestamp) to use.
*/
private static long getDaughterRegionIdTimestamp(final HRegionInfo hri) {
private static long getDaughterRegionIdTimestamp(final RegionInfo hri) {
long rid = EnvironmentEdgeManager.currentTime();
// Regionid is timestamp. Can't be less than that of parent else will insert
// at wrong location in hbase:meta (See HBASE-710).
@ -332,9 +344,9 @@ public class SplitTableRegionProcedure
final MasterProcedureProtos.SplitTableRegionStateData.Builder splitTableRegionMsg =
MasterProcedureProtos.SplitTableRegionStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
.setParentRegionInfo(HRegionInfo.convert(getRegion()))
.addChildRegionInfo(HRegionInfo.convert(daughter_1_HRI))
.addChildRegionInfo(HRegionInfo.convert(daughter_2_HRI));
.setParentRegionInfo(ProtobufUtil.toRegionInfo(getRegion()))
.addChildRegionInfo(ProtobufUtil.toRegionInfo(daughter_1_RI))
.addChildRegionInfo(ProtobufUtil.toRegionInfo(daughter_2_RI));
serializer.serialize(splitTableRegionMsg.build());
}
@ -346,10 +358,10 @@ public class SplitTableRegionProcedure
final MasterProcedureProtos.SplitTableRegionStateData splitTableRegionsMsg =
serializer.deserialize(MasterProcedureProtos.SplitTableRegionStateData.class);
setUser(MasterProcedureUtil.toUserInfo(splitTableRegionsMsg.getUserInfo()));
setRegion(HRegionInfo.convert(splitTableRegionsMsg.getParentRegionInfo()));
setRegion(ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getParentRegionInfo()));
assert(splitTableRegionsMsg.getChildRegionInfoCount() == 2);
daughter_1_HRI = HRegionInfo.convert(splitTableRegionsMsg.getChildRegionInfo(0));
daughter_2_HRI = HRegionInfo.convert(splitTableRegionsMsg.getChildRegionInfo(1));
daughter_1_RI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(0));
daughter_2_RI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(1));
}
@Override
@ -360,12 +372,12 @@ public class SplitTableRegionProcedure
sb.append(", parent=");
sb.append(getParentRegion().getShortNameToLog());
sb.append(", daughterA=");
sb.append(daughter_1_HRI.getShortNameToLog());
sb.append(daughter_1_RI.getShortNameToLog());
sb.append(", daughterB=");
sb.append(daughter_2_HRI.getShortNameToLog());
sb.append(daughter_2_RI.getShortNameToLog());
}
private HRegionInfo getParentRegion() {
private RegionInfo getParentRegion() {
return getRegion();
}
@ -380,7 +392,7 @@ public class SplitTableRegionProcedure
}
private byte[] getSplitRow() {
return daughter_2_HRI.getStartKey();
return daughter_2_RI.getStartKey();
}
private static State [] EXPECTED_SPLIT_STATES = new State [] {State.OPEN, State.CLOSED};
@ -394,7 +406,7 @@ public class SplitTableRegionProcedure
// Check whether the region is splittable
RegionStateNode node =
env.getAssignmentManager().getRegionStates().getRegionNode(getParentRegion());
HRegionInfo parentHRI = null;
RegionInfo parentHRI = null;
if (node != null) {
parentHRI = node.getRegionInfo();
@ -479,7 +491,7 @@ public class SplitTableRegionProcedure
final AssignProcedure[] procs = new AssignProcedure[regionReplication];
for (int i = 0; i < regionReplication; ++i) {
final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(getParentRegion(), i);
final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(getParentRegion(), i);
procs[i] = env.getAssignmentManager().createAssignProcedure(hri, serverName);
}
env.getMasterServices().getMasterProcedureExecutor().submitProcedures(procs);
@ -502,17 +514,17 @@ public class SplitTableRegionProcedure
Pair<Integer, Integer> expectedReferences = splitStoreFiles(env, regionFs);
assertReferenceFileCount(fs, expectedReferences.getFirst(),
regionFs.getSplitsDir(daughter_1_HRI));
regionFs.getSplitsDir(daughter_1_RI));
//Move the files from the temporary .splits to the final /table/region directory
regionFs.commitDaughterRegion(daughter_1_HRI);
regionFs.commitDaughterRegion(daughter_1_RI);
assertReferenceFileCount(fs, expectedReferences.getFirst(),
new Path(tabledir, daughter_1_HRI.getEncodedName()));
new Path(tabledir, daughter_1_RI.getEncodedName()));
assertReferenceFileCount(fs, expectedReferences.getSecond(),
regionFs.getSplitsDir(daughter_2_HRI));
regionFs.commitDaughterRegion(daughter_2_HRI);
regionFs.getSplitsDir(daughter_2_RI));
regionFs.commitDaughterRegion(daughter_2_RI);
assertReferenceFileCount(fs, expectedReferences.getSecond(),
new Path(tabledir, daughter_2_HRI.getEncodedName()));
new Path(tabledir, daughter_2_RI.getEncodedName()));
}
/**
@ -650,9 +662,9 @@ public class SplitTableRegionProcedure
final byte[] splitRow = getSplitRow();
final String familyName = Bytes.toString(family);
final Path path_first =
regionFs.splitStoreFile(this.daughter_1_HRI, familyName, sf, splitRow, false, null);
regionFs.splitStoreFile(this.daughter_1_RI, familyName, sf, splitRow, false, null);
final Path path_second =
regionFs.splitStoreFile(this.daughter_2_HRI, familyName, sf, splitRow, true, null);
regionFs.splitStoreFile(this.daughter_2_RI, familyName, sf, splitRow, true, null);
if (LOG.isDebugEnabled()) {
LOG.debug("pid=" + getProcId() + " splitting complete for store file: " +
sf.getPath() + " for region: " + getParentRegion().getShortNameToLog());
@ -702,7 +714,7 @@ public class SplitTableRegionProcedure
}
try {
for (Mutation p : metaEntries) {
HRegionInfo.parseRegionName(p.getRow());
RegionInfo.parseRegionName(p.getRow());
}
} catch (IOException e) {
LOG.error("pid=" + getProcId() + " row key of mutation from coprocessor not parsable as "
@ -720,7 +732,7 @@ public class SplitTableRegionProcedure
*/
private void updateMetaForDaughterRegions(final MasterProcedureEnv env) throws IOException {
env.getAssignmentManager().markRegionAsSplit(getParentRegion(), getParentRegionServerName(env),
daughter_1_HRI, daughter_2_HRI);
daughter_1_RI, daughter_2_RI);
}
/**
@ -742,7 +754,7 @@ public class SplitTableRegionProcedure
private void postSplitRegion(final MasterProcedureEnv env) throws IOException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.postCompletedSplitRegionAction(daughter_1_HRI, daughter_2_HRI, getUser());
cpHost.postCompletedSplitRegionAction(daughter_1_RI, daughter_2_RI, getUser());
}
}
@ -755,7 +767,7 @@ public class SplitTableRegionProcedure
final int regionReplication) {
final UnassignProcedure[] procs = new UnassignProcedure[regionReplication];
for (int i = 0; i < procs.length; ++i) {
final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(getParentRegion(), i);
final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(getParentRegion(), i);
procs[i] = env.getAssignmentManager().createUnassignProcedure(hri, null, true);
}
return procs;
@ -767,11 +779,11 @@ public class SplitTableRegionProcedure
final AssignProcedure[] procs = new AssignProcedure[regionReplication * 2];
int procsIdx = 0;
for (int i = 0; i < regionReplication; ++i) {
final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(daughter_1_HRI, i);
final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(daughter_1_RI, i);
procs[procsIdx++] = env.getAssignmentManager().createAssignProcedure(hri, targetServer);
}
for (int i = 0; i < regionReplication; ++i) {
final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(daughter_2_HRI, i);
final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(daughter_2_RI, i);
procs[procsIdx++] = env.getAssignmentManager().createAssignProcedure(hri, targetServer);
}
return procs;

View File

@ -20,29 +20,31 @@
package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.ServerCrashException;
import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.RegionCloseOperation;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.master.procedure.ServerCrashException;
import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.UnassignRegionStateData;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
/**
* Procedure that describes the unassignment of a single region.
@ -90,12 +92,12 @@ public class UnassignProcedure extends RegionTransitionProcedure {
super();
}
public UnassignProcedure(final HRegionInfo regionInfo, final ServerName hostingServer,
public UnassignProcedure(final RegionInfo regionInfo, final ServerName hostingServer,
final boolean force) {
this(regionInfo, hostingServer, null, force);
}
public UnassignProcedure(final HRegionInfo regionInfo,
public UnassignProcedure(final RegionInfo regionInfo,
final ServerName hostingServer, final ServerName destinationServer, final boolean force) {
super(regionInfo);
this.hostingServer = hostingServer;
@ -128,7 +130,7 @@ public class UnassignProcedure extends RegionTransitionProcedure {
UnassignRegionStateData.Builder state = UnassignRegionStateData.newBuilder()
.setTransitionState(getTransitionState())
.setHostingServer(ProtobufUtil.toServerName(this.hostingServer))
.setRegionInfo(HRegionInfo.convert(getRegionInfo()));
.setRegionInfo(ProtobufUtil.toRegionInfo(getRegionInfo()));
if (this.destinationServer != null) {
state.setDestinationServer(ProtobufUtil.toServerName(destinationServer));
}
@ -144,7 +146,7 @@ public class UnassignProcedure extends RegionTransitionProcedure {
final UnassignRegionStateData state =
serializer.deserialize(UnassignRegionStateData.class);
setTransitionState(state.getTransitionState());
setRegionInfo(HRegionInfo.convert(state.getRegionInfo()));
setRegionInfo(ProtobufUtil.toRegionInfo(state.getRegionInfo()));
this.hostingServer = ProtobufUtil.toServerName(state.getHostingServer());
force = state.getForce();
if (state.hasDestinationServer()) {

View File

@ -19,11 +19,12 @@ package org.apache.hadoop.hbase.master.assignment;
import java.io.IOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@ -43,13 +44,13 @@ class Util {
* @throws IOException Let it out so can report this IOE as reason for failure
*/
static GetRegionInfoResponse getRegionInfoResponse(final MasterProcedureEnv env,
final ServerName regionLocation, final HRegionInfo hri)
final ServerName regionLocation, final RegionInfo hri)
throws IOException {
return getRegionInfoResponse(env, regionLocation, hri, false);
}
static GetRegionInfoResponse getRegionInfoResponse(final MasterProcedureEnv env,
final ServerName regionLocation, final HRegionInfo hri, boolean includeBestSplitRow)
final ServerName regionLocation, final RegionInfo hri, boolean includeBestSplitRow)
throws IOException {
// TODO: There is no timeout on this controller. Set one!
HBaseRpcController controller = env.getMasterServices().getClusterConnection().

View File

@ -43,10 +43,10 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.MasterServices;
@ -71,7 +71,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
protected static final int MIN_SERVER_BALANCE = 2;
private volatile boolean stopped = false;
private static final List<HRegionInfo> EMPTY_REGION_LIST = new ArrayList<>(0);
private static final List<RegionInfo> EMPTY_REGION_LIST = new ArrayList<>(0);
static final Predicate<ServerLoad> IDLE_SERVER_PREDICATOR
= load -> load.getNumberOfRegions() == 0;
@ -127,7 +127,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
boolean multiServersPerHost = false; // whether or not any host has more than one server
ArrayList<String> tables;
HRegionInfo[] regions;
RegionInfo[] regions;
Deque<BalancerRegionLoad>[] regionLoads;
private RegionLocationFinder regionFinder;
@ -160,7 +160,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
Map<String, Integer> hostsToIndex;
Map<String, Integer> racksToIndex;
Map<String, Integer> tablesToIndex;
Map<HRegionInfo, Integer> regionsToIndex;
Map<RegionInfo, Integer> regionsToIndex;
float[] localityPerServer;
int numServers;
@ -170,7 +170,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
int numRegions;
int numMovedRegions = 0; //num moved regions from the initial configuration
Map<ServerName, List<HRegionInfo>> clusterState;
Map<ServerName, List<RegionInfo>> clusterState;
protected final RackManager rackManager;
// Maps region -> rackIndex -> locality of region on rack
@ -179,7 +179,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
private int[][] regionsToMostLocalEntities;
protected Cluster(
Map<ServerName, List<HRegionInfo>> clusterState,
Map<ServerName, List<RegionInfo>> clusterState,
Map<String, Deque<BalancerRegionLoad>> loads,
RegionLocationFinder regionFinder,
RackManager rackManager) {
@ -188,8 +188,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
@SuppressWarnings("unchecked")
protected Cluster(
Collection<HRegionInfo> unassignedRegions,
Map<ServerName, List<HRegionInfo>> clusterState,
Collection<RegionInfo> unassignedRegions,
Map<ServerName, List<RegionInfo>> clusterState,
Map<String, Deque<BalancerRegionLoad>> loads,
RegionLocationFinder regionFinder,
RackManager rackManager) {
@ -247,7 +247,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
// Count how many regions there are.
for (Entry<ServerName, List<HRegionInfo>> entry : clusterState.entrySet()) {
for (Entry<ServerName, List<RegionInfo>> entry : clusterState.entrySet()) {
numRegions += entry.getValue().size();
}
numRegions += unassignedRegions.size();
@ -256,7 +256,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
servers = new ServerName[numServers];
serversPerHost = new int[numHosts][];
serversPerRack = new int[numRacks][];
regions = new HRegionInfo[numRegions];
regions = new RegionInfo[numRegions];
regionIndexToServerIndex = new int[numRegions];
initialRegionIndexToServerIndex = new int[numRegions];
regionIndexToTableIndex = new int[numRegions];
@ -279,7 +279,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
int tableIndex = 0, regionIndex = 0, regionPerServerIndex = 0;
for (Entry<ServerName, List<HRegionInfo>> entry : clusterState.entrySet()) {
for (Entry<ServerName, List<RegionInfo>> entry : clusterState.entrySet()) {
if (entry.getKey() == null) {
LOG.warn("SERVERNAME IS NULL, skipping " + entry.getValue());
continue;
@ -314,7 +314,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
racks[entry.getValue()] = entry.getKey();
}
for (Entry<ServerName, List<HRegionInfo>> entry : clusterState.entrySet()) {
for (Entry<ServerName, List<RegionInfo>> entry : clusterState.entrySet()) {
int serverIndex = serversToIndex.get(entry.getKey().getHostAndPort());
regionPerServerIndex = 0;
@ -324,14 +324,14 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
int rackIndex = racksToIndex.get(this.rackManager.getRack(entry.getKey()));
serverIndexToRackIndex[serverIndex] = rackIndex;
for (HRegionInfo region : entry.getValue()) {
for (RegionInfo region : entry.getValue()) {
registerRegion(region, regionIndex, serverIndex, loads, regionFinder);
regionsPerServer[serverIndex][regionPerServerIndex++] = regionIndex;
regionIndex++;
}
}
for (HRegionInfo region : unassignedRegions) {
for (RegionInfo region : unassignedRegions) {
registerRegion(region, regionIndex, -1, loads, regionFinder);
regionIndex++;
}
@ -378,12 +378,12 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
for (int i = 0; i < regions.length; i ++) {
HRegionInfo info = regions[i];
RegionInfo info = regions[i];
if (RegionReplicaUtil.isDefaultReplica(info)) {
regionIndexToPrimaryIndex[i] = i;
} else {
hasRegionReplicas = true;
HRegionInfo primaryInfo = RegionReplicaUtil.getRegionInfoForDefaultReplica(info);
RegionInfo primaryInfo = RegionReplicaUtil.getRegionInfoForDefaultReplica(info);
regionIndexToPrimaryIndex[i] = regionsToIndex.getOrDefault(primaryInfo, -1);
}
}
@ -453,7 +453,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
/** Helper for Cluster constructor to handle a region */
private void registerRegion(HRegionInfo region, int regionIndex,
private void registerRegion(RegionInfo region, int regionIndex,
int serverIndex, Map<String, Deque<BalancerRegionLoad>> loads,
RegionLocationFinder regionFinder) {
String tableName = region.getTable().getNameAsString();
@ -731,7 +731,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* of the region in question
* @return true or false
*/
boolean wouldLowerAvailability(HRegionInfo regionInfo, ServerName serverName) {
boolean wouldLowerAvailability(RegionInfo regionInfo, ServerName serverName) {
if (!serversToIndex.containsKey(serverName.getHostAndPort())) {
return false; // safeguard against race between cluster.servers and servers from LB method args
}
@ -783,7 +783,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
return false;
}
void doAssignRegion(HRegionInfo regionInfo, ServerName serverName) {
void doAssignRegion(RegionInfo regionInfo, ServerName serverName) {
if (!serversToIndex.containsKey(serverName.getHostAndPort())) {
return;
}
@ -1047,20 +1047,20 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* Check if a region belongs to some system table.
* If so, the primary replica may be expected to be put on the master regionserver.
*/
public boolean shouldBeOnMaster(HRegionInfo region) {
public boolean shouldBeOnMaster(RegionInfo region) {
return this.onlySystemTablesOnMaster && region.isSystemTable();
}
/**
* Balance the regions that should be on master regionserver.
*/
protected List<RegionPlan> balanceMasterRegions(Map<ServerName, List<HRegionInfo>> clusterMap) {
protected List<RegionPlan> balanceMasterRegions(Map<ServerName, List<RegionInfo>> clusterMap) {
if (masterServerName == null || clusterMap == null || clusterMap.size() <= 1) return null;
List<RegionPlan> plans = null;
List<HRegionInfo> regions = clusterMap.get(masterServerName);
List<RegionInfo> regions = clusterMap.get(masterServerName);
if (regions != null) {
Iterator<ServerName> keyIt = null;
for (HRegionInfo region: regions) {
for (RegionInfo region: regions) {
if (shouldBeOnMaster(region)) continue;
// Find a non-master regionserver to host the region
@ -1083,9 +1083,9 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
plans.add(plan);
}
}
for (Map.Entry<ServerName, List<HRegionInfo>> server: clusterMap.entrySet()) {
for (Map.Entry<ServerName, List<RegionInfo>> server: clusterMap.entrySet()) {
if (masterServerName.equals(server.getKey())) continue;
for (HRegionInfo region: server.getValue()) {
for (RegionInfo region: server.getValue()) {
if (!shouldBeOnMaster(region)) continue;
// Move this region to the master regionserver
@ -1103,16 +1103,16 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* If master is configured to carry system tables only, in here is
* where we figure what to assign it.
*/
protected Map<ServerName, List<HRegionInfo>> assignMasterSystemRegions(
Collection<HRegionInfo> regions, List<ServerName> servers) {
protected Map<ServerName, List<RegionInfo>> assignMasterSystemRegions(
Collection<RegionInfo> regions, List<ServerName> servers) {
if (servers == null || regions == null || regions.isEmpty()) {
return null;
}
Map<ServerName, List<HRegionInfo>> assignments = new TreeMap<>();
Map<ServerName, List<RegionInfo>> assignments = new TreeMap<>();
if (this.onlySystemTablesOnMaster) {
if (masterServerName != null && servers.contains(masterServerName)) {
assignments.put(masterServerName, new ArrayList<>());
for (HRegionInfo region : regions) {
for (RegionInfo region : regions) {
if (shouldBeOnMaster(region)) {
assignments.get(masterServerName).add(region);
}
@ -1136,7 +1136,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
@Override
public void setClusterLoad(Map<TableName, Map<ServerName, List<HRegionInfo>>> clusterLoad){
public void setClusterLoad(Map<TableName, Map<ServerName, List<RegionInfo>>> clusterLoad){
}
@ -1169,7 +1169,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
int floor = (int) Math.floor(average * (1 - slop));
int ceiling = (int) Math.ceil(average * (1 + slop));
if (!(cs.getMaxLoad() > ceiling || cs.getMinLoad() < floor)) {
NavigableMap<ServerAndLoad, List<HRegionInfo>> serversByLoad = cs.getServersByLoad();
NavigableMap<ServerAndLoad, List<RegionInfo>> serversByLoad = cs.getServersByLoad();
if (LOG.isTraceEnabled()) {
// If nothing to balance, then don't say anything unless trace-level logging.
LOG.trace("Skipping load balancing because balanced cluster; " +
@ -1212,15 +1212,15 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* assignment is possible (ie. no regions or no servers)
*/
@Override
public Map<ServerName, List<HRegionInfo>> roundRobinAssignment(List<HRegionInfo> regions,
public Map<ServerName, List<RegionInfo>> roundRobinAssignment(List<RegionInfo> regions,
List<ServerName> servers) throws HBaseIOException {
metricsBalancer.incrMiscInvocations();
Map<ServerName, List<HRegionInfo>> assignments = assignMasterSystemRegions(regions, servers);
Map<ServerName, List<RegionInfo>> assignments = assignMasterSystemRegions(regions, servers);
if (assignments != null && !assignments.isEmpty()) {
servers = new ArrayList<>(servers);
// Guarantee not to put other regions on master
servers.remove(masterServerName);
List<HRegionInfo> masterRegions = assignments.get(masterServerName);
List<RegionInfo> masterRegions = assignments.get(masterServerName);
if (!masterRegions.isEmpty()) {
regions = new ArrayList<>(regions);
regions.removeAll(masterRegions);
@ -1248,20 +1248,20 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
Cluster cluster = createCluster(servers, regions, false);
List<HRegionInfo> unassignedRegions = new ArrayList<>();
List<RegionInfo> unassignedRegions = new ArrayList<>();
roundRobinAssignment(cluster, regions, unassignedRegions,
servers, assignments);
List<HRegionInfo> lastFewRegions = new ArrayList<>();
List<RegionInfo> lastFewRegions = new ArrayList<>();
// assign the remaining by going through the list and try to assign to servers one-by-one
int serverIdx = RANDOM.nextInt(numServers);
for (HRegionInfo region : unassignedRegions) {
for (RegionInfo region : unassignedRegions) {
boolean assigned = false;
for (int j = 0; j < numServers; j++) { // try all servers one by one
ServerName serverName = servers.get((j + serverIdx) % numServers);
if (!cluster.wouldLowerAvailability(region, serverName)) {
List<HRegionInfo> serverRegions =
List<RegionInfo> serverRegions =
assignments.computeIfAbsent(serverName, k -> new ArrayList<>());
serverRegions.add(region);
cluster.doAssignRegion(region, serverName);
@ -1276,10 +1276,10 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
// just sprinkle the rest of the regions on random regionservers. The balanceCluster will
// make it optimal later. we can end up with this if numReplicas > numServers.
for (HRegionInfo region : lastFewRegions) {
for (RegionInfo region : lastFewRegions) {
int i = RANDOM.nextInt(numServers);
ServerName server = servers.get(i);
List<HRegionInfo> serverRegions = assignments.computeIfAbsent(server, k -> new ArrayList<>());
List<RegionInfo> serverRegions = assignments.computeIfAbsent(server, k -> new ArrayList<>());
serverRegions.add(region);
cluster.doAssignRegion(region, server);
}
@ -1287,7 +1287,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
protected Cluster createCluster(List<ServerName> servers,
Collection<HRegionInfo> regions, boolean forceRefresh) {
Collection<RegionInfo> regions, boolean forceRefresh) {
if (forceRefresh && useRegionFinder) {
regionFinder.refreshAndWait(regions);
}
@ -1295,7 +1295,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
// a cluster out of it. Note that we might have replicas already assigned to some servers
// earlier. So we want to get the snapshot to see those assignments, but this will only contain
// replicas of the regions that are passed (for performance).
Map<ServerName, List<HRegionInfo>> clusterState = getRegionAssignmentsByServer(regions);
Map<ServerName, List<RegionInfo>> clusterState = getRegionAssignmentsByServer(regions);
for (ServerName server : servers) {
if (!clusterState.containsKey(server)) {
@ -1315,7 +1315,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* Used to assign a single region to a random server.
*/
@Override
public ServerName randomAssignment(HRegionInfo regionInfo, List<ServerName> servers)
public ServerName randomAssignment(RegionInfo regionInfo, List<ServerName> servers)
throws HBaseIOException {
metricsBalancer.incrMiscInvocations();
if (servers != null && servers.contains(masterServerName)) {
@ -1343,7 +1343,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
final List<ServerName> finalServers = idleServers.isEmpty() ?
servers : idleServers;
List<HRegionInfo> regions = Lists.newArrayList(regionInfo);
List<RegionInfo> regions = Lists.newArrayList(regionInfo);
Cluster cluster = createCluster(finalServers, regions, false);
return randomAssignment(cluster, regionInfo, finalServers);
}
@ -1366,16 +1366,16 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* @return map of servers and regions to be assigned to them
*/
@Override
public Map<ServerName, List<HRegionInfo>> retainAssignment(Map<HRegionInfo, ServerName> regions,
public Map<ServerName, List<RegionInfo>> retainAssignment(Map<RegionInfo, ServerName> regions,
List<ServerName> servers) throws HBaseIOException {
// Update metrics
metricsBalancer.incrMiscInvocations();
Map<ServerName, List<HRegionInfo>> assignments = assignMasterSystemRegions(regions.keySet(), servers);
Map<ServerName, List<RegionInfo>> assignments = assignMasterSystemRegions(regions.keySet(), servers);
if (assignments != null && !assignments.isEmpty()) {
servers = new ArrayList<>(servers);
// Guarantee not to put other regions on master
servers.remove(masterServerName);
List<HRegionInfo> masterRegions = assignments.get(masterServerName);
List<RegionInfo> masterRegions = assignments.get(masterServerName);
regions = regions.entrySet().stream().filter(e -> !masterRegions.contains(e.getKey()))
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
}
@ -1416,8 +1416,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
Cluster cluster = createCluster(servers, regions.keySet(), true);
for (Map.Entry<HRegionInfo, ServerName> entry : regions.entrySet()) {
HRegionInfo region = entry.getKey();
for (Map.Entry<RegionInfo, ServerName> entry : regions.entrySet()) {
RegionInfo region = entry.getKey();
ServerName oldServerName = entry.getValue();
List<ServerName> localServers = new ArrayList<>();
if (oldServerName != null) {
@ -1477,11 +1477,11 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
@Override
public void regionOnline(HRegionInfo regionInfo, ServerName sn) {
public void regionOnline(RegionInfo regionInfo, ServerName sn) {
}
@Override
public void regionOffline(HRegionInfo regionInfo) {
public void regionOffline(RegionInfo regionInfo) {
}
@Override
@ -1498,7 +1498,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
/**
* Used to assign a single region to a random server.
*/
private ServerName randomAssignment(Cluster cluster, HRegionInfo regionInfo,
private ServerName randomAssignment(Cluster cluster, RegionInfo regionInfo,
List<ServerName> servers) {
int numServers = servers.size(); // servers is not null, numServers > 1
ServerName sn = null;
@ -1517,9 +1517,9 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
/**
* Round robin a list of regions to a list of servers
*/
private void roundRobinAssignment(Cluster cluster, List<HRegionInfo> regions,
List<HRegionInfo> unassignedRegions, List<ServerName> servers,
Map<ServerName, List<HRegionInfo>> assignments) {
private void roundRobinAssignment(Cluster cluster, List<RegionInfo> regions,
List<RegionInfo> unassignedRegions, List<ServerName> servers,
Map<ServerName, List<RegionInfo>> assignments) {
int numServers = servers.size();
int numRegions = regions.size();
@ -1532,9 +1532,9 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
for (int j = 0; j < numServers; j++) {
ServerName server = servers.get((j + serverIdx) % numServers);
List<HRegionInfo> serverRegions = new ArrayList<>(max);
List<RegionInfo> serverRegions = new ArrayList<>(max);
for (int i = regionIdx; i < numRegions; i += numServers) {
HRegionInfo region = regions.get(i % numRegions);
RegionInfo region = regions.get(i % numRegions);
if (cluster.wouldLowerAvailability(region, server)) {
unassignedRegions.add(region);
} else {
@ -1547,8 +1547,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
}
protected Map<ServerName, List<HRegionInfo>> getRegionAssignmentsByServer(
Collection<HRegionInfo> regions) {
protected Map<ServerName, List<RegionInfo>> getRegionAssignmentsByServer(
Collection<RegionInfo> regions) {
if (this.services != null && this.services.getAssignmentManager() != null) {
return this.services.getAssignmentManager().getSnapShotOfAssignment(regions);
} else {

View File

@ -22,27 +22,27 @@ import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
/**
* Class used to hold the current state of the cluster and how balanced it is.
*/
public class ClusterLoadState {
private final Map<ServerName, List<HRegionInfo>> clusterState;
private final NavigableMap<ServerAndLoad, List<HRegionInfo>> serversByLoad;
private final Map<ServerName, List<RegionInfo>> clusterState;
private final NavigableMap<ServerAndLoad, List<RegionInfo>> serversByLoad;
private boolean emptyRegionServerPresent = false;
private int numRegions = 0;
private int numServers = 0;
public ClusterLoadState(Map<ServerName, List<HRegionInfo>> clusterState) {
public ClusterLoadState(Map<ServerName, List<RegionInfo>> clusterState) {
this.numRegions = 0;
this.numServers = clusterState.size();
this.clusterState = clusterState;
serversByLoad = new TreeMap<>();
// Iterate so we can count regions as we build the map
for (Map.Entry<ServerName, List<HRegionInfo>> server : clusterState.entrySet()) {
List<HRegionInfo> regions = server.getValue();
for (Map.Entry<ServerName, List<RegionInfo>> server : clusterState.entrySet()) {
List<RegionInfo> regions = server.getValue();
int sz = regions.size();
if (sz == 0) emptyRegionServerPresent = true;
numRegions += sz;
@ -50,11 +50,11 @@ public class ClusterLoadState {
}
}
Map<ServerName, List<HRegionInfo>> getClusterState() {
Map<ServerName, List<RegionInfo>> getClusterState() {
return clusterState;
}
NavigableMap<ServerAndLoad, List<HRegionInfo>> getServersByLoad() {
NavigableMap<ServerAndLoad, List<RegionInfo>> getServersByLoad() {
return serversByLoad;
}

View File

@ -17,11 +17,11 @@
*/
package org.apache.hadoop.hbase.master.balancer;
import static org.apache.hadoop.hbase.ServerName.NON_STARTCODE;
import static org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper.FAVORED_NODES_NUM;
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.PRIMARY;
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.SECONDARY;
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.TERTIARY;
import static org.apache.hadoop.hbase.ServerName.NON_STARTCODE;
import java.io.IOException;
import java.util.ArrayList;
@ -35,9 +35,9 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
@ -107,20 +107,20 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* secondary and tertiary as per favored nodes constraints.
*/
@Override
public Map<ServerName, List<HRegionInfo>> roundRobinAssignment(List<HRegionInfo> regions,
public Map<ServerName, List<RegionInfo>> roundRobinAssignment(List<RegionInfo> regions,
List<ServerName> servers) throws HBaseIOException {
metricsBalancer.incrMiscInvocations();
Set<HRegionInfo> regionSet = Sets.newHashSet(regions);
Map<ServerName, List<HRegionInfo>> assignmentMap = assignMasterSystemRegions(regions, servers);
Set<RegionInfo> regionSet = Sets.newHashSet(regions);
Map<ServerName, List<RegionInfo>> assignmentMap = assignMasterSystemRegions(regions, servers);
if (assignmentMap != null && !assignmentMap.isEmpty()) {
servers = new ArrayList<>(servers);
// Guarantee not to put other regions on master
servers.remove(masterServerName);
List<HRegionInfo> masterRegions = assignmentMap.get(masterServerName);
List<RegionInfo> masterRegions = assignmentMap.get(masterServerName);
if (!masterRegions.isEmpty()) {
for (HRegionInfo region: masterRegions) {
for (RegionInfo region: masterRegions) {
regionSet.remove(region);
}
}
@ -135,18 +135,18 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
new FavoredNodeAssignmentHelper(servers, fnm.getRackManager());
helper.initialize();
Set<HRegionInfo> systemRegions = FavoredNodesManager.filterNonFNApplicableRegions(regionSet);
Set<RegionInfo> systemRegions = FavoredNodesManager.filterNonFNApplicableRegions(regionSet);
regionSet.removeAll(systemRegions);
// Assign all system regions
Map<ServerName, List<HRegionInfo>> systemAssignments =
Map<ServerName, List<RegionInfo>> systemAssignments =
super.roundRobinAssignment(Lists.newArrayList(systemRegions), servers);
// Segregate favored and non-favored nodes regions and assign accordingly.
Pair<Map<ServerName,List<HRegionInfo>>, List<HRegionInfo>> segregatedRegions =
Pair<Map<ServerName,List<RegionInfo>>, List<RegionInfo>> segregatedRegions =
segregateRegionsAndAssignRegionsWithFavoredNodes(regionSet, servers);
Map<ServerName, List<HRegionInfo>> regionsWithFavoredNodesMap = segregatedRegions.getFirst();
Map<ServerName, List<HRegionInfo>> regionsWithoutFN =
Map<ServerName, List<RegionInfo>> regionsWithFavoredNodesMap = segregatedRegions.getFirst();
Map<ServerName, List<RegionInfo>> regionsWithoutFN =
generateFNForRegionsWithoutFN(helper, segregatedRegions.getSecond());
// merge the assignment maps
@ -161,16 +161,16 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
return assignmentMap;
}
private void mergeAssignmentMaps(Map<ServerName, List<HRegionInfo>> assignmentMap,
Map<ServerName, List<HRegionInfo>> otherAssignments) {
private void mergeAssignmentMaps(Map<ServerName, List<RegionInfo>> assignmentMap,
Map<ServerName, List<RegionInfo>> otherAssignments) {
if (otherAssignments == null || otherAssignments.isEmpty()) {
return;
}
for (Entry<ServerName, List<HRegionInfo>> entry : otherAssignments.entrySet()) {
for (Entry<ServerName, List<RegionInfo>> entry : otherAssignments.entrySet()) {
ServerName sn = entry.getKey();
List<HRegionInfo> regionsList = entry.getValue();
List<RegionInfo> regionsList = entry.getValue();
if (assignmentMap.get(sn) == null) {
assignmentMap.put(sn, Lists.newArrayList(regionsList));
} else {
@ -179,11 +179,11 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
}
}
private Map<ServerName, List<HRegionInfo>> generateFNForRegionsWithoutFN(
FavoredNodeAssignmentHelper helper, List<HRegionInfo> regions) throws IOException {
private Map<ServerName, List<RegionInfo>> generateFNForRegionsWithoutFN(
FavoredNodeAssignmentHelper helper, List<RegionInfo> regions) throws IOException {
Map<ServerName, List<HRegionInfo>> assignmentMap = Maps.newHashMap();
Map<HRegionInfo, List<ServerName>> regionsNoFNMap;
Map<ServerName, List<RegionInfo>> assignmentMap = Maps.newHashMap();
Map<RegionInfo, List<ServerName>> regionsNoFNMap;
if (regions.size() > 0) {
regionsNoFNMap = helper.generateFavoredNodesRoundRobin(assignmentMap, regions);
@ -196,16 +196,16 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* Return a pair - one with assignments when favored nodes are present and another with regions
* without favored nodes.
*/
private Pair<Map<ServerName, List<HRegionInfo>>, List<HRegionInfo>>
segregateRegionsAndAssignRegionsWithFavoredNodes(Collection<HRegionInfo> regions,
private Pair<Map<ServerName, List<RegionInfo>>, List<RegionInfo>>
segregateRegionsAndAssignRegionsWithFavoredNodes(Collection<RegionInfo> regions,
List<ServerName> onlineServers) throws HBaseIOException {
// Since we expect FN to be present most of the time, lets create map with same size
Map<ServerName, List<HRegionInfo>> assignmentMapForFavoredNodes =
Map<ServerName, List<RegionInfo>> assignmentMapForFavoredNodes =
new HashMap<>(onlineServers.size());
List<HRegionInfo> regionsWithNoFavoredNodes = new ArrayList<>();
List<RegionInfo> regionsWithNoFavoredNodes = new ArrayList<>();
for (HRegionInfo region : regions) {
for (RegionInfo region : regions) {
List<ServerName> favoredNodes = fnm.getFavoredNodes(region);
ServerName primaryHost = null;
ServerName secondaryHost = null;
@ -235,10 +235,10 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
return new Pair<>(assignmentMapForFavoredNodes, regionsWithNoFavoredNodes);
}
private void addRegionToMap(Map<ServerName, List<HRegionInfo>> assignmentMapForFavoredNodes,
HRegionInfo region, ServerName host) {
private void addRegionToMap(Map<ServerName, List<RegionInfo>> assignmentMapForFavoredNodes,
RegionInfo region, ServerName host) {
List<HRegionInfo> regionsOnServer;
List<RegionInfo> regionsOnServer;
if ((regionsOnServer = assignmentMapForFavoredNodes.get(host)) == null) {
regionsOnServer = Lists.newArrayList();
assignmentMapForFavoredNodes.put(host, regionsOnServer);
@ -265,7 +265,7 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* available (in that order).
*/
private void assignRegionToAvailableFavoredNode(
Map<ServerName, List<HRegionInfo>> assignmentMapForFavoredNodes, HRegionInfo region,
Map<ServerName, List<RegionInfo>> assignmentMapForFavoredNodes, RegionInfo region,
ServerName primaryHost, ServerName secondaryHost, ServerName tertiaryHost) {
if (primaryHost != null) {
@ -304,7 +304,7 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* destination. If we can't generate anything, lets fallback.
*/
@Override
public ServerName randomAssignment(HRegionInfo regionInfo, List<ServerName> servers)
public ServerName randomAssignment(RegionInfo regionInfo, List<ServerName> servers)
throws HBaseIOException {
if (servers != null && servers.contains(masterServerName)) {
@ -354,9 +354,9 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
return destination;
}
private void updateFavoredNodesForRegion(HRegionInfo regionInfo, List<ServerName> newFavoredNodes)
private void updateFavoredNodesForRegion(RegionInfo regionInfo, List<ServerName> newFavoredNodes)
throws IOException {
Map<HRegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
Map<RegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
regionFNMap.put(regionInfo, newFavoredNodes);
fnm.updateFavoredNodes(regionFNMap);
}
@ -365,11 +365,11 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* Reuse BaseLoadBalancer's retainAssignment, but generate favored nodes when its missing.
*/
@Override
public Map<ServerName, List<HRegionInfo>> retainAssignment(Map<HRegionInfo, ServerName> regions,
public Map<ServerName, List<RegionInfo>> retainAssignment(Map<RegionInfo, ServerName> regions,
List<ServerName> servers) throws HBaseIOException {
Map<ServerName, List<HRegionInfo>> assignmentMap = Maps.newHashMap();
Map<ServerName, List<HRegionInfo>> result = super.retainAssignment(regions, servers);
Map<ServerName, List<RegionInfo>> assignmentMap = Maps.newHashMap();
Map<ServerName, List<RegionInfo>> result = super.retainAssignment(regions, servers);
if (result == null || result.isEmpty()) {
LOG.warn("Nothing to assign to, probably no servers or no regions");
return null;
@ -386,15 +386,15 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
helper.initialize();
LOG.debug("Generating favored nodes for regions missing them.");
Map<HRegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
Map<RegionInfo, List<ServerName>> regionFNMap = Maps.newHashMap();
try {
for (Entry<ServerName, List<HRegionInfo>> entry : result.entrySet()) {
for (Entry<ServerName, List<RegionInfo>> entry : result.entrySet()) {
ServerName sn = entry.getKey();
ServerName primary = ServerName.valueOf(sn.getHostname(), sn.getPort(), NON_STARTCODE);
for (HRegionInfo hri : entry.getValue()) {
for (RegionInfo hri : entry.getValue()) {
if (FavoredNodesManager.isFavoredNodeApplicable(hri)) {
List<ServerName> favoredNodes = fnm.getFavoredNodes(hri);
@ -472,7 +472,7 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
}
}
public synchronized List<ServerName> getFavoredNodes(HRegionInfo regionInfo) {
public synchronized List<ServerName> getFavoredNodes(RegionInfo regionInfo) {
return this.fnm.getFavoredNodes(regionInfo);
}
@ -487,10 +487,10 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* randomly. This would give us better distribution over a period of time after enough splits.
*/
@Override
public void generateFavoredNodesForDaughter(List<ServerName> servers, HRegionInfo parent,
HRegionInfo regionA, HRegionInfo regionB) throws IOException {
public void generateFavoredNodesForDaughter(List<ServerName> servers, RegionInfo parent,
RegionInfo regionA, RegionInfo regionB) throws IOException {
Map<HRegionInfo, List<ServerName>> result = new HashMap<>();
Map<RegionInfo, List<ServerName>> result = new HashMap<>();
FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper(servers, rackManager);
helper.initialize();
@ -542,8 +542,8 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* keep it simple.
*/
@Override
public void generateFavoredNodesForMergedRegion(HRegionInfo merged, HRegionInfo regionA,
HRegionInfo regionB) throws IOException {
public void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo regionA,
RegionInfo regionB) throws IOException {
updateFavoredNodesForRegion(merged, fnm.getFavoredNodes(regionA));
}
@ -572,7 +572,7 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
return Cluster.NullAction;
}
HRegionInfo hri = cluster.regions[thisRegion];
RegionInfo hri = cluster.regions[thisRegion];
List<ServerName> favoredNodes = fnm.getFavoredNodes(hri);
int otherServer;
if (favoredNodes == null) {
@ -628,7 +628,7 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
cluster.sortServersByRegionCount();
int thisServer = pickMostLoadedServer(cluster);
int thisRegion = pickRandomRegion(cluster, thisServer, 0);
HRegionInfo hri = cluster.regions[thisRegion];
RegionInfo hri = cluster.regions[thisRegion];
int otherServer;
List<ServerName> favoredNodes = fnm.getFavoredNodes(hri);
if (favoredNodes == null) {
@ -693,20 +693,20 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
* implementation. For the misplaced regions, we assign a bogus server to it and AM takes care.
*/
@Override
public List<RegionPlan> balanceCluster(Map<ServerName, List<HRegionInfo>> clusterState) {
public List<RegionPlan> balanceCluster(Map<ServerName, List<RegionInfo>> clusterState) {
if (this.services != null) {
List<RegionPlan> regionPlans = Lists.newArrayList();
Map<ServerName, List<HRegionInfo>> correctAssignments = new HashMap<>();
Map<ServerName, List<RegionInfo>> correctAssignments = new HashMap<>();
int misplacedRegions = 0;
for (Entry<ServerName, List<HRegionInfo>> entry : clusterState.entrySet()) {
for (Entry<ServerName, List<RegionInfo>> entry : clusterState.entrySet()) {
ServerName current = entry.getKey();
List<HRegionInfo> regions = Lists.newArrayList();
List<RegionInfo> regions = Lists.newArrayList();
correctAssignments.put(current, regions);
for (HRegionInfo hri : entry.getValue()) {
for (RegionInfo hri : entry.getValue()) {
List<ServerName> favoredNodes = fnm.getFavoredNodes(hri);
if (FavoredNodesPlan.getFavoredServerPosition(favoredNodes, current) != null ||
!FavoredNodesManager.isFavoredNodeApplicable(hri)) {

View File

@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.master.balancer;
import java.util.Comparator;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.RegionInfo;
/**
* The following comparator assumes that RegionId from HRegionInfo can represent
@ -27,9 +27,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
* comparator is used in balanceCluster() to account for the out-of-band regions
* which were assigned to the server after some other region server crashed.
*/
class RegionInfoComparator implements Comparator<HRegionInfo> {
class RegionInfoComparator implements Comparator<RegionInfo> {
@Override
public int compare(HRegionInfo l, HRegionInfo r) {
public int compare(RegionInfo l, RegionInfo r) {
long diff = r.getRegionId() - l.getRegionId();
if (diff < 0) return -1;
if (diff > 0) return 1;

View File

@ -33,15 +33,15 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.cache.CacheBuilder;
import org.apache.hadoop.hbase.shaded.com.google.common.cache.CacheLoader;
@ -71,10 +71,10 @@ class RegionLocationFinder {
// Do not scheduleFullRefresh at master startup
private long lastFullRefresh = EnvironmentEdgeManager.currentTime();
private CacheLoader<HRegionInfo, HDFSBlocksDistribution> loader =
new CacheLoader<HRegionInfo, HDFSBlocksDistribution>() {
private CacheLoader<RegionInfo, HDFSBlocksDistribution> loader =
new CacheLoader<RegionInfo, HDFSBlocksDistribution>() {
public ListenableFuture<HDFSBlocksDistribution> reload(final HRegionInfo hri,
public ListenableFuture<HDFSBlocksDistribution> reload(final RegionInfo hri,
HDFSBlocksDistribution oldValue) throws Exception {
return executor.submit(new Callable<HDFSBlocksDistribution>() {
@Override
@ -85,13 +85,13 @@ class RegionLocationFinder {
}
@Override
public HDFSBlocksDistribution load(HRegionInfo key) throws Exception {
public HDFSBlocksDistribution load(RegionInfo key) throws Exception {
return internalGetTopBlockLocation(key);
}
};
// The cache for where regions are located.
private LoadingCache<HRegionInfo, HDFSBlocksDistribution> cache = null;
private LoadingCache<RegionInfo, HDFSBlocksDistribution> cache = null;
RegionLocationFinder() {
this.cache = createCache();
@ -109,7 +109,7 @@ class RegionLocationFinder {
* @param time time to cache the locations
* @return A new Cache.
*/
private LoadingCache<HRegionInfo, HDFSBlocksDistribution> createCache() {
private LoadingCache<RegionInfo, HDFSBlocksDistribution> createCache() {
return CacheBuilder.newBuilder()
.expireAfterWrite(CACHE_TIME, TimeUnit.MILLISECONDS)
.build(loader);
@ -155,14 +155,14 @@ class RegionLocationFinder {
// TODO: Should this refresh all the regions or only the ones assigned?
boolean includesUserTables = false;
for (final HRegionInfo hri : am.getAssignedRegions()) {
for (final RegionInfo hri : am.getAssignedRegions()) {
cache.refresh(hri);
includesUserTables = includesUserTables || !hri.isSystemTable();
}
return includesUserTables;
}
protected List<ServerName> getTopBlockLocations(HRegionInfo region) {
protected List<ServerName> getTopBlockLocations(RegionInfo region) {
List<String> topHosts = getBlockDistribution(region).getTopHosts();
return mapHostNameToServerName(topHosts);
}
@ -171,7 +171,7 @@ class RegionLocationFinder {
* Returns an ordered list of hosts which have better locality for this region
* than the current host.
*/
protected List<ServerName> getTopBlockLocations(HRegionInfo region, String currentHost) {
protected List<ServerName> getTopBlockLocations(RegionInfo region, String currentHost) {
HDFSBlocksDistribution blocksDistribution = getBlockDistribution(region);
List<String> topHosts = new ArrayList<>();
for (String host : blocksDistribution.getTopHosts()) {
@ -192,7 +192,7 @@ class RegionLocationFinder {
* @param region region
* @return ordered list of hosts holding blocks of the specified region
*/
protected HDFSBlocksDistribution internalGetTopBlockLocation(HRegionInfo region) {
protected HDFSBlocksDistribution internalGetTopBlockLocation(RegionInfo region) {
try {
TableDescriptor tableDescriptor = getTableDescriptor(region.getTable());
if (tableDescriptor != null) {
@ -272,7 +272,7 @@ class RegionLocationFinder {
return topServerNames;
}
public HDFSBlocksDistribution getBlockDistribution(HRegionInfo hri) {
public HDFSBlocksDistribution getBlockDistribution(RegionInfo hri) {
HDFSBlocksDistribution blockDistbn = null;
try {
if (cache.asMap().containsKey(hri)) {
@ -294,7 +294,7 @@ class RegionLocationFinder {
}
private ListenableFuture<HDFSBlocksDistribution> asyncGetBlockDistribution(
HRegionInfo hri) {
RegionInfo hri) {
try {
return loader.reload(hri, EMPTY_BLOCK_DISTRIBUTION);
} catch (Exception e) {
@ -302,13 +302,13 @@ class RegionLocationFinder {
}
}
public void refreshAndWait(Collection<HRegionInfo> hris) {
public void refreshAndWait(Collection<RegionInfo> hris) {
ArrayList<ListenableFuture<HDFSBlocksDistribution>> regionLocationFutures = new ArrayList<>(hris.size());
for (HRegionInfo hregionInfo : hris) {
for (RegionInfo hregionInfo : hris) {
regionLocationFutures.add(asyncGetBlockDistribution(hregionInfo));
}
int index = 0;
for (HRegionInfo hregionInfo : hris) {
for (RegionInfo hregionInfo : hris) {
ListenableFuture<HDFSBlocksDistribution> future = regionLocationFutures
.get(index);
try {
@ -325,7 +325,7 @@ class RegionLocationFinder {
}
// For test
LoadingCache<HRegionInfo, HDFSBlocksDistribution> getCache() {
LoadingCache<RegionInfo, HDFSBlocksDistribution> getCache() {
return cache;
}
}

View File

@ -33,12 +33,12 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.MinMaxPriorityQueue;
@ -77,9 +77,9 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
private int nextRegionForUnload;
private int numRegionsAdded;
private List<HRegionInfo> hriList;
private List<RegionInfo> hriList;
public BalanceInfo(int nextRegionForUnload, int numRegionsAdded, List<HRegionInfo> hriList) {
public BalanceInfo(int nextRegionForUnload, int numRegionsAdded, List<RegionInfo> hriList) {
this.nextRegionForUnload = nextRegionForUnload;
this.numRegionsAdded = numRegionsAdded;
this.hriList = hriList;
@ -97,7 +97,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
this.numRegionsAdded = numAdded;
}
List<HRegionInfo> getHriList() {
List<RegionInfo> getHriList() {
return hriList;
}
@ -107,11 +107,11 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
}
public void setClusterLoad(Map<TableName, Map<ServerName, List<HRegionInfo>>> clusterLoad){
public void setClusterLoad(Map<TableName, Map<ServerName, List<RegionInfo>>> clusterLoad){
serverLoadList = new ArrayList<>();
float sum = 0;
for(Map.Entry<TableName, Map<ServerName, List<HRegionInfo>>> clusterEntry : clusterLoad.entrySet()){
for(Map.Entry<ServerName, List<HRegionInfo>> entry : clusterEntry.getValue().entrySet()){
for(Map.Entry<TableName, Map<ServerName, List<RegionInfo>>> clusterEntry : clusterLoad.entrySet()){
for(Map.Entry<ServerName, List<RegionInfo>> entry : clusterEntry.getValue().entrySet()){
if(entry.getKey().equals(masterServerName)) continue; // we shouldn't include master as potential assignee
serverLoadList.add(new ServerAndLoad(entry.getKey(), entry.getValue().size()));
sum += entry.getValue().size();
@ -245,7 +245,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
*/
@Override
public List<RegionPlan> balanceCluster(
Map<ServerName, List<HRegionInfo>> clusterMap) {
Map<ServerName, List<RegionInfo>> clusterMap) {
List<RegionPlan> regionsToReturn = balanceMasterRegions(clusterMap);
if (regionsToReturn != null || clusterMap == null || clusterMap.size() <= 1) {
return regionsToReturn;
@ -267,7 +267,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
ClusterLoadState cs = new ClusterLoadState(clusterMap);
int numServers = cs.getNumServers();
NavigableMap<ServerAndLoad, List<HRegionInfo>> serversByLoad = cs.getServersByLoad();
NavigableMap<ServerAndLoad, List<RegionInfo>> serversByLoad = cs.getServersByLoad();
int numRegions = cs.getNumRegions();
float average = cs.getLoadAverage();
int max = (int)Math.ceil(average);
@ -291,7 +291,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
// flag used to fetch regions from head and tail of list, alternately
boolean fetchFromTail = false;
Map<ServerName, BalanceInfo> serverBalanceInfo = new TreeMap<>();
for (Map.Entry<ServerAndLoad, List<HRegionInfo>> server:
for (Map.Entry<ServerAndLoad, List<RegionInfo>> server:
serversByLoad.descendingMap().entrySet()) {
ServerAndLoad sal = server.getKey();
int load = sal.getLoad();
@ -300,14 +300,14 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
continue;
}
serversOverloaded++;
List<HRegionInfo> regions = server.getValue();
List<RegionInfo> regions = server.getValue();
int numToOffload = Math.min(load - max, regions.size());
// account for the out-of-band regions which were assigned to this server
// after some other region server crashed
Collections.sort(regions, riComparator);
int numTaken = 0;
for (int i = 0; i <= numToOffload; ) {
HRegionInfo hri = regions.get(i); // fetch from head
RegionInfo hri = regions.get(i); // fetch from head
if (fetchFromTail) {
hri = regions.get(regions.size() - 1 - i);
}
@ -330,7 +330,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
Map<ServerName, Integer> underloadedServers = new HashMap<>();
int maxToTake = numRegions - min;
for (Map.Entry<ServerAndLoad, List<HRegionInfo>> server:
for (Map.Entry<ServerAndLoad, List<RegionInfo>> server:
serversByLoad.entrySet()) {
if (maxToTake == 0) break; // no more to take
int load = server.getKey().getLoad();
@ -378,14 +378,14 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
// If we need more to fill min, grab one from each most loaded until enough
if (neededRegions != 0) {
// Walk down most loaded, grabbing one from each until we get enough
for (Map.Entry<ServerAndLoad, List<HRegionInfo>> server :
for (Map.Entry<ServerAndLoad, List<RegionInfo>> server :
serversByLoad.descendingMap().entrySet()) {
BalanceInfo balanceInfo =
serverBalanceInfo.get(server.getKey().getServerName());
int idx =
balanceInfo == null ? 0 : balanceInfo.getNextRegionForUnload();
if (idx >= server.getValue().size()) break;
HRegionInfo region = server.getValue().get(idx);
RegionInfo region = server.getValue().get(idx);
if (region.isMetaRegion()) continue; // Don't move meta regions.
regionsToMove.add(new RegionPlan(region, server.getKey().getServerName(), null));
balanceInfo.setNumRegionsAdded(balanceInfo.getNumRegionsAdded() - 1);
@ -402,7 +402,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
// Assign each underloaded up to the min, then if leftovers, assign to max
// Walk down least loaded, assigning to each to fill up to min
for (Map.Entry<ServerAndLoad, List<HRegionInfo>> server :
for (Map.Entry<ServerAndLoad, List<RegionInfo>> server :
serversByLoad.entrySet()) {
int regionCount = server.getKey().getLoad();
if (regionCount >= min) break;
@ -434,7 +434,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
", numServers=" + numServers + ", serversOverloaded=" + serversOverloaded +
", serversUnderloaded=" + serversUnderloaded);
StringBuilder sb = new StringBuilder();
for (Map.Entry<ServerName, List<HRegionInfo>> e: clusterMap.entrySet()) {
for (Map.Entry<ServerName, List<RegionInfo>> e: clusterMap.entrySet()) {
if (sb.length() > 0) sb.append(", ");
sb.append(e.getKey().toString());
sb.append(" ");
@ -481,10 +481,10 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
BalanceInfo balanceInfo = serverBalanceInfo.get(serverload.getServerName());
setLoad(serverLoadList, i, balanceInfo.getNumRegionsAdded());
if (balanceInfo.getHriList().size() + balanceInfo.getNumRegionsAdded() == max) {
HRegionInfo hriToPlan;
RegionInfo hriToPlan;
if (balanceInfo.getHriList().isEmpty()) {
LOG.debug("During balanceOverall, we found " + serverload.getServerName()
+ " has no HRegionInfo, no operation needed");
+ " has no RegionInfo, no operation needed");
continue;
} else if (balanceInfo.getNextRegionForUnload() >= balanceInfo.getHriList().size()) {
continue;
@ -587,7 +587,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
@Override
public List<RegionPlan> balanceCluster(TableName tableName,
Map<ServerName, List<HRegionInfo>> clusterState) throws HBaseIOException {
Map<ServerName, List<RegionInfo>> clusterState) throws HBaseIOException {
LOG.debug("Start Generate Balance plan for table: " + tableName);
return balanceCluster(clusterState);
}

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hbase.master.balancer;
import com.google.common.annotations.VisibleForTesting;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Arrays;
@ -38,11 +36,11 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action;
@ -58,6 +56,8 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Optional;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import com.google.common.annotations.VisibleForTesting;
/**
* <p>This is a best effort load balancer. Given a Cost function F(C) =&gt; x It will
* randomly try and mutate the cluster to Cprime. If F(Cprime) &lt; F(C) then the
@ -315,7 +315,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
@Override
public synchronized List<RegionPlan> balanceCluster(TableName tableName, Map<ServerName,
List<HRegionInfo>> clusterState) {
List<RegionInfo>> clusterState) {
this.tableName = tableName;
return balanceCluster(clusterState);
}
@ -332,7 +332,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
*/
@Override
public synchronized List<RegionPlan> balanceCluster(Map<ServerName,
List<HRegionInfo>> clusterState) {
List<RegionInfo>> clusterState) {
List<RegionPlan> plans = balanceMasterRegions(clusterState);
if (plans != null || clusterState == null || clusterState.size() <= 1) {
return plans;
@ -504,7 +504,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
int newServerIndex = cluster.regionIndexToServerIndex[regionIndex];
if (initialServerIndex != newServerIndex) {
HRegionInfo region = cluster.regions[regionIndex];
RegionInfo region = cluster.regions[regionIndex];
ServerName initialServer = cluster.servers[initialServerIndex];
ServerName newServer = cluster.servers[newServerIndex];
@ -624,7 +624,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
* @param server index of the server
* @param chanceOfNoSwap Chance that this will decide to try a move rather
* than a swap.
* @return a random {@link HRegionInfo} or null if an asymmetrical move is
* @return a random {@link RegionInfo} or null if an asymmetrical move is
* suggested.
*/
protected int pickRandomRegion(Cluster cluster, int server, double chanceOfNoSwap) {

View File

@ -24,13 +24,14 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.procedure2.LockType;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.util.NonceKey;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* Functions to acquire lock on table/namespace/regions.
@ -60,7 +61,7 @@ public final class LockManager {
return new MasterLock(tableName, type, description);
}
public MasterLock createMasterLock(final HRegionInfo[] regionInfos, final String description) {
public MasterLock createMasterLock(final RegionInfo[] regionInfos, final String description) {
return new MasterLock(regionInfos, description);
}
@ -81,7 +82,7 @@ public final class LockManager {
public class MasterLock {
private final String namespace;
private final TableName tableName;
private final HRegionInfo[] regionInfos;
private final RegionInfo[] regionInfos;
private final LockType type;
private final String description;
@ -105,7 +106,7 @@ public final class LockManager {
this.description = description;
}
public MasterLock(final HRegionInfo[] regionInfos, final String description) {
public MasterLock(final RegionInfo[] regionInfos, final String description) {
this.namespace = null;
this.tableName = null;
this.regionInfos = regionInfos;
@ -229,7 +230,7 @@ public final class LockManager {
/**
* @throws IllegalArgumentException if all regions are not from same table.
*/
public long requestRegionsLock(final HRegionInfo[] regionInfos, final String description,
public long requestRegionsLock(final RegionInfo[] regionInfos, final String description,
final NonceKey nonceKey)
throws IllegalArgumentException, IOException {
master.getMasterCoprocessorHost().preRequestLock(null, null, regionInfos,

View File

@ -19,12 +19,16 @@
package org.apache.hadoop.hbase.master.locking;
import java.io.IOException;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
import org.apache.hadoop.hbase.procedure2.LockType;
@ -32,16 +36,13 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockProcedureData;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
import java.io.IOException;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
/**
* Procedure to allow blessed clients and external admin tools to take our internal Schema locks
* used by the procedure framework isolating procedures doing creates/deletes etc. on
@ -68,7 +69,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
private String namespace;
private TableName tableName;
private HRegionInfo[] regionInfos;
private RegionInfo[] regionInfos;
private LockType type;
// underlying namespace/table/region lock.
private LockInterface lock;
@ -160,12 +161,12 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
* Useful for locks acquired locally from master process.
* @throws IllegalArgumentException if all regions are not from same table.
*/
public LockProcedure(final Configuration conf, final HRegionInfo[] regionInfos,
public LockProcedure(final Configuration conf, final RegionInfo[] regionInfos,
final LockType type, final String description, final CountDownLatch lockAcquireLatch)
throws IllegalArgumentException {
this(conf, type, description, lockAcquireLatch);
// Build HRegionInfo from region names.
// Build RegionInfo from region names.
if (regionInfos.length == 0) {
throw new IllegalArgumentException("No regions specified for region lock");
}
@ -269,7 +270,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
.setDescription(description);
if (regionInfos != null) {
for (int i = 0; i < regionInfos.length; ++i) {
builder.addRegionInfo(HRegionInfo.convert(regionInfos[i]));
builder.addRegionInfo(ProtobufUtil.toRegionInfo(regionInfos[i]));
}
} else if (namespace != null) {
builder.setNamespace(namespace);
@ -289,9 +290,9 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
type = LockType.valueOf(state.getLockType().name());
description = state.getDescription();
if (state.getRegionInfoCount() > 0) {
regionInfos = new HRegionInfo[state.getRegionInfoCount()];
regionInfos = new RegionInfo[state.getRegionInfoCount()];
for (int i = 0; i < state.getRegionInfoCount(); ++i) {
regionInfos[i] = HRegionInfo.convert(state.getRegionInfo(i));
regionInfos[i] = ProtobufUtil.toRegionInfo(state.getRegionInfo(i));
}
} else if (state.hasNamespace()) {
namespace = state.getNamespace();

View File

@ -18,14 +18,13 @@
*/
package org.apache.hadoop.hbase.master.normalizer;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
import java.io.IOException;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Normalization plan to merge regions (smallest region in the table with its smallest neighbor).
@ -34,10 +33,10 @@ import java.io.IOException;
public class MergeNormalizationPlan implements NormalizationPlan {
private static final Log LOG = LogFactory.getLog(MergeNormalizationPlan.class.getName());
private final HRegionInfo firstRegion;
private final HRegionInfo secondRegion;
private final RegionInfo firstRegion;
private final RegionInfo secondRegion;
public MergeNormalizationPlan(HRegionInfo firstRegion, HRegionInfo secondRegion) {
public MergeNormalizationPlan(RegionInfo firstRegion, RegionInfo secondRegion) {
this.firstRegion = firstRegion;
this.secondRegion = secondRegion;
}
@ -47,11 +46,11 @@ public class MergeNormalizationPlan implements NormalizationPlan {
return PlanType.MERGE;
}
HRegionInfo getFirstRegion() {
RegionInfo getFirstRegion() {
return firstRegion;
}
HRegionInfo getSecondRegion() {
RegionInfo getSecondRegion() {
return secondRegion;
}

View File

@ -21,12 +21,12 @@ package org.apache.hadoop.hbase.master.normalizer;
import java.util.List;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterRpcServices;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Performs "normalization" of regions on the cluster, making sure that suboptimal
@ -68,8 +68,8 @@ public interface RegionNormalizer {
* @param hri the region which is involved in the plan
* @param type type of plan
*/
void planSkipped(HRegionInfo hri, PlanType type);
void planSkipped(RegionInfo hri, PlanType type);
/**
* @param type type of plan for which skipped count is to be returned
* @return the count of plans of specified type which were skipped

View File

@ -26,15 +26,16 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterRpcServices;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
/**
@ -80,7 +81,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
}
@Override
public void planSkipped(HRegionInfo hri, PlanType type) {
public void planSkipped(RegionInfo hri, PlanType type) {
skippedCount[type.ordinal()]++;
}
@ -119,7 +120,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
}
List<NormalizationPlan> plans = new ArrayList<>();
List<HRegionInfo> tableRegions = masterServices.getAssignmentManager().getRegionStates().
List<RegionInfo> tableRegions = masterServices.getAssignmentManager().getRegionStates().
getRegionsOfTable(table);
//TODO: should we make min number of regions a config param?
@ -137,7 +138,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
int acutalRegionCnt = 0;
for (int i = 0; i < tableRegions.size(); i++) {
HRegionInfo hri = tableRegions.get(i);
RegionInfo hri = tableRegions.get(i);
long regionSize = getRegionSize(hri);
if (regionSize > 0) {
acutalRegionCnt++;
@ -165,7 +166,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
LOG.debug("Unable to determine whether split is enabled", e);
}
while (candidateIdx < tableRegions.size()) {
HRegionInfo hri = tableRegions.get(candidateIdx);
RegionInfo hri = tableRegions.get(candidateIdx);
long regionSize = getRegionSize(hri);
// if the region is > 2 times larger than average, we split it, split
// is more high priority normalization action than merge.
@ -180,7 +181,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
break;
}
if (mergeEnabled) {
HRegionInfo hri2 = tableRegions.get(candidateIdx+1);
RegionInfo hri2 = tableRegions.get(candidateIdx+1);
long regionSize2 = getRegionSize(hri2);
if (regionSize >= 0 && regionSize2 >= 0 && regionSize + regionSize2 < avgRegionSize) {
LOG.info("Table " + table + ", small region size: " + regionSize
@ -201,7 +202,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
return plans;
}
private long getRegionSize(HRegionInfo hri) {
private long getRegionSize(RegionInfo hri) {
ServerName sn = masterServices.getAssignmentManager().getRegionStates().
getRegionServerOfRegion(hri);
RegionLoad regionLoad = masterServices.getServerManager().getLoad(sn).

Some files were not shown because too many files have changed in this diff Show More