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. * Gets the table name from the specified region name.
* @param regionName * @param regionName
* @return Table name. * @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; int offset = -1;
for (int i = 0; i < regionName.length; i++) { for (int i = 0; i < regionName.length; i++) {
if (regionName[i] == HConstants.DELIMITER) { if (regionName[i] == HConstants.DELIMITER) {
@ -413,7 +415,18 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
} }
byte[] buff = new byte[offset]; byte[] buff = new byte[offset];
System.arraycopy(regionName, 0, buff, 0, 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 * Get current table name of the region
* @return byte array of table name * @return byte array of table name
* @deprecated Since 0.96.0; use #getTable()
*/ */
public TableName getTableName() { @Deprecated
if (tableName == null || tableName.getName().length == 0) { public byte [] getTableName() {
tableName = getTableName(getRegionName()); return getTable().toBytes();
} }
return tableName;
/**
* 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 = getTable(getRegionName());
}
return this.tableName;
} }
/** /**
@ -570,7 +597,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
/** @return true if this region is a meta region */ /** @return true if this region is a meta region */
public boolean isMetaRegion() { 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) { public static RegionInfo convert(final HRegionInfo info) {
if (info == null) return null; if (info == null) return null;
RegionInfo.Builder builder = RegionInfo.newBuilder(); RegionInfo.Builder builder = RegionInfo.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName(info.getTableName())); builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));
builder.setRegionId(info.getRegionId()); builder.setRegionId(info.getRegionId());
if (info.getStartKey() != null) { if (info.getStartKey() != null) {
builder.setStartKey(ByteString.copyFrom(info.getStartKey())); builder.setStartKey(ByteString.copyFrom(info.getStartKey()));

View File

@ -545,6 +545,16 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
values.remove(key); 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 * 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. * 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); Pair<HRegionInfo, ServerName> region = HRegionInfo.getHRegionInfoAndServerName(r);
HRegionInfo hri = region.getFirst(); HRegionInfo hri = region.getFirst();
if (hri == null) return true; if (hri == null) return true;
if (hri.getTableName() == null) return true; if (hri.getTable() == null) return true;
if (disabledTables.contains( if (disabledTables.contains(
hri.getTableName())) return true; hri.getTable())) return true;
// Are we to include split parents in the list? // Are we to include split parents in the list?
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true; if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
regions.put(hri, region.getSecond()); regions.put(hri, region.getSecond());
@ -362,7 +362,7 @@ public class MetaReader {
* <code>tableName</code> * <code>tableName</code>
*/ */
static boolean isInsideTable(final HRegionInfo current, final TableName tableName) { 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)); 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 * 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)); 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 * Create a new ClientScanner for the specified table Note that the passed {@link Scan}'s start
* row maybe changed changed. * row maybe changed changed.
@ -158,7 +176,16 @@ public class ClientScanner extends AbstractClientScanner {
return this.connection; 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; return this.tableName;
} }
@ -253,7 +280,7 @@ public class ClientScanner extends AbstractClientScanner {
int nbRows) { int nbRows) {
scan.setStartRow(localStartKey); scan.setStartRow(localStartKey);
ScannerCallable s = new ScannerCallable(getConnection(), ScannerCallable s = new ScannerCallable(getConnection(),
getTableName(), scan, this.scanMetrics); getTable(), scan, this.scanMetrics);
s.setCaching(nbRows); s.setCaching(nbRows);
return s; return s;
} }

View File

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

View File

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

View File

@ -1006,6 +1006,18 @@ public class HTable implements HTableInterface {
return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL); 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} * {@inheritDoc}
*/ */

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.client;
import com.google.protobuf.Service; import com.google.protobuf.Service;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -384,6 +385,14 @@ public interface HTableInterface extends Closeable {
long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
long amount, Durability durability) throws IOException; 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. * Tells whether or not 'auto-flush' is turned on.
* *

View File

@ -19,19 +19,6 @@
*/ */
package org.apache.hadoop.hbase.client; 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.io.IOException;
import java.util.AbstractMap.SimpleEntry; import java.util.AbstractMap.SimpleEntry;
import java.util.ArrayList; import java.util.ArrayList;
@ -46,6 +33,18 @@ import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong; 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. * 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. * 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 { HRegionLocation oldLoc) throws IOException {
Put failedPut = failedPutStatus.getPut(); Put failedPut = failedPutStatus.getPut();
// The currentPut is failed. So get the table name for the currentPut. // 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 // Decrease the retry count
int retryCount = failedPutStatus.getRetryCount() - 1; 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"); 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) { if (info == null) {
return true; return true;
} }
if (!(info.getTableName().equals(tableName))) { if (!(info.getTable().equals(tableName))) {
return false; return false;
} }
return super.processRow(rowResult); return super.processRow(rowResult);

View File

@ -101,6 +101,15 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
logCutOffLatency = conf.getInt(LOG_SCANNER_LATENCY_CUTOFF, 1000); 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 * @param reload force reload of server location
* @throws IOException * @throws IOException

View File

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

View File

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

View File

@ -140,7 +140,7 @@ public class UserPermission extends TablePermission {
* @param perm a TablePermission * @param perm a TablePermission
*/ */
public UserPermission(byte[] user, TablePermission perm) { 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); perm.actions);
this.user = user; this.user = user;
} }

View File

@ -129,8 +129,9 @@ public final class TableName implements Comparable<TableName> {
return tableName; return tableName;
} }
public static void isLegalTableQualifierName(final byte[] qualifierName){ public static byte [] isLegalTableQualifierName(final byte[] qualifierName){
isLegalTableQualifierName(qualifierName, 0, qualifierName.length); 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) public static void archiveRegion(Configuration conf, FileSystem fs, HRegionInfo info)
throws IOException { throws IOException {
Path rootDir = FSUtils.getRootDir(conf); 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)); HRegion.getRegionDir(rootDir, info));
} }

