HBASE-10347 HRegionInfo changes for adding replicaId and MetaEditor/MetaReader changes for region replicas

git-svn-id: https://svn.apache.org/repos/asf/hbase/branches/hbase-10070@1565041 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Enis Soztutar 2014-02-06 02:04:53 +00:00
parent e04009c989
commit d4b82224cc
28 changed files with 2242 additions and 499 deletions

View File

@ -29,7 +29,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -47,15 +46,34 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.io.DataInputBuffer;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* HRegion information.
* Contains HRegion id, start and end keys, a reference to this HRegions' table descriptor, etc.
* Information about a region. A region is a range of keys in the whole keyspace of a table, an
* identifier (a timestamp) for differentiating between subset ranges (after region split)
* and a replicaId for differentiating the instance for the same range and some status information
* about the region.
*
* On a big cluster, each client will have thousands of instances of this object, often
* 100 000 of them if not million. It's important to keep the object size as small
* as possible.
* The region has a unique name which consists of the following fields:
* <li> tableName : The name of the table </li>
* <li> startKey : The startKey for the region. </li>
* <li> regionId : A timestamp when the region is created. </li>
* <li> replicaId : An id starting from 0 to differentiate replicas of the same region range
* but hosted in separated servers. The same region range can be hosted in multiple locations.</li>
* <li> encodedName : An MD5 encoded string for the region name.</li>
*
* <br> Other than the fields in the region name, region info contains:
* <li> endKey : the endKey for the region (exclusive) </li>
* <li> split : Whether the region is split </li>
* <li> offline : Whether the region is offline </li>
*
* In 0.98 or before, a list of table's regions would fully cover the total keyspace, and at any
* point in time, a row key always belongs to a single region, which is hosted in a single server.
* In 0.99+, a region can have multiple instances (called replicas), and thus a range (or row) can
* correspond to multiple HRegionInfo's. These HRI's share the same fields however except the
* replicaId field. If the replicaId is not set, it defaults to 0, which is compatible with the
* previous behavior of a range corresponding to 1 region.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@ -118,6 +136,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
/** A non-capture group so that this can be embedded. */
public static final String ENCODED_REGION_NAME_REGEX = "(?:[a-f0-9]+)";
// to keep appended int's sorted in string format. Only allows 2 bytes to be
// sorted for replicaId
public static final String REPLICA_ID_FORMAT = "%04X";
public static final byte REPLICA_ID_DELIMITER = (byte)'_';
private static final int MAX_REPLICA_ID = 0xFFFF;
private static final int DEFAULT_REPLICA_ID = 0;
/**
* Does region name contain its encoded name?
* @param regionName region name
@ -190,6 +216,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
public static final String NO_HASH = null;
private String encodedName = null;
private byte [] encodedNameAsBytes = null;
private int replicaId = DEFAULT_REPLICA_ID;
// Current TableName
private TableName tableName = null;
@ -205,6 +232,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
result ^= Arrays.hashCode(this.endKey);
result ^= Boolean.valueOf(this.offLine).hashCode();
result ^= Arrays.hashCode(this.tableName.getName());
result ^= this.replicaId;
this.hashCode = result;
}
@ -248,7 +276,6 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
this(tableName, startKey, endKey, false);
}
/**
* Construct HRegionInfo with explicit parameters
*
@ -265,7 +292,6 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
this(tableName, startKey, endKey, split, System.currentTimeMillis());
}
/**
* Construct HRegionInfo with explicit parameters
*
@ -280,7 +306,25 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
public HRegionInfo(final TableName tableName, final byte[] startKey,
final byte[] endKey, final boolean split, final long regionid)
throws IllegalArgumentException {
this(tableName, startKey, endKey, split, regionid, DEFAULT_REPLICA_ID);
}
/**
* Construct HRegionInfo 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
*/
public HRegionInfo(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");
@ -288,8 +332,12 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
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 = createRegionName(this.tableName, startKey, regionId, true);
this.regionName = createRegionName(this.tableName, startKey, regionId, replicaId, true);
this.split = split;
this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone();
@ -315,8 +363,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
this.hashCode = other.hashCode();
this.encodedName = other.getEncodedName();
this.tableName = other.tableName;
this.replicaId = other.replicaId;
}
public HRegionInfo(HRegionInfo other, int replicaId) {
this(other);
this.replicaId = replicaId;
this.setHashCode();
}
/**
* Make a region name of passed parameters.
@ -346,6 +400,22 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
return createRegionName(tableName, startKey, Bytes.toBytes(id), newFormat);
}
/**
* Make a region name of passed parameters.
* @param tableName
* @param startKey Can be null
* @param regionid Region id (Usually timestamp from when region was created).
* @param replicaId
* @param newFormat should we create the region name in the new format
* (such that it contains its encoded name?).
* @return Region name made of passed tableName, startKey, id and replicaId
*/
public static byte [] createRegionName(final TableName tableName,
final byte [] startKey, final long regionid, int replicaId, boolean newFormat) {
return createRegionName(tableName, startKey, Bytes.toBytes(Long.toString(regionid)),
replicaId, newFormat);
}
/**
* Make a region name of passed parameters.
* @param tableName
@ -357,9 +427,35 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
*/
public static byte [] createRegionName(final TableName tableName,
final byte [] startKey, final byte [] id, boolean newFormat) {
byte [] b = new byte [tableName.getName().length + 2 + id.length +
(startKey == null? 0: startKey.length) +
(newFormat ? (MD5_HEX_LENGTH + 2) : 0)];
return createRegionName(tableName, startKey, id, DEFAULT_REPLICA_ID, newFormat);
}
/**
* Make a region name of passed parameters.
* @param tableName
* @param startKey Can be null
* @param id Region id (Usually timestamp from when region was created).
* @param replicaId
* @param newFormat should we create the region name in the new format
* @return Region name made of passed tableName, startKey, id and replicaId
*/
public static byte [] createRegionName(final TableName tableName,
final byte [] startKey, final byte [] id, final int replicaId, boolean newFormat) {
int len = tableName.getName().length + 2 + id.length +
(startKey == null? 0: startKey.length);
if (newFormat) {
len += MD5_HEX_LENGTH + 2;
}
byte[] replicaIdBytes = null;
// Special casing: replicaId is only appended if replicaId is greater than
// 0. This is because all regions in meta would have to be migrated to the new
// name otherwise
if (replicaId > 0) {
// use string representation for replica id
replicaIdBytes = Bytes.toBytes(String.format(REPLICA_ID_FORMAT, replicaId));
len += 1 + replicaIdBytes.length;
}
byte [] b = new byte [len];
int offset = tableName.getName().length;
System.arraycopy(tableName.getName(), 0, b, 0, offset);
@ -372,11 +468,17 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
System.arraycopy(id, 0, b, offset, id.length);
offset += id.length;
if (replicaIdBytes != null) {
b[offset++] = REPLICA_ID_DELIMITER;
System.arraycopy(replicaIdBytes, 0, b, offset, replicaIdBytes.length);
offset += replicaIdBytes.length;
}
if (newFormat) {
//
// Encoded name should be built into the region name.
//
// Use the region name thus far (namely, <tablename>,<startKey>,<id>)
// Use the region name thus far (namely, <tablename>,<startKey>,<id>_<replicaId>)
// to compute a MD5 hash to be used as the encoded name, and append
// it to the byte buffer.
//
@ -447,6 +549,11 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
*/
public static byte [][] parseRegionName(final byte [] regionName)
throws IOException {
// Region name is of the format:
// tablename,startkey,regionIdTimestamp[_replicaId][.encodedName.]
// startkey can contain the delimiter (',') so we parse from the start and end
// parse from start
int offset = -1;
for (int i = 0; i < regionName.length; i++) {
if (regionName[i] == HConstants.DELIMITER) {
@ -458,8 +565,27 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
byte[] tableName = new byte[offset];
System.arraycopy(regionName, 0, tableName, 0, offset);
offset = -1;
for (int i = regionName.length - 1; i > 0; i--) {
if(regionName[i] == HConstants.DELIMITER) {
int endOffset = regionName.length;
// check whether regionName contains encodedName
if (regionName.length > MD5_HEX_LENGTH + 2
&& regionName[regionName.length-1] == ENC_SEPARATOR
&& regionName[regionName.length-MD5_HEX_LENGTH-2] == ENC_SEPARATOR) {
endOffset = endOffset - MD5_HEX_LENGTH - 2;
}
// parse from end
byte[] replicaId = null;
int idEndOffset = endOffset;
for (int i = endOffset - 1; i > 0; i--) {
if (regionName[i] == REPLICA_ID_DELIMITER) { //replicaId may or may not be present
replicaId = new byte[endOffset - i - 1];
System.arraycopy(regionName, i + 1, replicaId, 0,
endOffset - i - 1);
idEndOffset = i;
// do not break, continue to search for id
}
if (regionName[i] == HConstants.DELIMITER) {
offset = i;
break;
}
@ -471,13 +597,17 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
System.arraycopy(regionName, tableName.length + 1, startKey, 0,
offset - tableName.length - 1);
}
byte [] id = new byte[regionName.length - offset - 1];
byte [] id = new byte[idEndOffset - offset - 1];
System.arraycopy(regionName, offset + 1, id, 0,
regionName.length - offset - 1);
byte [][] elements = new byte[3][];
idEndOffset - offset - 1);
byte [][] elements = new byte[replicaId == null ? 3 : 4][];
elements[0] = tableName;
elements[1] = startKey;
elements[2] = id;
if (replicaId != null) {
elements[3] = replicaId;
}
return elements;
}
@ -631,7 +761,6 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
this.offLine = offLine;
}
/**
* @return True if this is a split parent region.
*/
@ -643,6 +772,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
return true;
}
/**
* Returns the region replica id
* @return returns region replica id
*/
public int getReplicaId() {
return replicaId;
}
/**
* @see java.lang.Object#toString()
*/
@ -654,7 +791,8 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
Bytes.toStringBinary(this.startKey) + "', ENDKEY => '" +
Bytes.toStringBinary(this.endKey) + "'" +
(isOffline()? ", OFFLINE => true": "") +
(isSplit()? ", SPLIT => true": "") + "}";
(isSplit()? ", SPLIT => true": "") +
((replicaId > 0)? ", REPLICA_ID => " + replicaId : "") + "}";
}
/**
@ -766,6 +904,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
// Comparable
//
@Override
public int compareTo(HRegionInfo o) {
if (o == null) {
return 1;
@ -806,6 +945,9 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
return -1;
}
int replicaDiff = this.getReplicaId() - o.getReplicaId();
if (replicaDiff != 0) return replicaDiff;
if (this.offLine == o.offLine)
return 0;
if (this.offLine == true) return -1;
@ -849,6 +991,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
}
builder.setOffline(info.isOffline());
builder.setSplit(info.isSplit());
builder.setReplicaId(info.getReplicaId());
return builder.build();
}
@ -866,6 +1009,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
return FIRST_META_REGIONINFO;
}
long regionId = proto.getRegionId();
int replicaId = proto.hasReplicaId() ? proto.getReplicaId() : DEFAULT_REPLICA_ID;
byte[] startKey = null;
byte[] endKey = null;
if (proto.hasStartKey()) {
@ -881,7 +1025,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
HRegionInfo hri = new HRegionInfo(
tableName,
startKey,
endKey, split, regionId);
endKey, split, regionId, replicaId);
if (proto.hasOffline()) {
hri.setOffline(proto.getOffline());
}
@ -980,7 +1124,9 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* @return A pair of the {@link HRegionInfo} and the {@link ServerName}
* (or null for server address if no address set in hbase:meta).
* @throws IOException
* @deprecated use MetaReader methods for interacting with meta layouts
*/
@Deprecated
public static Pair<HRegionInfo, ServerName> getHRegionInfoAndServerName(final Result r) {
HRegionInfo info =
getHRegionInfo(r, HConstants.REGIONINFO_QUALIFIER);
@ -994,7 +1140,9 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* table Result.
* @param data a Result object from the catalog table scan
* @return HRegionInfo or null
* @deprecated use MetaReader methods for interacting with meta layouts
*/
@Deprecated
public static HRegionInfo getHRegionInfo(Result data) {
return getHRegionInfo(data, HConstants.REGIONINFO_QUALIFIER);
}
@ -1005,21 +1153,25 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* @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
* parent
* @deprecated use MetaReader methods for interacting with meta layouts
*/
@Deprecated
public static PairOfSameType<HRegionInfo> getDaughterRegions(Result data) throws IOException {
HRegionInfo splitA = getHRegionInfo(data, HConstants.SPLITA_QUALIFIER);
HRegionInfo splitB = getHRegionInfo(data, HConstants.SPLITB_QUALIFIER);
return new PairOfSameType<HRegionInfo>(splitA, splitB);
}
/**
* 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
* parent
* @deprecated use MetaReader methods for interacting with meta layouts
*/
@Deprecated
public static PairOfSameType<HRegionInfo> getMergeRegions(Result data) throws IOException {
HRegionInfo mergeA = getHRegionInfo(data, HConstants.MERGEA_QUALIFIER);
HRegionInfo mergeB = getHRegionInfo(data, HConstants.MERGEB_QUALIFIER);
@ -1035,7 +1187,9 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* {@link HConstants#SPLITA_QUALIFIER}, {@link HConstants#SPLITB_QUALIFIER} or
* {@link HConstants#REGIONINFO_QUALIFIER}.
* @return An HRegionInfo instance or null.
* @deprecated use MetaReader methods for interacting with meta layouts
*/
@Deprecated
public static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier) {
Cell cell = r.getColumnLatestCell(
HConstants.CATALOG_FAMILY, qualifier);
@ -1044,11 +1198,9 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
}
/**
* Returns a {@link ServerName} from catalog table {@link Result}.
* @param r Result to pull from
* @return A ServerName instance or null if necessary fields not found or empty.
* @deprecated use MetaReader methods for interacting with meta layouts
*/
// TODO: ServerName should be a better place for this method?
@Deprecated
public static ServerName getServerName(final Result r) {
Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
if (cell == null || cell.getValueLength() == 0) return null;
@ -1071,7 +1223,9 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* E.g. the seqNum when the result of {@link #getServerName(Result)} was written.
* @param r Result to pull the seqNum from
* @return SeqNum, or HConstants.NO_SEQNUM if there's no value written.
* @deprecated use MetaReader methods for interacting with meta layouts
*/
@Deprecated
public static long getSeqNumDuringOpen(final Result r) {
Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY, HConstants.SEQNUM_QUALIFIER);
if (cell == null || cell.getValueLength() == 0) return HConstants.NO_SEQNUM;
@ -1191,5 +1345,4 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
}
return false;
}
}

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hbase;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.util.Addressing;
/**
@ -32,9 +31,10 @@ import org.apache.hadoop.hbase.util.Addressing;
* On a big cluster, each client will have thousands of instances of this object, often
* 100 000 of them if not million. It's important to keep the object size as small
* as possible.
* <br>This interface has been marked InterfaceAudience.Public in 0.96 and 0.98, it is
* no longer the case.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class HRegionLocation implements Comparable<HRegionLocation> {
private final HRegionInfo regionInfo;
private final ServerName serverName;
@ -112,6 +112,7 @@ public class HRegionLocation implements Comparable<HRegionLocation> {
return serverName;
}
@Override
public int compareTo(HRegionLocation o) {
return serverName.compareTo(o.getServerName());
}

View File

@ -0,0 +1,291 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import java.util.Collection;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Container for holding a list of {@link HRegionLocation}'s that correspond to the
* same range. The list is indexed by the replicaId. This is an immutable list,
* however mutation operations are provided which returns a new List via copy-on-write
* (assuming small number of locations)
*/
@InterfaceAudience.Private
public class RegionLocations {
private final int numNonNullElements;
private final HRegionLocation[] locations; // replicaId -> HRegionLocation.
/**
* Constructs the region location list. The locations array should
* contain all the locations for known replicas for the region, and should be
* sorted in replicaId ascending order.
* @param locations an array of HRegionLocations for the same region range
*/
public RegionLocations(HRegionLocation... locations) {
int numNonNullElements = 0;
int maxReplicaId = -1;
for (HRegionLocation loc : locations) {
if (loc != null) {
numNonNullElements++;
if (loc.getRegionInfo().getReplicaId() > maxReplicaId) {
maxReplicaId = loc.getRegionInfo().getReplicaId();
}
}
}
this.numNonNullElements = numNonNullElements;
if (maxReplicaId + 1 == locations.length) {
this.locations = locations;
} else {
this.locations = new HRegionLocation[maxReplicaId + 1];
for (HRegionLocation loc : locations) {
if (loc != null) {
this.locations[loc.getRegionInfo().getReplicaId()] = loc;
}
}
}
}
public RegionLocations(Collection<HRegionLocation> locations) {
this(locations.toArray(new HRegionLocation[locations.size()]));
}
/**
* Returns the size of the list even if some of the elements
* might be null.
* @return the size of the list (corresponding to the max replicaId)
*/
public int size() {
return locations.length;
}
/**
* Returns the size of not-null locations
* @return the size of not-null locations
*/
public int numNonNullElements() {
return numNonNullElements;
}
/**
* Returns whether there are non-null elements in the list
* @return whether there are non-null elements in the list
*/
public boolean isEmpty() {
return numNonNullElements == 0;
}
/**
* Returns a new HRegionLocationList with the locations removed (set to null)
* which have the destination server as given.
* @param serverName the serverName to remove locations of
* @return an HRegionLocationList object with removed locations or the same object
* if nothing is removed
*/
public RegionLocations removeByServer(ServerName serverName) {
HRegionLocation[] newLocations = null;
for (int i = 0; i < locations.length; i++) {
// check whether something to remove
if (locations[i] != null && serverName.equals(locations[i].getServerName())) {
if (newLocations == null) { //first time
newLocations = new HRegionLocation[locations.length];
System.arraycopy(locations, 0, newLocations, 0, i);
}
newLocations[i] = null;
} else if (newLocations != null) {
newLocations[i] = locations[i];
}
}
return newLocations == null ? this : new RegionLocations(newLocations);
}
/**
* Removes the given location from the list
* @param location the location to remove
* @return an HRegionLocationList object with removed locations or the same object
* if nothing is removed
*/
public RegionLocations remove(HRegionLocation location) {
HRegionLocation[] newLocations = null;
for (int i = 0; i < locations.length; i++) {
// check whether something to remove. HRL.compareTo() compares ONLY the
// serverName. We want to compare the HRI's as well.
if (locations[i] != null
&& location.getRegionInfo().equals(locations[i].getRegionInfo())
&& location.equals(locations[i])) {
if (newLocations == null) { //first time
newLocations = new HRegionLocation[locations.length];
System.arraycopy(locations, 0, newLocations, 0, i);
}
newLocations[i] = null;
} else if (newLocations != null) {
newLocations[i] = locations[i];
}
}
return newLocations == null ? this : new RegionLocations(newLocations);
}
/**
* Merges this HRegionLocation list with the given list assuming
* same range, and keeping the most up to date version of the
* HRegionLocation entries from either list according to seqNum. If seqNums
* are equal, the location from the argument (other) is taken.
* @param other the locations to merge with
* @return an HRegionLocationList object with merged locations or the same object
* if nothing is merged
*/
public RegionLocations mergeLocations(RegionLocations other) {
assert other != null;
HRegionLocation[] newLocations = null;
int max = Math.max(this.locations.length, other.locations.length);
for (int i = 0; i < max; i++) {
HRegionLocation thisLoc = this.getRegionLocation(i);
HRegionLocation otherLoc = other.getRegionLocation(i);
HRegionLocation selectedLoc = selectRegionLocation(thisLoc,
otherLoc, true, false);
if (selectedLoc != thisLoc) {
if (newLocations == null) {
newLocations = new HRegionLocation[max];
System.arraycopy(locations, 0, newLocations, 0, i);
}
}
if (newLocations != null) {
newLocations[i] = selectedLoc;
}
}
return newLocations == null ? this : new RegionLocations(newLocations);
}
private HRegionLocation selectRegionLocation(HRegionLocation oldLocation,
HRegionLocation location, boolean checkForEquals, boolean force) {
if (location == null) {
return oldLocation == null ? null : oldLocation;
}
if (oldLocation == null) {
return location;
}
if (force
|| isGreaterThan(location.getSeqNum(), oldLocation.getSeqNum(), checkForEquals)) {
return location;
}
return oldLocation;
}
/**
* Updates the location with new only if the new location has a higher
* seqNum than the old one or force is true.
* @param location the location to add or update
* @param checkForEquals whether to update the location if seqNums for the
* HRegionLocations for the old and new location are the same
* @param force whether to force update
* @return an HRegionLocationList object with updated locations or the same object
* if nothing is updated
*/
public RegionLocations updateLocation(HRegionLocation location,
boolean checkForEquals, boolean force) {
assert location != null;
int replicaId = location.getRegionInfo().getReplicaId();
HRegionLocation oldLoc = getRegionLocation(location.getRegionInfo().getReplicaId());
HRegionLocation selectedLoc = selectRegionLocation(oldLoc, location,
checkForEquals, force);
if (selectedLoc == oldLoc) {
return this;
}
HRegionLocation[] newLocations = new HRegionLocation[Math.max(locations.length, replicaId +1)];
System.arraycopy(locations, 0, newLocations, 0, locations.length);
newLocations[replicaId] = location;
return new RegionLocations(newLocations);
}
private boolean isGreaterThan(long a, long b, boolean checkForEquals) {
return a > b || (checkForEquals && (a == b));
}
public HRegionLocation getRegionLocation(int replicaId) {
if (replicaId >= locations.length) {
return null;
}
return locations[replicaId];
}
/**
* Returns the region location from the list for matching regionName, which can
* be regionName or encodedRegionName
* @param regionName regionName or encodedRegionName
* @return HRegionLocation found or null
*/
public HRegionLocation getRegionLocationByRegionName(byte[] regionName) {
for (HRegionLocation loc : locations) {
if (loc != null) {
if (Bytes.equals(loc.getRegionInfo().getRegionName(), regionName)
|| Bytes.equals(loc.getRegionInfo().getEncodedNameAsBytes(), regionName)) {
return loc;
}
}
}
return null;
}
public HRegionLocation[] getRegionLocations() {
return locations;
}
/**
* Returns the first not-null region location in the list
*/
public HRegionLocation getRegionLocation() {
for (HRegionLocation loc : locations) {
if (loc != null) {
return loc;
}
}
return null;
}
@Override
public String toString() {
StringBuilder builder = new StringBuilder("[");
for (HRegionLocation loc : locations) {
if (loc != null) {
if (builder.length() > 1) {
builder.append(", ");
}
builder.append(loc);
}
}
builder.append("]");
return builder.toString();
}
}

View File

@ -17,37 +17,83 @@
*/
package org.apache.hadoop.hbase.catalog;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map.Entry;
import java.util.NavigableMap;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.TableName;
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.HTableDescriptor;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.PairOfSameType;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.Set;
import java.util.TreeMap;
import com.google.common.annotations.VisibleForTesting;
/**
* Reads region and assignment information from <code>hbase:meta</code>.
*/
@InterfaceAudience.Private
public class MetaReader {
/*
* HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the
* same table range (table, startKey, endKey). For every range, there will be at least one
* HRI defined which is called default replica.
*
* Meta layout (as of 0.98 + HBASE-10070) is like:
* For each table range, there is a single row, formatted like:
* <tableName>,<startKey>,<regionId>,<encodedRegionName>. This row corresponds to the regionName
* of the default region replica.
* Columns are:
* info:regioninfo => contains serialized HRI for the default region replica
* info:server => contains hostname:port (in string form) for the server hosting
* the default regionInfo replica
* info:server_<replicaId> => contains hostname:port (in string form) for the server hosting the
* regionInfo replica with replicaId
* info:serverstartcode => contains server start code (in binary long form) for the server
* hosting the default regionInfo replica
* info:serverstartcode_<replicaId> => contains server start code (in binary long form) for the
* server hosting the regionInfo replica with replicaId
* info:seqnumDuringOpen => contains seqNum (in binary long form) for the region at the time
* the server opened the region with default replicaId
* info:seqnumDuringOpen_<replicaId> => contains seqNum (in binary long form) for the region at
* the time the server opened the region with replicaId
* info:splitA => contains a serialized HRI for the first daughter region if the
* region is split
* info:splitB => contains a serialized HRI for the second daughter region if the
* region is split
* info:mergeA => contains a serialized HRI for the first parent region if the
* region is the result of a merge
* info:mergeB => contains a serialized HRI for the second parent region if the
* region is the result of a merge
*
* The actual layout of meta should be encapsulated inside MetaReader and MetaEditor methods,
* and should not leak out of those (through Result objects, etc)
*/
// TODO: Strip CatalogTracker from this class. Its all over and in the end
// its only used to get its Configuration so we can get associated
// Connection.
@ -63,59 +109,12 @@ public class MetaReader {
META_REGION_PREFIX, 0, len);
}
/**
* Performs a full scan of <code>hbase:meta</code>, skipping regions from any
* tables in the specified set of disabled tables.
* @param catalogTracker
* @param disabledTables set of disabled tables that will not be returned
* @return Returns a map of every region to it's currently assigned server,
* according to META. If the region does not have an assignment it will have
* a null value in the map.
* @throws IOException
*/
public static Map<HRegionInfo, ServerName> fullScan(
CatalogTracker catalogTracker, final Set<TableName> disabledTables)
throws IOException {
return fullScan(catalogTracker, disabledTables, false);
}
/** The delimiter for meta columns for replicaIds > 0 */
protected static final char META_REPLICA_ID_DELIMITER = '_';
/**
* Performs a full scan of <code>hbase:meta</code>, skipping regions from any
* tables in the specified set of disabled tables.
* @param catalogTracker
* @param disabledTables set of disabled tables that will not be returned
* @param excludeOfflinedSplitParents If true, do not include offlined split
* parents in the return.
* @return Returns a map of every region to it's currently assigned server,
* according to META. If the region does not have an assignment it will have
* a null value in the map.
* @throws IOException
*/
public static Map<HRegionInfo, ServerName> fullScan(
CatalogTracker catalogTracker, final Set<TableName> disabledTables,
final boolean excludeOfflinedSplitParents)
throws IOException {
final Map<HRegionInfo, ServerName> regions =
new TreeMap<HRegionInfo, ServerName>();
Visitor v = new Visitor() {
@Override
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
Pair<HRegionInfo, ServerName> region = HRegionInfo.getHRegionInfoAndServerName(r);
HRegionInfo hri = region.getFirst();
if (hri == null) return true;
if (hri.getTable() == null) return true;
if (disabledTables.contains(
hri.getTable())) return true;
// Are we to include split parents in the list?
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
regions.put(hri, region.getSecond());
return true;
}
};
fullScan(catalogTracker, v);
return regions;
}
/** A regex for parsing server columns from meta. See above javadoc for meta layout */
private static final Pattern SERVER_COLUMN_PATTERN
= Pattern.compile("^server(_[0-9a-fA-F]{4})?$");
/**
* Performs a full scan of <code>hbase:meta</code>.
@ -206,34 +205,82 @@ public class MetaReader {
}
}
/**
* Reads the location of the specified region
* @param catalogTracker
* @param regionName region whose location we are after
* @return location of region as a {@link ServerName} or null if not found
* @throws IOException
*/
static ServerName readRegionLocation(CatalogTracker catalogTracker,
byte [] regionName)
throws IOException {
Pair<HRegionInfo, ServerName> pair = getRegion(catalogTracker, regionName);
return (pair == null || pair.getSecond() == null)? null: pair.getSecond();
}
/**
* Gets the region info and assignment for the specified region.
* @param catalogTracker
* @param regionName Region to lookup.
* @return Location and HRegionInfo for <code>regionName</code>
* @throws IOException
* @deprecated use {@link #getRegionLocation(CatalogTracker, byte[])} instead
*/
@Deprecated
public static Pair<HRegionInfo, ServerName> getRegion(
CatalogTracker catalogTracker, byte [] regionName)
throws IOException {
Get get = new Get(regionName);
HRegionLocation location = getRegionLocation(catalogTracker, regionName);
return location == null
? null
: new Pair<HRegionInfo, ServerName>(location.getRegionInfo(), location.getServerName());
}
/**
* Returns the HRegionLocation from meta for the given region
* @param catalogTracker
* @param regionName
* @return HRegionLocation for the given region
* @throws IOException
*/
public static HRegionLocation getRegionLocation(CatalogTracker catalogTracker,
byte[] regionName) throws IOException {
byte[] row = regionName;
HRegionInfo parsedInfo = null;
try {
parsedInfo = parseRegionInfoFromRegionName(regionName);
row = getMetaKeyForRegion(parsedInfo);
} catch (Exception parseEx) {
LOG.warn("Received parse exception:" + parseEx);
}
Get get = new Get(row);
get.addFamily(HConstants.CATALOG_FAMILY);
Result r = get(getCatalogHTable(catalogTracker), get);
return (r == null || r.isEmpty())? null: HRegionInfo.getHRegionInfoAndServerName(r);
RegionLocations locations = getRegionLocations(r);
return locations == null
? null
: locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
}
/**
* Returns the HRegionLocation from meta for the given region
* @param catalogTracker
* @param regionInfo
* @return HRegionLocation for the given region
* @throws IOException
*/
public static HRegionLocation getRegionLocation(CatalogTracker catalogTracker,
HRegionInfo regionInfo) throws IOException {
byte[] row = getMetaKeyForRegion(regionInfo);
Get get = new Get(row);
get.addFamily(HConstants.CATALOG_FAMILY);
Result r = get(getCatalogHTable(catalogTracker), get);
return getRegionLocation(r, regionInfo, regionInfo.getReplicaId());
}
/** Returns the row key to use for this regionInfo */
protected static byte[] getMetaKeyForRegion(HRegionInfo regionInfo) {
return RegionReplicaUtil.getRegionInfoForDefaultReplica(regionInfo).getRegionName();
}
/** Returns an HRI parsed from this regionName. Not all the fields of the HRI
* is stored in the name, so the returned object should only be used for the fields
* in the regionName.
*/
protected static HRegionInfo parseRegionInfoFromRegionName(byte[] regionName)
throws IOException {
byte[][] fields = HRegionInfo.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);
}
/**
@ -258,10 +305,8 @@ public class MetaReader {
public static Pair<HRegionInfo, HRegionInfo> getRegionsFromMergeQualifier(
CatalogTracker catalogTracker, byte[] regionName) throws IOException {
Result result = getRegionResult(catalogTracker, regionName);
HRegionInfo mergeA = HRegionInfo.getHRegionInfo(result,
HConstants.MERGEA_QUALIFIER);
HRegionInfo mergeB = HRegionInfo.getHRegionInfo(result,
HConstants.MERGEB_QUALIFIER);
HRegionInfo mergeA = getHRegionInfo(result, HConstants.MERGEA_QUALIFIER);
HRegionInfo mergeB = getHRegionInfo(result, HConstants.MERGEB_QUALIFIER);
if (mergeA == null && mergeB == null) {
return null;
}
@ -289,8 +334,12 @@ public class MetaReader {
@Override
public boolean visit(Result r) throws IOException {
this.current =
HRegionInfo.getHRegionInfo(r, HConstants.REGIONINFO_QUALIFIER);
RegionLocations locations = getRegionLocations(r);
if (locations == null || locations.getRegionLocation().getRegionInfo() == null) {
LOG.warn("No serialized HRegionInfo in " + r);
return true;
}
this.current = locations.getRegionLocation().getRegionInfo();
if (this.current == null) {
LOG.warn("No serialized HRegionInfo in " + r);
return true;
@ -438,28 +487,33 @@ public class MetaReader {
// Make a version of CollectingVisitor that collects HRegionInfo and ServerAddress
CollectingVisitor<Pair<HRegionInfo, ServerName>> visitor =
new CollectingVisitor<Pair<HRegionInfo, ServerName>>() {
private Pair<HRegionInfo, ServerName> current = null;
private RegionLocations current = null;
@Override
public boolean visit(Result r) throws IOException {
HRegionInfo hri =
HRegionInfo.getHRegionInfo(r, HConstants.REGIONINFO_QUALIFIER);
if (hri == null) {
current = getRegionLocations(r);
if (current == null || current.getRegionLocation().getRegionInfo() == null) {
LOG.warn("No serialized HRegionInfo in " + r);
return true;
}
HRegionInfo hri = current.getRegionLocation().getRegionInfo();
if (!isInsideTable(hri, tableName)) return false;
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
ServerName sn = HRegionInfo.getServerName(r);
// Populate this.current so available when we call #add
this.current = new Pair<HRegionInfo, ServerName>(hri, sn);
// Else call super and add this Result to the collection.
return super.visit(r);
}
@Override
void add(Result r) {
this.results.add(this.current);
if (current == null) {
return;
}
for (HRegionLocation loc : current.getRegionLocations()) {
if (loc != null) {
this.results.add(new Pair<HRegionInfo, ServerName>(
loc.getRegionInfo(), loc.getServerName()));
}
}
}
};
fullScan(catalogTracker, visitor, getTableStartRowForMeta(tableName));
@ -483,22 +537,18 @@ public class MetaReader {
@Override
void add(Result r) {
if (r == null || r.isEmpty()) return;
if (HRegionInfo.getHRegionInfo(r) == null) return;
ServerName sn = HRegionInfo.getServerName(r);
if (sn != null && sn.equals(serverName)) {
this.results.add(r);
RegionLocations locations = getRegionLocations(r);
if (locations == null) return;
for (HRegionLocation loc : locations.getRegionLocations()) {
if (loc != null) {
if (loc.getServerName() != null && loc.getServerName().equals(serverName)) {
hris.put(loc.getRegionInfo(), r);
}
}
}
}
};
fullScan(catalogTracker, v);
List<Result> results = v.getResults();
if (results != null && !results.isEmpty()) {
// Convert results to Map keyed by HRI
for (Result r: results) {
HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
if (hri != null) hris.put(hri, r);
}
}
return hris;
}
@ -509,8 +559,13 @@ public class MetaReader {
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r);
HRegionInfo hrim = HRegionInfo.getHRegionInfo(r);
LOG.info("fullScanMetaAndPrint.HRI Print= " + hrim);
RegionLocations locations = getRegionLocations(r);
if (locations == null) return true;
for (HRegionLocation loc : locations.getRegionLocations()) {
if (loc != null) {
LOG.info("fullScanMetaAndPrint.HRI Print= " + loc.getRegionInfo());
}
}
return true;
}
};
@ -554,6 +609,215 @@ public class MetaReader {
return;
}
/**
* Returns the column family used for meta columns.
* @return HConstants.CATALOG_FAMILY.
*/
protected static byte[] getFamily() {
return HConstants.CATALOG_FAMILY;
}
/**
* Returns the column qualifier for serialized region info
* @return HConstants.REGIONINFO_QUALIFIER
*/
protected static byte[] getRegionInfoColumn() {
return HConstants.REGIONINFO_QUALIFIER;
}
/**
* Returns the column qualifier for server column for replicaId
* @param replicaId the replicaId of the region
* @return a byte[] for server column qualifier
*/
protected static byte[] getServerColumn(int replicaId) {
return replicaId == 0
? HConstants.SERVER_QUALIFIER
: Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
* Returns the column qualifier for server start code column for replicaId
* @param replicaId the replicaId of the region
* @return a byte[] for server start code column qualifier
*/
protected static byte[] getStartCodeColumn(int replicaId) {
return replicaId == 0
? HConstants.STARTCODE_QUALIFIER
: Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
* Returns the column qualifier for seqNum column for replicaId
* @param replicaId the replicaId of the region
* @return a byte[] for seqNum column qualifier
*/
protected static byte[] getSeqNumColumn(int replicaId) {
return replicaId == 0
? HConstants.SEQNUM_QUALIFIER
: Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
* Parses the replicaId from the server column qualifier. See top of the class javadoc
* for the actual meta layout
* @param serverColumn the column qualifier
* @return an int for the replicaId
*/
@VisibleForTesting
static int parseReplicaIdFromServerColumn(byte[] serverColumn) {
String serverStr = Bytes.toString(serverColumn);
Matcher matcher = SERVER_COLUMN_PATTERN.matcher(serverStr);
if (matcher.matches() && matcher.groupCount() > 0) {
String group = matcher.group(1);
if (group != null && group.length() > 0) {
return Integer.parseInt(group.substring(1), 16);
} else {
return 0;
}
}
return -1;
}
/**
* Returns a {@link ServerName} from catalog table {@link Result}.
* @param r Result to pull from
* @return A ServerName instance or null if necessary fields not found or empty.
*/
private static ServerName getServerName(final Result r, final int replicaId) {
byte[] serverColumn = getServerColumn(replicaId);
Cell cell = r.getColumnLatestCell(getFamily(), serverColumn);
if (cell == null || cell.getValueLength() == 0) return null;
String hostAndPort = Bytes.toString(
cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
byte[] startcodeColumn = getStartCodeColumn(replicaId);
cell = r.getColumnLatestCell(getFamily(), startcodeColumn);
if (cell == null || cell.getValueLength() == 0) return null;
return ServerName.valueOf(hostAndPort,
Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
}
/**
* The latest seqnum that the server writing to meta observed when opening the region.
* E.g. the seqNum when the result of {@link #getServerName(Result)} was written.
* @param r Result to pull the seqNum from
* @return SeqNum, or HConstants.NO_SEQNUM if there's no value written.
*/
private static long getSeqNumDuringOpen(final Result r, final int replicaId) {
Cell cell = r.getColumnLatestCell(getFamily(), getSeqNumColumn(replicaId));
if (cell == null || cell.getValueLength() == 0) return HConstants.NO_SEQNUM;
return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
}
/**
* Returns an HRegionLocationList extracted from the result.
* @return an HRegionLocationList containing all locations for the region range
*/
public static RegionLocations getRegionLocations(final Result r) {
if (r == null) return null;
HRegionInfo regionInfo = getHRegionInfo(r, getRegionInfoColumn());
if (regionInfo == null) return null;
List<HRegionLocation> locations = new ArrayList<HRegionLocation>(1);
NavigableMap<byte[],NavigableMap<byte[],byte[]>> familyMap = r.getNoVersionMap();
locations.add(getRegionLocation(r, regionInfo, 0));
NavigableMap<byte[], byte[]> infoMap = familyMap.get(getFamily());
if (infoMap == null) return new RegionLocations(locations);
// iterate until all serverName columns are seen
int replicaId = 0;
byte[] serverColumn = getServerColumn(replicaId);
SortedMap<byte[], byte[]> serverMap = infoMap.tailMap(serverColumn, false);
if (serverMap.isEmpty()) return new RegionLocations(locations);
for (Entry<byte[], byte[]> entry : serverMap.entrySet()) {
replicaId = parseReplicaIdFromServerColumn(entry.getKey());
if (replicaId < 0) {
break;
}
locations.add(getRegionLocation(r, regionInfo, replicaId));
}
return new RegionLocations(locations);
}
/**
* Returns the HRegionLocation parsed from the given meta row Result
* for the given regionInfo and replicaId. The regionInfo can be the default region info
* for the replica.
* @param r the meta row result
* @param regionInfo RegionInfo for default replica
* @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,
final int replicaId) {
ServerName serverName = getServerName(r, replicaId);
long seqNum = getSeqNumDuringOpen(r, replicaId);
HRegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
return new HRegionLocation(replicaInfo, serverName, seqNum);
}
/**
* Returns HRegionInfo 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
*/
public static HRegionInfo getHRegionInfo(Result data) {
return getHRegionInfo(data, HConstants.REGIONINFO_QUALIFIER);
}
/**
* Returns the HRegionInfo 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.
*/
private static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier) {
Cell cell = r.getColumnLatestCell(getFamily(), qualifier);
if (cell == null) return null;
return HRegionInfo.parseFromOrNull(cell.getValueArray(),
cell.getValueOffset(), cell.getValueLength());
}
/**
* 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
* parent
*/
public static PairOfSameType<HRegionInfo> getDaughterRegions(Result data) throws IOException {
HRegionInfo splitA = getHRegionInfo(data, HConstants.SPLITA_QUALIFIER);
HRegionInfo splitB = getHRegionInfo(data, HConstants.SPLITB_QUALIFIER);
return new PairOfSameType<HRegionInfo>(splitA, splitB);
}
/**
* 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
* parent
*/
public static PairOfSameType<HRegionInfo> getMergeRegions(Result data) throws IOException {
HRegionInfo mergeA = getHRegionInfo(data, HConstants.MERGEA_QUALIFIER);
HRegionInfo mergeB = getHRegionInfo(data, HConstants.MERGEB_QUALIFIER);
return new PairOfSameType<HRegionInfo>(mergeA, mergeB);
}
/**
* Implementations 'visit' a catalog table row.
*/

View File

@ -92,7 +92,7 @@ class AsyncProcess {
private static final Log LOG = LogFactory.getLog(AsyncProcess.class);
protected static final AtomicLong COUNTER = new AtomicLong();
/**
/**
* The context used to wait for results from one submit call.
* 1) If AsyncProcess is set to track errors globally, and not per call (for HTable puts),
* then errors and failed operations in this object will reflect global errors.
@ -110,10 +110,15 @@ class AsyncProcess {
/** Return value from a submit that didn't contain any requests. */
private static final AsyncRequestFuture NO_REQS_RESULT = new AsyncRequestFuture() {
public final Object[] result = new Object[0];
@Override
public boolean hasError() { return false; }
@Override
public RetriesExhaustedWithDetailsException getErrors() { return null; }
@Override
public List<? extends Row> getFailedOperations() { return null; }
@Override
public Object[] getResults() { return result; }
@Override
public void waitUntilDone() throws InterruptedIOException {}
};
@ -683,7 +688,7 @@ class AsyncProcess {
// Do not use the exception for updating cache because it might be coming from
// any of the regions in the MultiAction.
byte[] row = rsActions.actions.values().iterator().next().get(0).getAction().getRow();
hConnection.updateCachedLocations(tableName, row, null, server);
hConnection.updateCachedLocations(tableName, null, row, null, server);
errorsByServer.reportServerError(server);
boolean canRetry = errorsByServer.canRetryMore(numAttempt);
@ -789,7 +794,7 @@ class AsyncProcess {
if (!regionFailureRegistered) { // We're doing this once per location.
regionFailureRegistered = true;
// The location here is used as a server name.
hConnection.updateCachedLocations(tableName, row.getRow(), result, server);
hConnection.updateCachedLocations(tableName, regionName, row.getRow(), result, server);
if (failureCount == 0) {
errorsByServer.reportServerError(server);
canRetry = errorsByServer.canRetryMore(numAttempt);
@ -834,7 +839,7 @@ class AsyncProcess {
canRetry = errorsByServer.canRetryMore(numAttempt);
}
hConnection.updateCachedLocations(
tableName, actions.get(0).getAction().getRow(), throwable, server);
tableName, region, actions.get(0).getAction().getRow(), throwable, server);
failureCount += actions.size();
for (Action<Row> action : actions) {
@ -990,7 +995,7 @@ class AsyncProcess {
}
}
/**
/**
* Only used w/useGlobalErrors ctor argument, for HTable backward compat.
* @return Whether there were any errors in any request since the last time
* {@link #waitForAllPreviousOpsAndReset(List)} was called, or AP was created.

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
interface ClusterConnection extends HConnection {
/** @return - true if the master server is running */
@Override
boolean isMasterRunning()
throws MasterNotRunningException, ZooKeeperConnectionException;
@ -52,9 +53,10 @@ interface ClusterConnection extends HConnection {
* @throws IOException
* if a remote or network exception occurs
*/
@Override
boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws
IOException;
/**
* Find the location of the region of <i>tableName</i> that <i>row</i>
* lives in.
@ -64,12 +66,14 @@ interface ClusterConnection extends HConnection {
* question
* @throws IOException if a remote or network exception occurs
*/
@Override
public HRegionLocation locateRegion(final TableName tableName,
final byte [] row) throws IOException;
/**
* Allows flushing the region cache.
*/
@Override
void clearRegionCache();
/**
@ -78,12 +82,14 @@ interface ClusterConnection extends HConnection {
* @param tableName Name of the table whose regions we are to remove from
* cache.
*/
@Override
void clearRegionCache(final TableName tableName);
/**
* Deletes cached locations for the specific region.
* @param location The location object for the region, to be purged from cache.
*/
@Override
void deleteCachedRegionLocation(final HRegionLocation location);
/**
@ -95,6 +101,7 @@ interface ClusterConnection extends HConnection {
* question
* @throws IOException if a remote or network exception occurs
*/
@Override
HRegionLocation relocateRegion(final TableName tableName,
final byte [] row) throws IOException;
@ -102,11 +109,13 @@ interface ClusterConnection extends HConnection {
* Update the location cache. This is used internally by HBase, in most cases it should not be
* used by the client application.
* @param tableName the table name
* @param regionName the region name
* @param rowkey the row
* @param exception the exception if any. Can be null.
* @param source the previous location
*/
void updateCachedLocations(TableName tableName, byte[] rowkey,
@Override
void updateCachedLocations(TableName tableName, byte[] regionName, byte[] rowkey,
Object exception, ServerName source);
@ -117,6 +126,7 @@ interface ClusterConnection extends HConnection {
* question
* @throws IOException if a remote or network exception occurs
*/
@Override
HRegionLocation locateRegion(final byte[] regionName)
throws IOException;
@ -126,6 +136,7 @@ interface ClusterConnection extends HConnection {
* @return list of region locations for all regions of table
* @throws IOException
*/
@Override
List<HRegionLocation> locateRegions(final TableName tableName) throws IOException;
/**
@ -137,6 +148,7 @@ interface ClusterConnection extends HConnection {
* @return list of region locations for all regions of table
* @throws IOException
*/
@Override
List<HRegionLocation> locateRegions(final TableName tableName,
final boolean useCache,
final boolean offlined) throws IOException;
@ -145,6 +157,7 @@ interface ClusterConnection extends HConnection {
/**
* Returns a {@link MasterKeepAliveConnection} to the active master
*/
@Override
MasterService.BlockingInterface getMaster() throws IOException;
@ -154,6 +167,7 @@ interface ClusterConnection extends HConnection {
* @return proxy for HRegionServer
* @throws IOException if a remote or network exception occurs
*/
@Override
AdminService.BlockingInterface getAdmin(final ServerName serverName) throws IOException;
/**
@ -165,6 +179,7 @@ interface ClusterConnection extends HConnection {
* @throws IOException if a remote or network exception occurs
*
*/
@Override
ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException;
/**
@ -175,6 +190,7 @@ interface ClusterConnection extends HConnection {
* @return Location of row.
* @throws IOException if a remote or network exception occurs
*/
@Override
HRegionLocation getRegionLocation(TableName tableName, byte [] row,
boolean reload)
throws IOException;
@ -183,6 +199,7 @@ interface ClusterConnection extends HConnection {
* Clear any caches that pertain to server name <code>sn</code>.
* @param sn A server name
*/
@Override
void clearCaches(final ServerName sn);
/**
@ -191,6 +208,7 @@ interface ClusterConnection extends HConnection {
* @return The shared instance. Never returns null.
* @throws MasterNotRunningException
*/
@Override
@Deprecated
MasterKeepAliveConnection getKeepAliveMasterService()
throws MasterNotRunningException;
@ -199,12 +217,14 @@ interface ClusterConnection extends HConnection {
* @param serverName
* @return true if the server is known as dead, false otherwise.
* @deprecated internal method, do not use thru HConnection */
@Override
@Deprecated
boolean isDeadServer(ServerName serverName);
/**
* @return Nonce generator for this HConnection; may be null if disabled in configuration.
*/
@Override
public NonceGenerator getNonceGenerator();
/**

View File

@ -241,9 +241,9 @@ class ConnectionAdapter implements ClusterConnection {
}
@Override
public void updateCachedLocations(TableName tableName, byte[] rowkey,
public void updateCachedLocations(TableName tableName, byte[] regionName, byte[] rowkey,
Object exception, ServerName source) {
wrappedConnection.updateCachedLocations(tableName, rowkey, exception, source);
wrappedConnection.updateCachedLocations(tableName, regionName, rowkey, exception, source);
}
@Override

View File

@ -34,8 +34,6 @@ import java.util.NavigableMap;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
@ -54,6 +52,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.RegionTooBusyException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Stoppable;
@ -61,6 +60,7 @@ 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.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
@ -350,6 +350,7 @@ class ConnectionManager {
* @param conf configuration whose identity is used to find {@link HConnection} instance.
* @deprecated
*/
@Deprecated
public static void deleteConnection(Configuration conf) {
deleteConnection(new HConnectionKey(conf), false);
}
@ -361,6 +362,7 @@ class ConnectionManager {
* @param connection
* @deprecated
*/
@Deprecated
public static void deleteStaleConnection(HConnection connection) {
deleteConnection(connection, true);
}
@ -371,6 +373,7 @@ class ConnectionManager {
* staleConnection to true.
* @deprecated
*/
@Deprecated
public static void deleteAllConnections(boolean staleConnection) {
synchronized (CONNECTION_INSTANCES) {
Set<HConnectionKey> connectionKeys = new HashSet<HConnectionKey>();
@ -496,19 +499,7 @@ class ConnectionManager {
// Client rpc instance.
private RpcClient rpcClient;
/**
* Map of table to table {@link HRegionLocation}s.
*/
private final ConcurrentMap<TableName, ConcurrentSkipListMap<byte[], HRegionLocation>>
cachedRegionLocations =
new ConcurrentHashMap<TableName, ConcurrentSkipListMap<byte[], HRegionLocation>>();
// The presence of a server in the map implies it's likely that there is an
// entry in cachedRegionLocations that map to this server; but the absence
// of a server in this map guarentees that there is no entry in cache that
// maps to the absent server.
// The access to this attribute must be protected by a lock on cachedRegionLocations
private final Set<ServerName> cachedServers = new ConcurrentSkipListSet<ServerName>();
private MetaCache metaCache = new MetaCache();
private int refCount;
@ -731,6 +722,7 @@ class ConnectionManager {
* An identifier that will remain the same for a given connection.
* @return
*/
@Override
public String toString(){
return "hconnection-0x" + Integer.toHexString(hashCode());
}
@ -902,8 +894,9 @@ class ConnectionManager {
@Override
public HRegionLocation locateRegion(final byte[] regionName) throws IOException {
return locateRegion(HRegionInfo.getTable(regionName),
HRegionInfo.getStartKey(regionName), false, true);
RegionLocations locations = locateRegion(HRegionInfo.getTable(regionName),
HRegionInfo.getStartKey(regionName), false, true);
return locations == null ? null : locations.getRegionLocation();
}
@Override
@ -934,7 +927,14 @@ class ConnectionManager {
tableName, offlined);
final List<HRegionLocation> locations = new ArrayList<HRegionLocation>();
for (HRegionInfo regionInfo : regions.keySet()) {
locations.add(locateRegion(tableName, regionInfo.getStartKey(), useCache, true));
RegionLocations list = locateRegion(tableName, regionInfo.getStartKey(), useCache, true);
if (list != null) {
for (HRegionLocation loc : list.getRegionLocations()) {
if (loc != null) {
locations.add(loc);
}
}
}
}
return locations;
}
@ -949,7 +949,8 @@ class ConnectionManager {
public HRegionLocation locateRegion(final TableName tableName,
final byte [] row)
throws IOException{
return locateRegion(tableName, row, true, true);
RegionLocations locations = locateRegion(tableName, row, true, true);
return locations == null ? null : locations.getRegionLocation();
}
@Override
@ -969,7 +970,8 @@ class ConnectionManager {
throw new TableNotEnabledException(tableName.getNameAsString() + " is disabled.");
}
return locateRegion(tableName, row, false, true);
RegionLocations locations = locateRegion(tableName, row, false, true);
return locations == null ? null : locations.getRegionLocation();
}
@Override
@ -979,7 +981,7 @@ class ConnectionManager {
}
private HRegionLocation locateRegion(final TableName tableName,
private RegionLocations locateRegion(final TableName tableName,
final byte [] row, boolean useCache, boolean retry)
throws IOException {
if (this.closed) throw new IOException(toString() + " closed");
@ -1000,15 +1002,15 @@ class ConnectionManager {
* Search the hbase:meta table for the HRegionLocation
* info that contains the table and row we're seeking.
*/
private HRegionLocation locateRegionInMeta(TableName tableName, byte[] row,
private RegionLocations locateRegionInMeta(TableName tableName, byte[] row,
boolean useCache, boolean retry) throws IOException {
// If we are supposed to be using the cache, look in the cache to see if
// we already have the region.
if (useCache) {
HRegionLocation location = getCachedLocation(tableName, row);
if (location != null) {
return location;
RegionLocations locations = getCachedLocation(tableName, row);
if (locations != null) {
return locations;
}
}
@ -1033,9 +1035,9 @@ class ConnectionManager {
" after " + localNumRetries + " tries.");
}
if (useCache) {
HRegionLocation location = getCachedLocation(tableName, row);
if (location != null) {
return location;
RegionLocations locations = getCachedLocation(tableName, row);
if (locations != null) {
return locations;
}
}
@ -1057,7 +1059,8 @@ class ConnectionManager {
}
// convert the row result into the HRegionLocation we need!
HRegionInfo regionInfo = MetaScanner.getHRegionInfo(regionInfoRow);
RegionLocations locations = MetaReader.getRegionLocations(regionInfoRow);
HRegionInfo regionInfo = locations.getRegionLocation().getRegionInfo();
if (regionInfo == null) {
throw new IOException("HRegionInfo was null or empty in " +
TableName.META_TABLE_NAME + ", row=" + regionInfoRow);
@ -1081,7 +1084,7 @@ class ConnectionManager {
regionInfo.getRegionNameAsString());
}
ServerName serverName = HRegionInfo.getServerName(regionInfoRow);
ServerName serverName = locations.getRegionLocation().getServerName();
if (serverName == null) {
throw new NoServerForRegionException("No server address listed " +
"in " + TableName.META_TABLE_NAME + " for region " +
@ -1096,10 +1099,8 @@ class ConnectionManager {
}
// Instantiate the location
HRegionLocation location = new HRegionLocation(regionInfo, serverName,
HRegionInfo.getSeqNumDuringOpen(regionInfoRow));
cacheLocation(tableName, null, location);
return location;
cacheLocation(tableName, locations);
return locations;
} catch (TableNotFoundException e) {
// if we got this error, probably means the table just plain doesn't
@ -1138,7 +1139,16 @@ class ConnectionManager {
}
}
/*
/**
* Put a newly discovered HRegionLocation into the cache.
* @param tableName The table name.
* @param location the new location
*/
private void cacheLocation(final TableName tableName, final RegionLocations location) {
metaCache.cacheLocation(tableName, location);
}
/**
* Search the cache for a location that fits our table and row key.
* Return null if no suitable region is located.
*
@ -1146,52 +1156,13 @@ class ConnectionManager {
* @param row
* @return Null or region location found in cache.
*/
HRegionLocation getCachedLocation(final TableName tableName,
RegionLocations getCachedLocation(final TableName tableName,
final byte [] row) {
ConcurrentSkipListMap<byte[], HRegionLocation> tableLocations =
getTableLocations(tableName);
Entry<byte[], HRegionLocation> e = tableLocations.floorEntry(row);
if (e == null) {
return null;
}
HRegionLocation possibleRegion = e.getValue();
// make sure that the end key is greater than the row we're looking
// for, otherwise the row actually belongs in the next region, not
// this one. the exception case is when the endkey is
// HConstants.EMPTY_END_ROW, signifying that the region we're
// checking is actually the last region in the table.
byte[] endKey = possibleRegion.getRegionInfo().getEndKey();
if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) ||
tableName.getRowComparator().compareRows(
endKey, 0, endKey.length, row, 0, row.length) > 0) {
return possibleRegion;
}
// Passed all the way through, so we got nothing - complete cache miss
return null;
return metaCache.getCachedLocation(tableName, row);
}
/**
* Delete a cached location, no matter what it is. Called when we were told to not use cache.
* @param tableName tableName
* @param row
*/
void forceDeleteCachedLocation(final TableName tableName, final byte [] row) {
HRegionLocation rl = null;
Map<byte[], HRegionLocation> tableLocations = getTableLocations(tableName);
// start to examine the cache. we can only do cache actions
// if there's something in the cache for this table.
rl = getCachedLocation(tableName, row);
if (rl != null) {
tableLocations.remove(rl.getRegionInfo().getStartKey());
}
if ((rl != null) && LOG.isDebugEnabled()) {
LOG.debug("Removed " + rl.getHostname() + ":" + rl.getPort()
+ " as a location of " + rl.getRegionInfo().getRegionNameAsString() +
" for tableName=" + tableName + " from cache");
}
public void clearRegionCache(final TableName tableName, byte[] row) {
metaCache.clearCache(tableName, row);
}
/*
@ -1199,66 +1170,17 @@ class ConnectionManager {
*/
@Override
public void clearCaches(final ServerName serverName) {
if (!this.cachedServers.contains(serverName)) {
return;
}
boolean deletedSomething = false;
synchronized (this.cachedServers) {
// We block here, because if there is an error on a server, it's likely that multiple
// threads will get the error simultaneously. If there are hundreds of thousand of
// region location to check, it's better to do this only once. A better pattern would
// be to check if the server is dead when we get the region location.
if (!this.cachedServers.contains(serverName)) {
return;
}
for (Map<byte[], HRegionLocation> tableLocations : cachedRegionLocations.values()) {
for (Entry<byte[], HRegionLocation> e : tableLocations.entrySet()) {
HRegionLocation value = e.getValue();
if (value != null
&& serverName.equals(value.getServerName())) {
tableLocations.remove(e.getKey());
deletedSomething = true;
}
}
}
this.cachedServers.remove(serverName);
}
if (deletedSomething && LOG.isDebugEnabled()) {
LOG.debug("Removed all cached region locations that map to " + serverName);
}
}
/*
* @param tableName
* @return Map of cached locations for passed <code>tableName</code>
*/
private ConcurrentSkipListMap<byte[], HRegionLocation> getTableLocations(
final TableName tableName) {
// find the map of cached locations for this table
ConcurrentSkipListMap<byte[], HRegionLocation> result;
result = this.cachedRegionLocations.get(tableName);
// if tableLocations for this table isn't built yet, make one
if (result == null) {
result = new ConcurrentSkipListMap<byte[], HRegionLocation>(Bytes.BYTES_COMPARATOR);
ConcurrentSkipListMap<byte[], HRegionLocation> old =
this.cachedRegionLocations.putIfAbsent(tableName, result);
if (old != null) {
return old;
}
}
return result;
metaCache.clearCache(serverName);
}
@Override
public void clearRegionCache() {
this.cachedRegionLocations.clear();
this.cachedServers.clear();
metaCache.clearCache();
}
@Override
public void clearRegionCache(final TableName tableName) {
this.cachedRegionLocations.remove(tableName);
metaCache.clearCache(tableName);
}
@Override
@ -1274,37 +1196,7 @@ class ConnectionManager {
*/
private void cacheLocation(final TableName tableName, final ServerName source,
final HRegionLocation location) {
boolean isFromMeta = (source == null);
byte [] startKey = location.getRegionInfo().getStartKey();
ConcurrentMap<byte[], HRegionLocation> tableLocations = getTableLocations(tableName);
HRegionLocation oldLocation = tableLocations.putIfAbsent(startKey, location);
boolean isNewCacheEntry = (oldLocation == null);
if (isNewCacheEntry) {
cachedServers.add(location.getServerName());
return;
}
boolean updateCache;
// If the server in cache sends us a redirect, assume it's always valid.
if (oldLocation.getServerName().equals(source)) {
updateCache = true;
} else {
long newLocationSeqNum = location.getSeqNum();
// Meta record is stale - some (probably the same) server has closed the region
// with later seqNum and told us about the new location.
boolean isStaleMetaRecord = isFromMeta && (oldLocation.getSeqNum() > newLocationSeqNum);
// Same as above for redirect. However, in this case, if the number is equal to previous
// record, the most common case is that first the region was closed with seqNum, and then
// opened with the same seqNum; hence we will ignore the redirect.
// There are so many corner cases with various combinations of opens and closes that
// an additional counter on top of seqNum would be necessary to handle them all.
boolean isStaleRedirect = !isFromMeta && (oldLocation.getSeqNum() >= newLocationSeqNum);
boolean isStaleUpdate = (isStaleMetaRecord || isStaleRedirect);
updateCache = (!isStaleUpdate);
}
if (updateCache) {
tableLocations.replace(startKey, oldLocation, location);
cachedServers.add(location.getServerName());
}
metaCache.cacheLocation(tableName, source, location);
}
// Map keyed by service name + regionserver to service stub implementation
@ -1987,7 +1879,7 @@ class ConnectionManager {
}
};
}
private static void release(MasterServiceState mss) {
if (mss != null && mss.connection != null) {
@ -2046,37 +1938,17 @@ class ConnectionManager {
cacheLocation(hri.getTable(), source, newHrl);
}
/**
* Deletes the cached location of the region if necessary, based on some error from source.
* @param hri The region in question.
* @param source The source of the error that prompts us to invalidate cache.
*/
void deleteCachedLocation(HRegionInfo hri, ServerName source) {
getTableLocations(hri.getTable()).remove(hri.getStartKey());
}
@Override
public void deleteCachedRegionLocation(final HRegionLocation location) {
if (location == null || location.getRegionInfo() == null) {
return;
}
HRegionLocation removedLocation;
TableName tableName = location.getRegionInfo().getTable();
Map<byte[], HRegionLocation> tableLocations = getTableLocations(tableName);
removedLocation = tableLocations.remove(location.getRegionInfo().getStartKey());
if (LOG.isDebugEnabled() && removedLocation != null) {
LOG.debug("Removed " +
location.getRegionInfo().getRegionNameAsString() +
" for tableName=" + tableName +
" from cache");
}
metaCache.clearCache(location);
}
@Override
public void updateCachedLocations(final TableName tableName, byte[] rowkey,
final Object exception, final HRegionLocation source) {
updateCachedLocations(tableName, rowkey, exception, source.getServerName());
final Object exception, final HRegionLocation source) {
assert source != null;
updateCachedLocations(tableName, source.getRegionInfo().getRegionName()
, rowkey, exception, source.getServerName());
}
/**
@ -2088,7 +1960,7 @@ class ConnectionManager {
* @param source server that is the source of the location update.
*/
@Override
public void updateCachedLocations(final TableName tableName, byte[] rowkey,
public void updateCachedLocations(final TableName tableName, byte[] regionName, byte[] rowkey,
final Object exception, final ServerName source) {
if (rowkey == null || tableName == null) {
LOG.warn("Coding error, see method javadoc. row=" + (rowkey == null ? "null" : rowkey) +
@ -2101,8 +1973,18 @@ class ConnectionManager {
return;
}
if (regionName == null) {
// we do not know which region, so just remove the cache entry for the row and server
metaCache.clearCache(tableName, rowkey, source);
return;
}
// Is it something we have already updated?
final HRegionLocation oldLocation = getCachedLocation(tableName, rowkey);
final RegionLocations oldLocations = getCachedLocation(tableName, rowkey);
HRegionLocation oldLocation = null;
if (oldLocations != null) {
oldLocation = oldLocations.getRegionLocationByRegionName(regionName);
}
if (oldLocation == null || !source.equals(oldLocation.getServerName())) {
// There is no such location in the cache (it's been removed already) or
// the cache has already been refreshed with a different location. => nothing to do
@ -2133,8 +2015,8 @@ class ConnectionManager {
}
// If we're here, it means that can cannot be sure about the location, so we remove it from
// the cache.
deleteCachedLocation(regionInfo, source);
// the cache. Do not send the source because source can be a new server in the same host:port
metaCache.clearCache(regionInfo);
}
@Override
@ -2221,24 +2103,9 @@ class ConnectionManager {
* Return the number of cached region for a table. It will only be called
* from a unit test.
*/
@VisibleForTesting
int getNumberOfCachedRegionLocations(final TableName tableName) {
Map<byte[], HRegionLocation> tableLocs = this.cachedRegionLocations.get(tableName);
if (tableLocs == null) {
return 0;
}
return tableLocs.values().size();
}
/**
* Check the region cache to see whether a region is cached yet or not.
* Called by unit tests.
* @param tableName tableName
* @param row row
* @return Region cached or not.
*/
boolean isRegionCached(TableName tableName, final byte[] row) {
HRegionLocation location = getCachedLocation(tableName, row);
return location != null;
return metaCache.getNumberOfCachedRegionLocations(tableName);
}
@Override
@ -2567,7 +2434,7 @@ class ConnectionManager {
* Look for an exception we know in the remote exception:
* - hadoop.ipc wrapped exceptions
* - nested exceptions
*
*
* Looks for: RegionMovedException / RegionOpeningException / RegionTooBusyException
* @return null if we didn't find the exception, the exception otherwise.
*/

View File

@ -27,11 +27,11 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
* keeps a cache of locations and then knows how to re-calibrate after they move. You need one
* of these to talk to your HBase cluster. {@link HConnectionManager} manages instances of this
* class. See it for how to get one of these.
*
*
* <p>This is NOT a connection to a particular server but to ALL servers in the cluster. Individual
* connections are managed at a lower level.
*
@ -264,7 +264,8 @@ public interface HConnection extends Abortable, Closeable {
* @return HRegionLocation that describes where to find the region in
* question
* @throws IOException if a remote or network exception occurs
* @deprecated internal method, do not use thru HConnection */
* @deprecated internal method, do not use thru HConnection
*/
@Deprecated
public HRegionLocation locateRegion(final TableName tableName,
final byte [] row) throws IOException;
@ -323,12 +324,14 @@ public interface HConnection extends Abortable, Closeable {
* Update the location cache. This is used internally by HBase, in most cases it should not be
* used by the client application.
* @param tableName the table name
* @param regionName the regionName
* @param rowkey the row
* @param exception the exception if any. Can be null.
* @param source the previous location
* @deprecated internal method, do not use thru HConnection */
* @deprecated internal method, do not use thru HConnection
*/
@Deprecated
void updateCachedLocations(TableName tableName, byte[] rowkey,
void updateCachedLocations(TableName tableName, byte[] regionName, byte[] rowkey,
Object exception, ServerName source);
@Deprecated
@ -366,7 +369,8 @@ public interface HConnection extends Abortable, Closeable {
* regions from returned list.
* @return list of region locations for all regions of table
* @throws IOException
* @deprecated internal method, do not use thru HConnection */
* @deprecated internal method, do not use thru HConnection
*/
@Deprecated
public List<HRegionLocation> locateRegions(final TableName tableName,
final boolean useCache,
@ -412,6 +416,7 @@ public interface HConnection extends Abortable, Closeable {
* @throws IOException if a remote or network exception occurs
* @deprecated You can pass master flag but nothing special is done.
*/
@Deprecated
AdminService.BlockingInterface getAdmin(final ServerName serverName, boolean getMaster)
throws IOException;
@ -506,6 +511,7 @@ public interface HConnection extends Abortable, Closeable {
* @throws IOException if a remote or network exception occurs
* @deprecated This method will be changed from public to package protected.
*/
@Deprecated
int getCurrentNrHRS() throws IOException;
/**

View File

@ -243,6 +243,7 @@ public class HConnectionManager {
* @param conf configuration whose identity is used to find {@link HConnection} instance.
* @deprecated
*/
@Deprecated
public static void deleteConnection(Configuration conf) {
ConnectionManager.deleteConnection(conf);
}
@ -254,6 +255,7 @@ public class HConnectionManager {
* @param connection
* @deprecated
*/
@Deprecated
public static void deleteStaleConnection(HConnection connection) {
ConnectionManager.deleteStaleConnection(connection);
}
@ -264,6 +266,7 @@ public class HConnectionManager {
* staleConnection to true.
* @deprecated
*/
@Deprecated
public static void deleteAllConnections(boolean staleConnection) {
ConnectionManager.deleteAllConnections(staleConnection);
}

View File

@ -43,7 +43,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
@ -51,6 +50,7 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
@ -76,11 +76,10 @@ import org.apache.hadoop.hbase.util.Threads;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Message;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.Service;
import com.google.protobuf.ServiceException;
import com.google.common.annotations.VisibleForTesting;
/**
* <p>Used to communicate with a single HBase table. An implementation of
* {@link HTableInterface}. Instances of this class can be constructed directly but it is
@ -608,12 +607,15 @@ public class HTable implements HTableInterface {
* @return Pair of arrays of region starting and ending row keys
* @throws IOException if a remote or network exception occurs
*/
// TODO: these are not in HTableInterface. Should we add them there or move these to HBaseAdmin?
public Pair<byte[][],byte[][]> getStartEndKeys() throws IOException {
NavigableMap<HRegionInfo, ServerName> regions = getRegionLocations();
List<RegionLocations> regions = listRegionLocations();
final List<byte[]> startKeyList = new ArrayList<byte[]>(regions.size());
final List<byte[]> endKeyList = new ArrayList<byte[]>(regions.size());
for (HRegionInfo region : regions.keySet()) {
for (RegionLocations locations : regions) {
HRegionInfo region = locations.getRegionLocation().getRegionInfo();
startKeyList.add(region.getStartKey());
endKeyList.add(region.getEndKey());
}
@ -623,13 +625,20 @@ public class HTable implements HTableInterface {
endKeyList.toArray(new byte[endKeyList.size()][]));
}
@VisibleForTesting
List<RegionLocations> listRegionLocations() throws IOException {
return MetaScanner.listTableRegionLocations(getConfiguration(), this.connection, getName());
}
/**
* Gets all the regions and their address for this table.
* <p>
* This is mainly useful for the MapReduce integration.
* @return A map of HRegionInfo with it's server address
* @throws IOException if a remote or network exception occurs
* @deprecated This is no longer a public API
*/
@Deprecated
public NavigableMap<HRegionInfo, ServerName> getRegionLocations() throws IOException {
// TODO: Odd that this returns a Map of HRI to SN whereas getRegionLocation, singular, returns an HRegionLocation.
return MetaScanner.allTableRegions(getConfiguration(), this.connection, getName(), false);
@ -643,7 +652,9 @@ public class HTable implements HTableInterface {
* @return A list of HRegionLocations corresponding to the regions that
* contain the specified range
* @throws IOException if a remote or network exception occurs
* @deprecated This is no longer a public API
*/
@Deprecated
public List<HRegionLocation> getRegionsInRange(final byte [] startKey,
final byte [] endKey) throws IOException {
return getRegionsInRange(startKey, endKey, false);
@ -658,7 +669,9 @@ public class HTable implements HTableInterface {
* @return A list of HRegionLocations corresponding to the regions that
* contain the specified range
* @throws IOException if a remote or network exception occurs
* @deprecated This is no longer a public API
*/
@Deprecated
public List<HRegionLocation> getRegionsInRange(final byte [] startKey,
final byte [] endKey, final boolean reload) throws IOException {
return getKeysAndRegionsInRange(startKey, endKey, false, reload).getSecond();
@ -674,7 +687,9 @@ public class HTable implements HTableInterface {
* @return A pair of list of start keys and list of HRegionLocations that
* contain the specified range
* @throws IOException if a remote or network exception occurs
* @deprecated This is no longer a public API
*/
@Deprecated
private Pair<List<byte[]>, List<HRegionLocation>> getKeysAndRegionsInRange(
final byte[] startKey, final byte[] endKey, final boolean includeEndKey)
throws IOException {
@ -692,7 +707,9 @@ public class HTable implements HTableInterface {
* @return A pair of list of start keys and list of HRegionLocations that
* contain the specified range
* @throws IOException if a remote or network exception occurs
* @deprecated This is no longer a public API
*/
@Deprecated
private Pair<List<byte[]>, List<HRegionLocation>> getKeysAndRegionsInRange(
final byte[] startKey, final byte[] endKey, final boolean includeEndKey,
final boolean reload) throws IOException {
@ -727,7 +744,8 @@ public class HTable implements HTableInterface {
throws IOException {
RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
tableName, row) {
public Result call(int callTimeout) throws IOException {
@Override
public Result call(int callTimeout) throws IOException {
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
controller.setPriority(tableName);
controller.setCallTimeout(callTimeout);
@ -801,6 +819,7 @@ public class HTable implements HTableInterface {
public Result get(final Get get) throws IOException {
RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
getName(), get.getRow()) {
@Override
public Result call(int callTimeout) throws IOException {
ClientProtos.GetRequest request =
RequestConverter.buildGetRequest(getLocation().getRegionInfo().getRegionName(), get);
@ -862,6 +881,7 @@ public class HTable implements HTableInterface {
* @deprecated If any exception is thrown by one of the actions, there is no way to
* retrieve the partially executed results. Use {@link #batch(List, Object[])} instead.
*/
@Deprecated
@Override
public Object[] batch(final List<? extends Row> actions)
throws InterruptedException, IOException {
@ -887,6 +907,7 @@ public class HTable implements HTableInterface {
* {@link #batchCallback(List, Object[], org.apache.hadoop.hbase.client.coprocessor.Batch.Callback)}
* instead.
*/
@Deprecated
@Override
public <R> Object[] batchCallback(
final List<? extends Row> actions, final Batch.Callback<R> callback) throws IOException,
@ -904,6 +925,7 @@ public class HTable implements HTableInterface {
throws IOException {
RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(connection,
tableName, delete.getRow()) {
@Override
public Boolean call(int callTimeout) throws IOException {
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
controller.setPriority(tableName);
@ -1044,6 +1066,7 @@ public class HTable implements HTableInterface {
public void mutateRow(final RowMutations rm) throws IOException {
RegionServerCallable<Void> callable =
new RegionServerCallable<Void>(connection, getName(), rm.getRow()) {
@Override
public Void call(int callTimeout) throws IOException {
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
controller.setPriority(tableName);
@ -1078,6 +1101,7 @@ public class HTable implements HTableInterface {
final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
RegionServerCallable<Result> callable =
new RegionServerCallable<Result>(this.connection, getName(), append.getRow()) {
@Override
public Result call(int callTimeout) throws IOException {
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
controller.setPriority(getTableName());
@ -1109,6 +1133,7 @@ public class HTable implements HTableInterface {
final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
getName(), increment.getRow()) {
@Override
public Result call(int callTimeout) throws IOException {
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
controller.setPriority(getTableName());
@ -1172,6 +1197,7 @@ public class HTable implements HTableInterface {
final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
RegionServerCallable<Long> callable =
new RegionServerCallable<Long>(connection, getName(), row) {
@Override
public Long call(int callTimeout) throws IOException {
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
controller.setPriority(getTableName());
@ -1202,6 +1228,7 @@ public class HTable implements HTableInterface {
throws IOException {
RegionServerCallable<Boolean> callable =
new RegionServerCallable<Boolean>(connection, getName(), row) {
@Override
public Boolean call(int callTimeout) throws IOException {
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
controller.setPriority(tableName);
@ -1230,6 +1257,7 @@ public class HTable implements HTableInterface {
throws IOException {
RegionServerCallable<Boolean> callable =
new RegionServerCallable<Boolean>(connection, getName(), row) {
@Override
public Boolean call(int callTimeout) throws IOException {
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
controller.setPriority(tableName);
@ -1259,6 +1287,7 @@ public class HTable implements HTableInterface {
throws IOException {
RegionServerCallable<Boolean> callable =
new RegionServerCallable<Boolean>(connection, getName(), row) {
@Override
public Boolean call(int callTimeout) throws IOException {
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
controller.setPriority(tableName);
@ -1287,6 +1316,7 @@ public class HTable implements HTableInterface {
throws IOException {
RegionServerCallable<Boolean> callable =
new RegionServerCallable<Boolean>(connection, getName(), row) {
@Override
public Boolean call(int callTimeout) throws IOException {
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
controller.setPriority(tableName);
@ -1474,6 +1504,7 @@ public class HTable implements HTableInterface {
* @param writeBufferSize The new write buffer size, in bytes.
* @throws IOException if a remote or network exception occurs.
*/
@Override
public void setWriteBufferSize(long writeBufferSize) throws IOException {
this.writeBufferSize = writeBufferSize;
if(currentWriteBufferSize > writeBufferSize) {
@ -1595,6 +1626,7 @@ public class HTable implements HTableInterface {
/**
* {@inheritDoc}
*/
@Override
public CoprocessorRpcChannel coprocessorService(byte[] row) {
return new RegionCoprocessorRpcChannel(connection, tableName, row);
}
@ -1609,6 +1641,7 @@ public class HTable implements HTableInterface {
final Map<byte[],R> results = Collections.synchronizedMap(
new TreeMap<byte[], R>(Bytes.BYTES_COMPARATOR));
coprocessorService(service, startKey, endKey, callable, new Batch.Callback<R>() {
@Override
public void update(byte[] region, byte[] row, R value) {
if (region != null) {
results.put(region, value);
@ -1636,6 +1669,7 @@ public class HTable implements HTableInterface {
new RegionCoprocessorRpcChannel(connection, tableName, r);
Future<R> future = pool.submit(
new Callable<R>() {
@Override
public R call() throws Exception {
T instance = ProtobufUtil.newServiceStub(service, channel);
R result = callable.call(instance);

View File

@ -0,0 +1,343 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.ConcurrentSkipListSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.Bytes;
/**
* A cache implementation for region locations from meta.
*/
@InterfaceAudience.Private
public class MetaCache {
private static final Log LOG = LogFactory.getLog(MetaCache.class);
/**
* Map of table to table {@link HRegionLocation}s.
*/
private final ConcurrentMap<TableName, ConcurrentSkipListMap<byte[], RegionLocations>>
cachedRegionLocations =
new ConcurrentHashMap<TableName, ConcurrentSkipListMap<byte[], RegionLocations>>();
// The presence of a server in the map implies it's likely that there is an
// entry in cachedRegionLocations that map to this server; but the absence
// of a server in this map guarentees that there is no entry in cache that
// maps to the absent server.
// The access to this attribute must be protected by a lock on cachedRegionLocations
private final Set<ServerName> cachedServers = new ConcurrentSkipListSet<ServerName>();
/**
* Search the cache for a location that fits our table and row key.
* Return null if no suitable region is located.
*
* @param tableName
* @param row
* @return Null or region location found in cache.
*/
public RegionLocations getCachedLocation(final TableName tableName, final byte [] row) {
ConcurrentSkipListMap<byte[], RegionLocations> tableLocations =
getTableLocations(tableName);
Entry<byte[], RegionLocations> e = tableLocations.floorEntry(row);
if (e == null) {
return null;
}
RegionLocations possibleRegion = e.getValue();
// make sure that the end key is greater than the row we're looking
// for, otherwise the row actually belongs in the next region, not
// this one. the exception case is when the endkey is
// HConstants.EMPTY_END_ROW, signifying that the region we're
// checking is actually the last region in the table.
byte[] endKey = possibleRegion.getRegionLocation().getRegionInfo().getEndKey();
if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) ||
tableName.getRowComparator().compareRows(
endKey, 0, endKey.length, row, 0, row.length) > 0) {
return possibleRegion;
}
// Passed all the way through, so we got nothing - complete cache miss
return null;
}
/**
* Put a newly discovered HRegionLocation into the cache.
* @param tableName The table name.
* @param source the source of the new location
* @param location the new location
*/
public void cacheLocation(final TableName tableName, final ServerName source,
final HRegionLocation location) {
assert source != null;
byte [] startKey = location.getRegionInfo().getStartKey();
ConcurrentMap<byte[], RegionLocations> tableLocations = getTableLocations(tableName);
RegionLocations locations = new RegionLocations(new HRegionLocation[] {location}) ;
RegionLocations oldLocations = tableLocations.putIfAbsent(startKey, locations);
boolean isNewCacheEntry = (oldLocations == null);
if (isNewCacheEntry) {
addToCachedServers(locations);
return;
}
// If the server in cache sends us a redirect, assume it's always valid.
HRegionLocation oldLocation = oldLocations.getRegionLocation(
location.getRegionInfo().getReplicaId());
boolean force = oldLocation != null && oldLocation.getServerName() != null
&& oldLocation.getServerName().equals(source);
// For redirect if the number is equal to previous
// record, the most common case is that first the region was closed with seqNum, and then
// opened with the same seqNum; hence we will ignore the redirect.
// There are so many corner cases with various combinations of opens and closes that
// an additional counter on top of seqNum would be necessary to handle them all.
RegionLocations updatedLocations = oldLocations.updateLocation(location, false, force);
if (oldLocations != updatedLocations) {
tableLocations.replace(startKey, oldLocations, updatedLocations);
addToCachedServers(updatedLocations);
}
}
/**
* Put a newly discovered HRegionLocation into the cache.
* @param tableName The table name.
* @param location the new location
*/
public void cacheLocation(final TableName tableName, final RegionLocations location) {
byte [] startKey = location.getRegionLocation().getRegionInfo().getStartKey();
ConcurrentMap<byte[], RegionLocations> tableLocations = getTableLocations(tableName);
RegionLocations oldLocation = tableLocations.putIfAbsent(startKey, location);
boolean isNewCacheEntry = (oldLocation == null);
if (isNewCacheEntry) {
addToCachedServers(location);
return;
}
// merge old and new locations and add it to the cache
// Meta record might be stale - some (probably the same) server has closed the region
// with later seqNum and told us about the new location.
RegionLocations mergedLocation = oldLocation.mergeLocations(location);
tableLocations.replace(startKey, oldLocation, mergedLocation);
addToCachedServers(location);
}
private void addToCachedServers(RegionLocations locations) {
for (HRegionLocation loc : locations.getRegionLocations()) {
if (loc != null) {
cachedServers.add(loc.getServerName());
}
}
}
/**
* @param tableName
* @return Map of cached locations for passed <code>tableName</code>
*/
private ConcurrentSkipListMap<byte[], RegionLocations>
getTableLocations(final TableName tableName) {
// find the map of cached locations for this table
ConcurrentSkipListMap<byte[], RegionLocations> result;
result = this.cachedRegionLocations.get(tableName);
// if tableLocations for this table isn't built yet, make one
if (result == null) {
result = new ConcurrentSkipListMap<byte[], RegionLocations>(Bytes.BYTES_COMPARATOR);
ConcurrentSkipListMap<byte[], RegionLocations> old =
this.cachedRegionLocations.putIfAbsent(tableName, result);
if (old != null) {
return old;
}
}
return result;
}
/**
* Check the region cache to see whether a region is cached yet or not.
* @param tableName tableName
* @param row row
* @return Region cached or not.
*/
public boolean isRegionCached(TableName tableName, final byte[] row) {
RegionLocations location = getCachedLocation(tableName, row);
return location != null;
}
/**
* Return the number of cached region for a table. It will only be called
* from a unit test.
*/
public int getNumberOfCachedRegionLocations(final TableName tableName) {
Map<byte[], RegionLocations> tableLocs = this.cachedRegionLocations.get(tableName);
if (tableLocs == null) {
return 0;
}
int numRegions = 0;
for (RegionLocations tableLoc : tableLocs.values()) {
numRegions += tableLoc.numNonNullElements();
}
return numRegions;
}
/**
* Delete all cached entries.
*/
public void clearCache() {
this.cachedRegionLocations.clear();
this.cachedServers.clear();
}
/**
* Delete all cached entries of a server.
*/
public void clearCache(final ServerName serverName) {
if (!this.cachedServers.contains(serverName)) {
return;
}
boolean deletedSomething = false;
synchronized (this.cachedServers) {
// We block here, because if there is an error on a server, it's likely that multiple
// threads will get the error simultaneously. If there are hundreds of thousand of
// region location to check, it's better to do this only once. A better pattern would
// be to check if the server is dead when we get the region location.
if (!this.cachedServers.contains(serverName)) {
return;
}
for (ConcurrentMap<byte[], RegionLocations> tableLocations : cachedRegionLocations.values()){
for (Entry<byte[], RegionLocations> e : tableLocations.entrySet()) {
RegionLocations regionLocations = e.getValue();
if (regionLocations != null) {
RegionLocations updatedLocations = regionLocations.removeByServer(serverName);
deletedSomething |= regionLocations == updatedLocations;
if (updatedLocations != regionLocations) {
if (updatedLocations.isEmpty()) {
tableLocations.remove(e.getKey(), regionLocations);
} else {
tableLocations.replace(e.getKey(), regionLocations, updatedLocations);
}
}
}
}
}
this.cachedServers.remove(serverName);
}
if (deletedSomething && LOG.isDebugEnabled()) {
LOG.debug("Removed all cached region locations that map to " + serverName);
}
}
/**
* Delete all cached entries of a table.
*/
public void clearCache(final TableName tableName) {
this.cachedRegionLocations.remove(tableName);
}
/**
* Delete a cached location, no matter what it is. Called when we were told to not use cache.
* @param tableName tableName
* @param row
*/
public void clearCache(final TableName tableName, final byte [] row) {
ConcurrentMap<byte[], RegionLocations> tableLocations = getTableLocations(tableName);
RegionLocations regionLocations = getCachedLocation(tableName, row);
if (regionLocations != null) {
byte[] startKey = regionLocations.getRegionLocation().getRegionInfo().getStartKey();
boolean removed = tableLocations.remove(startKey, regionLocations);
if (removed && LOG.isDebugEnabled()) {
LOG.debug("Removed " + regionLocations + " from cache");
}
}
}
/**
* Delete a cached location for a table, row and server
*/
public void clearCache(final TableName tableName, final byte [] row, ServerName serverName) {
ConcurrentMap<byte[], RegionLocations> tableLocations = getTableLocations(tableName);
RegionLocations regionLocations = getCachedLocation(tableName, row);
if (regionLocations != null) {
RegionLocations updatedLocations = regionLocations.removeByServer(serverName);
if (updatedLocations != regionLocations) {
byte[] startKey = regionLocations.getRegionLocation().getRegionInfo().getStartKey();
if (updatedLocations.isEmpty()) {
tableLocations.remove(startKey, regionLocations);
} else {
tableLocations.replace(startKey, regionLocations, updatedLocations);
}
}
}
}
/**
* Deletes the cached location of the region if necessary, based on some error from source.
* @param hri The region in question.
*/
public void clearCache(HRegionInfo hri) {
ConcurrentMap<byte[], RegionLocations> tableLocations = getTableLocations(hri.getTable());
RegionLocations regionLocations = tableLocations.get(hri.getStartKey());
if (regionLocations != null) {
HRegionLocation oldLocation = regionLocations.getRegionLocation(hri.getReplicaId());
RegionLocations updatedLocations = regionLocations.remove(oldLocation);
if (updatedLocations != regionLocations) {
if (updatedLocations.isEmpty()) {
tableLocations.remove(hri.getStartKey(), regionLocations);
} else {
tableLocations.replace(hri.getStartKey(), regionLocations, updatedLocations);
}
}
}
}
public void clearCache(final HRegionLocation location) {
if (location == null) {
return;
}
TableName tableName = location.getRegionInfo().getTable();
ConcurrentMap<byte[], RegionLocations> tableLocations = getTableLocations(tableName);
RegionLocations rll = tableLocations.get(location.getRegionInfo().getStartKey());
RegionLocations updatedLocations = rll.remove(location);
if (updatedLocations.isEmpty()) {
tableLocations.remove(location.getRegionInfo().getStartKey(), rll);
}
if (LOG.isDebugEnabled() && (rll == updatedLocations)) {
LOG.debug("Removed " +
location.getRegionInfo().getRegionNameAsString() +
" for tableName=" + tableName +
" from cache");
}
}
}

View File

@ -30,11 +30,14 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ExceptionUtil;
@ -50,6 +53,7 @@ import org.apache.hadoop.hbase.util.ExceptionUtil;
* see HBASE-5986, and {@link DefaultMetaScannerVisitor} for details. </p>
*/
@InterfaceAudience.Private
//TODO: merge this to MetaReader, get rid of it.
public class MetaScanner {
private static final Log LOG = LogFactory.getLog(MetaScanner.class);
/**
@ -216,14 +220,14 @@ public class MetaScanner {
* table Result.
* @param data a Result object from the catalog table scan
* @return HRegionInfo or null
* @deprecated Use {@link MetaReader#getRegionLocations(Result)}
*/
@Deprecated
public static HRegionInfo getHRegionInfo(Result data) {
return HRegionInfo.getHRegionInfo(data);
}
/**
* Used in tests.
*
* Lists all of the regions currently in META.
* @param conf
* @param offlined True if we are to include offlined regions, false and we'll
@ -234,22 +238,23 @@ public class MetaScanner {
public static List<HRegionInfo> listAllRegions(Configuration conf, final boolean offlined)
throws IOException {
final List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
MetaScannerVisitor visitor = new DefaultMetaScannerVisitor() {
MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
@Override
public boolean processRowInternal(Result result) throws IOException {
public boolean processRow(Result result) throws IOException {
if (result == null || result.isEmpty()) {
return true;
}
HRegionInfo regionInfo = getHRegionInfo(result);
if (regionInfo == null) {
LOG.warn("Null REGIONINFO_QUALIFIER: " + result);
return true;
RegionLocations locations = MetaReader.getRegionLocations(result);
if (locations == null) return true;
for (HRegionLocation loc : locations.getRegionLocations()) {
if (loc != null) {
HRegionInfo regionInfo = loc.getRegionInfo();
// If region offline AND we are not to include offlined regions, return.
if (regionInfo.isOffline() && !offlined) continue;
regions.add(regionInfo);
}
}
// If region offline AND we are not to include offlined regions, return.
if (regionInfo.isOffline() && !offlined) return true;
regions.add(regionInfo);
return true;
}
};
@ -272,10 +277,34 @@ public class MetaScanner {
new TreeMap<HRegionInfo, ServerName>();
MetaScannerVisitor visitor = new TableMetaScannerVisitor(tableName) {
@Override
public boolean processRowInternal(Result rowResult) throws IOException {
HRegionInfo info = getHRegionInfo(rowResult);
ServerName serverName = HRegionInfo.getServerName(rowResult);
regions.put(new UnmodifyableHRegionInfo(info), serverName);
public boolean processRowInternal(Result result) throws IOException {
RegionLocations locations = MetaReader.getRegionLocations(result);
if (locations == null) return true;
for (HRegionLocation loc : locations.getRegionLocations()) {
if (loc != null) {
HRegionInfo regionInfo = loc.getRegionInfo();
regions.put(new UnmodifyableHRegionInfo(regionInfo), loc.getServerName());
}
}
return true;
}
};
metaScan(conf, connection, visitor, tableName);
return regions;
}
/**
* Lists table regions and locations grouped by region range from META.
*/
public static List<RegionLocations> listTableRegionLocations(Configuration conf,
ClusterConnection connection, final TableName tableName) throws IOException {
final List<RegionLocations> regions = new ArrayList<RegionLocations>();
MetaScannerVisitor visitor = new TableMetaScannerVisitor(tableName) {
@Override
public boolean processRowInternal(Result result) throws IOException {
RegionLocations locations = MetaReader.getRegionLocations(result);
if (locations == null) return true;
regions.add(locations);
return true;
}
};

View File

@ -0,0 +1,65 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
/**
* Utility methods which contain the logic for regions and replicas.
*/
@InterfaceAudience.Private
public class RegionReplicaUtil {
/**
* The default replicaId for the region
*/
private 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.
*/
public static HRegionInfo getRegionInfoForReplica(HRegionInfo regionInfo, int replicaId) {
if (regionInfo.getReplicaId() == replicaId) {
return regionInfo;
}
HRegionInfo replicaInfo = new HRegionInfo(regionInfo.getTable(), regionInfo.getStartKey(),
regionInfo.getEndKey(), regionInfo.isSplit(), regionInfo.getRegionId(), replicaId);
replicaInfo.setOffline(regionInfo.isOffline());
return replicaInfo;
}
/**
* Returns the HRegionInfo for the default replicaId (0). 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.
* @return an HRegionInfo object corresponding to the same range (table, start and
* end key), but for the default replicaId.
*/
public static HRegionInfo getRegionInfoForDefaultReplica(HRegionInfo regionInfo) {
return getRegionInfoForReplica(regionInfo, DEFAULT_REPLICA_ID);
}
}

View File

@ -73,6 +73,7 @@ public abstract class RegionServerCallable<T> implements RetryingCallable<T> {
* @param reload Set this to true if connection should re-find the region
* @throws IOException e
*/
@Override
public void prepare(final boolean reload) throws IOException {
this.location = connection.getRegionLocation(tableName, row, reload);
if (this.location == null) {
@ -124,7 +125,7 @@ public abstract class RegionServerCallable<T> implements RetryingCallable<T> {
// hbase:meta again to find the new location
if (this.location != null) getConnection().clearCaches(location.getServerName());
} else if (t instanceof RegionMovedException) {
getConnection().updateCachedLocations(tableName, row, t, location.getServerName());
getConnection().updateCachedLocations(tableName, row, t, location);
} else if (t instanceof NotServingRegionException && !retrying) {
// Purge cache entries for this specific region from hbase:meta cache
// since we don't call connect(true) when number of retries is 1.

View File

@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionLocation;
/**
* Cluster registry.
@ -36,7 +36,7 @@ interface Registry {
* @return Meta region location
* @throws IOException
*/
HRegionLocation getMetaRegionLocation() throws IOException;
RegionLocations getMetaRegionLocation() throws IOException;
/**
* @return Cluster id.

View File

@ -22,10 +22,11 @@ import java.io.InterruptedIOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
@ -49,7 +50,7 @@ class ZooKeeperRegistry implements Registry {
}
@Override
public HRegionLocation getMetaRegionLocation() throws IOException {
public RegionLocations getMetaRegionLocation() throws IOException {
ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher();
try {
@ -62,7 +63,8 @@ class ZooKeeperRegistry implements Registry {
"; serverName=" + ((servername == null) ? "null" : servername));
}
if (servername == null) return null;
return new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, servername, 0);
HRegionLocation loc = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, servername, 0);
return new RegionLocations(new HRegionLocation[] {loc});
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return null;

View File

@ -0,0 +1,275 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import org.junit.Test;
public class TestRegionLocations {
ServerName sn0 = ServerName.valueOf("host0", 10, 10);
ServerName sn1 = ServerName.valueOf("host1", 10, 10);
ServerName sn2 = ServerName.valueOf("host2", 10, 10);
ServerName sn3 = ServerName.valueOf("host3", 10, 10);
HRegionInfo info0 = hri(0);
HRegionInfo info1 = hri(1);
HRegionInfo info2 = hri(2);
HRegionInfo info9 = hri(9);
@Test
public void testSizeMethods() {
RegionLocations list = new RegionLocations();
assertTrue(list.isEmpty());
assertEquals(0, list.size());
assertEquals(0, list.numNonNullElements());
list = hrll((HRegionLocation)null);
assertTrue(list.isEmpty());
assertEquals(0, list.size());
assertEquals(0, list.numNonNullElements());
HRegionInfo info0 = hri(0);
list = hrll(hrl(info0, null));
assertFalse(list.isEmpty());
assertEquals(1, list.size());
assertEquals(1, list.numNonNullElements());
HRegionInfo info9 = hri(9);
list = hrll(hrl(info9, null));
assertFalse(list.isEmpty());
assertEquals(10, list.size());
assertEquals(1, list.numNonNullElements());
list = hrll(hrl(info0, null), hrl(info9, null));
assertFalse(list.isEmpty());
assertEquals(10, list.size());
assertEquals(2, list.numNonNullElements());
}
private HRegionInfo hri(int replicaId) {
TableName table = TableName.valueOf("table");
byte[] startKey = HConstants.EMPTY_START_ROW;
byte[] endKey = HConstants.EMPTY_END_ROW;
long regionId = System.currentTimeMillis();
HRegionInfo info = new HRegionInfo(table, startKey, endKey, false, regionId, replicaId);
return info;
}
private HRegionLocation hrl(HRegionInfo hri, ServerName sn) {
return new HRegionLocation(hri, sn);
}
private HRegionLocation hrl(HRegionInfo hri, ServerName sn, long seqNum) {
return new HRegionLocation(hri, sn, seqNum);
}
private RegionLocations hrll(HRegionLocation ... locations) {
return new RegionLocations(locations);
}
@Test
public void testRemoveByServer() {
RegionLocations list;
// test remove from empty list
list = new RegionLocations();
assertTrue(list == list.removeByServer(sn0));
// test remove from single element list
list = hrll(hrl(info0, sn0));
assertTrue(list == list.removeByServer(sn1));
list = list.removeByServer(sn0);
assertTrue(list.isEmpty());
// test remove from multi element list
list = hrll(hrl(info0, sn0), hrl(info1, sn1), hrl(info2, sn2), hrl(info9, sn2));
assertTrue(list == list.removeByServer(sn3)); // no region is mapped to sn3
list = list.removeByServer(sn0);
assertNull(list.getRegionLocation(0));
assertEquals(sn1, list.getRegionLocation(1).getServerName());
assertEquals(sn2, list.getRegionLocation(2).getServerName());
assertNull(list.getRegionLocation(5));
assertEquals(sn2, list.getRegionLocation(9).getServerName());
// test multi-element remove from multi element list
list = hrll(hrl(info0, sn1), hrl(info1, sn1), hrl(info2, sn0), hrl(info9, sn0));
list = list.removeByServer(sn0);
assertEquals(sn1, list.getRegionLocation(0).getServerName());
assertEquals(sn1, list.getRegionLocation(1).getServerName());
assertNull(list.getRegionLocation(2));
assertNull(list.getRegionLocation(5));
assertNull(list.getRegionLocation(9));
}
@Test
public void testRemove() {
RegionLocations list;
// test remove from empty list
list = new RegionLocations();
assertTrue(list == list.remove(hrl(info0, sn0)));
// test remove from single element list
list = hrll(hrl(info0, sn0));
assertTrue(list == list.remove(hrl(info0, sn1)));
list = list.remove(hrl(info0, sn0));
assertTrue(list.isEmpty());
// test remove from multi element list
list = hrll(hrl(info0, sn0), hrl(info1, sn1), hrl(info2, sn2), hrl(info9, sn2));
assertTrue(list == list.remove(hrl(info1, sn3))); // no region is mapped to sn3
list = list.remove(hrl(info0, sn0));
assertNull(list.getRegionLocation(0));
assertEquals(sn1, list.getRegionLocation(1).getServerName());
assertEquals(sn2, list.getRegionLocation(2).getServerName());
assertNull(list.getRegionLocation(5));
assertEquals(sn2, list.getRegionLocation(9).getServerName());
list = list.remove(hrl(info9, sn2));
assertNull(list.getRegionLocation(0));
assertEquals(sn1, list.getRegionLocation(1).getServerName());
assertEquals(sn2, list.getRegionLocation(2).getServerName());
assertNull(list.getRegionLocation(5));
assertNull(list.getRegionLocation(9));
// test multi-element remove from multi element list
list = hrll(hrl(info0, sn1), hrl(info1, sn1), hrl(info2, sn0), hrl(info9, sn0));
list = list.remove(hrl(info9, sn0));
assertEquals(sn1, list.getRegionLocation(0).getServerName());
assertEquals(sn1, list.getRegionLocation(1).getServerName());
assertEquals(sn0, list.getRegionLocation(2).getServerName());
assertNull(list.getRegionLocation(5));
assertNull(list.getRegionLocation(9));
}
@Test
public void testUpdateLocation() {
RegionLocations list;
// test add to empty list
list = new RegionLocations();
list = list.updateLocation(hrl(info0, sn1), false, false);
assertEquals(sn1, list.getRegionLocation(0).getServerName());
// test add to non-empty list
list = list.updateLocation(hrl(info9, sn3, 10), false, false);
assertEquals(sn3, list.getRegionLocation(9).getServerName());
assertEquals(10, list.size());
list = list.updateLocation(hrl(info2, sn2, 10), false, false);
assertEquals(sn2, list.getRegionLocation(2).getServerName());
assertEquals(10, list.size());
// test update greater SeqNum
list = list.updateLocation(hrl(info2, sn3, 11), false, false);
assertEquals(sn3, list.getRegionLocation(2).getServerName());
assertEquals(sn3, list.getRegionLocation(9).getServerName());
// test update equal SeqNum
list = list.updateLocation(hrl(info2, sn1, 11), false, false); // should not update
assertEquals(sn3, list.getRegionLocation(2).getServerName());
assertEquals(sn3, list.getRegionLocation(9).getServerName());
list = list.updateLocation(hrl(info2, sn1, 11), true, false); // should update
assertEquals(sn1, list.getRegionLocation(2).getServerName());
assertEquals(sn3, list.getRegionLocation(9).getServerName());
// test force update
list = list.updateLocation(hrl(info2, sn2, 9), false, true); // should update
assertEquals(sn2, list.getRegionLocation(2).getServerName());
assertEquals(sn3, list.getRegionLocation(9).getServerName());
}
@Test
public void testMergeLocations() {
RegionLocations list1, list2;
// test merge empty lists
list1 = new RegionLocations();
list2 = new RegionLocations();
assertTrue(list1 == list1.mergeLocations(list2));
// test merge non-empty and empty
list2 = hrll(hrl(info0, sn0));
list1 = list1.mergeLocations(list2);
assertEquals(sn0, list1.getRegionLocation(0).getServerName());
// test merge empty and non empty
list1 = hrll();
list1 = list2.mergeLocations(list1);
assertEquals(sn0, list1.getRegionLocation(0).getServerName());
// test merge non intersecting
list1 = hrll(hrl(info0, sn0), hrl(info1, sn1));
list2 = hrll(hrl(info2, sn2));
list1 = list2.mergeLocations(list1);
assertEquals(sn0, list1.getRegionLocation(0).getServerName());
assertEquals(sn1, list1.getRegionLocation(1).getServerName());
assertEquals(sn2, list1.getRegionLocation(2).getServerName());
// do the other way merge as well
list1 = hrll(hrl(info0, sn0), hrl(info1, sn1));
list2 = hrll(hrl(info2, sn2));
list1 = list1.mergeLocations(list2);
assertEquals(sn0, list1.getRegionLocation(0).getServerName());
assertEquals(sn1, list1.getRegionLocation(1).getServerName());
assertEquals(sn2, list1.getRegionLocation(2).getServerName());
// test intersecting lists same seqNum
list1 = hrll(hrl(info0, sn0), hrl(info1, sn1));
list2 = hrll(hrl(info0, sn2), hrl(info1, sn2), hrl(info9, sn3));
list1 = list2.mergeLocations(list1); // list1 should override
assertEquals(10, list1.size());
assertEquals(sn0, list1.getRegionLocation(0).getServerName());
assertEquals(sn1, list1.getRegionLocation(1).getServerName());
assertEquals(sn3, list1.getRegionLocation(9).getServerName());
// do the other way
list1 = hrll(hrl(info0, sn0), hrl(info1, sn1));
list2 = hrll(hrl(info0, sn2), hrl(info1, sn2), hrl(info9, sn3));
list1 = list1.mergeLocations(list2); // list2 should override
assertEquals(10, list1.size());
assertEquals(sn2, list1.getRegionLocation(0).getServerName());
assertEquals(sn2, list1.getRegionLocation(1).getServerName());
assertEquals(sn3, list1.getRegionLocation(9).getServerName());
// test intersecting lists different seqNum
list1 = hrll(hrl(info0, sn0, 10), hrl(info1, sn1, 10));
list2 = hrll(hrl(info0, sn2, 11), hrl(info1, sn2, 11), hrl(info9, sn3, 11));
list1 = list1.mergeLocations(list2); // list2 should override because of seqNum
assertEquals(10, list1.size());
assertEquals(sn2, list1.getRegionLocation(0).getServerName());
assertEquals(sn2, list1.getRegionLocation(1).getServerName());
assertEquals(sn3, list1.getRegionLocation(9).getServerName());
// do the other way
list1 = hrll(hrl(info0, sn0, 10), hrl(info1, sn1, 10));
list2 = hrll(hrl(info0, sn2, 11), hrl(info1, sn2, 11), hrl(info9, sn3, 11));
list1 = list1.mergeLocations(list2); // list2 should override
assertEquals(10, list1.size());
assertEquals(sn2, list1.getRegionLocation(0).getServerName());
assertEquals(sn2, list1.getRegionLocation(1).getServerName());
assertEquals(sn3, list1.getRegionLocation(9).getServerName());
}
}

View File

@ -43,6 +43,7 @@ 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.RegionLocations;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.RegionTooBusyException;
import org.apache.hadoop.hbase.ServerName;
@ -117,8 +118,9 @@ public class TestClientNoCluster extends Configured implements Tool {
}
@Override
public HRegionLocation getMetaRegionLocation() throws IOException {
return new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, META_HOST);
public RegionLocations getMetaRegionLocation() throws IOException {
return new RegionLocations(
new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, META_HOST));
}
@Override
@ -142,7 +144,7 @@ public class TestClientNoCluster extends Configured implements Tool {
* Remove the @Ignore to try out timeout and retry asettings
* @throws IOException
*/
@Ignore
@Ignore
@Test
public void testTimeoutAndRetries() throws IOException {
Configuration localConfig = HBaseConfiguration.create(this.conf);
@ -759,7 +761,7 @@ public class TestClientNoCluster extends Configured implements Tool {
// an exception is thrown -- usually RegionTooBusyException when we have more than
// hbase.test.multi.too.many requests outstanding at any time.
getConf().setInt("hbase.client.start.log.errors.counter", 0);
// Ugly but this is only way to pass in configs.into ManyServersManyRegionsConnection class.
getConf().setInt("hbase.test.regions", regions);
getConf().setLong("hbase.test.namespace.span", namespaceSpan);

View File

@ -397,13 +397,19 @@ public final class HConstants {
public static final byte [] REGIONINFO_QUALIFIER = Bytes.toBytes(REGIONINFO_QUALIFIER_STR);
/** The server column qualifier */
public static final byte [] SERVER_QUALIFIER = Bytes.toBytes("server");
public static final String SERVER_QUALIFIER_STR = "server";
/** The server column qualifier */
public static final byte [] SERVER_QUALIFIER = Bytes.toBytes(SERVER_QUALIFIER_STR);
/** The startcode column qualifier */
public static final byte [] STARTCODE_QUALIFIER = Bytes.toBytes("serverstartcode");
public static final String STARTCODE_QUALIFIER_STR = "serverstartcode";
/** The startcode column qualifier */
public static final byte [] STARTCODE_QUALIFIER = Bytes.toBytes(STARTCODE_QUALIFIER_STR);
/** The open seqnum column qualifier */
public static final byte [] SEQNUM_QUALIFIER = Bytes.toBytes("seqnumDuringOpen");
public static final String SEQNUM_QUALIFIER_STR = "seqnumDuringOpen";
/** The open seqnum column qualifier */
public static final byte [] SEQNUM_QUALIFIER = Bytes.toBytes(SEQNUM_QUALIFIER_STR);
/** The state column qualifier */
public static final byte [] STATE_QUALIFIER = Bytes.toBytes("state");

View File

@ -3687,6 +3687,16 @@ public final class HBaseProtos {
* <code>optional bool split = 6;</code>
*/
boolean getSplit();
// optional int32 replica_id = 7 [default = 0];
/**
* <code>optional int32 replica_id = 7 [default = 0];</code>
*/
boolean hasReplicaId();
/**
* <code>optional int32 replica_id = 7 [default = 0];</code>
*/
int getReplicaId();
}
/**
* Protobuf type {@code RegionInfo}
@ -3782,6 +3792,11 @@ public final class HBaseProtos {
split_ = input.readBool();
break;
}
case 56: {
bitField0_ |= 0x00000040;
replicaId_ = input.readInt32();
break;
}
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -3924,6 +3939,22 @@ public final class HBaseProtos {
return split_;
}
// optional int32 replica_id = 7 [default = 0];
public static final int REPLICA_ID_FIELD_NUMBER = 7;
private int replicaId_;
/**
* <code>optional int32 replica_id = 7 [default = 0];</code>
*/
public boolean hasReplicaId() {
return ((bitField0_ & 0x00000040) == 0x00000040);
}
/**
* <code>optional int32 replica_id = 7 [default = 0];</code>
*/
public int getReplicaId() {
return replicaId_;
}
private void initFields() {
regionId_ = 0L;
tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
@ -3931,6 +3962,7 @@ public final class HBaseProtos {
endKey_ = com.google.protobuf.ByteString.EMPTY;
offline_ = false;
split_ = false;
replicaId_ = 0;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@ -3974,6 +4006,9 @@ public final class HBaseProtos {
if (((bitField0_ & 0x00000020) == 0x00000020)) {
output.writeBool(6, split_);
}
if (((bitField0_ & 0x00000040) == 0x00000040)) {
output.writeInt32(7, replicaId_);
}
getUnknownFields().writeTo(output);
}
@ -4007,6 +4042,10 @@ public final class HBaseProtos {
size += com.google.protobuf.CodedOutputStream
.computeBoolSize(6, split_);
}
if (((bitField0_ & 0x00000040) == 0x00000040)) {
size += com.google.protobuf.CodedOutputStream
.computeInt32Size(7, replicaId_);
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@ -4060,6 +4099,11 @@ public final class HBaseProtos {
result = result && (getSplit()
== other.getSplit());
}
result = result && (hasReplicaId() == other.hasReplicaId());
if (hasReplicaId()) {
result = result && (getReplicaId()
== other.getReplicaId());
}
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@ -4097,6 +4141,10 @@ public final class HBaseProtos {
hash = (37 * hash) + SPLIT_FIELD_NUMBER;
hash = (53 * hash) + hashBoolean(getSplit());
}
if (hasReplicaId()) {
hash = (37 * hash) + REPLICA_ID_FIELD_NUMBER;
hash = (53 * hash) + getReplicaId();
}
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@ -4228,6 +4276,8 @@ public final class HBaseProtos {
bitField0_ = (bitField0_ & ~0x00000010);
split_ = false;
bitField0_ = (bitField0_ & ~0x00000020);
replicaId_ = 0;
bitField0_ = (bitField0_ & ~0x00000040);
return this;
}
@ -4284,6 +4334,10 @@ public final class HBaseProtos {
to_bitField0_ |= 0x00000020;
}
result.split_ = split_;
if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
to_bitField0_ |= 0x00000040;
}
result.replicaId_ = replicaId_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@ -4318,6 +4372,9 @@ public final class HBaseProtos {
if (other.hasSplit()) {
setSplit(other.getSplit());
}
if (other.hasReplicaId()) {
setReplicaId(other.getReplicaId());
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@ -4645,6 +4702,39 @@ public final class HBaseProtos {
return this;
}
// optional int32 replica_id = 7 [default = 0];
private int replicaId_ ;
/**
* <code>optional int32 replica_id = 7 [default = 0];</code>
*/
public boolean hasReplicaId() {
return ((bitField0_ & 0x00000040) == 0x00000040);
}
/**
* <code>optional int32 replica_id = 7 [default = 0];</code>
*/
public int getReplicaId() {
return replicaId_;
}
/**
* <code>optional int32 replica_id = 7 [default = 0];</code>
*/
public Builder setReplicaId(int value) {
bitField0_ |= 0x00000040;
replicaId_ = value;
onChanged();
return this;
}
/**
* <code>optional int32 replica_id = 7 [default = 0];</code>
*/
public Builder clearReplicaId() {
bitField0_ = (bitField0_ & ~0x00000040);
replicaId_ = 0;
onChanged();
return this;
}
// @@protoc_insertion_point(builder_scope:RegionInfo)
}
@ -16234,43 +16324,43 @@ public final class HBaseProtos {
"ingPair\"o\n\022ColumnFamilySchema\022\014\n\004name\030\001 " +
"\002(\014\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPair" +
"\022&\n\rconfiguration\030\003 \003(\0132\017.NameStringPair" +
"\"\203\001\n\nRegionInfo\022\021\n\tregion_id\030\001 \002(\004\022\036\n\nta",
"\"\232\001\n\nRegionInfo\022\021\n\tregion_id\030\001 \002(\004\022\036\n\nta",
"ble_name\030\002 \002(\0132\n.TableName\022\021\n\tstart_key\030" +
"\003 \001(\014\022\017\n\007end_key\030\004 \001(\014\022\017\n\007offline\030\005 \001(\010\022" +
"\r\n\005split\030\006 \001(\010\"1\n\014FavoredNodes\022!\n\014favore" +
"d_node\030\001 \003(\0132\013.ServerName\"\225\001\n\017RegionSpec" +
"ifier\0222\n\004type\030\001 \002(\0162$.RegionSpecifier.Re" +
"gionSpecifierType\022\r\n\005value\030\002 \002(\014\"?\n\023Regi" +
"onSpecifierType\022\017\n\013REGION_NAME\020\001\022\027\n\023ENCO" +
"DED_REGION_NAME\020\002\"%\n\tTimeRange\022\014\n\004from\030\001" +
" \001(\004\022\n\n\002to\030\002 \001(\004\"A\n\nServerName\022\021\n\thost_n" +
"ame\030\001 \002(\t\022\014\n\004port\030\002 \001(\r\022\022\n\nstart_code\030\003 ",
"\001(\004\"\033\n\013Coprocessor\022\014\n\004name\030\001 \002(\t\"-\n\016Name" +
"StringPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\"" +
",\n\rNameBytesPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030" +
"\002 \001(\014\"/\n\016BytesBytesPair\022\r\n\005first\030\001 \002(\014\022\016" +
"\n\006second\030\002 \002(\014\",\n\rNameInt64Pair\022\014\n\004name\030" +
"\001 \001(\t\022\r\n\005value\030\002 \001(\003\"\275\001\n\023SnapshotDescrip" +
"tion\022\014\n\004name\030\001 \002(\t\022\r\n\005table\030\002 \001(\t\022\030\n\rcre" +
"ation_time\030\003 \001(\003:\0010\022.\n\004type\030\004 \001(\0162\031.Snap" +
"shotDescription.Type:\005FLUSH\022\017\n\007version\030\005" +
" \001(\005\".\n\004Type\022\014\n\010DISABLED\020\000\022\t\n\005FLUSH\020\001\022\r\n",
"\tSKIPFLUSH\020\002\"}\n\024ProcedureDescription\022\021\n\t" +
"signature\030\001 \002(\t\022\020\n\010instance\030\002 \001(\t\022\030\n\rcre" +
"ation_time\030\003 \001(\003:\0010\022&\n\rconfiguration\030\004 \003" +
"(\0132\017.NameStringPair\"\n\n\010EmptyMsg\"\033\n\007LongM" +
"sg\022\020\n\010long_msg\030\001 \002(\003\"\037\n\tDoubleMsg\022\022\n\ndou" +
"ble_msg\030\001 \002(\001\"\'\n\rBigDecimalMsg\022\026\n\016bigdec" +
"imal_msg\030\001 \002(\014\"5\n\004UUID\022\026\n\016least_sig_bits" +
"\030\001 \002(\004\022\025\n\rmost_sig_bits\030\002 \002(\004\"K\n\023Namespa" +
"ceDescriptor\022\014\n\004name\030\001 \002(\014\022&\n\rconfigurat" +
"ion\030\002 \003(\0132\017.NameStringPair\"$\n\020RegionServ",
"erInfo\022\020\n\010infoPort\030\001 \001(\005*r\n\013CompareType\022" +
"\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022" +
"\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007" +
"GREATER\020\005\022\t\n\005NO_OP\020\006B>\n*org.apache.hadoo" +
"p.hbase.protobuf.generatedB\013HBaseProtosH" +
"\001\240\001\001"
"\r\n\005split\030\006 \001(\010\022\025\n\nreplica_id\030\007 \001(\005:\0010\"1\n" +
"\014FavoredNodes\022!\n\014favored_node\030\001 \003(\0132\013.Se" +
"rverName\"\225\001\n\017RegionSpecifier\0222\n\004type\030\001 \002" +
"(\0162$.RegionSpecifier.RegionSpecifierType" +
"\022\r\n\005value\030\002 \002(\014\"?\n\023RegionSpecifierType\022\017" +
"\n\013REGION_NAME\020\001\022\027\n\023ENCODED_REGION_NAME\020\002" +
"\"%\n\tTimeRange\022\014\n\004from\030\001 \001(\004\022\n\n\002to\030\002 \001(\004\"" +
"A\n\nServerName\022\021\n\thost_name\030\001 \002(\t\022\014\n\004port",
"\030\002 \001(\r\022\022\n\nstart_code\030\003 \001(\004\"\033\n\013Coprocesso" +
"r\022\014\n\004name\030\001 \002(\t\"-\n\016NameStringPair\022\014\n\004nam" +
"e\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\rNameBytesPair\022" +
"\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016BytesByt" +
"esPair\022\r\n\005first\030\001 \002(\014\022\016\n\006second\030\002 \002(\014\",\n" +
"\rNameInt64Pair\022\014\n\004name\030\001 \001(\t\022\r\n\005value\030\002 " +
"\001(\003\"\275\001\n\023SnapshotDescription\022\014\n\004name\030\001 \002(" +
"\t\022\r\n\005table\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(\003:" +
"\0010\022.\n\004type\030\004 \001(\0162\031.SnapshotDescription.T" +
"ype:\005FLUSH\022\017\n\007version\030\005 \001(\005\".\n\004Type\022\014\n\010D",
"ISABLED\020\000\022\t\n\005FLUSH\020\001\022\r\n\tSKIPFLUSH\020\002\"}\n\024P" +
"rocedureDescription\022\021\n\tsignature\030\001 \002(\t\022\020" +
"\n\010instance\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(\003:" +
"\0010\022&\n\rconfiguration\030\004 \003(\0132\017.NameStringPa" +
"ir\"\n\n\010EmptyMsg\"\033\n\007LongMsg\022\020\n\010long_msg\030\001 " +
"\002(\003\"\037\n\tDoubleMsg\022\022\n\ndouble_msg\030\001 \002(\001\"\'\n\r" +
"BigDecimalMsg\022\026\n\016bigdecimal_msg\030\001 \002(\014\"5\n" +
"\004UUID\022\026\n\016least_sig_bits\030\001 \002(\004\022\025\n\rmost_si" +
"g_bits\030\002 \002(\004\"K\n\023NamespaceDescriptor\022\014\n\004n" +
"ame\030\001 \002(\014\022&\n\rconfiguration\030\002 \003(\0132\017.NameS",
"tringPair\"$\n\020RegionServerInfo\022\020\n\010infoPor" +
"t\030\001 \001(\005*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS" +
"_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n" +
"\020GREATER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_O" +
"P\020\006B>\n*org.apache.hadoop.hbase.protobuf." +
"generatedB\013HBaseProtosH\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -16300,7 +16390,7 @@ public final class HBaseProtos {
internal_static_RegionInfo_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_RegionInfo_descriptor,
new java.lang.String[] { "RegionId", "TableName", "StartKey", "EndKey", "Offline", "Split", });
new java.lang.String[] { "RegionId", "TableName", "StartKey", "EndKey", "Offline", "Split", "ReplicaId", });
internal_static_FavoredNodes_descriptor =
getDescriptor().getMessageTypes().get(4);
internal_static_FavoredNodes_fieldAccessorTable = new

View File

@ -64,6 +64,7 @@ message RegionInfo {
optional bytes end_key = 4;
optional bool offline = 5;
optional bool split = 6;
optional int32 replica_id = 7 [default = 0];
}
/**

View File

@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.catalog;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.ConnectException;
import java.util.ArrayList;
import java.util.List;
@ -47,11 +46,10 @@ import com.google.protobuf.ServiceException;
/**
* Writes region and assignment information to <code>hbase:meta</code>.
* TODO: Put MetaReader and MetaEditor together; doesn't make sense having
* them distinct. see HBASE-3475.
*/
@InterfaceAudience.Private
public class MetaEditor {
public class MetaEditor extends MetaReader {
// TODO: Strip CatalogTracker from this class. Its all over and in the end
// its only used to get its Configuration so we can get associated
// Connection.
@ -290,7 +288,7 @@ public class MetaEditor {
Put put = new Put(regionInfo.getRegionName());
addRegionInfo(put, regionInfo);
if (sn != null) {
addLocation(put, sn, openSeqNum);
addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
}
putToMetaTable(catalogTracker, put);
LOG.info("Added daughter " + regionInfo.getEncodedName() +
@ -327,7 +325,7 @@ public class MetaEditor {
Delete deleteB = makeDeleteFromRegionInfo(regionB);
// The merged is a new region, openSeqNum = 1 is fine.
addLocation(putOfMerged, sn, 1);
addLocation(putOfMerged, sn, 1, mergedRegion.getReplicaId());
byte[] tableRow = Bytes.toBytes(mergedRegion.getRegionNameAsString()
+ HConstants.DELIMITER);
@ -365,8 +363,8 @@ public class MetaEditor {
Put putA = makePutFromRegionInfo(splitA);
Put putB = makePutFromRegionInfo(splitB);
addLocation(putA, sn, 1); //these are new regions, openSeqNum = 1 is fine.
addLocation(putB, sn, 1);
addLocation(putA, sn, 1, splitA.getReplicaId()); //new regions, openSeqNum = 1 is fine.
addLocation(putB, sn, 1, splitB.getReplicaId());
byte[] tableRow = Bytes.toBytes(parent.getRegionNameAsString() + HConstants.DELIMITER);
multiMutate(meta, tableRow, putParent, putA, putB);
@ -401,29 +399,6 @@ public class MetaEditor {
}
}
/**
* Updates the location of the specified hbase:meta region in ROOT to be the
* specified server hostname and startcode.
* <p>
* Uses passed catalog tracker to get a connection to the server hosting
* ROOT and makes edits to that region.
*
* @param catalogTracker catalog tracker
* @param regionInfo region to update location of
* @param sn Server name
* @param openSeqNum the latest sequence number obtained when the region was open
* @throws IOException
* @throws ConnectException Usually because the regionserver carrying hbase:meta
* is down.
* @throws NullPointerException Because no -ROOT- server connection
*/
public static void updateMetaLocation(CatalogTracker catalogTracker,
HRegionInfo regionInfo, ServerName sn, long openSeqNum)
throws IOException, ConnectException {
updateLocation(catalogTracker, regionInfo, sn, openSeqNum);
}
/**
* Updates the location of the specified region in hbase:meta to be the specified
* server hostname and startcode.
@ -458,8 +433,9 @@ public class MetaEditor {
private static void updateLocation(final CatalogTracker catalogTracker,
HRegionInfo regionInfo, ServerName sn, long openSeqNum)
throws IOException {
Put put = new Put(regionInfo.getRegionName());
addLocation(put, sn, openSeqNum);
// region replicas are kept in the primary region's row
Put put = new Put(getMetaKeyForRegion(regionInfo));
addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
putToCatalogTable(catalogTracker, put);
LOG.info("Updated row " + regionInfo.getRegionNameAsString() +
" with server=" + sn);
@ -568,12 +544,12 @@ public class MetaEditor {
return p;
}
private static Put addLocation(final Put p, final ServerName sn, long openSeqNum) {
p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
private static Put addLocation(final Put p, final ServerName sn, long openSeqNum, int replicaId){
p.addImmutable(HConstants.CATALOG_FAMILY, MetaReader.getServerColumn(replicaId),
Bytes.toBytes(sn.getHostAndPort()));
p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
p.addImmutable(HConstants.CATALOG_FAMILY, MetaReader.getStartCodeColumn(replicaId),
Bytes.toBytes(sn.getStartcode()));
p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.SEQNUM_QUALIFIER,
p.addImmutable(HConstants.CATALOG_FAMILY, MetaReader.getSeqNumColumn(replicaId),
Bytes.toBytes(openSeqNum));
return p;
}

View File

@ -85,6 +85,7 @@ public class CoprocessorHConnection implements ClusterConnection {
this.delegate = delegate;
}
@Override
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService.BlockingInterface
getClient(ServerName serverName) throws IOException {
// client is trying to reach off-server, so we can't do anything special
@ -96,262 +97,323 @@ public class CoprocessorHConnection implements ClusterConnection {
return server.getRSRpcServices();
}
@Override
public void abort(String why, Throwable e) {
delegate.abort(why, e);
}
@Override
public boolean isAborted() {
return delegate.isAborted();
}
@Override
public Configuration getConfiguration() {
return delegate.getConfiguration();
}
@Override
public HTableInterface getTable(String tableName) throws IOException {
return delegate.getTable(tableName);
}
@Override
public HTableInterface getTable(byte[] tableName) throws IOException {
return delegate.getTable(tableName);
}
@Override
public HTableInterface getTable(TableName tableName) throws IOException {
return delegate.getTable(tableName);
}
@Override
public HTableInterface getTable(String tableName, ExecutorService pool) throws IOException {
return delegate.getTable(tableName, pool);
}
@Override
public HTableInterface getTable(byte[] tableName, ExecutorService pool) throws IOException {
return delegate.getTable(tableName, pool);
}
@Override
public HTableInterface getTable(TableName tableName, ExecutorService pool) throws IOException {
return delegate.getTable(tableName, pool);
}
@Override
public Admin getAdmin() throws IOException { return delegate.getAdmin(); }
@Override
public boolean isMasterRunning() throws MasterNotRunningException, ZooKeeperConnectionException {
return delegate.isMasterRunning();
}
@Override
public boolean isTableEnabled(TableName tableName) throws IOException {
return delegate.isTableEnabled(tableName);
}
@Override
public boolean isTableEnabled(byte[] tableName) throws IOException {
return delegate.isTableEnabled(tableName);
}
@Override
public boolean isTableDisabled(TableName tableName) throws IOException {
return delegate.isTableDisabled(tableName);
}
@Override
public boolean isTableDisabled(byte[] tableName) throws IOException {
return delegate.isTableDisabled(tableName);
}
@Override
public boolean isTableAvailable(TableName tableName) throws IOException {
return delegate.isTableAvailable(tableName);
}
@Override
public boolean isTableAvailable(byte[] tableName) throws IOException {
return delegate.isTableAvailable(tableName);
}
@Override
public boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException {
return delegate.isTableAvailable(tableName, splitKeys);
}
@Override
public boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws IOException {
return delegate.isTableAvailable(tableName, splitKeys);
}
@Override
public HTableDescriptor[] listTables() throws IOException {
return delegate.listTables();
}
@Override
public String[] getTableNames() throws IOException {
return delegate.getTableNames();
}
@Override
public TableName[] listTableNames() throws IOException {
return delegate.listTableNames();
}
@Override
public HTableDescriptor getHTableDescriptor(TableName tableName) throws IOException {
return delegate.getHTableDescriptor(tableName);
}
@Override
public HTableDescriptor getHTableDescriptor(byte[] tableName) throws IOException {
return delegate.getHTableDescriptor(tableName);
}
@Override
public HRegionLocation locateRegion(TableName tableName, byte[] row) throws IOException {
return delegate.locateRegion(tableName, row);
}
@Override
public HRegionLocation locateRegion(byte[] tableName, byte[] row) throws IOException {
return delegate.locateRegion(tableName, row);
}
@Override
public void clearRegionCache() {
delegate.clearRegionCache();
}
@Override
public void clearRegionCache(TableName tableName) {
delegate.clearRegionCache(tableName);
}
@Override
public void clearRegionCache(byte[] tableName) {
delegate.clearRegionCache(tableName);
}
@Override
public HRegionLocation relocateRegion(TableName tableName, byte[] row) throws IOException {
return delegate.relocateRegion(tableName, row);
}
@Override
public HRegionLocation relocateRegion(byte[] tableName, byte[] row) throws IOException {
return delegate.relocateRegion(tableName, row);
}
public void updateCachedLocations(TableName tableName, byte[] rowkey, Object exception,
ServerName source) {
delegate.updateCachedLocations(tableName, rowkey, exception, source);
@Override
public void updateCachedLocations(TableName tableName, byte[] regionName, byte[] rowkey,
Object exception, ServerName source) {
delegate.updateCachedLocations(tableName, regionName, rowkey, exception, source);
}
@Override
public void updateCachedLocations(TableName tableName, byte[] rowkey, Object exception,
HRegionLocation source) {
delegate.updateCachedLocations(tableName, rowkey, exception, source);
}
@Override
public void updateCachedLocations(byte[] tableName, byte[] rowkey, Object exception,
HRegionLocation source) {
delegate.updateCachedLocations(tableName, rowkey, exception, source);
}
@Override
public HRegionLocation locateRegion(byte[] regionName) throws IOException {
return delegate.locateRegion(regionName);
}
@Override
public List<HRegionLocation> locateRegions(TableName tableName) throws IOException {
return delegate.locateRegions(tableName);
}
@Override
public List<HRegionLocation> locateRegions(byte[] tableName) throws IOException {
return delegate.locateRegions(tableName);
}
@Override
public List<HRegionLocation>
locateRegions(TableName tableName, boolean useCache, boolean offlined) throws IOException {
return delegate.locateRegions(tableName, useCache, offlined);
}
@Override
public List<HRegionLocation> locateRegions(byte[] tableName, boolean useCache, boolean offlined)
throws IOException {
return delegate.locateRegions(tableName, useCache, offlined);
}
@Override
public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService.BlockingInterface getMaster()
throws IOException {
return delegate.getMaster();
}
@Override
public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface
getAdmin(ServerName serverName) throws IOException {
return delegate.getAdmin(serverName);
}
@Override
public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface
getAdmin(ServerName serverName, boolean getMaster) throws IOException {
return delegate.getAdmin(serverName, getMaster);
}
@Override
public HRegionLocation getRegionLocation(TableName tableName, byte[] row, boolean reload)
throws IOException {
return delegate.getRegionLocation(tableName, row, reload);
}
@Override
public HRegionLocation getRegionLocation(byte[] tableName, byte[] row, boolean reload)
throws IOException {
return delegate.getRegionLocation(tableName, row, reload);
}
@Override
public void processBatch(List<? extends Row> actions, TableName tableName, ExecutorService pool,
Object[] results) throws IOException, InterruptedException {
delegate.processBatch(actions, tableName, pool, results);
}
@Override
public void processBatch(List<? extends Row> actions, byte[] tableName, ExecutorService pool,
Object[] results) throws IOException, InterruptedException {
delegate.processBatch(actions, tableName, pool, results);
}
@Override
public <R> void processBatchCallback(List<? extends Row> list, TableName tableName,
ExecutorService pool, Object[] results, Callback<R> callback) throws IOException,
InterruptedException {
delegate.processBatchCallback(list, tableName, pool, results, callback);
}
@Override
public <R> void processBatchCallback(List<? extends Row> list, byte[] tableName,
ExecutorService pool, Object[] results, Callback<R> callback) throws IOException,
InterruptedException {
delegate.processBatchCallback(list, tableName, pool, results, callback);
}
@Override
public void setRegionCachePrefetch(TableName tableName, boolean enable) {
delegate.setRegionCachePrefetch(tableName, enable);
}
@Override
public void setRegionCachePrefetch(byte[] tableName, boolean enable) {
delegate.setRegionCachePrefetch(tableName, enable);
}
@Override
public boolean getRegionCachePrefetch(TableName tableName) {
return delegate.getRegionCachePrefetch(tableName);
}
@Override
public boolean getRegionCachePrefetch(byte[] tableName) {
return delegate.getRegionCachePrefetch(tableName);
}
@Override
public int getCurrentNrHRS() throws IOException {
return delegate.getCurrentNrHRS();
}
@Override
public HTableDescriptor[] getHTableDescriptorsByTableName(List<TableName> tableNames)
throws IOException {
return delegate.getHTableDescriptorsByTableName(tableNames);
}
@Override
public HTableDescriptor[] getHTableDescriptors(List<String> tableNames) throws IOException {
return delegate.getHTableDescriptors(tableNames);
}
@Override
public boolean isClosed() {
return delegate.isClosed();
}
@Override
public void clearCaches(ServerName sn) {
delegate.clearCaches(sn);
}
@Override
public void close() throws IOException {
delegate.close();
}
@Override
public void deleteCachedRegionLocation(HRegionLocation location) {
delegate.deleteCachedRegionLocation(location);
}
@Override
public MasterKeepAliveConnection getKeepAliveMasterService()
throws MasterNotRunningException {
return delegate.getKeepAliveMasterService();
}
@Override
public boolean isDeadServer(ServerName serverName) {
return delegate.isDeadServer(serverName);
}

View File

@ -25,7 +25,7 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.List;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -35,11 +35,14 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -274,5 +277,100 @@ public class TestMetaReaderEditor {
pair.getFirst().getEncodedName());
}
@Test
public void testParseReplicaIdFromServerColumn() {
String column1 = HConstants.SERVER_QUALIFIER_STR;
assertEquals(0, MetaReader.parseReplicaIdFromServerColumn(Bytes.toBytes(column1)));
String column2 = column1 + MetaReader.META_REPLICA_ID_DELIMITER;
assertEquals(-1, MetaReader.parseReplicaIdFromServerColumn(Bytes.toBytes(column2)));
String column3 = column2 + "00";
assertEquals(-1, MetaReader.parseReplicaIdFromServerColumn(Bytes.toBytes(column3)));
String column4 = column3 + "2A";
assertEquals(42, MetaReader.parseReplicaIdFromServerColumn(Bytes.toBytes(column4)));
String column5 = column4 + "2A";
assertEquals(-1, MetaReader.parseReplicaIdFromServerColumn(Bytes.toBytes(column5)));
String column6 = HConstants.STARTCODE_QUALIFIER_STR;
assertEquals(-1, MetaReader.parseReplicaIdFromServerColumn(Bytes.toBytes(column6)));
}
@Test
public void testMetaReaderGetColumnMethods() {
Assert.assertArrayEquals(HConstants.SERVER_QUALIFIER, MetaReader.getServerColumn(0));
Assert.assertArrayEquals(Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR
+ MetaReader.META_REPLICA_ID_DELIMITER + "002A"), MetaReader.getServerColumn(42));
Assert.assertArrayEquals(HConstants.STARTCODE_QUALIFIER, MetaReader.getStartCodeColumn(0));
Assert.assertArrayEquals(Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR
+ MetaReader.META_REPLICA_ID_DELIMITER + "002A"), MetaReader.getStartCodeColumn(42));
Assert.assertArrayEquals(HConstants.SEQNUM_QUALIFIER, MetaReader.getSeqNumColumn(0));
Assert.assertArrayEquals(Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR
+ MetaReader.META_REPLICA_ID_DELIMITER + "002A"), MetaReader.getSeqNumColumn(42));
}
@Test
public void testMetaLocationsForRegionReplicas() throws IOException {
Random random = new Random();
ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong());
ServerName serverName1 = ServerName.valueOf("bar", 60010, random.nextLong());
ServerName serverName100 = ServerName.valueOf("baz", 60010, random.nextLong());
long regionId = System.currentTimeMillis();
HRegionInfo primary = new HRegionInfo(TableName.valueOf("table_foo"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 0);
HRegionInfo replica1 = new HRegionInfo(TableName.valueOf("table_foo"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 1);
HRegionInfo replica100 = new HRegionInfo(TableName.valueOf("table_foo"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 100);
long seqNum0 = random.nextLong();
long seqNum1 = random.nextLong();
long seqNum100 = random.nextLong();
HTable meta = MetaReader.getMetaHTable(CT);
try {
MetaEditor.updateRegionLocation(CT, primary, serverName0, seqNum0);
// assert that the server, startcode and seqNum columns are there for the primary region
assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
// add replica = 1
MetaEditor.updateRegionLocation(CT, replica1, serverName1, seqNum1);
// check whether the primary is still there
assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
// now check for replica 1
assertMetaLocation(meta, primary.getRegionName(), serverName1, seqNum1, 1, true);
// add replica = 1
MetaEditor.updateRegionLocation(CT, replica100, serverName100, seqNum100);
// check whether the primary is still there
assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
// check whether the replica 1 is still there
assertMetaLocation(meta, primary.getRegionName(), serverName1, seqNum1, 1, true);
// now check for replica 1
assertMetaLocation(meta, primary.getRegionName(), serverName100, seqNum100, 100, true);
} finally {
meta.close();
}
}
public static void assertMetaLocation(HTable meta, byte[] row, ServerName serverName,
long seqNum, int replicaId, boolean checkSeqNum) throws IOException {
Get get = new Get(row);
Result result = meta.get(get);
assertTrue(Bytes.equals(
result.getValue(HConstants.CATALOG_FAMILY, MetaReader.getServerColumn(replicaId)),
Bytes.toBytes(serverName.getHostAndPort())));
assertTrue(Bytes.equals(
result.getValue(HConstants.CATALOG_FAMILY, MetaReader.getStartCodeColumn(replicaId)),
Bytes.toBytes(serverName.getStartcode())));
if (checkSeqNum) {
assertTrue(Bytes.equals(
result.getValue(HConstants.CATALOG_FAMILY, MetaReader.getSeqNumColumn(replicaId)),
Bytes.toBytes(seqNum)));
}
}
}

View File

@ -26,6 +26,8 @@ import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.lang.reflect.Method;
@ -60,6 +62,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
@ -95,6 +98,7 @@ import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
import org.apache.hadoop.hbase.regionserver.Store;
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.zookeeper.ZooKeeperWatcher;
import org.apache.log4j.Level;
import org.junit.After;
@ -260,7 +264,7 @@ public class TestFromClientSide {
result = table.get(get);
assertNull(result.getValue(FAMILY, COLUMN));
// major compaction, purged future deletes
// major compaction, purged future deletes
TEST_UTIL.getHBaseAdmin().flush(TABLENAME);
TEST_UTIL.getHBaseAdmin().majorCompact(TABLENAME);
@ -284,7 +288,7 @@ public class TestFromClientSide {
get = new Get(ROW);
result = table.get(get);
assertArrayEquals(VALUE, result.getValue(FAMILY, COLUMN));
table.close();
}
@ -6045,7 +6049,6 @@ public class TestFromClientSide {
table.close();
}
/**
* Tests reversed scan under multi regions
*/
@ -6210,4 +6213,44 @@ public class TestFromClientSide {
}
assertEquals(4, count); // 003 004 005 006
}
@Test
public void testGetStartEndKeysWithRegionReplicas() throws IOException {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testGetStartEndKeys"));
HColumnDescriptor fam = new HColumnDescriptor(FAMILY);
htd.addFamily(fam);
byte[][] KEYS = HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE;
TEST_UTIL.getHBaseAdmin().createTable(htd, KEYS);
List<HRegionInfo> regions = TEST_UTIL.getHBaseAdmin().getTableRegions(htd.getTableName());
for (int regionReplication = 1; regionReplication < 4 ; regionReplication++) {
List<RegionLocations> regionLocations = new ArrayList<RegionLocations>();
// mock region locations coming from meta with multiple replicas
for (HRegionInfo region : regions) {
HRegionLocation[] arr = new HRegionLocation[regionReplication];
for (int i = 0; i < arr.length; i++) {
arr[i] = new HRegionLocation(RegionReplicaUtil.getRegionInfoForReplica(region, i), null);
}
regionLocations.add(new RegionLocations(arr));
}
HTable table = spy(new HTable(TEST_UTIL.getConfiguration(), htd.getTableName()));
when(table.listRegionLocations()).thenReturn(regionLocations);
Pair<byte[][], byte[][]> startEndKeys = table.getStartEndKeys();
assertEquals(KEYS.length + 1, startEndKeys.getFirst().length);
for (int i = 0; i < KEYS.length + 1; i++) {
byte[] startKey = i == 0 ? HConstants.EMPTY_START_ROW : KEYS[i - 1];
byte[] endKey = i == KEYS.length ? HConstants.EMPTY_END_ROW : KEYS[i];
assertArrayEquals(startKey, startEndKeys.getFirst()[i]);
assertArrayEquals(endKey, startEndKeys.getSecond()[i]);
}
table.close();
}
}
}

View File

@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
@ -603,15 +604,16 @@ public class TestHCM {
assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW));
final int nextPort = conn.getCachedLocation(TABLE_NAME, ROW).getPort() + 1;
HRegionLocation loc = conn.getCachedLocation(TABLE_NAME, ROW);
final int nextPort = conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation().getPort() + 1;
HRegionLocation loc = conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation();
conn.updateCachedLocation(loc.getRegionInfo(), loc.getServerName(),
ServerName.valueOf("127.0.0.1", nextPort,
HConstants.LATEST_TIMESTAMP), HConstants.LATEST_TIMESTAMP);
Assert.assertEquals(conn.getCachedLocation(TABLE_NAME, ROW).getPort(), nextPort);
Assert.assertEquals(conn.getCachedLocation(TABLE_NAME, ROW)
.getRegionLocation().getPort(), nextPort);
conn.forceDeleteCachedLocation(TABLE_NAME, ROW.clone());
HRegionLocation rl = conn.getCachedLocation(TABLE_NAME, ROW);
conn.clearRegionCache(TABLE_NAME, ROW.clone());
RegionLocations rl = conn.getCachedLocation(TABLE_NAME, ROW);
assertNull("What is this location?? " + rl, rl);
// We're now going to move the region and check that it works for the client
@ -633,7 +635,7 @@ public class TestHCM {
}
// Now moving the region to the second server
HRegionLocation toMove = conn.getCachedLocation(TABLE_NAME, ROW);
HRegionLocation toMove = conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation();
byte[] regionName = toMove.getRegionInfo().getRegionName();
byte[] encodedRegionNameBytes = toMove.getRegionInfo().getEncodedNameAsBytes();
@ -682,7 +684,8 @@ public class TestHCM {
// Cache was NOT updated and points to the wrong server
Assert.assertFalse(
conn.getCachedLocation(TABLE_NAME, ROW).getPort() == destServerName.getPort());
conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation()
.getPort() == destServerName.getPort());
// This part relies on a number of tries equals to 1.
// We do a put and expect the cache to be updated, even if we don't retry
@ -706,10 +709,13 @@ public class TestHCM {
Assert.assertNotNull("Cached connection is null", conn.getCachedLocation(TABLE_NAME, ROW));
Assert.assertEquals(
"Previous server was " + curServer.getServerName().getHostAndPort(),
destServerName.getPort(), conn.getCachedLocation(TABLE_NAME, ROW).getPort());
destServerName.getPort(),
conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation().getPort());
Assert.assertFalse(destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes));
Assert.assertFalse(curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes));
Assert.assertFalse(destServer.getRegionsInTransitionInRS()
.containsKey(encodedRegionNameBytes));
Assert.assertFalse(curServer.getRegionsInTransitionInRS()
.containsKey(encodedRegionNameBytes));
// We move it back to do another test with a scan
LOG.info("Move starting region=" + toMove.getRegionInfo().getRegionNameAsString());
@ -732,7 +738,7 @@ public class TestHCM {
LOG.info("Move finished for region=" + toMove.getRegionInfo().getRegionNameAsString());
// Cache was NOT updated and points to the wrong server
Assert.assertFalse(conn.getCachedLocation(TABLE_NAME, ROW).getPort() ==
Assert.assertFalse(conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation().getPort() ==
curServer.getServerName().getPort());
Scan sc = new Scan();
@ -756,7 +762,8 @@ public class TestHCM {
Assert.assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW));
Assert.assertEquals(
"Previous server was "+destServer.getServerName().getHostAndPort(),
curServer.getServerName().getPort(), conn.getCachedLocation(TABLE_NAME, ROW).getPort());
curServer.getServerName().getPort(),
conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation().getPort());
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
@ -796,7 +803,7 @@ public class TestHCM {
ConnectionManager.HConnectionImplementation conn =
(ConnectionManager.HConnectionImplementation)table.getConnection();
HRegionLocation location = conn.getCachedLocation(TABLE_NAME2, ROW);
HRegionLocation location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation();
assertNotNull(location);
ServerName anySource = ServerName.valueOf(location.getHostname(), location.getPort() - 1, 0L);
@ -805,28 +812,28 @@ public class TestHCM {
int nextPort = location.getPort() + 1;
conn.updateCachedLocation(location.getRegionInfo(), location.getServerName(),
ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
location = conn.getCachedLocation(TABLE_NAME2, ROW);
location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation();
Assert.assertEquals(nextPort, location.getPort());
// No source specified - same.
nextPort = location.getPort() + 1;
conn.updateCachedLocation(location.getRegionInfo(), location.getServerName(),
ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
location = conn.getCachedLocation(TABLE_NAME2, ROW);
location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation();
Assert.assertEquals(nextPort, location.getPort());
// Higher seqNum - overwrites lower seqNum.
nextPort = location.getPort() + 1;
conn.updateCachedLocation(location.getRegionInfo(), anySource,
ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() + 1);
location = conn.getCachedLocation(TABLE_NAME2, ROW);
location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation();
Assert.assertEquals(nextPort, location.getPort());
// Lower seqNum - does not overwrite higher seqNum.
nextPort = location.getPort() + 1;
conn.updateCachedLocation(location.getRegionInfo(), anySource,
ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
location = conn.getCachedLocation(TABLE_NAME2, ROW);
location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation();
Assert.assertEquals(nextPort - 1, location.getPort());
table.close();
}
@ -1024,7 +1031,7 @@ public class TestHCM {
table.put(put);
// Now moving the region to the second server
HRegionLocation toMove = conn.getCachedLocation(TABLE_NAME3, ROW_X);
HRegionLocation toMove = conn.getCachedLocation(TABLE_NAME3, ROW_X).getRegionLocation();
byte[] regionName = toMove.getRegionInfo().getRegionName();
byte[] encodedRegionNameBytes = toMove.getRegionInfo().getEncodedNameAsBytes();
@ -1090,7 +1097,8 @@ public class TestHCM {
// Cache was NOT updated and points to the wrong server
Assert.assertFalse(
conn.getCachedLocation(TABLE_NAME3, ROW_X).getPort() == destServerName.getPort());
conn.getCachedLocation(TABLE_NAME3, ROW_X).getRegionLocation()
.getPort() == destServerName.getPort());
// Hijack the number of retry to fail after 2 tries
final int prevNumRetriesVal = setNumTries(conn, 2);

View File

@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@ -33,11 +34,15 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.MD5Hash;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import com.google.protobuf.ByteString;
@Category(SmallTests.class)
public class TestHRegionInfo {
@Test
@ -101,7 +106,7 @@ public class TestHRegionInfo {
+ id + "." + md5HashInHex + ".",
nameStr);
}
@Test
public void testContainsRange() {
HTableDescriptor tableDesc = new HTableDescriptor(TableName.valueOf("testtable"));
@ -121,7 +126,7 @@ public class TestHRegionInfo {
assertFalse(hri.containsRange(Bytes.toBytes("g"), Bytes.toBytes("g")));
// Single row range entirely outside
assertFalse(hri.containsRange(Bytes.toBytes("z"), Bytes.toBytes("z")));
// Degenerate range
try {
hri.containsRange(Bytes.toBytes("z"), Bytes.toBytes("a"));
@ -149,13 +154,106 @@ public class TestHRegionInfo {
public void testComparator() {
TableName tablename = TableName.valueOf("comparatorTablename");
byte[] empty = new byte[0];
HRegionInfo older = new HRegionInfo(tablename, empty, empty, false, 0L);
HRegionInfo newer = new HRegionInfo(tablename, empty, empty, false, 1L);
HRegionInfo older = new HRegionInfo(tablename, empty, empty, false, 0L);
HRegionInfo newer = new HRegionInfo(tablename, empty, empty, false, 1L);
assertTrue(older.compareTo(newer) < 0);
assertTrue(newer.compareTo(older) > 0);
assertTrue(older.compareTo(older) == 0);
assertTrue(newer.compareTo(newer) == 0);
}
@Test
public void testRegionNameForRegionReplicas() throws Exception {
String tableName = "tablename";
final TableName tn = TableName.valueOf(tableName);
String startKey = "startkey";
final byte[] sk = Bytes.toBytes(startKey);
String id = "id";
// assert with only the region name without encoding
// primary, replicaId = 0
byte [] name = HRegionInfo.createRegionName(tn, sk, Bytes.toBytes(id), 0, false);
String nameStr = Bytes.toString(name);
assertEquals(tableName + "," + startKey + "," + id, nameStr);
// replicaId = 1
name = HRegionInfo.createRegionName(tn, sk, Bytes.toBytes(id), 1, false);
nameStr = Bytes.toString(name);
assertEquals(tableName + "," + startKey + "," + id + "_" +
String.format(HRegionInfo.REPLICA_ID_FORMAT, 1), nameStr);
// replicaId = max
name = HRegionInfo.createRegionName(tn, sk, Bytes.toBytes(id), 0xFFFF, false);
nameStr = Bytes.toString(name);
assertEquals(tableName + "," + startKey + "," + id + "_" +
String.format(HRegionInfo.REPLICA_ID_FORMAT, 0xFFFF), nameStr);
}
@Test
public void testParseName() throws IOException {
TableName tableName = TableName.valueOf("testParseName");
byte[] startKey = Bytes.toBytes("startKey");
long regionId = System.currentTimeMillis();
int replicaId = 42;
// test without replicaId
byte[] regionName = HRegionInfo.createRegionName(tableName, startKey, regionId, false);
byte[][] fields = HRegionInfo.parseRegionName(regionName);
assertArrayEquals(Bytes.toString(fields[0]),tableName.getName(), fields[0]);
assertArrayEquals(Bytes.toString(fields[1]),startKey, fields[1]);
assertArrayEquals(Bytes.toString(fields[2]), Bytes.toBytes(Long.toString(regionId)),fields[2]);
assertEquals(3, fields.length);
// test with replicaId
regionName = HRegionInfo.createRegionName(tableName, startKey, regionId,
replicaId, false);
fields = HRegionInfo.parseRegionName(regionName);
assertArrayEquals(Bytes.toString(fields[0]),tableName.getName(), fields[0]);
assertArrayEquals(Bytes.toString(fields[1]),startKey, fields[1]);
assertArrayEquals(Bytes.toString(fields[2]), Bytes.toBytes(Long.toString(regionId)),fields[2]);
assertArrayEquals(Bytes.toString(fields[3]), Bytes.toBytes(
String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId)), fields[3]);
}
@Test
public void testConvert() {
TableName tableName = TableName.valueOf("ns1:table1");
byte[] startKey = Bytes.toBytes("startKey");
byte[] endKey = Bytes.toBytes("endKey");
boolean split = false;
long regionId = System.currentTimeMillis();
int replicaId = 42;
HRegionInfo hri = new HRegionInfo(tableName, startKey, endKey, split,
regionId, replicaId);
// convert two times, compare
HRegionInfo convertedHri = HRegionInfo.convert(HRegionInfo.convert(hri));
assertEquals(hri, convertedHri);
// test convert RegionInfo without replicaId
RegionInfo info = RegionInfo.newBuilder()
.setTableName(HBaseProtos.TableName.newBuilder()
.setQualifier(ByteString.copyFrom(tableName.getQualifier()))
.setNamespace(ByteString.copyFrom(tableName.getNamespace()))
.build())
.setStartKey(ByteString.copyFrom(startKey))
.setEndKey(ByteString.copyFrom(endKey))
.setSplit(split)
.setRegionId(regionId)
.build();
convertedHri = HRegionInfo.convert(info);
HRegionInfo expectedHri = new HRegionInfo(tableName, startKey, endKey, split,
regionId, 0); // expecting default replicaId
assertEquals(expectedHri, convertedHri);
}
}