HBASE-9508 Restore some API mistakenly removed in client, mapred*, and common

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1523141 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2013-09-13 23:15:00 +00:00
parent a7211896df
commit ed509f750c
75 changed files with 366 additions and 252 deletions

View File

@ -402,8 +402,10 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* Gets the table name from the specified region name.
* @param regionName
* @return Table name.
* @deprecated Since 0.96.0; use #getTable(byte[])
*/
public static TableName getTableName(byte[] regionName) {
@Deprecated
public static byte [] getTableName(byte[] regionName) {
int offset = -1;
for (int i = 0; i < regionName.length; i++) {
if (regionName[i] == HConstants.DELIMITER) {
@ -413,7 +415,18 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
}
byte[] buff = new byte[offset];
System.arraycopy(regionName, 0, buff, 0, offset);
return TableName.valueOf(buff);
return buff;
}
/**
* Gets the table name from the specified region name.
* Like {@link #getTableName(byte[])} only returns a {@link TableName} rather than a byte array.
* @param regionName
* @return Table name
* @see #getTableName(byte[])
*/
public static TableName getTable(final byte [] regionName) {
return TableName.valueOf(getTableName(regionName));
}
/**
@ -523,12 +536,26 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
/**
* Get current table name of the region
* @return byte array of table name
* @deprecated Since 0.96.0; use #getTable()
*/
public TableName getTableName() {
@Deprecated
public byte [] getTableName() {
return getTable().toBytes();
}
/**
* Get current table name of the region
* @return TableName
* @see #getTableName()
*/
public TableName getTable() {
// This method name should be getTableName but there was already a method getTableName
// that returned a byte array. It is unfortunate given everwhere else, getTableName returns
// a TableName instance.
if (tableName == null || tableName.getName().length == 0) {
tableName = getTableName(getRegionName());
tableName = getTable(getRegionName());
}
return tableName;
return this.tableName;
}
/**
@ -570,7 +597,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
/** @return true if this region is a meta region */
public boolean isMetaRegion() {
return tableName.equals(HRegionInfo.FIRST_META_REGIONINFO.getTableName());
return tableName.equals(HRegionInfo.FIRST_META_REGIONINFO.getTable());
}
/**
@ -813,7 +840,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
public static RegionInfo convert(final HRegionInfo info) {
if (info == null) return null;
RegionInfo.Builder builder = RegionInfo.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName(info.getTableName()));
builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));
builder.setRegionId(info.getRegionId());
if (info.getStartKey() != null) {
builder.setStartKey(ByteString.copyFrom(info.getStartKey()));

View File

@ -545,6 +545,16 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
values.remove(key);
}
/**
* Remove metadata represented by the key from the {@link #values} map
*
* @param key Key whose key and value we're to remove from HTableDescriptor
* parameters.
*/
public void remove(final byte [] key) {
remove(new ImmutableBytesWritable(key));
}
/**
* Check if the readOnly flag of the table is set. If the readOnly flag is
* set then the contents of the table can only be read from but not modified.

View File

@ -103,9 +103,9 @@ public class MetaReader {
Pair<HRegionInfo, ServerName> region = HRegionInfo.getHRegionInfoAndServerName(r);
HRegionInfo hri = region.getFirst();
if (hri == null) return true;
if (hri.getTableName() == null) return true;
if (hri.getTable() == null) return true;
if (disabledTables.contains(
hri.getTableName())) return true;
hri.getTable())) return true;
// Are we to include split parents in the list?
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
regions.put(hri, region.getSecond());
@ -362,7 +362,7 @@ public class MetaReader {
* <code>tableName</code>
*/
static boolean isInsideTable(final HRegionInfo current, final TableName tableName) {
return tableName.equals(current.getTableName());
return tableName.equals(current.getTable());
}
/**

View File

@ -85,6 +85,15 @@ public class ClientScanner extends AbstractClientScanner {
this(conf, scan, tableName, HConnectionManager.getConnection(conf));
}
/**
* @deprecated Use {@link #ClientScanner(Configuration, Scan, TableName)}
*/
@Deprecated
public ClientScanner(final Configuration conf, final Scan scan,
final byte [] tableName) throws IOException {
this(conf, scan, TableName.valueOf(tableName));
}
/**
* Create a new ClientScanner for the specified table
@ -101,6 +110,15 @@ public class ClientScanner extends AbstractClientScanner {
this(conf, scan, tableName, connection, new RpcRetryingCallerFactory(conf));
}
/**
* @deprecated Use {@link #ClientScanner(Configuration, Scan, TableName, HConnection)}
*/
@Deprecated
public ClientScanner(final Configuration conf, final Scan scan, final byte [] tableName,
HConnection connection) throws IOException {
this(conf, scan, TableName.valueOf(tableName), connection, new RpcRetryingCallerFactory(conf));
}
/**
* Create a new ClientScanner for the specified table Note that the passed {@link Scan}'s start
* row maybe changed changed.
@ -158,7 +176,16 @@ public class ClientScanner extends AbstractClientScanner {
return this.connection;
}
protected TableName getTableName() {
/**
* @return Table name
* @deprecated Since 0.96.0; use {@link #getTable()}
*/
@Deprecated
protected byte [] getTableName() {
return this.tableName.getName();
}
protected TableName getTable() {
return this.tableName;
}
@ -253,7 +280,7 @@ public class ClientScanner extends AbstractClientScanner {
int nbRows) {
scan.setStartRow(localStartKey);
ScannerCallable s = new ScannerCallable(getConnection(),
getTableName(), scan, this.scanMetrics);
getTable(), scan, this.scanMetrics);
s.setCaching(nbRows);
return s;
}

View File

@ -494,7 +494,7 @@ public class HBaseAdmin implements Abortable, Closeable {
LOG.warn("No serialized HRegionInfo in " + rowResult);
return true;
}
if (!info.getTableName().equals(desc.getTableName())) {
if (!info.getTable().equals(desc.getTableName())) {
return false;
}
ServerName serverName = HRegionInfo.getServerName(rowResult);

View File

@ -926,15 +926,14 @@ public class HConnectionManager {
public boolean processRow(Result row) throws IOException {
HRegionInfo info = MetaScanner.getHRegionInfo(row);
if (info != null && !info.isSplitParent()) {
if (tableName.equals(info.getTableName())) {
if (tableName.equals(info.getTable())) {
ServerName server = HRegionInfo.getServerName(row);
if (server == null) {
available.set(false);
return false;
}
regionCount.incrementAndGet();
} else if (tableName.compareTo(
info.getTableName()) < 0) {
} else if (tableName.compareTo(info.getTable()) < 0) {
// Return if we are done with the current table
return false;
}
@ -961,7 +960,7 @@ public class HConnectionManager {
public boolean processRow(Result row) throws IOException {
HRegionInfo info = MetaScanner.getHRegionInfo(row);
if (info != null && !info.isSplitParent()) {
if (tableName.equals(info.getTableName())) {
if (tableName.equals(info.getTable())) {
ServerName server = HRegionInfo.getServerName(row);
if (server == null) {
available.set(false);
@ -979,7 +978,7 @@ public class HConnectionManager {
// Always empty start row should be counted
regionCount.incrementAndGet();
}
} else if (tableName.compareTo(info.getTableName()) < 0) {
} else if (tableName.compareTo(info.getTable()) < 0) {
// Return if we are done with the current table
return false;
}
@ -1000,7 +999,7 @@ public class HConnectionManager {
@Override
public HRegionLocation locateRegion(final byte[] regionName) throws IOException {
return locateRegion(HRegionInfo.getTableName(regionName),
return locateRegion(HRegionInfo.getTable(regionName),
HRegionInfo.getStartKey(regionName), false, true);
}
@ -1113,7 +1112,7 @@ public class HConnectionManager {
}
// possible we got a region of a different table...
if (!regionInfo.getTableName().equals(tableName)) {
if (!regionInfo.getTable().equals(tableName)) {
return false; // stop scanning
}
if (regionInfo.isOffline()) {
@ -1226,10 +1225,10 @@ public class HConnectionManager {
}
// possible we got a region of a different table...
if (!regionInfo.getTableName().equals(tableName)) {
if (!regionInfo.getTable().equals(tableName)) {
throw new TableNotFoundException(
"Table '" + tableName + "' was not found, got: " +
regionInfo.getTableName() + ".");
regionInfo.getTable() + ".");
}
if (regionInfo.isSplit()) {
throw new RegionOfflineException("the only available region for" +
@ -2306,7 +2305,7 @@ public class HConnectionManager {
ServerName serverName, long seqNum) {
HRegionLocation newHrl = new HRegionLocation(hri, serverName, seqNum);
synchronized (this.cachedRegionLocations) {
cacheLocation(hri.getTableName(), source, newHrl);
cacheLocation(hri.getTable(), source, newHrl);
}
}
@ -2320,7 +2319,7 @@ public class HConnectionManager {
HRegionLocation oldLocation;
synchronized (this.cachedRegionLocations) {
Map<byte[], HRegionLocation> tableLocations =
getTableLocations(hri.getTableName());
getTableLocations(hri.getTable());
oldLocation = tableLocations.get(hri.getStartKey());
if (oldLocation != null) {
// Do not delete the cache entry if it's not for the same server that gave us the error.
@ -2338,7 +2337,7 @@ public class HConnectionManager {
return;
}
synchronized (this.cachedRegionLocations) {
TableName tableName = location.getRegionInfo().getTableName();
TableName tableName = location.getRegionInfo().getTable();
Map<byte[], HRegionLocation> tableLocations =
getTableLocations(tableName);
if (!tableLocations.isEmpty()) {

View File

@ -1006,6 +1006,18 @@ public class HTable implements HTableInterface {
return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL);
}
/**
* @deprecated Use {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
*/
@Deprecated
@Override
public long incrementColumnValue(final byte [] row, final byte [] family,
final byte [] qualifier, final long amount, final boolean writeToWAL)
throws IOException {
return incrementColumnValue(row, family, qualifier, amount,
writeToWAL? Durability.SKIP_WAL: Durability.USE_DEFAULT);
}
/**
* {@inheritDoc}
*/

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.client;
import com.google.protobuf.Service;
import com.google.protobuf.ServiceException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -384,6 +385,14 @@ public interface HTableInterface extends Closeable {
long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
long amount, Durability durability) throws IOException;
/**
* @deprecated Use {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
*/
@Deprecated
long incrementColumnValue(final byte [] row, final byte [] family,
final byte [] qualifier, final long amount, final boolean writeToWAL)
throws IOException;
/**
* Tells whether or not 'auto-flush' is turned on.
*

View File

@ -19,19 +19,6 @@
*/
package org.apache.hadoop.hbase.client;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import java.io.IOException;
import java.util.AbstractMap.SimpleEntry;
import java.util.ArrayList;
@ -46,6 +33,18 @@ import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* HTableMultiplexer provides a thread-safe non blocking PUT API across all the tables.
* Each put will be sharded into different buffer queues based on its destination region server.
@ -450,7 +449,7 @@ public class HTableMultiplexer {
HRegionLocation oldLoc) throws IOException {
Put failedPut = failedPutStatus.getPut();
// The currentPut is failed. So get the table name for the currentPut.
TableName tableName = failedPutStatus.getRegionInfo().getTableName();
TableName tableName = failedPutStatus.getRegionInfo().getTable();
// Decrease the retry count
int retryCount = failedPutStatus.getRetryCount() - 1;

View File

@ -600,5 +600,11 @@ public class HTablePool implements Closeable {
throw new IllegalStateException("Table=" + new String(table.getTableName()) + " already closed");
}
}
@Override
public long incrementColumnValue(byte[] row, byte[] family,
byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
return table.incrementColumnValue(row, family, qualifier, amount, writeToWAL);
}
}
}

View File

@ -355,7 +355,7 @@ public class MetaScanner {
if (info == null) {
return true;
}
if (!(info.getTableName().equals(tableName))) {
if (!(info.getTable().equals(tableName))) {
return false;
}
return super.processRow(rowResult);

View File

@ -101,6 +101,15 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
logCutOffLatency = conf.getInt(LOG_SCANNER_LATENCY_CUTOFF, 1000);
}
/**
* @deprecated Use {@link #ScannerCallable(HConnection, TableName, Scan, ScanMetrics)}
*/
@Deprecated
public ScannerCallable (HConnection connection, final byte [] tableName, Scan scan,
ScanMetrics scanMetrics) {
this(connection, TableName.valueOf(tableName), scan, scanMetrics);
}
/**
* @param reload force reload of server location
* @throws IOException

View File

@ -1645,7 +1645,7 @@ public final class ProtobufUtil {
AccessControlProtos.TablePermission.Builder builder =
AccessControlProtos.TablePermission.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName(tablePerm.getTable()));
builder.setTableName(ProtobufUtil.toProtoTableName(tablePerm.getTableName()));
if (tablePerm.hasFamily()) {
builder.setFamily(ByteString.copyFrom(tablePerm.getFamily()));
}
@ -2195,7 +2195,7 @@ public final class ProtobufUtil {
// input / output paths are relative to the store dir
// store dir is relative to region dir
CompactionDescriptor.Builder builder = CompactionDescriptor.newBuilder()
.setTableName(ByteString.copyFrom(info.getTableName().getName()))
.setTableName(ByteString.copyFrom(info.getTableName()))
.setEncodedRegionName(ByteString.copyFrom(info.getEncodedNameAsBytes()))
.setFamilyName(ByteString.copyFrom(family))
.setStoreHomeDir(storeDir.getName()); //make relative

View File

@ -151,7 +151,7 @@ public class TablePermission extends Permission {
return table != null;
}
public TableName getTable() {
public TableName getTableName() {
return table;
}
@ -318,7 +318,7 @@ public class TablePermission extends Permission {
}
TablePermission other = (TablePermission)obj;
if (!(table.equals(other.getTable()) &&
if (!(table.equals(other.getTableName()) &&
((family == null && other.getFamily() == null) ||
Bytes.equals(family, other.getFamily())) &&
((qualifier == null && other.getQualifier() == null) ||

View File

@ -140,7 +140,7 @@ public class UserPermission extends TablePermission {
* @param perm a TablePermission
*/
public UserPermission(byte[] user, TablePermission perm) {
super(perm.getNamespace(), perm.getTable(), perm.getFamily(), perm.getQualifier(),
super(perm.getNamespace(), perm.getTableName(), perm.getFamily(), perm.getQualifier(),
perm.actions);
this.user = user;
}

View File

@ -129,8 +129,9 @@ public final class TableName implements Comparable<TableName> {
return tableName;
}
public static void isLegalTableQualifierName(final byte[] qualifierName){
public static byte [] isLegalTableQualifierName(final byte[] qualifierName){
isLegalTableQualifierName(qualifierName, 0, qualifierName.length);
return qualifierName;
}
/**

View File

@ -72,7 +72,7 @@ public class HFileArchiver {
public static void archiveRegion(Configuration conf, FileSystem fs, HRegionInfo info)
throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
archiveRegion(fs, rootDir, FSUtils.getTableDir(rootDir, info.getTableName()),
archiveRegion(fs, rootDir, FSUtils.getTableDir(rootDir, info.getTable()),
HRegion.getRegionDir(rootDir, info));
}

View File

@ -564,6 +564,12 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
public void setWriteBufferSize(long writeBufferSize) throws IOException {
table.setWriteBufferSize(writeBufferSize);
}
@Override
public long incrementColumnValue(byte[] row, byte[] family,
byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
return table.incrementColumnValue(row, family, qualifier, amount, writeToWAL);
}
}
/** The coprocessor */

View File

@ -263,7 +263,7 @@ public class HFileLink extends FileLink {
*/
public static String createHFileLinkName(final HRegionInfo hfileRegionInfo,
final String hfileName) {
return createHFileLinkName(hfileRegionInfo.getTableName(),
return createHFileLinkName(hfileRegionInfo.getTable(),
hfileRegionInfo.getEncodedName(), hfileName);
}
@ -300,7 +300,7 @@ public class HFileLink extends FileLink {
public static boolean create(final Configuration conf, final FileSystem fs,
final Path dstFamilyPath, final HRegionInfo hfileRegionInfo,
final String hfileName) throws IOException {
TableName linkedTable = hfileRegionInfo.getTableName();
TableName linkedTable = hfileRegionInfo.getTable();
String linkedRegion = hfileRegionInfo.getEncodedName();
return create(conf, fs, dstFamilyPath, linkedTable, linkedRegion, hfileName);
}

View File

@ -60,7 +60,8 @@ public class TableSplit implements InputSplit, Comparable<TableSplit> {
public TableSplit(byte [] tableName, byte [] startRow, byte [] endRow,
final String location) {
this(TableName.valueOf(tableName), startRow, endRow, location);
this(TableName.valueOf(tableName), startRow, endRow,
location);
}
/** @return table name */

View File

@ -517,6 +517,16 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
return null;
}
/**
* @deprecated Use {@link #tryAtomicRegionLoad(HConnection, TableName, byte[], Collection)}
*/
@Deprecated
protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn,
final byte [] tableName, final byte[] first, Collection<LoadQueueItem> lqis)
throws IOException {
return tryAtomicRegionLoad(conn, TableName.valueOf(tableName), first, lqis);
}
/**
* Attempts to do an atomic load of many hfiles into a region. If it fails,
* it returns a list of hfiles that need to be retried. If it is successful
@ -531,9 +541,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
* failure
*/
protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn,
final TableName tableName,
final byte[] first, Collection<LoadQueueItem> lqis) throws IOException {
final TableName tableName, final byte[] first, Collection<LoadQueueItem> lqis)
throws IOException {
final List<Pair<byte[], String>> famPaths =
new ArrayList<Pair<byte[], String>>(lqis.size());
for (LoadQueueItem lqi : lqis) {

View File

@ -85,10 +85,19 @@ implements Writable, Comparable<TableSplit> {
/** Default constructor. */
public TableSplit() {
this(null, null, HConstants.EMPTY_BYTE_ARRAY,
this((TableName)null, null, HConstants.EMPTY_BYTE_ARRAY,
HConstants.EMPTY_BYTE_ARRAY, "");
}
/**
* @deprecated Since 0.96.0; use {@link TableSplit#TableSplit(TableName, byte[], byte[], String)}
*/
@Deprecated
public TableSplit(final byte [] tableName, Scan scan, byte [] startRow, byte [] endRow,
final String location) {
this(TableName.valueOf(tableName), scan, startRow, endRow, location);
}
/**
* Creates a new instance while assigning all variables.
*
@ -111,7 +120,16 @@ implements Writable, Comparable<TableSplit> {
this.endRow = endRow;
this.regionLocation = location;
}
/**
* @deprecated Since 0.96.0; use {@link TableSplit#TableSplit(TableName, byte[], byte[], String)}
*/
@Deprecated
public TableSplit(final byte [] tableName, byte[] startRow, byte[] endRow,
final String location) {
this(TableName.valueOf(tableName), startRow, endRow, location);
}
/**
* Creates a new instance without a scanner.
*
@ -135,12 +153,24 @@ implements Writable, Comparable<TableSplit> {
return TableMapReduceUtil.convertStringToScan(this.scan);
}
/**
* Returns the table name converted to a byte array.
* @see #getTable()
* @return The table name.
*/
public byte [] getTableName() {
return tableName.getName();
}
/**
* Returns the table name.
*
* @return The table name.
*/
public TableName getTableName() {
public TableName getTable() {
// It is ugly that usually to get a TableName, the method is called getTableName. We can't do
// that in here though because there was an existing getTableName in place already since
// deprecated.
return tableName;
}
@ -268,7 +298,7 @@ implements Writable, Comparable<TableSplit> {
// If The table name of the two splits is the same then compare start row
// otherwise compare based on table names
int tableNameComparison =
getTableName().compareTo(split.getTableName());
getTable().compareTo(split.getTable());
return tableNameComparison != 0 ? tableNameComparison : Bytes.compareTo(
getStartRow(), split.getStartRow());
}

View File

@ -460,7 +460,7 @@ public class AssignmentManager extends ZooKeeperListener {
// its a clean cluster startup, else its a failover.
Map<HRegionInfo, ServerName> regions = regionStates.getRegionAssignments();
for (Map.Entry<HRegionInfo, ServerName> e: regions.entrySet()) {
if (!e.getKey().getTableName().isSystemTable()
if (!e.getKey().getTable().isSystemTable()
&& e.getValue() != null) {
LOG.debug("Found " + e + " out on cluster");
failover = true;
@ -1282,8 +1282,7 @@ public class AssignmentManager extends ZooKeeperListener {
if (rs.isOpened()) {
ServerName serverName = rs.getServerName();
regionOnline(regionInfo, serverName);
boolean disabled = getZKTable().isDisablingOrDisabledTable(
regionInfo.getTableName());
boolean disabled = getZKTable().isDisablingOrDisabledTable(regionInfo.getTable());
if (!serverManager.isServerOnline(serverName) && !disabled) {
LOG.info("Opened " + regionNameStr
+ "but the region server is offline, reassign the region");
@ -1898,7 +1897,7 @@ public class AssignmentManager extends ZooKeeperListener {
// When we have a case such as all the regions are added directly into hbase:meta and we call
// assignRegion then we need to make the table ENABLED. Hence in such case the table
// will not be in ENABLING or ENABLED state.
TableName tableName = region.getTableName();
TableName tableName = region.getTable();
if (!zkTable.isEnablingTable(tableName) && !zkTable.isEnabledTable(tableName)) {
LOG.debug("Setting table " + tableName + " to ENABLED state.");
setEnabledTable(tableName);
@ -2085,7 +2084,7 @@ public class AssignmentManager extends ZooKeeperListener {
}
private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
TableName tableName = region.getTableName();
TableName tableName = region.getTable();
boolean disabled = this.zkTable.isDisabledTable(tableName);
if (disabled || this.zkTable.isDisablingTable(tableName)) {
LOG.info("Table " + tableName + (disabled ? " disabled;" : " disabling;") +
@ -2584,7 +2583,7 @@ public class AssignmentManager extends ZooKeeperListener {
//remove system tables because they would have been assigned earlier
for(Iterator<HRegionInfo> iter = allRegions.keySet().iterator();
iter.hasNext();) {
if (iter.next().getTableName().isSystemTable()) {
if (iter.next().getTable().isSystemTable()) {
iter.remove();
}
}
@ -2603,7 +2602,7 @@ public class AssignmentManager extends ZooKeeperListener {
}
for (HRegionInfo hri : allRegions.keySet()) {
TableName tableName = hri.getTableName();
TableName tableName = hri.getTable();
if (!zkTable.isEnabledTable(tableName)) {
setEnabledTable(tableName);
}
@ -2664,7 +2663,7 @@ public class AssignmentManager extends ZooKeeperListener {
HRegionInfo regionInfo = region.getFirst();
ServerName regionLocation = region.getSecond();
if (regionInfo == null) continue;
TableName tableName = regionInfo.getTableName();
TableName tableName = regionInfo.getTable();
if (tableName.isSystemTable()) continue;
regionStates.createRegionState(regionInfo);
if (regionStates.isRegionInState(regionInfo, State.SPLIT)) {
@ -3161,7 +3160,7 @@ public class AssignmentManager extends ZooKeeperListener {
} catch (KeeperException ke) {
server.abort("Unexpected ZK exception deleting node " + hri, ke);
}
if (zkTable.isDisablingOrDisabledTable(hri.getTableName())) {
if (zkTable.isDisablingOrDisabledTable(hri.getTable())) {
it.remove();
regionStates.regionOffline(hri);
continue;
@ -3193,8 +3192,7 @@ public class AssignmentManager extends ZooKeeperListener {
// the master to disable, we need to make sure we close those regions in
// that case. This is not racing with the region server itself since RS
// report is done after the split transaction completed.
if (this.zkTable.isDisablingOrDisabledTable(
parent.getTableName())) {
if (this.zkTable.isDisablingOrDisabledTable(parent.getTable())) {
unassign(a);
unassign(b);
}
@ -3217,7 +3215,7 @@ public class AssignmentManager extends ZooKeeperListener {
// the master to disable, we need to make sure we close those regions in
// that case. This is not racing with the region server itself since RS
// report is done after the regions merge transaction completed.
if (this.zkTable.isDisablingOrDisabledTable(merged.getTableName())) {
if (this.zkTable.isDisablingOrDisabledTable(merged.getTable())) {
unassign(merged);
}
}

View File

@ -149,7 +149,7 @@ public class CatalogJanitor extends Chore {
HRegionInfo info = HRegionInfo.getHRegionInfo(r);
if (info == null) return true; // Keep scanning
if (isTableSpecified
&& info.getTableName().compareTo(tableName) > 0) {
&& info.getTable().compareTo(tableName) > 0) {
// Another table, stop scanning
return false;
}
@ -184,9 +184,8 @@ public class CatalogJanitor extends Chore {
final HRegionInfo regionA, final HRegionInfo regionB) throws IOException {
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
Path rootdir = this.services.getMasterFileSystem().getRootDir();
Path tabledir = FSUtils.getTableDir(rootdir,
mergedRegion.getTableName());
HTableDescriptor htd = getTableDescriptor(mergedRegion.getTableName());
Path tabledir = FSUtils.getTableDir(rootdir, mergedRegion.getTable());
HTableDescriptor htd = getTableDescriptor(mergedRegion.getTable());
HRegionFileSystem regionFs = null;
try {
regionFs = HRegionFileSystem.openRegionFromFileSystem(
@ -290,8 +289,7 @@ public class CatalogJanitor extends Chore {
if (left == null) return -1;
if (right == null) return 1;
// Same table name.
int result = left.getTableName().compareTo(
right.getTableName());
int result = left.getTable().compareTo(right.getTable());
if (result != 0) return result;
// Compare start keys.
result = Bytes.compareTo(left.getStartKey(), right.getStartKey());
@ -375,7 +373,7 @@ public class CatalogJanitor extends Chore {
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
Path rootdir = this.services.getMasterFileSystem().getRootDir();
Path tabledir = FSUtils.getTableDir(rootdir, daughter.getTableName());
Path tabledir = FSUtils.getTableDir(rootdir, daughter.getTable());
HRegionFileSystem regionFs = null;
try {
@ -387,7 +385,7 @@ public class CatalogJanitor extends Chore {
}
boolean references = false;
HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTableName());
HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTable());
for (HColumnDescriptor family: parentDescriptor.getFamilies()) {
if ((references = regionFs.hasReferences(family.getNameAsString()))) {
break;

View File

@ -1071,7 +1071,7 @@ MasterServices, Server {
MetaReader.fullScan(catalogTracker, disabledOrDisablingOrEnabling, true);
for(Iterator<HRegionInfo> iter = allRegions.keySet().iterator();
iter.hasNext();) {
if (!iter.next().getTableName().isSystemTable()) {
if (!iter.next().getTable().isSystemTable()) {
iter.remove();
}
}
@ -1111,8 +1111,8 @@ MasterServices, Server {
this.assignmentManager.regionOnline(regionInfo, currServer);
}
if (!this.assignmentManager.getZKTable().isEnabledTable(regionInfo.getTableName())) {
this.assignmentManager.setEnabledTable(regionInfo.getTableName());
if (!this.assignmentManager.getZKTable().isEnabledTable(regionInfo.getTable())) {
this.assignmentManager.setEnabledTable(regionInfo.getTable());
}
LOG.info("System region " + regionInfo.getRegionNameAsString() + " assigned, rit=" + rit +
", location=" + catalogTracker.getMetaLocation());
@ -2097,7 +2097,7 @@ MasterServices, Server {
if (pair == null) {
return false;
}
if (!pair.getFirst().getTableName().equals(tableName)) {
if (!pair.getFirst().getTable().equals(tableName)) {
return false;
}
result.set(pair);

View File

@ -588,7 +588,7 @@ public class MasterFileSystem {
public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName)
throws IOException {
// archive family store files
Path tableDir = FSUtils.getTableDir(rootdir, region.getTableName());
Path tableDir = FSUtils.getTableDir(rootdir, region.getTable());
HFileArchiver.archiveFamily(fs, conf, region, tableDir, familyName);
// delete the family folder

View File

@ -433,7 +433,7 @@ public class RegionStates {
// before all table's regions.
HRegionInfo boundary = new HRegionInfo(tableName, null, null, false, 0L);
for (HRegionInfo hri: regionAssignments.tailMap(boundary).keySet()) {
if(!hri.getTableName().equals(tableName)) break;
if(!hri.getTable().equals(tableName)) break;
tableRegions.add(hri);
}
return tableRegions;
@ -529,7 +529,7 @@ public class RegionStates {
for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
for (HRegionInfo hri: e.getValue()) {
if (hri.isMetaRegion()) continue;
TableName tablename = hri.getTableName();
TableName tablename = hri.getTable();
Map<ServerName, List<HRegionInfo>> svrToRegions = result.get(tablename);
if (svrToRegions == null) {
svrToRegions = new HashMap<ServerName, List<HRegionInfo>>(serverHoldings.size());

View File

@ -104,8 +104,8 @@ public class SnapshotOfRegionAssignmentFromMeta {
HRegionInfo.getHRegionInfoAndServerName(result);
HRegionInfo hri = regionAndServer.getFirst();
if (hri == null) return true;
if (hri.getTableName() == null) return true;
if (disabledTables.contains(hri.getTableName())) {
if (hri.getTable() == null) return true;
if (disabledTables.contains(hri.getTable())) {
return true;
}
// Are we to include split parents in the list?
@ -144,7 +144,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
regionNameToRegionInfoMap.put(regionInfo.getRegionNameAsString(), regionInfo);
// Process the table to region map
TableName tableName = regionInfo.getTableName();
TableName tableName = regionInfo.getTable();
List<HRegionInfo> regionList = tableToRegionMap.get(tableName);
if (regionList == null) {
regionList = new ArrayList<HRegionInfo>();

View File

@ -148,7 +148,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
regionPerServerIndex = 0;
for (HRegionInfo region : entry.getValue()) {
String tableName = region.getTableName().getNameAsString();
String tableName = region.getTable().getNameAsString();
Integer idx = tablesToIndex.get(tableName);
if (idx == null) {
tables.add(tableName);

View File

@ -99,7 +99,7 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer {
currentServer.getPort(), ServerName.NON_STARTCODE);
List<HRegionInfo> list = entry.getValue();
for (HRegionInfo region : list) {
if(region.getTableName().getNamespaceAsString()
if(region.getTable().getNamespaceAsString()
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
continue;
}

View File

@ -122,7 +122,7 @@ class RegionLocationFinder {
protected List<ServerName> internalGetTopBlockLocation(HRegionInfo region) {
List<ServerName> topServerNames = null;
try {
HTableDescriptor tableDescriptor = getTableDescriptor(region.getTableName());
HTableDescriptor tableDescriptor = getTableDescriptor(region.getTable());
if (tableDescriptor != null) {
HDFSBlocksDistribution blocksDistribution =
HRegion.computeHDFSBlocksDistribution(getConf(), tableDescriptor, region);

View File

@ -92,7 +92,7 @@ public class ClosedRegionHandler extends EventHandler implements TotesHRegionInf
LOG.debug("Handling CLOSED event for " + regionInfo.getEncodedName());
// Check if this table is being disabled or not
if (this.assignmentManager.getZKTable().
isDisablingOrDisabledTable(this.regionInfo.getTableName())) {
isDisablingOrDisabledTable(this.regionInfo.getTable())) {
assignmentManager.offlineDisabledRegion(regionInfo);
return;
}

View File

@ -230,7 +230,7 @@ public class EnableTableHandler extends EventHandler {
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Skipping assign for the region " + hri + " during enable table "
+ hri.getTableName() + " because its already in tranition or assigned.");
+ hri.getTable() + " because its already in tranition or assigned.");
}
}
}

View File

@ -69,7 +69,7 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
this.expectedVersion = expectedVersion;
if(regionInfo.isMetaRegion()) {
priority = OpenedPriority.META;
} else if(regionInfo.getTableName()
} else if(regionInfo.getTable()
.getNamespaceAsString().equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
priority = OpenedPriority.SYSTEM;
} else {
@ -113,8 +113,7 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
" because regions is NOT in RIT -- presuming this is because it SPLIT");
}
if (!openedNodeDeleted) {
if (this.assignmentManager.getZKTable().isDisablingOrDisabledTable(
regionInfo.getTableName())) {
if (this.assignmentManager.getZKTable().isDisablingOrDisabledTable(regionInfo.getTable())) {
debugLog(regionInfo, "Opened region "
+ regionInfo.getShortNameToLog() + " but "
+ "this table is disabled, triggering close of region");

View File

@ -251,7 +251,7 @@ public class ServerShutdownHandler extends EventHandler {
toAssignRegions.add(hri);
} else if (rit != null) {
if ((rit.isClosing() || rit.isPendingClose())
&& am.getZKTable().isDisablingOrDisabledTable(hri.getTableName())) {
&& am.getZKTable().isDisablingOrDisabledTable(hri.getTable())) {
// If the table was partially disabled and the RS went down, we should clear the RIT
// and remove the node for the region.
// The rit that we use may be stale in case the table was in DISABLING state
@ -324,18 +324,16 @@ public class ServerShutdownHandler extends EventHandler {
public static boolean processDeadRegion(HRegionInfo hri, Result result,
AssignmentManager assignmentManager, CatalogTracker catalogTracker)
throws IOException {
boolean tablePresent = assignmentManager.getZKTable().isTablePresent(
hri.getTableName());
boolean tablePresent = assignmentManager.getZKTable().isTablePresent(hri.getTable());
if (!tablePresent) {
LOG.info("The table " + hri.getTableName()
LOG.info("The table " + hri.getTable()
+ " was deleted. Hence not proceeding.");
return false;
}
// If table is not disabled but the region is offlined,
boolean disabled = assignmentManager.getZKTable().isDisabledTable(
hri.getTableName());
boolean disabled = assignmentManager.getZKTable().isDisabledTable(hri.getTable());
if (disabled){
LOG.info("The table " + hri.getTableName()
LOG.info("The table " + hri.getTable()
+ " was disabled. Hence not proceeding.");
return false;
}
@ -345,10 +343,9 @@ public class ServerShutdownHandler extends EventHandler {
//to the dead server. We don't have to do anything.
return false;
}
boolean disabling = assignmentManager.getZKTable().isDisablingTable(
hri.getTableName());
boolean disabling = assignmentManager.getZKTable().isDisablingTable(hri.getTable());
if (disabling) {
LOG.info("The table " + hri.getTableName()
LOG.info("The table " + hri.getTable()
+ " is disabled. Hence not assigning region" + hri.getEncodedName());
return false;
}

View File

@ -3924,11 +3924,11 @@ public class HRegion implements HeapSize { // , Writable{
final HLog hlog,
final boolean initialize, final boolean ignoreHLog)
throws IOException {
LOG.info("creating HRegion " + info.getTableName().getNameAsString()
LOG.info("creating HRegion " + info.getTable().getNameAsString()
+ " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
" Table name == " + info.getTableName().getNameAsString());
" Table name == " + info.getTable().getNameAsString());
Path tableDir = FSUtils.getTableDir(rootDir, info.getTableName());
Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
FileSystem fs = FileSystem.get(conf);
HRegionFileSystem rfs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info);
HLog effectiveHLog = hlog;
@ -4088,7 +4088,7 @@ public class HRegion implements HeapSize { // , Writable{
if (LOG.isDebugEnabled()) {
LOG.debug("Opening region: " + info);
}
Path dir = FSUtils.getTableDir(rootDir, info.getTableName());
Path dir = FSUtils.getTableDir(rootDir, info.getTable());
HRegion r = HRegion.newHRegion(dir, wal, fs, conf, info, htd, rsServices);
return r.openHRegion(reporter);
}
@ -4217,8 +4217,7 @@ public class HRegion implements HeapSize { // , Writable{
@Deprecated
public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
return new Path(
FSUtils.getTableDir(rootdir, info.getTableName()),
info.getEncodedName());
FSUtils.getTableDir(rootdir, info.getTable()), info.getEncodedName());
}
/**
@ -4277,8 +4276,7 @@ public class HRegion implements HeapSize { // , Writable{
* @throws IOException
*/
public static HRegion merge(final HRegion a, final HRegion b) throws IOException {
if (!a.getRegionInfo().getTableName().equals(
b.getRegionInfo().getTableName())) {
if (!a.getRegionInfo().getTable().equals(b.getRegionInfo().getTable())) {
throw new IOException("Regions do not belong to the same table");
}
@ -5197,7 +5195,7 @@ public class HRegion implements HeapSize { // , Writable{
public byte[] checkSplit() {
// Can't split META
if (this.getRegionInfo().isMetaTable() ||
TableName.NAMESPACE_TABLE_NAME.equals(this.getRegionInfo().getTableName())) {
TableName.NAMESPACE_TABLE_NAME.equals(this.getRegionInfo().getTable())) {
if (shouldForceSplit()) {
LOG.warn("Cannot split meta region in HBase 0.20 and above");
}

View File

@ -2347,7 +2347,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
synchronized (this.onlineRegions) {
for (HRegion region: this.onlineRegions.values()) {
HRegionInfo regionInfo = region.getRegionInfo();
if(regionInfo.getTableName().equals(tableName)) {
if(regionInfo.getTable().equals(tableName)) {
tableRegions.add(region);
}
}
@ -3532,10 +3532,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
}
}
LOG.info("Open " + region.getRegionNameAsString());
htd = htds.get(region.getTableName());
htd = htds.get(region.getTable());
if (htd == null) {
htd = this.tableDescriptors.get(region.getTableName());
htds.put(region.getTableName(), htd);
htd = this.tableDescriptors.get(region.getTable());
htds.put(region.getTable(), htd);
}
final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(

View File

@ -281,7 +281,7 @@ public class HStore implements Store {
@Override
public TableName getTableName() {
return this.getRegionInfo().getTableName();
return this.getRegionInfo().getTable();
}
@Override

View File

@ -146,7 +146,7 @@ public class RegionCoprocessorHost
loadSystemCoprocessors(conf, REGION_COPROCESSOR_CONF_KEY);
// load system default cp's for user tables from configuration.
if (!region.getRegionInfo().getTableName().isSystemTable()) {
if (!region.getRegionInfo().getTable().isSystemTable()) {
loadSystemCoprocessors(conf, USER_REGION_COPROCESSOR_CONF_KEY);
}

View File

@ -433,7 +433,7 @@ public class RegionMergeTransaction {
}
// Merged region is sorted between two merging regions in META
HRegionInfo mergedRegionInfo = new HRegionInfo(a.getTableName(), startKey,
HRegionInfo mergedRegionInfo = new HRegionInfo(a.getTable(), startKey,
endKey, false, rid);
return mergedRegionInfo;
}

View File

@ -168,8 +168,8 @@ public class SplitTransaction {
return false;
}
long rid = getDaughterRegionIdTimestamp(hri);
this.hri_a = new HRegionInfo(hri.getTableName(), startKey, this.splitrow, false, rid);
this.hri_b = new HRegionInfo(hri.getTableName(), this.splitrow, endKey, false, rid);
this.hri_a = new HRegionInfo(hri.getTable(), startKey, this.splitrow, false, rid);
this.hri_b = new HRegionInfo(hri.getTable(), this.splitrow, endKey, false, rid);
return true;
}

View File

@ -1615,7 +1615,7 @@ public class HLogSplitter {
final long pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
boolean reloadLocation = false;
TableName tableName = loc.getRegionInfo().getTableName();
TableName tableName = loc.getRegionInfo().getTable();
int tries = 0;
Throwable cause = null;
while (endTime > EnvironmentEdgeManager.currentTimeMillis()) {

View File

@ -779,4 +779,10 @@ public class RemoteHTable implements HTableInterface {
public void setWriteBufferSize(long writeBufferSize) throws IOException {
throw new IOException("setWriteBufferSize not supported");
}
@Override
public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
long amount, boolean writeToWAL) throws IOException {
throw new IOException("incrementColumnValue not supported");
}
}

View File

@ -299,7 +299,7 @@ public class AccessControlLists {
} else if(userPerm.isGlobal()) {
row = ACL_GLOBAL_NAME;
} else {
row = userPerm.getTable().getName();
row = userPerm.getTableName().getName();
}
return row;
}
@ -502,7 +502,7 @@ public class AccessControlLists {
for (Map.Entry<String, TablePermission> entry : allPerms.entries()) {
UserPermission up = new UserPermission(Bytes.toBytes(entry.getKey()),
entry.getValue().getTable(), entry.getValue().getFamily(),
entry.getValue().getTableName(), entry.getValue().getFamily(),
entry.getValue().getQualifier(), entry.getValue().getActions());
perms.add(up);
}

View File

@ -208,7 +208,7 @@ public class AccessController extends BaseRegionObserver
RegionCoprocessorEnvironment e,
Map<byte [], ? extends Collection<?>> families) {
HRegionInfo hri = e.getRegion().getRegionInfo();
TableName tableName = hri.getTableName();
TableName tableName = hri.getTable();
// 1. All users need read access to hbase:meta table.
// this is a very common operation, so deal with it quickly.
@ -470,7 +470,7 @@ public class AccessController extends BaseRegionObserver
Map<byte[], ? extends Set<byte[]>> familyMap)
throws IOException {
HRegionInfo hri = env.getRegion().getRegionInfo();
TableName tableName = hri.getTableName();
TableName tableName = hri.getTable();
if (user == null) {
return false;
@ -702,7 +702,7 @@ public class AccessController extends BaseRegionObserver
@Override
public void preMove(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo region,
ServerName srcServer, ServerName destServer) throws IOException {
requirePermission("move", region.getTableName(), null, null, Action.ADMIN);
requirePermission("move", region.getTable(), null, null, Action.ADMIN);
}
@Override
@ -713,7 +713,7 @@ public class AccessController extends BaseRegionObserver
@Override
public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo)
throws IOException {
requirePermission("assign", regionInfo.getTableName(), null, null, Action.ADMIN);
requirePermission("assign", regionInfo.getTable(), null, null, Action.ADMIN);
}
@Override
@ -723,7 +723,7 @@ public class AccessController extends BaseRegionObserver
@Override
public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo,
boolean force) throws IOException {
requirePermission("unassign", regionInfo.getTableName(), null, null, Action.ADMIN);
requirePermission("unassign", regionInfo.getTable(), null, null, Action.ADMIN);
}
@Override
@ -733,7 +733,7 @@ public class AccessController extends BaseRegionObserver
@Override
public void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> c,
HRegionInfo regionInfo) throws IOException {
requirePermission("regionOffline", regionInfo.getTableName(), null, null, Action.ADMIN);
requirePermission("regionOffline", regionInfo.getTable(), null, null, Action.ADMIN);
}
@Override
@ -1263,7 +1263,7 @@ public class AccessController extends BaseRegionObserver
switch(request.getUserPermission().getPermission().getType()) {
case Global :
case Table :
requirePermission("grant", perm.getTable(), perm.getFamily(),
requirePermission("grant", perm.getTableName(), perm.getFamily(),
perm.getQualifier(), Action.ADMIN);
break;
case Namespace :
@ -1303,7 +1303,7 @@ public class AccessController extends BaseRegionObserver
switch(request.getUserPermission().getPermission().getType()) {
case Global :
case Table :
requirePermission("revoke", perm.getTable(), perm.getFamily(),
requirePermission("revoke", perm.getTableName(), perm.getFamily(),
perm.getQualifier(), Action.ADMIN);
break;
case Namespace :
@ -1379,11 +1379,11 @@ public class AccessController extends BaseRegionObserver
if (permission instanceof TablePermission) {
TablePermission tperm = (TablePermission) permission;
for (Permission.Action action : permission.getActions()) {
if (!tperm.getTable().equals(tableName)) {
if (!tperm.getTableName().equals(tableName)) {
throw new CoprocessorException(AccessController.class, String.format("This method "
+ "can only execute at the table specified in TablePermission. " +
"Table of the region:%s , requested table:%s", tableName,
tperm.getTable()));
tperm.getTableName()));
}
Map<byte[], Set<byte[]>> familyMap = new TreeMap<byte[], Set<byte[]>>(Bytes.BYTES_COMPARATOR);
@ -1425,7 +1425,7 @@ public class AccessController extends BaseRegionObserver
if (region != null) {
HRegionInfo regionInfo = region.getRegionInfo();
if (regionInfo != null) {
tableName = regionInfo.getTableName();
tableName = regionInfo.getTable();
}
}
return tableName;
@ -1457,7 +1457,7 @@ public class AccessController extends BaseRegionObserver
}
private boolean isSpecialTable(HRegionInfo regionInfo) {
TableName tableName = regionInfo.getTableName();
TableName tableName = regionInfo.getTable();
return tableName.equals(AccessControlLists.ACL_TABLE_NAME)
|| tableName.equals(TableName.NAMESPACE_TABLE_NAME)
|| tableName.equals(TableName.META_TABLE_NAME);

View File

@ -1270,7 +1270,7 @@ public class HBaseFsck extends Configured implements Tool {
* Check if the specified region's table is disabled.
*/
private boolean isTableDisabled(HRegionInfo regionInfo) {
return disabledTables.contains(regionInfo.getTableName());
return disabledTables.contains(regionInfo.getTable());
}
/**
@ -1820,7 +1820,7 @@ public class HBaseFsck extends Configured implements Tool {
if (hbi.deployedOn.size() == 0) continue;
// We should be safe here
TableName tableName = hbi.metaEntry.getTableName();
TableName tableName = hbi.metaEntry.getTable();
TableInfo modTInfo = tablesInfo.get(tableName);
if (modTInfo == null) {
modTInfo = new TableInfo(tableName);
@ -2460,7 +2460,7 @@ public class HBaseFsck extends Configured implements Tool {
// pick only those tables that were not modified in the last few milliseconds.
if (info != null && info.getStartKey().length == 0 && !info.isMetaRegion()) {
if (info.modTime + timelag < now) {
tableNames.add(info.getTableName());
tableNames.add(info.getTable());
} else {
numSkipped.incrementAndGet(); // one more in-flux table
}
@ -2584,7 +2584,7 @@ public class HBaseFsck extends Configured implements Tool {
sn = pair.getSecond();
}
HRegionInfo hri = pair.getFirst();
if (!(isTableIncluded(hri.getTableName())
if (!(isTableIncluded(hri.getTable())
|| hri.isMetaRegion())) {
return true;
}
@ -2661,7 +2661,7 @@ public class HBaseFsck extends Configured implements Tool {
hash ^= Arrays.hashCode(getStartKey());
hash ^= Arrays.hashCode(getEndKey());
hash ^= Boolean.valueOf(isOffline()).hashCode();
hash ^= getTableName().hashCode();
hash ^= getTable().hashCode();
if (regionServer != null) {
hash ^= regionServer.hashCode();
}
@ -2752,7 +2752,7 @@ public class HBaseFsck extends Configured implements Tool {
public TableName getTableName() {
if (this.metaEntry != null) {
return this.metaEntry.getTableName();
return this.metaEntry.getTable();
} else if (this.hdfsEntry != null) {
// we are only guaranteed to have a path and not an HRI for hdfsEntry,
// so we get the name from the Path
@ -3108,7 +3108,7 @@ public class HBaseFsck extends Configured implements Tool {
List<HRegionInfo> ret = Lists.newArrayList();
for (HRegionInfo hri : regions) {
if (hri.isMetaTable() || (!hbck.checkMetaOnly
&& hbck.isTableIncluded(hri.getTableName()))) {
&& hbck.isTableIncluded(hri.getTable()))) {
ret.add(hri);
}
}

View File

@ -254,7 +254,7 @@ class HMerge {
Bytes.toString(HConstants.CATALOG_FAMILY) + ":" +
Bytes.toString(HConstants.REGIONINFO_QUALIFIER));
}
if (!region.getTableName().equals(this.tableName)) {
if (!region.getTable().equals(this.tableName)) {
return null;
}
return region;
@ -283,7 +283,7 @@ class HMerge {
continue;
}
HRegionInfo region = HRegionInfo.getHRegionInfo(currentRow);
if (!region.getTableName().equals(this.tableName)) {
if (!region.getTable().equals(this.tableName)) {
currentRow = metaScanner.next();
continue;
}

View File

@ -1915,7 +1915,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
continue;
}
if (info.getTableName().equals(tableName)) {
if (info.getTable().equals(tableName)) {
LOG.info("getMetaTableRows: row -> " +
Bytes.toStringBinary(result.getRow()) + info);
rows.add(result.getRow());
@ -2654,7 +2654,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
while ((r = s.next()) != null) {
byte [] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
HRegionInfo info = HRegionInfo.parseFromOrNull(b);
if (info != null && info.getTableName().equals(tableName)) {
if (info != null && info.getTable().equals(tableName)) {
b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
allRegionsAssigned &= (b != null);
}

View File

@ -1325,7 +1325,7 @@ public class TestAdmin {
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs);
for (HRegionInfo regionInfo : onlineRegions) {
if (!regionInfo.getTableName().isSystemTable()) {
if (!regionInfo.getTable().isSystemTable()) {
info = regionInfo;
admin.closeRegionWithEncodedRegionName(regionInfo.getEncodedName(), rs
.getServerName().getServerName());

View File

@ -142,7 +142,7 @@ public class TestMetaScanner {
//select a random region
HRegionInfo parent = regions.get(random.nextInt(regions.size()));
if (parent == null || !TABLENAME.equals(parent.getTableName())) {
if (parent == null || !TABLENAME.equals(parent.getTable())) {
continue;
}

View File

@ -87,7 +87,7 @@ implements WALObserver {
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException {
boolean bypass = false;
// check table name matches or not.
if (!Bytes.equals(info.getTableName().getName(), this.tableName)) {
if (!Bytes.equals(info.getTableName(), this.tableName)) {
return bypass;
}
preWALWriteCalled = true;

View File

@ -493,7 +493,7 @@ public class TestRegionObserverInterface {
try {
for (JVMClusterUtil.RegionServerThread t : cluster.getRegionServerThreads()) {
for (HRegionInfo r : ProtobufUtil.getOnlineRegions(t.getRegionServer())) {
if (!r.getTableName().equals(tableName)) {
if (!r.getTable().equals(tableName)) {
continue;
}
RegionCoprocessorHost cph = t.getRegionServer().getOnlineRegion(r.getRegionName()).

View File

@ -104,7 +104,7 @@ public class TestRegionServerCoprocessorExceptionWithAbort {
final Put put, final WALEdit edit,
final Durability durability) {
TableName tableName =
c.getEnvironment().getRegion().getRegionInfo().getTableName();
c.getEnvironment().getRegion().getRegionInfo().getTable();
if (TABLE_NAME.equals(tableName) && Bytes.equals(put.getRow(), ROW)) {
throw new NullPointerException("Buggy coprocessor: " + put);
}

View File

@ -53,8 +53,7 @@ public class TestRegionServerCoprocessorExceptionWithRemove {
final Put put, final WALEdit edit,
final Durability durability) {
String tableName =
c.getEnvironment().getRegion().getRegionInfo()
.getTableName().getNameAsString();
c.getEnvironment().getRegion().getRegionInfo().getTable().getNameAsString();
if (tableName.equals("observed_table")) {
Integer i = null;
i = i + 1;

View File

@ -186,7 +186,7 @@ public class TestWALObserver {
// it's where WAL write cp should occur.
long now = EnvironmentEdgeManager.currentTimeMillis();
log.append(hri, hri.getTableName(), edit, now, htd);
log.append(hri, hri.getTable(), edit, now, htd);
// the edit shall have been change now by the coprocessor.
foundFamily0 = false;

View File

@ -158,7 +158,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
.toBytes(table));
for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(hrs)) {
if (Bytes.equals(hri.getTableName().getName(), Bytes.toBytes(table))) {
if (Bytes.equals(hri.getTable().getName(), Bytes.toBytes(table))) {
// splitRegion doesn't work if startkey/endkey are null
ProtobufUtil.split(hrs, hri, rowkey(ROWCOUNT / 2)); // hard code split
}
@ -169,7 +169,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
do {
regions = 0;
for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(hrs)) {
if (Bytes.equals(hri.getTableName().getName(), Bytes.toBytes(table))) {
if (Bytes.equals(hri.getTable().getName(), Bytes.toBytes(table))) {
regions++;
}
}

View File

@ -501,7 +501,7 @@ public class TestAssignmentManager {
// adding region in pending close.
am.getRegionStates().updateRegionState(
REGIONINFO, State.SPLITTING, SERVERNAME_A);
am.getZKTable().setEnabledTable(REGIONINFO.getTableName());
am.getZKTable().setEnabledTable(REGIONINFO.getTable());
RegionTransition data = RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_SPLITTING,
REGIONINFO.getRegionName(), SERVERNAME_A);
String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
@ -550,9 +550,9 @@ public class TestAssignmentManager {
// adding region in pending close.
am.getRegionStates().updateRegionState(REGIONINFO, State.PENDING_CLOSE);
if (state == Table.State.DISABLING) {
am.getZKTable().setDisablingTable(REGIONINFO.getTableName());
am.getZKTable().setDisablingTable(REGIONINFO.getTable());
} else {
am.getZKTable().setDisabledTable(REGIONINFO.getTableName());
am.getZKTable().setDisabledTable(REGIONINFO.getTable());
}
RegionTransition data = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_CLOSING,
REGIONINFO.getRegionName(), SERVERNAME_A);
@ -576,7 +576,7 @@ public class TestAssignmentManager {
am.getRegionStates().isRegionsInTransition());
}
} finally {
am.setEnabledTable(REGIONINFO.getTableName());
am.setEnabledTable(REGIONINFO.getTable());
executor.shutdown();
am.shutdown();
// Clean up all znodes
@ -888,7 +888,7 @@ public class TestAssignmentManager {
}
try{
// set table in disabling state.
am.getZKTable().setDisablingTable(REGIONINFO.getTableName());
am.getZKTable().setDisablingTable(REGIONINFO.getTable());
am.joinCluster();
// should not call retainAssignment if we get empty regions in assignAllUserRegions.
assertFalse(
@ -896,12 +896,12 @@ public class TestAssignmentManager {
gate.get());
// need to change table state from disabling to disabled.
assertTrue("Table should be disabled.",
am.getZKTable().isDisabledTable(REGIONINFO.getTableName()));
am.getZKTable().isDisabledTable(REGIONINFO.getTable()));
} finally {
this.server.getConfiguration().setClass(
HConstants.HBASE_MASTER_LOADBALANCER_CLASS, DefaultLoadBalancer.class,
LoadBalancer.class);
am.getZKTable().setEnabledTable(REGIONINFO.getTableName());
am.getZKTable().setEnabledTable(REGIONINFO.getTable());
am.shutdown();
}
}
@ -927,17 +927,17 @@ public class TestAssignmentManager {
this.serverManager);
try {
// set table in enabling state.
am.getZKTable().setEnablingTable(REGIONINFO.getTableName());
new EnableTableHandler(server, REGIONINFO.getTableName(),
am.getZKTable().setEnablingTable(REGIONINFO.getTable());
new EnableTableHandler(server, REGIONINFO.getTable(),
am.getCatalogTracker(), am, new NullTableLockManager(), true).prepare()
.process();
assertEquals("Number of assignments should be 1.", 1, assignmentCount);
assertTrue("Table should be enabled.",
am.getZKTable().isEnabledTable(REGIONINFO.getTableName()));
am.getZKTable().isEnabledTable(REGIONINFO.getTable()));
} finally {
enabling = false;
assignmentCount = 0;
am.getZKTable().setEnabledTable(REGIONINFO.getTableName());
am.getZKTable().setEnabledTable(REGIONINFO.getTable());
am.shutdown();
ZKAssign.deleteAllNodes(this.watcher);
}
@ -964,7 +964,7 @@ public class TestAssignmentManager {
// adding region plan
am.regionPlans.put(REGIONINFO.getEncodedName(),
new RegionPlan(REGIONINFO, SERVERNAME_B, SERVERNAME_A));
am.getZKTable().setEnabledTable(REGIONINFO.getTableName());
am.getZKTable().setEnabledTable(REGIONINFO.getTable());
try {
am.assignInvoked = false;

View File

@ -833,7 +833,7 @@ public class TestCatalogJanitor {
// archive directory. Otherwise, it just seems to pick the first root directory it can find (so
// the single test passes, but when the full suite is run, things get borked).
FSUtils.setRootDir(fs.getConf(), rootdir);
Path tabledir = FSUtils.getTableDir(rootdir, parent.getTableName());
Path tabledir = FSUtils.getTableDir(rootdir, parent.getTable());
Path storedir = HStore.getStoreHomedir(tabledir, parent, htd.getColumnFamilies()[0].getName());
System.out.println("Old root:" + rootdir);
System.out.println("Old table:" + tabledir);
@ -915,7 +915,7 @@ public class TestCatalogJanitor {
final HRegionInfo daughter, final byte [] midkey, final boolean top)
throws IOException {
Path rootdir = services.getMasterFileSystem().getRootDir();
Path tabledir = FSUtils.getTableDir(rootdir, parent.getTableName());
Path tabledir = FSUtils.getTableDir(rootdir, parent.getTable());
Path storedir = HStore.getStoreHomedir(tabledir, daughter,
htd.getColumnFamilies()[0].getName());
Reference ref =

View File

@ -208,7 +208,7 @@ public class TestDistributedLogSplitting {
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs);
for (HRegionInfo region : regions) {
if (region.getTableName().getNameAsString().equalsIgnoreCase("table")) {
if (region.getTable().getNameAsString().equalsIgnoreCase("table")) {
foundRs = true;
break;
}
@ -222,7 +222,7 @@ public class TestDistributedLogSplitting {
Iterator<HRegionInfo> it = regions.iterator();
while (it.hasNext()) {
HRegionInfo region = it.next();
if (region.getTableName().getNamespaceAsString()
if (region.getTable().getNamespaceAsString()
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
it.remove();
}
@ -664,12 +664,12 @@ public class TestDistributedLogSplitting {
break;
}
if (tableName != null &&
!tableName.equalsIgnoreCase(region.getTableName().getNameAsString())) {
!tableName.equalsIgnoreCase(region.getTable().getNameAsString())) {
// make sure that we find a RS has online regions for both "table" and "disableTable"
hasRegionsForBothTables = true;
break;
} else if (tableName == null) {
tableName = region.getTableName().getNameAsString();
tableName = region.getTable().getNameAsString();
}
}
if (isCarryingMeta) {
@ -1123,7 +1123,7 @@ public class TestDistributedLogSplitting {
HRegionServer hrs = rst.getRegionServer();
List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs);
for (HRegionInfo hri : hris) {
if (hri.getTableName().isSystemTable()) {
if (hri.getTable().isSystemTable()) {
continue;
}
LOG.debug("adding data to rs = " + rst.getName() +
@ -1148,7 +1148,7 @@ public class TestDistributedLogSplitting {
for(Iterator<HRegionInfo> iter = regions.iterator(); iter.hasNext(); ) {
HRegionInfo regionInfo = iter.next();
if(regionInfo.getTableName().isSystemTable()) {
if(regionInfo.getTable().isSystemTable()) {
iter.remove();
}
}
@ -1157,7 +1157,7 @@ public class TestDistributedLogSplitting {
List<HRegionInfo> hris = new ArrayList<HRegionInfo>();
for (HRegionInfo region : regions) {
if (!region.getTableName().getNameAsString().equalsIgnoreCase(tname)) {
if (!region.getTable().getNameAsString().equalsIgnoreCase(tname)) {
continue;
}
hris.add(region);
@ -1330,7 +1330,7 @@ public class TestDistributedLogSplitting {
if (region.isMetaRegion()) {
isCarryingMeta = true;
}
if (tableName == null || region.getTableName().getNameAsString().equals(tableName)) {
if (tableName == null || region.getTable().getNameAsString().equals(tableName)) {
foundTableRegion = true;
}
if (foundTableRegion && (isCarryingMeta || !hasMetaRegion)) {

View File

@ -489,12 +489,12 @@ public class TestMasterTransitions {
for (Result r = null; (r = s.next()) != null;) {
HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
if (hri == null) break;
if (!hri.getTableName().getNameAsString().equals(TABLENAME)) {
if (!hri.getTable().getNameAsString().equals(TABLENAME)) {
continue;
}
// If start key, add 'aaa'.
if(!hri.getTableName().getNameAsString().equals(TABLENAME)) {
if(!hri.getTable().getNameAsString().equals(TABLENAME)) {
continue;
}
byte [] row = getStartKey(hri);
@ -511,24 +511,6 @@ public class TestMasterTransitions {
return rows;
}
/*
* @return Count of rows in TABLENAME
* @throws IOException
*/
private static int count() throws IOException {
HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
int rows = 0;
Scan scan = new Scan();
ResultScanner s = t.getScanner(scan);
for (Result r = null; (r = s.next()) != null;) {
rows++;
}
s.close();
LOG.info("Counted=" + rows);
t.close();
return rows;
}
/*
* @param hri
* @return Start key for hri (If start key is '', then return 'aaa'.
@ -545,6 +527,4 @@ public class TestMasterTransitions {
private static byte [] getTestQualifier() {
return getTestFamily();
}
}
}

View File

@ -64,8 +64,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -307,7 +305,7 @@ public class TestRegionPlacement {
TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getOnlineRegionsLocalContext();
isNamespaceServer = false;
for (HRegion r : regs) {
if (r.getRegionInfo().getTableName().getNamespaceAsString()
if (r.getRegionInfo().getTable().getNamespaceAsString()
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
isNamespaceServer = true;
break;
@ -606,7 +604,7 @@ public class TestRegionPlacement {
public boolean processRow(Result result) throws IOException {
try {
HRegionInfo info = MetaScanner.getHRegionInfo(result);
if(info.getTableName().getNamespaceAsString()
if(info.getTable().getNamespaceAsString()
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
return true;
}

View File

@ -283,7 +283,7 @@ public class TestZKBasedOpenCloseRegion {
for (Result r = null; (r = s.next()) != null;) {
HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
if (hri == null) break;
if(!hri.getTableName().equals(TABLENAME)) {
if(!hri.getTable().equals(TABLENAME)) {
continue;
}
// If start key, add 'aaa'.

View File

@ -480,7 +480,7 @@ public class TestEndToEndSplitTransaction {
throws IOException, InterruptedException {
log("blocking until region is opened for reading:" + hri.getRegionNameAsString());
long start = System.currentTimeMillis();
HTable table = new HTable(conf, hri.getTableName());
HTable table = new HTable(conf, hri.getTable());
try {
byte [] row = hri.getStartKey();

View File

@ -60,7 +60,7 @@ public class TestHRegionFileSystem {
// Create a Region
HRegionInfo hri = new HRegionInfo(TableName.valueOf("TestTable"));
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs,
FSUtils.getTableDir(rootDir, hri.getTableName()), hri);
FSUtils.getTableDir(rootDir, hri.getTable()), hri);
// Verify if the region is on disk
Path regionDir = regionFs.getRegionDir();
@ -72,12 +72,12 @@ public class TestHRegionFileSystem {
// Open the region
regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, fs,
FSUtils.getTableDir(rootDir, hri.getTableName()), hri, false);
FSUtils.getTableDir(rootDir, hri.getTable()), hri, false);
assertEquals(regionDir, regionFs.getRegionDir());
// Delete the region
HRegionFileSystem.deleteRegionFromFileSystem(conf, fs,
FSUtils.getTableDir(rootDir, hri.getTableName()), hri);
FSUtils.getTableDir(rootDir, hri.getTable()), hri);
assertFalse("The region folder should be removed", fs.exists(regionDir));
fs.delete(rootDir, true);

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver;
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 static org.junit.Assert.fail;
@ -28,12 +27,12 @@ import java.io.IOException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
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.util.Bytes;
import org.apache.hadoop.hbase.util.MD5Hash;
import org.junit.Test;

View File

@ -185,7 +185,7 @@ public class TestRegionMergeTransactionOnCluster {
FileSystem fs = master.getMasterFileSystem().getFileSystem();
Path rootDir = master.getMasterFileSystem().getRootDir();
Path tabledir = FSUtils.getTableDir(rootDir, mergedRegionInfo.getTableName());
Path tabledir = FSUtils.getTableDir(rootDir, mergedRegionInfo.getTable());
Path regionAdir = new Path(tabledir, regionA.getEncodedName());
Path regionBdir = new Path(tabledir, regionB.getEncodedName());
assertTrue(fs.exists(regionAdir));

View File

@ -307,7 +307,7 @@ public class TestRegionServerNoMaster {
hri.getEncodedNameAsBytes()));
// Let's start the open handler
HTableDescriptor htd = getRS().tableDescriptors.get(hri.getTableName());
HTableDescriptor htd = getRS().tableDescriptors.get(hri.getTable());
getRS().service.submit(new OpenRegionHandler(getRS(), getRS(), hri, htd, 0));
// The open handler should have removed the region from RIT but kept the region closed
@ -361,7 +361,7 @@ public class TestRegionServerNoMaster {
// 1) There is no ZK node
// 2) The region in RIT was changed.
// The order is more or less implementation dependant.
HTableDescriptor htd = getRS().tableDescriptors.get(hri.getTableName());
HTableDescriptor htd = getRS().tableDescriptors.get(hri.getTable());
getRS().service.submit(new OpenRegionHandler(getRS(), getRS(), hri, htd, 0));
// The open handler should have removed the region from RIT but kept the region closed

View File

@ -217,11 +217,11 @@ public class TestSplitTransactionOnCluster {
RegionStates regionStates = cluster.getMaster().getAssignmentManager().getRegionStates();
Map<String, RegionState> rit = regionStates.getRegionsInTransition();
for (int i=0; rit.containsKey(hri.getTableName()) && i<100; i++) {
for (int i=0; rit.containsKey(hri.getTable()) && i<100; i++) {
Thread.sleep(100);
}
assertFalse("region still in transition", rit.containsKey(
rit.containsKey(hri.getTableName())));
rit.containsKey(hri.getTable())));
List<HRegion> onlineRegions = regionServer.getOnlineRegions(tableName);
// Region server side split is successful.
@ -923,7 +923,7 @@ public class TestSplitTransactionOnCluster {
@Override
void transitionZKNode(Server server, RegionServerServices services, HRegion a, HRegion b)
throws IOException {
if (this.currentRegion.getRegionInfo().getTableName().getNameAsString()
if (this.currentRegion.getRegionInfo().getTable().getNameAsString()
.equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
try {
if (!secondSplit){
@ -935,14 +935,14 @@ public class TestSplitTransactionOnCluster {
}
super.transitionZKNode(server, services, a, b);
if (this.currentRegion.getRegionInfo().getTableName().getNameAsString()
if (this.currentRegion.getRegionInfo().getTable().getNameAsString()
.equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
firstSplitCompleted = true;
}
}
@Override
public boolean rollback(Server server, RegionServerServices services) throws IOException {
if (this.currentRegion.getRegionInfo().getTableName().getNameAsString()
if (this.currentRegion.getRegionInfo().getTable().getNameAsString()
.equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
if(secondSplit){
super.rollback(server, services);

View File

@ -93,7 +93,7 @@ public class TestStoreFile extends HBaseTestCase {
final HRegionInfo hri =
new HRegionInfo(TableName.valueOf("testBasicHalfMapFileTb"));
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(
conf, fs, new Path(this.testDir, hri.getTableName().getNameAsString()), hri);
conf, fs, new Path(this.testDir, hri.getTable().getNameAsString()), hri);
StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs, 2 * 1024)
.withFilePath(regionFs.createTempName())
@ -142,7 +142,7 @@ public class TestStoreFile extends HBaseTestCase {
public void testReference() throws IOException {
final HRegionInfo hri = new HRegionInfo(TableName.valueOf("testReferenceTb"));
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(
conf, fs, new Path(this.testDir, hri.getTableName().getNameAsString()), hri);
conf, fs, new Path(this.testDir, hri.getTable().getNameAsString()), hri);
// Make a store file and write data to it.
StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs, 8 * 1024)
@ -162,7 +162,7 @@ public class TestStoreFile extends HBaseTestCase {
kv = KeyValue.createKeyValueFromKey(reader.getLastKey());
byte [] finalRow = kv.getRow();
// Make a reference
HRegionInfo splitHri = new HRegionInfo(hri.getTableName(), null, midRow);
HRegionInfo splitHri = new HRegionInfo(hri.getTable(), null, midRow);
Path refPath = splitStoreFile(regionFs, splitHri, TEST_FAMILY, hsf, midRow, true);
StoreFile refHsf = new StoreFile(this.fs, refPath, conf, cacheConf,
BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
@ -186,7 +186,7 @@ public class TestStoreFile extends HBaseTestCase {
Configuration testConf = new Configuration(this.conf);
FSUtils.setRootDir(testConf, this.testDir);
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(
testConf, fs, FSUtils.getTableDir(this.testDir, hri.getTableName()), hri);
testConf, fs, FSUtils.getTableDir(this.testDir, hri.getTable()), hri);
// Make a store file and write data to it.
StoreFile.Writer writer = new StoreFile.WriterBuilder(testConf, cacheConf, this.fs, 8 * 1024)
@ -228,7 +228,7 @@ public class TestStoreFile extends HBaseTestCase {
// adding legal table name chars to verify regex handles it.
HRegionInfo hri = new HRegionInfo(TableName.valueOf("_original-evil-name"));
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(
testConf, fs, FSUtils.getTableDir(this.testDir, hri.getTableName()), hri);
testConf, fs, FSUtils.getTableDir(this.testDir, hri.getTable()), hri);
// Make a store file and write data to it. <root>/<tablename>/<rgn>/<cf>/<file>
StoreFile.Writer writer = new StoreFile.WriterBuilder(testConf, cacheConf, this.fs, 8 * 1024)
@ -240,7 +240,7 @@ public class TestStoreFile extends HBaseTestCase {
// create link to store file. <root>/clone/region/<cf>/<hfile>-<region>-<table>
HRegionInfo hriClone = new HRegionInfo(TableName.valueOf("clone"));
HRegionFileSystem cloneRegionFs = HRegionFileSystem.createRegionOnFileSystem(
testConf, fs, FSUtils.getTableDir(this.testDir, hri.getTableName()),
testConf, fs, FSUtils.getTableDir(this.testDir, hri.getTable()),
hriClone);
Path dstPath = cloneRegionFs.getStoreDir(TEST_FAMILY);
HFileLink.create(testConf, this.fs, dstPath, hri, storeFilePath.getName());
@ -250,8 +250,8 @@ public class TestStoreFile extends HBaseTestCase {
// create splits of the link.
// <root>/clone/splitA/<cf>/<reftohfilelink>,
// <root>/clone/splitB/<cf>/<reftohfilelink>
HRegionInfo splitHriA = new HRegionInfo(hri.getTableName(), null, SPLITKEY);
HRegionInfo splitHriB = new HRegionInfo(hri.getTableName(), SPLITKEY, null);
HRegionInfo splitHriA = new HRegionInfo(hri.getTable(), null, SPLITKEY);
HRegionInfo splitHriB = new HRegionInfo(hri.getTable(), SPLITKEY, null);
StoreFile f = new StoreFile(fs, linkFilePath, testConf, cacheConf, BloomType.NONE,
NoOpDataBlockEncoder.INSTANCE);
Path pathA = splitStoreFile(cloneRegionFs, splitHriA, TEST_FAMILY, f, SPLITKEY, true); // top
@ -300,11 +300,11 @@ public class TestStoreFile extends HBaseTestCase {
KeyValue midKV = KeyValue.createKeyValueFromKey(midkey);
byte [] midRow = midKV.getRow();
// Create top split.
HRegionInfo topHri = new HRegionInfo(regionFs.getRegionInfo().getTableName(),
HRegionInfo topHri = new HRegionInfo(regionFs.getRegionInfo().getTable(),
null, midRow);
Path topPath = splitStoreFile(regionFs, topHri, TEST_FAMILY, f, midRow, true);
// Create bottom split.
HRegionInfo bottomHri = new HRegionInfo(regionFs.getRegionInfo().getTableName(),
HRegionInfo bottomHri = new HRegionInfo(regionFs.getRegionInfo().getTable(),
midRow, null);
Path bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, midRow, false);
// Make readers on top and bottom.

View File

@ -103,10 +103,10 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool
addFamilyMapToWALEdit(put.getFamilyCellMap(), walEdit);
HRegionInfo hri = region.getRegionInfo();
if (this.noSync) {
hlog.appendNoSync(hri, hri.getTableName(), walEdit,
hlog.appendNoSync(hri, hri.getTable(), walEdit,
new ArrayList<UUID>(), now, htd);
} else {
hlog.append(hri, hri.getTableName(), walEdit, now, htd);
hlog.append(hri, hri.getTable(), walEdit, now, htd);
}
}
long totalTime = (System.currentTimeMillis() - startTime);

View File

@ -191,7 +191,7 @@ public class TestTablePermissions {
assertEquals("Should have 1 permission for george", 1, userPerms.size());
TablePermission permission = userPerms.get(0);
assertEquals("Permission should be for " + TEST_TABLE,
TEST_TABLE, permission.getTable());
TEST_TABLE, permission.getTableName());
assertNull("Column family should be empty", permission.getFamily());
// check actions
@ -206,7 +206,7 @@ public class TestTablePermissions {
assertEquals("Should have 1 permission for hubert", 1, userPerms.size());
permission = userPerms.get(0);
assertEquals("Permission should be for " + TEST_TABLE,
TEST_TABLE, permission.getTable());
TEST_TABLE, permission.getTableName());
assertNull("Column family should be empty", permission.getFamily());
// check actions
@ -221,7 +221,7 @@ public class TestTablePermissions {
assertEquals("Should have 1 permission for humphrey", 1, userPerms.size());
permission = userPerms.get(0);
assertEquals("Permission should be for " + TEST_TABLE,
TEST_TABLE, permission.getTable());
TEST_TABLE, permission.getTableName());
assertTrue("Permission should be for family " + TEST_FAMILY,
Bytes.equals(TEST_FAMILY, permission.getFamily()));
assertTrue("Permission should be for qualifier " + TEST_QUALIFIER,
@ -249,7 +249,7 @@ public class TestTablePermissions {
assertNotNull(userPerms);
assertEquals(1, userPerms.size());
permission = userPerms.get(0);
assertEquals(TEST_TABLE, permission.getTable());
assertEquals(TEST_TABLE, permission.getTableName());
assertEquals(1, permission.getActions().length);
assertEquals(TablePermission.Action.READ, permission.getActions()[0]);
@ -257,7 +257,7 @@ public class TestTablePermissions {
assertNotNull(userPerms);
assertEquals(1, userPerms.size());
permission = userPerms.get(0);
assertEquals(TEST_TABLE2, permission.getTable());
assertEquals(TEST_TABLE2, permission.getTableName());
assertEquals(2, permission.getActions().length);
actions = Arrays.asList(permission.getActions());
assertTrue(actions.contains(TablePermission.Action.READ));

View File

@ -37,17 +37,16 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.util.Bytes;
@ -202,7 +201,7 @@ public class TestExportSnapshot {
// First region, simple with one plain hfile.
HRegionInfo hri = new HRegionInfo(tableWithRefsName);
HRegionFileSystem r0fs = HRegionFileSystem.createRegionOnFileSystem(conf,
fs, FSUtils.getTableDir(archiveDir, hri.getTableName()), hri);
fs, FSUtils.getTableDir(archiveDir, hri.getTable()), hri);
Path storeFile = new Path(rootDir, TEST_HFILE);
FSDataOutputStream out = fs.create(storeFile);
out.write(Bytes.toBytes("Test Data"));
@ -213,7 +212,7 @@ public class TestExportSnapshot {
// This region contains a reference to the hfile in the first region.
hri = new HRegionInfo(tableWithRefsName);
HRegionFileSystem r1fs = HRegionFileSystem.createRegionOnFileSystem(conf,
fs, new Path(archiveDir, hri.getTableName().getNameAsString()), hri);
fs, new Path(archiveDir, hri.getTable().getNameAsString()), hri);
storeFile = new Path(rootDir, TEST_HFILE + '.' + r0fs.getRegionInfo().getEncodedName());
out = fs.create(storeFile);
out.write(Bytes.toBytes("Test Data"));

View File

@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
@ -160,7 +159,7 @@ public class TestRestoreSnapshotHelper {
// First region, simple with one plain hfile.
HRegionInfo hri = new HRegionInfo(htd.getTableName());
HRegionFileSystem r0fs = HRegionFileSystem.createRegionOnFileSystem(conf,
fs, FSUtils.getTableDir(archiveDir, hri.getTableName()), hri);
fs, FSUtils.getTableDir(archiveDir, hri.getTable()), hri);
Path storeFile = new Path(rootDir, TEST_HFILE);
fs.createNewFile(storeFile);
r0fs.commitStoreFile(TEST_FAMILY, storeFile);
@ -169,7 +168,7 @@ public class TestRestoreSnapshotHelper {
// This region contains a reference to the hfile in the first region.
hri = new HRegionInfo(htd.getTableName());
HRegionFileSystem r1fs = HRegionFileSystem.createRegionOnFileSystem(conf,
fs, FSUtils.getTableDir(archiveDir, hri.getTableName()), hri);
fs, FSUtils.getTableDir(archiveDir, hri.getTable()), hri);
storeFile = new Path(rootDir, TEST_HFILE + '.' + r0fs.getRegionInfo().getEncodedName());
fs.createNewFile(storeFile);
r1fs.commitStoreFile(TEST_FAMILY, storeFile);

View File

@ -1868,7 +1868,7 @@ public class TestHBaseFsck {
@Override
public boolean processRow(Result rowResult) throws IOException {
if(!MetaScanner.getHRegionInfo(rowResult).getTableName().isSystemTable()) {
if(!MetaScanner.getHRegionInfo(rowResult).getTable().isSystemTable()) {
Delete delete = new Delete(rowResult.getRow());
delete.deleteColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
deletes.add(delete);

View File

@ -235,8 +235,7 @@ public class OfflineMetaRebuildTestCore {
for (Result r : scanner) {
HRegionInfo info =
HRegionInfo.getHRegionInfo(r);
if(info != null &&
!info.getTableName().getNamespaceAsString()
if(info != null && !info.getTable().getNamespaceAsString()
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
Delete d = new Delete(r.getRow());
dels.add(d);