View File

@ -564,6 +564,12 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
public void setWriteBufferSize(long writeBufferSize) throws IOException { public void setWriteBufferSize(long writeBufferSize) throws IOException {
table.setWriteBufferSize(writeBufferSize); 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 */ /** The coprocessor */

View File

@ -263,7 +263,7 @@ public class HFileLink extends FileLink {
*/ */
public static String createHFileLinkName(final HRegionInfo hfileRegionInfo, public static String createHFileLinkName(final HRegionInfo hfileRegionInfo,
final String hfileName) { final String hfileName) {
return createHFileLinkName(hfileRegionInfo.getTableName(), return createHFileLinkName(hfileRegionInfo.getTable(),
hfileRegionInfo.getEncodedName(), hfileName); hfileRegionInfo.getEncodedName(), hfileName);
} }
@ -300,7 +300,7 @@ public class HFileLink extends FileLink {
public static boolean create(final Configuration conf, final FileSystem fs, public static boolean create(final Configuration conf, final FileSystem fs,
final Path dstFamilyPath, final HRegionInfo hfileRegionInfo, final Path dstFamilyPath, final HRegionInfo hfileRegionInfo,
final String hfileName) throws IOException { final String hfileName) throws IOException {
TableName linkedTable = hfileRegionInfo.getTableName(); TableName linkedTable = hfileRegionInfo.getTable();
String linkedRegion = hfileRegionInfo.getEncodedName(); String linkedRegion = hfileRegionInfo.getEncodedName();
return create(conf, fs, dstFamilyPath, linkedTable, linkedRegion, hfileName); 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, public TableSplit(byte [] tableName, byte [] startRow, byte [] endRow,
final String location) { final String location) {
this(TableName.valueOf(tableName), startRow, endRow, location); this(TableName.valueOf(tableName), startRow, endRow,
location);
} }
/** @return table name */ /** @return table name */

View File

@ -517,6 +517,16 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
return null; 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, * 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 * 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 * failure
*/ */
protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn, protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn,
final TableName tableName, final TableName tableName, final byte[] first, Collection<LoadQueueItem> lqis)
final byte[] first, Collection<LoadQueueItem> lqis) throws IOException { throws IOException {
final List<Pair<byte[], String>> famPaths = final List<Pair<byte[], String>> famPaths =
new ArrayList<Pair<byte[], String>>(lqis.size()); new ArrayList<Pair<byte[], String>>(lqis.size());
for (LoadQueueItem lqi : lqis) { for (LoadQueueItem lqi : lqis) {

View File

@ -85,10 +85,19 @@ implements Writable, Comparable<TableSplit> {
/** Default constructor. */ /** Default constructor. */
public TableSplit() { public TableSplit() {
this(null, null, HConstants.EMPTY_BYTE_ARRAY, this((TableName)null, null, HConstants.EMPTY_BYTE_ARRAY,
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. * Creates a new instance while assigning all variables.
* *
@ -112,6 +121,15 @@ implements Writable, Comparable<TableSplit> {
this.regionLocation = location; 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. * Creates a new instance without a scanner.
* *
@ -135,12 +153,24 @@ implements Writable, Comparable<TableSplit> {
return TableMapReduceUtil.convertStringToScan(this.scan); 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. * Returns the table name.
* *
* @return 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; 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 // If The table name of the two splits is the same then compare start row
// otherwise compare based on table names // otherwise compare based on table names
int tableNameComparison = int tableNameComparison =
getTableName().compareTo(split.getTableName()); getTable().compareTo(split.getTable());
return tableNameComparison != 0 ? tableNameComparison : Bytes.compareTo( return tableNameComparison != 0 ? tableNameComparison : Bytes.compareTo(
getStartRow(), split.getStartRow()); getStartRow(), split.getStartRow());
} }

View File

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

View File

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

View File

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

View File

@ -588,7 +588,7 @@ public class MasterFileSystem {
public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName) public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName)
throws IOException { throws IOException {
// archive family store files // 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); HFileArchiver.archiveFamily(fs, conf, region, tableDir, familyName);
// delete the family folder // delete the family folder

View File

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

View File

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

View File

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

View File

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

View File

@ -122,7 +122,7 @@ class RegionLocationFinder {
protected List<ServerName> internalGetTopBlockLocation(HRegionInfo region) { protected List<ServerName> internalGetTopBlockLocation(HRegionInfo region) {
List<ServerName> topServerNames = null; List<ServerName> topServerNames = null;
try { try {
HTableDescriptor tableDescriptor = getTableDescriptor(region.getTableName()); HTableDescriptor tableDescriptor = getTableDescriptor(region.getTable());
if (tableDescriptor != null) { if (tableDescriptor != null) {
HDFSBlocksDistribution blocksDistribution = HDFSBlocksDistribution blocksDistribution =
HRegion.computeHDFSBlocksDistribution(getConf(), tableDescriptor, region); 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()); LOG.debug("Handling CLOSED event for " + regionInfo.getEncodedName());
// Check if this table is being disabled or not // Check if this table is being disabled or not
if (this.assignmentManager.getZKTable(). if (this.assignmentManager.getZKTable().
isDisablingOrDisabledTable(this.regionInfo.getTableName())) { isDisablingOrDisabledTable(this.regionInfo.getTable())) {
assignmentManager.offlineDisabledRegion(regionInfo); assignmentManager.offlineDisabledRegion(regionInfo);
return; return;
} }

View File

@ -230,7 +230,7 @@ public class EnableTableHandler extends EventHandler {
} else { } else {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Skipping assign for the region " + hri + " during enable table " 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; this.expectedVersion = expectedVersion;
if(regionInfo.isMetaRegion()) { if(regionInfo.isMetaRegion()) {
priority = OpenedPriority.META; priority = OpenedPriority.META;
} else if(regionInfo.getTableName() } else if(regionInfo.getTable()
.getNamespaceAsString().equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) { .getNamespaceAsString().equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
priority = OpenedPriority.SYSTEM; priority = OpenedPriority.SYSTEM;
} else { } else {
@ -113,8 +113,7 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
" because regions is NOT in RIT -- presuming this is because it SPLIT"); " because regions is NOT in RIT -- presuming this is because it SPLIT");
} }
if (!openedNodeDeleted) { if (!openedNodeDeleted) {
if (this.assignmentManager.getZKTable().isDisablingOrDisabledTable( if (this.assignmentManager.getZKTable().isDisablingOrDisabledTable(regionInfo.getTable())) {
regionInfo.getTableName())) {
debugLog(regionInfo, "Opened region " debugLog(regionInfo, "Opened region "
+ regionInfo.getShortNameToLog() + " but " + regionInfo.getShortNameToLog() + " but "
+ "this table is disabled, triggering close of region"); + "this table is disabled, triggering close of region");

View File

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

View File

@ -3924,11 +3924,11 @@ public class HRegion implements HeapSize { // , Writable{
final HLog hlog, final HLog hlog,
final boolean initialize, final boolean ignoreHLog) final boolean initialize, final boolean ignoreHLog)
throws IOException { throws IOException {
LOG.info("creating HRegion " + info.getTableName().getNameAsString() LOG.info("creating HRegion " + info.getTable().getNameAsString()
+ " HTD == " + hTableDescriptor + " RootDir = " + rootDir + + " 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); FileSystem fs = FileSystem.get(conf);
HRegionFileSystem rfs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info); HRegionFileSystem rfs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info);
HLog effectiveHLog = hlog; HLog effectiveHLog = hlog;
@ -4088,7 +4088,7 @@ public class HRegion implements HeapSize { // , Writable{
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Opening region: " + info); 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); HRegion r = HRegion.newHRegion(dir, wal, fs, conf, info, htd, rsServices);
return r.openHRegion(reporter); return r.openHRegion(reporter);
} }
@ -4217,8 +4217,7 @@ public class HRegion implements HeapSize { // , Writable{
@Deprecated @Deprecated
public static Path getRegionDir(final Path rootdir, final HRegionInfo info) { public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
return new Path( return new Path(
FSUtils.getTableDir(rootdir, info.getTableName()), FSUtils.getTableDir(rootdir, info.getTable()), info.getEncodedName());
info.getEncodedName());
} }
/** /**
@ -4277,8 +4276,7 @@ public class HRegion implements HeapSize { // , Writable{
* @throws IOException * @throws IOException
*/ */
public static HRegion merge(final HRegion a, final HRegion b) throws IOException { public static HRegion merge(final HRegion a, final HRegion b) throws IOException {
if (!a.getRegionInfo().getTableName().equals( if (!a.getRegionInfo().getTable().equals(b.getRegionInfo().getTable())) {
b.getRegionInfo().getTableName())) {
throw new IOException("Regions do not belong to the same table"); throw new IOException("Regions do not belong to the same table");
} }
@ -5197,7 +5195,7 @@ public class HRegion implements HeapSize { // , Writable{
public byte[] checkSplit() { public byte[] checkSplit() {
// Can't split META // Can't split META
if (this.getRegionInfo().isMetaTable() || if (this.getRegionInfo().isMetaTable() ||
TableName.NAMESPACE_TABLE_NAME.equals(this.getRegionInfo().getTableName())) { TableName.NAMESPACE_TABLE_NAME.equals(this.getRegionInfo().getTable())) {
if (shouldForceSplit()) { if (shouldForceSplit()) {
LOG.warn("Cannot split meta region in HBase 0.20 and above"); 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) { synchronized (this.onlineRegions) {
for (HRegion region: this.onlineRegions.values()) { for (HRegion region: this.onlineRegions.values()) {
HRegionInfo regionInfo = region.getRegionInfo(); HRegionInfo regionInfo = region.getRegionInfo();
if(regionInfo.getTableName().equals(tableName)) { if(regionInfo.getTable().equals(tableName)) {
tableRegions.add(region); tableRegions.add(region);
} }
} }
@ -3532,10 +3532,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
} }
} }
LOG.info("Open " + region.getRegionNameAsString()); LOG.info("Open " + region.getRegionNameAsString());
htd = htds.get(region.getTableName()); htd = htds.get(region.getTable());
if (htd == null) { if (htd == null) {
htd = this.tableDescriptors.get(region.getTableName()); htd = this.tableDescriptors.get(region.getTable());
htds.put(region.getTableName(), htd); htds.put(region.getTable(), htd);
} }
final Boolean previous = this.regionsInTransitionInRS.putIfAbsent( final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(

View File

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

View File

@ -146,7 +146,7 @@ public class RegionCoprocessorHost
loadSystemCoprocessors(conf, REGION_COPROCESSOR_CONF_KEY); loadSystemCoprocessors(conf, REGION_COPROCESSOR_CONF_KEY);
// load system default cp's for user tables from configuration. // 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); 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 // 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); endKey, false, rid);
return mergedRegionInfo; return mergedRegionInfo;
} }

View File

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

View File

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

View File

@ -779,4 +779,10 @@ public class RemoteHTable implements HTableInterface {
public void setWriteBufferSize(long writeBufferSize) throws IOException { public void setWriteBufferSize(long writeBufferSize) throws IOException {
throw new IOException("setWriteBufferSize not supported"); 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()) { } else if(userPerm.isGlobal()) {
row = ACL_GLOBAL_NAME; row = ACL_GLOBAL_NAME;
} else { } else {
row = userPerm.getTable().getName(); row = userPerm.getTableName().getName();
} }
return row; return row;
} }
@ -502,7 +502,7 @@ public class AccessControlLists {
for (Map.Entry<String, TablePermission> entry : allPerms.entries()) { for (Map.Entry<String, TablePermission> entry : allPerms.entries()) {
UserPermission up = new UserPermission(Bytes.toBytes(entry.getKey()), 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()); entry.getValue().getQualifier(), entry.getValue().getActions());
perms.add(up); perms.add(up);
} }

View File

@ -208,7 +208,7 @@ public class AccessController extends BaseRegionObserver
RegionCoprocessorEnvironment e, RegionCoprocessorEnvironment e,
Map<byte [], ? extends Collection<?>> families) { Map<byte [], ? extends Collection<?>> families) {
HRegionInfo hri = e.getRegion().getRegionInfo(); HRegionInfo hri = e.getRegion().getRegionInfo();
TableName tableName = hri.getTableName(); TableName tableName = hri.getTable();
// 1. All users need read access to hbase:meta table. // 1. All users need read access to hbase:meta table.
// this is a very common operation, so deal with it quickly. // 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) Map<byte[], ? extends Set<byte[]>> familyMap)
throws IOException { throws IOException {
HRegionInfo hri = env.getRegion().getRegionInfo(); HRegionInfo hri = env.getRegion().getRegionInfo();
TableName tableName = hri.getTableName(); TableName tableName = hri.getTable();
if (user == null) { if (user == null) {
return false; return false;
@ -702,7 +702,7 @@ public class AccessController extends BaseRegionObserver
@Override @Override
public void preMove(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo region, public void preMove(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo region,
ServerName srcServer, ServerName destServer) throws IOException { ServerName srcServer, ServerName destServer) throws IOException {
requirePermission("move", region.getTableName(), null, null, Action.ADMIN); requirePermission("move", region.getTable(), null, null, Action.ADMIN);
} }
@Override @Override
@ -713,7 +713,7 @@ public class AccessController extends BaseRegionObserver
@Override @Override
public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo) public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo)
throws IOException { throws IOException {
requirePermission("assign", regionInfo.getTableName(), null, null, Action.ADMIN); requirePermission("assign", regionInfo.getTable(), null, null, Action.ADMIN);
} }
@Override @Override
@ -723,7 +723,7 @@ public class AccessController extends BaseRegionObserver
@Override @Override
public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo, public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo,
boolean force) throws IOException { boolean force) throws IOException {
requirePermission("unassign", regionInfo.getTableName(), null, null, Action.ADMIN); requirePermission("unassign", regionInfo.getTable(), null, null, Action.ADMIN);
} }
@Override @Override
@ -733,7 +733,7 @@ public class AccessController extends BaseRegionObserver
@Override @Override
public void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> c, public void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> c,
HRegionInfo regionInfo) throws IOException { HRegionInfo regionInfo) throws IOException {
requirePermission("regionOffline", regionInfo.getTableName(), null, null, Action.ADMIN); requirePermission("regionOffline", regionInfo.getTable(), null, null, Action.ADMIN);
} }
@Override @Override
@ -1263,7 +1263,7 @@ public class AccessController extends BaseRegionObserver
switch(request.getUserPermission().getPermission().getType()) { switch(request.getUserPermission().getPermission().getType()) {
case Global : case Global :
case Table : case Table :
requirePermission("grant", perm.getTable(), perm.getFamily(), requirePermission("grant", perm.getTableName(), perm.getFamily(),
perm.getQualifier(), Action.ADMIN); perm.getQualifier(), Action.ADMIN);
break; break;
case Namespace : case Namespace :
@ -1303,7 +1303,7 @@ public class AccessController extends BaseRegionObserver
switch(request.getUserPermission().getPermission().getType()) { switch(request.getUserPermission().getPermission().getType()) {
case Global : case Global :
case Table : case Table :
requirePermission("revoke", perm.getTable(), perm.getFamily(), requirePermission("revoke", perm.getTableName(), perm.getFamily(),
perm.getQualifier(), Action.ADMIN); perm.getQualifier(), Action.ADMIN);
break; break;
case Namespace : case Namespace :
@ -1379,11 +1379,11 @@ public class AccessController extends BaseRegionObserver
if (permission instanceof TablePermission) { if (permission instanceof TablePermission) {
TablePermission tperm = (TablePermission) permission; TablePermission tperm = (TablePermission) permission;
for (Permission.Action action : permission.getActions()) { 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 " throw new CoprocessorException(AccessController.class, String.format("This method "
+ "can only execute at the table specified in TablePermission. " + + "can only execute at the table specified in TablePermission. " +
"Table of the region:%s , requested table:%s", tableName, "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); 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) { if (region != null) {
HRegionInfo regionInfo = region.getRegionInfo(); HRegionInfo regionInfo = region.getRegionInfo();
if (regionInfo != null) { if (regionInfo != null) {
tableName = regionInfo.getTableName(); tableName = regionInfo.getTable();
} }
} }
return tableName; return tableName;
@ -1457,7 +1457,7 @@ public class AccessController extends BaseRegionObserver
} }
private boolean isSpecialTable(HRegionInfo regionInfo) { private boolean isSpecialTable(HRegionInfo regionInfo) {
TableName tableName = regionInfo.getTableName(); TableName tableName = regionInfo.getTable();
return tableName.equals(AccessControlLists.ACL_TABLE_NAME) return tableName.equals(AccessControlLists.ACL_TABLE_NAME)
|| tableName.equals(TableName.NAMESPACE_TABLE_NAME) || tableName.equals(TableName.NAMESPACE_TABLE_NAME)
|| tableName.equals(TableName.META_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. * Check if the specified region's table is disabled.
*/ */
private boolean isTableDisabled(HRegionInfo regionInfo) { 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; if (hbi.deployedOn.size() == 0) continue;
// We should be safe here // We should be safe here
TableName tableName = hbi.metaEntry.getTableName(); TableName tableName = hbi.metaEntry.getTable();
TableInfo modTInfo = tablesInfo.get(tableName); TableInfo modTInfo = tablesInfo.get(tableName);
if (modTInfo == null) { if (modTInfo == null) {
modTInfo = new TableInfo(tableName); 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. // pick only those tables that were not modified in the last few milliseconds.
if (info != null && info.getStartKey().length == 0 && !info.isMetaRegion()) { if (info != null && info.getStartKey().length == 0 && !info.isMetaRegion()) {
if (info.modTime + timelag < now) { if (info.modTime + timelag < now) {
tableNames.add(info.getTableName()); tableNames.add(info.getTable());
} else { } else {
numSkipped.incrementAndGet(); // one more in-flux table numSkipped.incrementAndGet(); // one more in-flux table
} }
@ -2584,7 +2584,7 @@ public class HBaseFsck extends Configured implements Tool {
sn = pair.getSecond(); sn = pair.getSecond();
} }
HRegionInfo hri = pair.getFirst(); HRegionInfo hri = pair.getFirst();
if (!(isTableIncluded(hri.getTableName()) if (!(isTableIncluded(hri.getTable())
|| hri.isMetaRegion())) { || hri.isMetaRegion())) {
return true; return true;
} }
@ -2661,7 +2661,7 @@ public class HBaseFsck extends Configured implements Tool {
hash ^= Arrays.hashCode(getStartKey()); hash ^= Arrays.hashCode(getStartKey());
hash ^= Arrays.hashCode(getEndKey()); hash ^= Arrays.hashCode(getEndKey());
hash ^= Boolean.valueOf(isOffline()).hashCode(); hash ^= Boolean.valueOf(isOffline()).hashCode();
hash ^= getTableName().hashCode(); hash ^= getTable().hashCode();
if (regionServer != null) { if (regionServer != null) {
hash ^= regionServer.hashCode(); hash ^= regionServer.hashCode();
} }
@ -2752,7 +2752,7 @@ public class HBaseFsck extends Configured implements Tool {
public TableName getTableName() { public TableName getTableName() {
if (this.metaEntry != null) { if (this.metaEntry != null) {
return this.metaEntry.getTableName(); return this.metaEntry.getTable();
} else if (this.hdfsEntry != null) { } else if (this.hdfsEntry != null) {
// we are only guaranteed to have a path and not an HRI for hdfsEntry, // we are only guaranteed to have a path and not an HRI for hdfsEntry,
// so we get the name from the Path // so we get the name from the Path
@ -3108,7 +3108,7 @@ public class HBaseFsck extends Configured implements Tool {
List<HRegionInfo> ret = Lists.newArrayList(); List<HRegionInfo> ret = Lists.newArrayList();
for (HRegionInfo hri : regions) { for (HRegionInfo hri : regions) {
if (hri.isMetaTable() || (!hbck.checkMetaOnly if (hri.isMetaTable() || (!hbck.checkMetaOnly
&& hbck.isTableIncluded(hri.getTableName()))) { && hbck.isTableIncluded(hri.getTable()))) {
ret.add(hri); ret.add(hri);
} }
} }

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -186,7 +186,7 @@ public class TestWALObserver {
// it's where WAL write cp should occur. // it's where WAL write cp should occur.
long now = EnvironmentEdgeManager.currentTimeMillis(); 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. // the edit shall have been change now by the coprocessor.
foundFamily0 = false; foundFamily0 = false;

View File

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

View File

@ -501,7 +501,7 @@ public class TestAssignmentManager {
// adding region in pending close. // adding region in pending close.
am.getRegionStates().updateRegionState( am.getRegionStates().updateRegionState(
REGIONINFO, State.SPLITTING, SERVERNAME_A); REGIONINFO, State.SPLITTING, SERVERNAME_A);
am.getZKTable().setEnabledTable(REGIONINFO.getTableName()); am.getZKTable().setEnabledTable(REGIONINFO.getTable());
RegionTransition data = RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_SPLITTING, RegionTransition data = RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_SPLITTING,
REGIONINFO.getRegionName(), SERVERNAME_A); REGIONINFO.getRegionName(), SERVERNAME_A);
String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName()); String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
@ -550,9 +550,9 @@ public class TestAssignmentManager {
// adding region in pending close. // adding region in pending close.
am.getRegionStates().updateRegionState(REGIONINFO, State.PENDING_CLOSE); am.getRegionStates().updateRegionState(REGIONINFO, State.PENDING_CLOSE);
if (state == Table.State.DISABLING) { if (state == Table.State.DISABLING) {
am.getZKTable().setDisablingTable(REGIONINFO.getTableName()); am.getZKTable().setDisablingTable(REGIONINFO.getTable());
} else { } else {
am.getZKTable().setDisabledTable(REGIONINFO.getTableName()); am.getZKTable().setDisabledTable(REGIONINFO.getTable());
} }
RegionTransition data = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_CLOSING, RegionTransition data = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_CLOSING,
REGIONINFO.getRegionName(), SERVERNAME_A); REGIONINFO.getRegionName(), SERVERNAME_A);
@ -576,7 +576,7 @@ public class TestAssignmentManager {
am.getRegionStates().isRegionsInTransition()); am.getRegionStates().isRegionsInTransition());
} }
} finally { } finally {
am.setEnabledTable(REGIONINFO.getTableName()); am.setEnabledTable(REGIONINFO.getTable());
executor.shutdown(); executor.shutdown();
am.shutdown(); am.shutdown();
// Clean up all znodes // Clean up all znodes
@ -888,7 +888,7 @@ public class TestAssignmentManager {
} }
try{ try{
// set table in disabling state. // set table in disabling state.
am.getZKTable().setDisablingTable(REGIONINFO.getTableName()); am.getZKTable().setDisablingTable(REGIONINFO.getTable());
am.joinCluster(); am.joinCluster();
// should not call retainAssignment if we get empty regions in assignAllUserRegions. // should not call retainAssignment if we get empty regions in assignAllUserRegions.
assertFalse( assertFalse(
@ -896,12 +896,12 @@ public class TestAssignmentManager {
gate.get()); gate.get());
// need to change table state from disabling to disabled. // need to change table state from disabling to disabled.
assertTrue("Table should be disabled.", assertTrue("Table should be disabled.",
am.getZKTable().isDisabledTable(REGIONINFO.getTableName())); am.getZKTable().isDisabledTable(REGIONINFO.getTable()));
} finally { } finally {
this.server.getConfiguration().setClass( this.server.getConfiguration().setClass(
HConstants.HBASE_MASTER_LOADBALANCER_CLASS, DefaultLoadBalancer.class, HConstants.HBASE_MASTER_LOADBALANCER_CLASS, DefaultLoadBalancer.class,
LoadBalancer.class); LoadBalancer.class);
am.getZKTable().setEnabledTable(REGIONINFO.getTableName()); am.getZKTable().setEnabledTable(REGIONINFO.getTable());
am.shutdown(); am.shutdown();
} }
} }
@ -927,17 +927,17 @@ public class TestAssignmentManager {
this.serverManager); this.serverManager);
try { try {
// set table in enabling state. // set table in enabling state.
am.getZKTable().setEnablingTable(REGIONINFO.getTableName()); am.getZKTable().setEnablingTable(REGIONINFO.getTable());
new EnableTableHandler(server, REGIONINFO.getTableName(), new EnableTableHandler(server, REGIONINFO.getTable(),
am.getCatalogTracker(), am, new NullTableLockManager(), true).prepare() am.getCatalogTracker(), am, new NullTableLockManager(), true).prepare()
.process(); .process();
assertEquals("Number of assignments should be 1.", 1, assignmentCount); assertEquals("Number of assignments should be 1.", 1, assignmentCount);
assertTrue("Table should be enabled.", assertTrue("Table should be enabled.",
am.getZKTable().isEnabledTable(REGIONINFO.getTableName())); am.getZKTable().isEnabledTable(REGIONINFO.getTable()));
} finally { } finally {
enabling = false; enabling = false;
assignmentCount = 0; assignmentCount = 0;
am.getZKTable().setEnabledTable(REGIONINFO.getTableName()); am.getZKTable().setEnabledTable(REGIONINFO.getTable());
am.shutdown(); am.shutdown();
ZKAssign.deleteAllNodes(this.watcher); ZKAssign.deleteAllNodes(this.watcher);
} }
@ -964,7 +964,7 @@ public class TestAssignmentManager {
// adding region plan // adding region plan
am.regionPlans.put(REGIONINFO.getEncodedName(), am.regionPlans.put(REGIONINFO.getEncodedName(),
new RegionPlan(REGIONINFO, SERVERNAME_B, SERVERNAME_A)); new RegionPlan(REGIONINFO, SERVERNAME_B, SERVERNAME_A));
am.getZKTable().setEnabledTable(REGIONINFO.getTableName()); am.getZKTable().setEnabledTable(REGIONINFO.getTable());
try { try {
am.assignInvoked = false; 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 // 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). // the single test passes, but when the full suite is run, things get borked).
FSUtils.setRootDir(fs.getConf(), rootdir); 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()); Path storedir = HStore.getStoreHomedir(tabledir, parent, htd.getColumnFamilies()[0].getName());
System.out.println("Old root:" + rootdir); System.out.println("Old root:" + rootdir);
System.out.println("Old table:" + tabledir); System.out.println("Old table:" + tabledir);
@ -915,7 +915,7 @@ public class TestCatalogJanitor {
final HRegionInfo daughter, final byte [] midkey, final boolean top) final HRegionInfo daughter, final byte [] midkey, final boolean top)
throws IOException { throws IOException {
Path rootdir = services.getMasterFileSystem().getRootDir(); 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, Path storedir = HStore.getStoreHomedir(tabledir, daughter,
htd.getColumnFamilies()[0].getName()); htd.getColumnFamilies()[0].getName());
Reference ref = Reference ref =

View File

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

View File

@ -489,12 +489,12 @@ public class TestMasterTransitions {
for (Result r = null; (r = s.next()) != null;) { for (Result r = null; (r = s.next()) != null;) {
HRegionInfo hri = HRegionInfo.getHRegionInfo(r); HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
if (hri == null) break; if (hri == null) break;
if (!hri.getTableName().getNameAsString().equals(TABLENAME)) { if (!hri.getTable().getNameAsString().equals(TABLENAME)) {
continue; continue;
} }
// If start key, add 'aaa'. // If start key, add 'aaa'.
if(!hri.getTableName().getNameAsString().equals(TABLENAME)) { if(!hri.getTable().getNameAsString().equals(TABLENAME)) {
continue; continue;
} }
byte [] row = getStartKey(hri); byte [] row = getStartKey(hri);
@ -511,24 +511,6 @@ public class TestMasterTransitions {
return rows; 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 * @param hri
* @return Start key for hri (If start key is '', then return 'aaa'. * @return Start key for hri (If start key is '', then return 'aaa'.
@ -545,6 +527,4 @@ public class TestMasterTransitions {
private static byte [] getTestQualifier() { private static byte [] getTestQualifier() {
return getTestFamily(); 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.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair; 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.apache.zookeeper.KeeperException;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
@ -307,7 +305,7 @@ public class TestRegionPlacement {
TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getOnlineRegionsLocalContext(); TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getOnlineRegionsLocalContext();
isNamespaceServer = false; isNamespaceServer = false;
for (HRegion r : regs) { for (HRegion r : regs) {
if (r.getRegionInfo().getTableName().getNamespaceAsString() if (r.getRegionInfo().getTable().getNamespaceAsString()
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) { .equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
isNamespaceServer = true; isNamespaceServer = true;
break; break;
@ -606,7 +604,7 @@ public class TestRegionPlacement {
public boolean processRow(Result result) throws IOException { public boolean processRow(Result result) throws IOException {
try { try {
HRegionInfo info = MetaScanner.getHRegionInfo(result); HRegionInfo info = MetaScanner.getHRegionInfo(result);
if(info.getTableName().getNamespaceAsString() if(info.getTable().getNamespaceAsString()
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) { .equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
return true; return true;
} }

View File

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

View File

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

View File

@ -60,7 +60,7 @@ public class TestHRegionFileSystem {
// Create a Region // Create a Region
HRegionInfo hri = new HRegionInfo(TableName.valueOf("TestTable")); HRegionInfo hri = new HRegionInfo(TableName.valueOf("TestTable"));
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, 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 // Verify if the region is on disk
Path regionDir = regionFs.getRegionDir(); Path regionDir = regionFs.getRegionDir();
@ -72,12 +72,12 @@ public class TestHRegionFileSystem {
// Open the region // Open the region
regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, fs, regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, fs,
FSUtils.getTableDir(rootDir, hri.getTableName()), hri, false); FSUtils.getTableDir(rootDir, hri.getTable()), hri, false);
assertEquals(regionDir, regionFs.getRegionDir()); assertEquals(regionDir, regionFs.getRegionDir());
// Delete the region // Delete the region
HRegionFileSystem.deleteRegionFromFileSystem(conf, fs, 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)); assertFalse("The region folder should be removed", fs.exists(regionDir));
fs.delete(rootDir, true); 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.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; 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.FileStatus;
import org.apache.hadoop.fs.Path; 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.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.SmallTests; 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.Bytes;
import org.apache.hadoop.hbase.util.MD5Hash; import org.apache.hadoop.hbase.util.MD5Hash;
import org.junit.Test; import org.junit.Test;

View File

@ -185,7 +185,7 @@ public class TestRegionMergeTransactionOnCluster {
FileSystem fs = master.getMasterFileSystem().getFileSystem(); FileSystem fs = master.getMasterFileSystem().getFileSystem();
Path rootDir = master.getMasterFileSystem().getRootDir(); 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 regionAdir = new Path(tabledir, regionA.getEncodedName());
Path regionBdir = new Path(tabledir, regionB.getEncodedName()); Path regionBdir = new Path(tabledir, regionB.getEncodedName());
assertTrue(fs.exists(regionAdir)); assertTrue(fs.exists(regionAdir));

View File

@ -307,7 +307,7 @@ public class TestRegionServerNoMaster {
hri.getEncodedNameAsBytes())); hri.getEncodedNameAsBytes()));
// Let's start the open handler // 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)); 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 // 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 // 1) There is no ZK node
// 2) The region in RIT was changed. // 2) The region in RIT was changed.
// The order is more or less implementation dependant. // 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)); 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 // 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(); RegionStates regionStates = cluster.getMaster().getAssignmentManager().getRegionStates();
Map<String, RegionState> rit = regionStates.getRegionsInTransition(); 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); Thread.sleep(100);
} }
assertFalse("region still in transition", rit.containsKey( assertFalse("region still in transition", rit.containsKey(
rit.containsKey(hri.getTableName()))); rit.containsKey(hri.getTable())));
List<HRegion> onlineRegions = regionServer.getOnlineRegions(tableName); List<HRegion> onlineRegions = regionServer.getOnlineRegions(tableName);
// Region server side split is successful. // Region server side split is successful.
@ -923,7 +923,7 @@ public class TestSplitTransactionOnCluster {
@Override @Override
void transitionZKNode(Server server, RegionServerServices services, HRegion a, HRegion b) void transitionZKNode(Server server, RegionServerServices services, HRegion a, HRegion b)
throws IOException { throws IOException {
if (this.currentRegion.getRegionInfo().getTableName().getNameAsString() if (this.currentRegion.getRegionInfo().getTable().getNameAsString()
.equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) { .equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
try { try {
if (!secondSplit){ if (!secondSplit){
@ -935,14 +935,14 @@ public class TestSplitTransactionOnCluster {
} }
super.transitionZKNode(server, services, a, b); super.transitionZKNode(server, services, a, b);
if (this.currentRegion.getRegionInfo().getTableName().getNameAsString() if (this.currentRegion.getRegionInfo().getTable().getNameAsString()
.equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) { .equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
firstSplitCompleted = true; firstSplitCompleted = true;
} }
} }
@Override @Override
public boolean rollback(Server server, RegionServerServices services) throws IOException { public boolean rollback(Server server, RegionServerServices services) throws IOException {
if (this.currentRegion.getRegionInfo().getTableName().getNameAsString() if (this.currentRegion.getRegionInfo().getTable().getNameAsString()
.equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) { .equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
if(secondSplit){ if(secondSplit){
super.rollback(server, services); super.rollback(server, services);

View File

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

View File

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

View File

@ -191,7 +191,7 @@ public class TestTablePermissions {
assertEquals("Should have 1 permission for george", 1, userPerms.size()); assertEquals("Should have 1 permission for george", 1, userPerms.size());
TablePermission permission = userPerms.get(0); TablePermission permission = userPerms.get(0);
assertEquals("Permission should be for " + TEST_TABLE, assertEquals("Permission should be for " + TEST_TABLE,
TEST_TABLE, permission.getTable()); TEST_TABLE, permission.getTableName());
assertNull("Column family should be empty", permission.getFamily()); assertNull("Column family should be empty", permission.getFamily());
// check actions // check actions
@ -206,7 +206,7 @@ public class TestTablePermissions {
assertEquals("Should have 1 permission for hubert", 1, userPerms.size()); assertEquals("Should have 1 permission for hubert", 1, userPerms.size());
permission = userPerms.get(0); permission = userPerms.get(0);
assertEquals("Permission should be for " + TEST_TABLE, assertEquals("Permission should be for " + TEST_TABLE,
TEST_TABLE, permission.getTable()); TEST_TABLE, permission.getTableName());
assertNull("Column family should be empty", permission.getFamily()); assertNull("Column family should be empty", permission.getFamily());
// check actions // check actions
@ -221,7 +221,7 @@ public class TestTablePermissions {
assertEquals("Should have 1 permission for humphrey", 1, userPerms.size()); assertEquals("Should have 1 permission for humphrey", 1, userPerms.size());
permission = userPerms.get(0); permission = userPerms.get(0);
assertEquals("Permission should be for " + TEST_TABLE, assertEquals("Permission should be for " + TEST_TABLE,
TEST_TABLE, permission.getTable()); TEST_TABLE, permission.getTableName());
assertTrue("Permission should be for family " + TEST_FAMILY, assertTrue("Permission should be for family " + TEST_FAMILY,
Bytes.equals(TEST_FAMILY, permission.getFamily())); Bytes.equals(TEST_FAMILY, permission.getFamily()));
assertTrue("Permission should be for qualifier " + TEST_QUALIFIER, assertTrue("Permission should be for qualifier " + TEST_QUALIFIER,
@ -249,7 +249,7 @@ public class TestTablePermissions {
assertNotNull(userPerms); assertNotNull(userPerms);
assertEquals(1, userPerms.size()); assertEquals(1, userPerms.size());
permission = userPerms.get(0); permission = userPerms.get(0);
assertEquals(TEST_TABLE, permission.getTable()); assertEquals(TEST_TABLE, permission.getTableName());
assertEquals(1, permission.getActions().length); assertEquals(1, permission.getActions().length);
assertEquals(TablePermission.Action.READ, permission.getActions()[0]); assertEquals(TablePermission.Action.READ, permission.getActions()[0]);
@ -257,7 +257,7 @@ public class TestTablePermissions {
assertNotNull(userPerms); assertNotNull(userPerms);
assertEquals(1, userPerms.size()); assertEquals(1, userPerms.size());
permission = userPerms.get(0); permission = userPerms.get(0);
assertEquals(TEST_TABLE2, permission.getTable()); assertEquals(TEST_TABLE2, permission.getTableName());
assertEquals(2, permission.getActions().length); assertEquals(2, permission.getActions().length);
actions = Arrays.asList(permission.getActions()); actions = Arrays.asList(permission.getActions());
assertTrue(actions.contains(TablePermission.Action.READ)); 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.FileSystem;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests; 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.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; 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.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -202,7 +201,7 @@ public class TestExportSnapshot {
// First region, simple with one plain hfile. // First region, simple with one plain hfile.
HRegionInfo hri = new HRegionInfo(tableWithRefsName); HRegionInfo hri = new HRegionInfo(tableWithRefsName);
HRegionFileSystem r0fs = HRegionFileSystem.createRegionOnFileSystem(conf, 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); Path storeFile = new Path(rootDir, TEST_HFILE);
FSDataOutputStream out = fs.create(storeFile); FSDataOutputStream out = fs.create(storeFile);
out.write(Bytes.toBytes("Test Data")); 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. // This region contains a reference to the hfile in the first region.
hri = new HRegionInfo(tableWithRefsName); hri = new HRegionInfo(tableWithRefsName);
HRegionFileSystem r1fs = HRegionFileSystem.createRegionOnFileSystem(conf, 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()); storeFile = new Path(rootDir, TEST_HFILE + '.' + r0fs.getRegionInfo().getEncodedName());
out = fs.create(storeFile); out = fs.create(storeFile);
out.write(Bytes.toBytes("Test Data")); 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.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.monitoring.MonitoredTask; 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.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
@ -160,7 +159,7 @@ public class TestRestoreSnapshotHelper {
// First region, simple with one plain hfile. // First region, simple with one plain hfile.
HRegionInfo hri = new HRegionInfo(htd.getTableName()); HRegionInfo hri = new HRegionInfo(htd.getTableName());
HRegionFileSystem r0fs = HRegionFileSystem.createRegionOnFileSystem(conf, 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); Path storeFile = new Path(rootDir, TEST_HFILE);
fs.createNewFile(storeFile); fs.createNewFile(storeFile);
r0fs.commitStoreFile(TEST_FAMILY, 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. // This region contains a reference to the hfile in the first region.
hri = new HRegionInfo(htd.getTableName()); hri = new HRegionInfo(htd.getTableName());
HRegionFileSystem r1fs = HRegionFileSystem.createRegionOnFileSystem(conf, 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()); storeFile = new Path(rootDir, TEST_HFILE + '.' + r0fs.getRegionInfo().getEncodedName());
fs.createNewFile(storeFile); fs.createNewFile(storeFile);
r1fs.commitStoreFile(TEST_FAMILY, storeFile); r1fs.commitStoreFile(TEST_FAMILY, storeFile);

View File

@ -1868,7 +1868,7 @@ public class TestHBaseFsck {
@Override @Override
public boolean processRow(Result rowResult) throws IOException { 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 delete = new Delete(rowResult.getRow());
delete.deleteColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); delete.deleteColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
deletes.add(delete); deletes.add(delete);

View File

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