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:
parent
a7211896df
commit
ed509f750c
|
@ -402,8 +402,10 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
* Gets the table name from the specified region name.
|
||||
* @param regionName
|
||||
* @return Table name.
|
||||
* @deprecated Since 0.96.0; use #getTable(byte[])
|
||||
*/
|
||||
public static TableName getTableName(byte[] regionName) {
|
||||
@Deprecated
|
||||
public static byte [] getTableName(byte[] regionName) {
|
||||
int offset = -1;
|
||||
for (int i = 0; i < regionName.length; i++) {
|
||||
if (regionName[i] == HConstants.DELIMITER) {
|
||||
|
@ -413,7 +415,18 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
}
|
||||
byte[] buff = new byte[offset];
|
||||
System.arraycopy(regionName, 0, buff, 0, offset);
|
||||
return TableName.valueOf(buff);
|
||||
return buff;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the table name from the specified region name.
|
||||
* Like {@link #getTableName(byte[])} only returns a {@link TableName} rather than a byte array.
|
||||
* @param regionName
|
||||
* @return Table name
|
||||
* @see #getTableName(byte[])
|
||||
*/
|
||||
public static TableName getTable(final byte [] regionName) {
|
||||
return TableName.valueOf(getTableName(regionName));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -523,12 +536,26 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
/**
|
||||
* Get current table name of the region
|
||||
* @return byte array of table name
|
||||
* @deprecated Since 0.96.0; use #getTable()
|
||||
*/
|
||||
public TableName getTableName() {
|
||||
@Deprecated
|
||||
public byte [] getTableName() {
|
||||
return getTable().toBytes();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get current table name of the region
|
||||
* @return TableName
|
||||
* @see #getTableName()
|
||||
*/
|
||||
public TableName getTable() {
|
||||
// This method name should be getTableName but there was already a method getTableName
|
||||
// that returned a byte array. It is unfortunate given everwhere else, getTableName returns
|
||||
// a TableName instance.
|
||||
if (tableName == null || tableName.getName().length == 0) {
|
||||
tableName = getTableName(getRegionName());
|
||||
tableName = getTable(getRegionName());
|
||||
}
|
||||
return tableName;
|
||||
return this.tableName;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -570,7 +597,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
|
||||
/** @return true if this region is a meta region */
|
||||
public boolean isMetaRegion() {
|
||||
return tableName.equals(HRegionInfo.FIRST_META_REGIONINFO.getTableName());
|
||||
return tableName.equals(HRegionInfo.FIRST_META_REGIONINFO.getTable());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -813,7 +840,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
public static RegionInfo convert(final HRegionInfo info) {
|
||||
if (info == null) return null;
|
||||
RegionInfo.Builder builder = RegionInfo.newBuilder();
|
||||
builder.setTableName(ProtobufUtil.toProtoTableName(info.getTableName()));
|
||||
builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));
|
||||
builder.setRegionId(info.getRegionId());
|
||||
if (info.getStartKey() != null) {
|
||||
builder.setStartKey(ByteString.copyFrom(info.getStartKey()));
|
||||
|
|
|
@ -545,6 +545,16 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
|
|||
values.remove(key);
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove metadata represented by the key from the {@link #values} map
|
||||
*
|
||||
* @param key Key whose key and value we're to remove from HTableDescriptor
|
||||
* parameters.
|
||||
*/
|
||||
public void remove(final byte [] key) {
|
||||
remove(new ImmutableBytesWritable(key));
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if the readOnly flag of the table is set. If the readOnly flag is
|
||||
* set then the contents of the table can only be read from but not modified.
|
||||
|
|
|
@ -103,9 +103,9 @@ public class MetaReader {
|
|||
Pair<HRegionInfo, ServerName> region = HRegionInfo.getHRegionInfoAndServerName(r);
|
||||
HRegionInfo hri = region.getFirst();
|
||||
if (hri == null) return true;
|
||||
if (hri.getTableName() == null) return true;
|
||||
if (hri.getTable() == null) return true;
|
||||
if (disabledTables.contains(
|
||||
hri.getTableName())) return true;
|
||||
hri.getTable())) return true;
|
||||
// Are we to include split parents in the list?
|
||||
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
|
||||
regions.put(hri, region.getSecond());
|
||||
|
@ -362,7 +362,7 @@ public class MetaReader {
|
|||
* <code>tableName</code>
|
||||
*/
|
||||
static boolean isInsideTable(final HRegionInfo current, final TableName tableName) {
|
||||
return tableName.equals(current.getTableName());
|
||||
return tableName.equals(current.getTable());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -85,6 +85,15 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
this(conf, scan, tableName, HConnectionManager.getConnection(conf));
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #ClientScanner(Configuration, Scan, TableName)}
|
||||
*/
|
||||
@Deprecated
|
||||
public ClientScanner(final Configuration conf, final Scan scan,
|
||||
final byte [] tableName) throws IOException {
|
||||
this(conf, scan, TableName.valueOf(tableName));
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create a new ClientScanner for the specified table
|
||||
|
@ -101,6 +110,15 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
this(conf, scan, tableName, connection, new RpcRetryingCallerFactory(conf));
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #ClientScanner(Configuration, Scan, TableName, HConnection)}
|
||||
*/
|
||||
@Deprecated
|
||||
public ClientScanner(final Configuration conf, final Scan scan, final byte [] tableName,
|
||||
HConnection connection) throws IOException {
|
||||
this(conf, scan, TableName.valueOf(tableName), connection, new RpcRetryingCallerFactory(conf));
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new ClientScanner for the specified table Note that the passed {@link Scan}'s start
|
||||
* row maybe changed changed.
|
||||
|
@ -158,7 +176,16 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
return this.connection;
|
||||
}
|
||||
|
||||
protected TableName getTableName() {
|
||||
/**
|
||||
* @return Table name
|
||||
* @deprecated Since 0.96.0; use {@link #getTable()}
|
||||
*/
|
||||
@Deprecated
|
||||
protected byte [] getTableName() {
|
||||
return this.tableName.getName();
|
||||
}
|
||||
|
||||
protected TableName getTable() {
|
||||
return this.tableName;
|
||||
}
|
||||
|
||||
|
@ -253,7 +280,7 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
int nbRows) {
|
||||
scan.setStartRow(localStartKey);
|
||||
ScannerCallable s = new ScannerCallable(getConnection(),
|
||||
getTableName(), scan, this.scanMetrics);
|
||||
getTable(), scan, this.scanMetrics);
|
||||
s.setCaching(nbRows);
|
||||
return s;
|
||||
}
|
||||
|
|
|
@ -494,7 +494,7 @@ public class HBaseAdmin implements Abortable, Closeable {
|
|||
LOG.warn("No serialized HRegionInfo in " + rowResult);
|
||||
return true;
|
||||
}
|
||||
if (!info.getTableName().equals(desc.getTableName())) {
|
||||
if (!info.getTable().equals(desc.getTableName())) {
|
||||
return false;
|
||||
}
|
||||
ServerName serverName = HRegionInfo.getServerName(rowResult);
|
||||
|
|
|
@ -926,15 +926,14 @@ public class HConnectionManager {
|
|||
public boolean processRow(Result row) throws IOException {
|
||||
HRegionInfo info = MetaScanner.getHRegionInfo(row);
|
||||
if (info != null && !info.isSplitParent()) {
|
||||
if (tableName.equals(info.getTableName())) {
|
||||
if (tableName.equals(info.getTable())) {
|
||||
ServerName server = HRegionInfo.getServerName(row);
|
||||
if (server == null) {
|
||||
available.set(false);
|
||||
return false;
|
||||
}
|
||||
regionCount.incrementAndGet();
|
||||
} else if (tableName.compareTo(
|
||||
info.getTableName()) < 0) {
|
||||
} else if (tableName.compareTo(info.getTable()) < 0) {
|
||||
// Return if we are done with the current table
|
||||
return false;
|
||||
}
|
||||
|
@ -961,7 +960,7 @@ public class HConnectionManager {
|
|||
public boolean processRow(Result row) throws IOException {
|
||||
HRegionInfo info = MetaScanner.getHRegionInfo(row);
|
||||
if (info != null && !info.isSplitParent()) {
|
||||
if (tableName.equals(info.getTableName())) {
|
||||
if (tableName.equals(info.getTable())) {
|
||||
ServerName server = HRegionInfo.getServerName(row);
|
||||
if (server == null) {
|
||||
available.set(false);
|
||||
|
@ -979,7 +978,7 @@ public class HConnectionManager {
|
|||
// Always empty start row should be counted
|
||||
regionCount.incrementAndGet();
|
||||
}
|
||||
} else if (tableName.compareTo(info.getTableName()) < 0) {
|
||||
} else if (tableName.compareTo(info.getTable()) < 0) {
|
||||
// Return if we are done with the current table
|
||||
return false;
|
||||
}
|
||||
|
@ -1000,7 +999,7 @@ public class HConnectionManager {
|
|||
|
||||
@Override
|
||||
public HRegionLocation locateRegion(final byte[] regionName) throws IOException {
|
||||
return locateRegion(HRegionInfo.getTableName(regionName),
|
||||
return locateRegion(HRegionInfo.getTable(regionName),
|
||||
HRegionInfo.getStartKey(regionName), false, true);
|
||||
}
|
||||
|
||||
|
@ -1113,7 +1112,7 @@ public class HConnectionManager {
|
|||
}
|
||||
|
||||
// possible we got a region of a different table...
|
||||
if (!regionInfo.getTableName().equals(tableName)) {
|
||||
if (!regionInfo.getTable().equals(tableName)) {
|
||||
return false; // stop scanning
|
||||
}
|
||||
if (regionInfo.isOffline()) {
|
||||
|
@ -1226,10 +1225,10 @@ public class HConnectionManager {
|
|||
}
|
||||
|
||||
// possible we got a region of a different table...
|
||||
if (!regionInfo.getTableName().equals(tableName)) {
|
||||
if (!regionInfo.getTable().equals(tableName)) {
|
||||
throw new TableNotFoundException(
|
||||
"Table '" + tableName + "' was not found, got: " +
|
||||
regionInfo.getTableName() + ".");
|
||||
regionInfo.getTable() + ".");
|
||||
}
|
||||
if (regionInfo.isSplit()) {
|
||||
throw new RegionOfflineException("the only available region for" +
|
||||
|
@ -2306,7 +2305,7 @@ public class HConnectionManager {
|
|||
ServerName serverName, long seqNum) {
|
||||
HRegionLocation newHrl = new HRegionLocation(hri, serverName, seqNum);
|
||||
synchronized (this.cachedRegionLocations) {
|
||||
cacheLocation(hri.getTableName(), source, newHrl);
|
||||
cacheLocation(hri.getTable(), source, newHrl);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2320,7 +2319,7 @@ public class HConnectionManager {
|
|||
HRegionLocation oldLocation;
|
||||
synchronized (this.cachedRegionLocations) {
|
||||
Map<byte[], HRegionLocation> tableLocations =
|
||||
getTableLocations(hri.getTableName());
|
||||
getTableLocations(hri.getTable());
|
||||
oldLocation = tableLocations.get(hri.getStartKey());
|
||||
if (oldLocation != null) {
|
||||
// Do not delete the cache entry if it's not for the same server that gave us the error.
|
||||
|
@ -2338,7 +2337,7 @@ public class HConnectionManager {
|
|||
return;
|
||||
}
|
||||
synchronized (this.cachedRegionLocations) {
|
||||
TableName tableName = location.getRegionInfo().getTableName();
|
||||
TableName tableName = location.getRegionInfo().getTable();
|
||||
Map<byte[], HRegionLocation> tableLocations =
|
||||
getTableLocations(tableName);
|
||||
if (!tableLocations.isEmpty()) {
|
||||
|
|
|
@ -1006,6 +1006,18 @@ public class HTable implements HTableInterface {
|
|||
return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL);
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
|
||||
*/
|
||||
@Deprecated
|
||||
@Override
|
||||
public long incrementColumnValue(final byte [] row, final byte [] family,
|
||||
final byte [] qualifier, final long amount, final boolean writeToWAL)
|
||||
throws IOException {
|
||||
return incrementColumnValue(row, family, qualifier, amount,
|
||||
writeToWAL? Durability.SKIP_WAL: Durability.USE_DEFAULT);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import com.google.protobuf.Service;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -384,6 +385,14 @@ public interface HTableInterface extends Closeable {
|
|||
long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
|
||||
long amount, Durability durability) throws IOException;
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
|
||||
*/
|
||||
@Deprecated
|
||||
long incrementColumnValue(final byte [] row, final byte [] family,
|
||||
final byte [] qualifier, final long amount, final boolean writeToWAL)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Tells whether or not 'auto-flush' is turned on.
|
||||
*
|
||||
|
|
|
@ -19,19 +19,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.AbstractMap.SimpleEntry;
|
||||
import java.util.ArrayList;
|
||||
|
@ -46,6 +33,18 @@ import java.util.concurrent.LinkedBlockingQueue;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
/**
|
||||
* HTableMultiplexer provides a thread-safe non blocking PUT API across all the tables.
|
||||
* Each put will be sharded into different buffer queues based on its destination region server.
|
||||
|
@ -450,7 +449,7 @@ public class HTableMultiplexer {
|
|||
HRegionLocation oldLoc) throws IOException {
|
||||
Put failedPut = failedPutStatus.getPut();
|
||||
// The currentPut is failed. So get the table name for the currentPut.
|
||||
TableName tableName = failedPutStatus.getRegionInfo().getTableName();
|
||||
TableName tableName = failedPutStatus.getRegionInfo().getTable();
|
||||
// Decrease the retry count
|
||||
int retryCount = failedPutStatus.getRetryCount() - 1;
|
||||
|
||||
|
|
|
@ -600,5 +600,11 @@ public class HTablePool implements Closeable {
|
|||
throw new IllegalStateException("Table=" + new String(table.getTableName()) + " already closed");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long incrementColumnValue(byte[] row, byte[] family,
|
||||
byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
|
||||
return table.incrementColumnValue(row, family, qualifier, amount, writeToWAL);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -355,7 +355,7 @@ public class MetaScanner {
|
|||
if (info == null) {
|
||||
return true;
|
||||
}
|
||||
if (!(info.getTableName().equals(tableName))) {
|
||||
if (!(info.getTable().equals(tableName))) {
|
||||
return false;
|
||||
}
|
||||
return super.processRow(rowResult);
|
||||
|
|
|
@ -101,6 +101,15 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
|
|||
logCutOffLatency = conf.getInt(LOG_SCANNER_LATENCY_CUTOFF, 1000);
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #ScannerCallable(HConnection, TableName, Scan, ScanMetrics)}
|
||||
*/
|
||||
@Deprecated
|
||||
public ScannerCallable (HConnection connection, final byte [] tableName, Scan scan,
|
||||
ScanMetrics scanMetrics) {
|
||||
this(connection, TableName.valueOf(tableName), scan, scanMetrics);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param reload force reload of server location
|
||||
* @throws IOException
|
||||
|
|
|
@ -1645,7 +1645,7 @@ public final class ProtobufUtil {
|
|||
|
||||
AccessControlProtos.TablePermission.Builder builder =
|
||||
AccessControlProtos.TablePermission.newBuilder();
|
||||
builder.setTableName(ProtobufUtil.toProtoTableName(tablePerm.getTable()));
|
||||
builder.setTableName(ProtobufUtil.toProtoTableName(tablePerm.getTableName()));
|
||||
if (tablePerm.hasFamily()) {
|
||||
builder.setFamily(ByteString.copyFrom(tablePerm.getFamily()));
|
||||
}
|
||||
|
@ -2195,7 +2195,7 @@ public final class ProtobufUtil {
|
|||
// input / output paths are relative to the store dir
|
||||
// store dir is relative to region dir
|
||||
CompactionDescriptor.Builder builder = CompactionDescriptor.newBuilder()
|
||||
.setTableName(ByteString.copyFrom(info.getTableName().getName()))
|
||||
.setTableName(ByteString.copyFrom(info.getTableName()))
|
||||
.setEncodedRegionName(ByteString.copyFrom(info.getEncodedNameAsBytes()))
|
||||
.setFamilyName(ByteString.copyFrom(family))
|
||||
.setStoreHomeDir(storeDir.getName()); //make relative
|
||||
|
|
|
@ -151,7 +151,7 @@ public class TablePermission extends Permission {
|
|||
return table != null;
|
||||
}
|
||||
|
||||
public TableName getTable() {
|
||||
public TableName getTableName() {
|
||||
return table;
|
||||
}
|
||||
|
||||
|
@ -318,7 +318,7 @@ public class TablePermission extends Permission {
|
|||
}
|
||||
TablePermission other = (TablePermission)obj;
|
||||
|
||||
if (!(table.equals(other.getTable()) &&
|
||||
if (!(table.equals(other.getTableName()) &&
|
||||
((family == null && other.getFamily() == null) ||
|
||||
Bytes.equals(family, other.getFamily())) &&
|
||||
((qualifier == null && other.getQualifier() == null) ||
|
||||
|
|
|
@ -140,7 +140,7 @@ public class UserPermission extends TablePermission {
|
|||
* @param perm a TablePermission
|
||||
*/
|
||||
public UserPermission(byte[] user, TablePermission perm) {
|
||||
super(perm.getNamespace(), perm.getTable(), perm.getFamily(), perm.getQualifier(),
|
||||
super(perm.getNamespace(), perm.getTableName(), perm.getFamily(), perm.getQualifier(),
|
||||
perm.actions);
|
||||
this.user = user;
|
||||
}
|
||||
|
|
|
@ -129,8 +129,9 @@ public final class TableName implements Comparable<TableName> {
|
|||
return tableName;
|
||||
}
|
||||
|
||||
public static void isLegalTableQualifierName(final byte[] qualifierName){
|
||||
public static byte [] isLegalTableQualifierName(final byte[] qualifierName){
|
||||
isLegalTableQualifierName(qualifierName, 0, qualifierName.length);
|
||||
return qualifierName;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -72,7 +72,7 @@ public class HFileArchiver {
|
|||
public static void archiveRegion(Configuration conf, FileSystem fs, HRegionInfo info)
|
||||
throws IOException {
|
||||
Path rootDir = FSUtils.getRootDir(conf);
|
||||
archiveRegion(fs, rootDir, FSUtils.getTableDir(rootDir, info.getTableName()),
|
||||
archiveRegion(fs, rootDir, FSUtils.getTableDir(rootDir, info.getTable()),
|
||||
HRegion.getRegionDir(rootDir, info));
|
||||
}
|
||||
|
||||
|
|
|
@ -564,6 +564,12 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
|
|||
public void setWriteBufferSize(long writeBufferSize) throws IOException {
|
||||
table.setWriteBufferSize(writeBufferSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long incrementColumnValue(byte[] row, byte[] family,
|
||||
byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
|
||||
return table.incrementColumnValue(row, family, qualifier, amount, writeToWAL);
|
||||
}
|
||||
}
|
||||
|
||||
/** The coprocessor */
|
||||
|
|
|
@ -263,7 +263,7 @@ public class HFileLink extends FileLink {
|
|||
*/
|
||||
public static String createHFileLinkName(final HRegionInfo hfileRegionInfo,
|
||||
final String hfileName) {
|
||||
return createHFileLinkName(hfileRegionInfo.getTableName(),
|
||||
return createHFileLinkName(hfileRegionInfo.getTable(),
|
||||
hfileRegionInfo.getEncodedName(), hfileName);
|
||||
}
|
||||
|
||||
|
@ -300,7 +300,7 @@ public class HFileLink extends FileLink {
|
|||
public static boolean create(final Configuration conf, final FileSystem fs,
|
||||
final Path dstFamilyPath, final HRegionInfo hfileRegionInfo,
|
||||
final String hfileName) throws IOException {
|
||||
TableName linkedTable = hfileRegionInfo.getTableName();
|
||||
TableName linkedTable = hfileRegionInfo.getTable();
|
||||
String linkedRegion = hfileRegionInfo.getEncodedName();
|
||||
return create(conf, fs, dstFamilyPath, linkedTable, linkedRegion, hfileName);
|
||||
}
|
||||
|
|
|
@ -60,7 +60,8 @@ public class TableSplit implements InputSplit, Comparable<TableSplit> {
|
|||
|
||||
public TableSplit(byte [] tableName, byte [] startRow, byte [] endRow,
|
||||
final String location) {
|
||||
this(TableName.valueOf(tableName), startRow, endRow, location);
|
||||
this(TableName.valueOf(tableName), startRow, endRow,
|
||||
location);
|
||||
}
|
||||
|
||||
/** @return table name */
|
||||
|
|
|
@ -517,6 +517,16 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #tryAtomicRegionLoad(HConnection, TableName, byte[], Collection)}
|
||||
*/
|
||||
@Deprecated
|
||||
protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn,
|
||||
final byte [] tableName, final byte[] first, Collection<LoadQueueItem> lqis)
|
||||
throws IOException {
|
||||
return tryAtomicRegionLoad(conn, TableName.valueOf(tableName), first, lqis);
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempts to do an atomic load of many hfiles into a region. If it fails,
|
||||
* it returns a list of hfiles that need to be retried. If it is successful
|
||||
|
@ -531,9 +541,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
* failure
|
||||
*/
|
||||
protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn,
|
||||
final TableName tableName,
|
||||
final byte[] first, Collection<LoadQueueItem> lqis) throws IOException {
|
||||
|
||||
final TableName tableName, final byte[] first, Collection<LoadQueueItem> lqis)
|
||||
throws IOException {
|
||||
final List<Pair<byte[], String>> famPaths =
|
||||
new ArrayList<Pair<byte[], String>>(lqis.size());
|
||||
for (LoadQueueItem lqi : lqis) {
|
||||
|
|
|
@ -85,10 +85,19 @@ implements Writable, Comparable<TableSplit> {
|
|||
|
||||
/** Default constructor. */
|
||||
public TableSplit() {
|
||||
this(null, null, HConstants.EMPTY_BYTE_ARRAY,
|
||||
this((TableName)null, null, HConstants.EMPTY_BYTE_ARRAY,
|
||||
HConstants.EMPTY_BYTE_ARRAY, "");
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Since 0.96.0; use {@link TableSplit#TableSplit(TableName, byte[], byte[], String)}
|
||||
*/
|
||||
@Deprecated
|
||||
public TableSplit(final byte [] tableName, Scan scan, byte [] startRow, byte [] endRow,
|
||||
final String location) {
|
||||
this(TableName.valueOf(tableName), scan, startRow, endRow, location);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new instance while assigning all variables.
|
||||
*
|
||||
|
@ -111,7 +120,16 @@ implements Writable, Comparable<TableSplit> {
|
|||
this.endRow = endRow;
|
||||
this.regionLocation = location;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @deprecated Since 0.96.0; use {@link TableSplit#TableSplit(TableName, byte[], byte[], String)}
|
||||
*/
|
||||
@Deprecated
|
||||
public TableSplit(final byte [] tableName, byte[] startRow, byte[] endRow,
|
||||
final String location) {
|
||||
this(TableName.valueOf(tableName), startRow, endRow, location);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new instance without a scanner.
|
||||
*
|
||||
|
@ -135,12 +153,24 @@ implements Writable, Comparable<TableSplit> {
|
|||
return TableMapReduceUtil.convertStringToScan(this.scan);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the table name converted to a byte array.
|
||||
* @see #getTable()
|
||||
* @return The table name.
|
||||
*/
|
||||
public byte [] getTableName() {
|
||||
return tableName.getName();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the table name.
|
||||
*
|
||||
* @return The table name.
|
||||
*/
|
||||
public TableName getTableName() {
|
||||
public TableName getTable() {
|
||||
// It is ugly that usually to get a TableName, the method is called getTableName. We can't do
|
||||
// that in here though because there was an existing getTableName in place already since
|
||||
// deprecated.
|
||||
return tableName;
|
||||
}
|
||||
|
||||
|
@ -268,7 +298,7 @@ implements Writable, Comparable<TableSplit> {
|
|||
// If The table name of the two splits is the same then compare start row
|
||||
// otherwise compare based on table names
|
||||
int tableNameComparison =
|
||||
getTableName().compareTo(split.getTableName());
|
||||
getTable().compareTo(split.getTable());
|
||||
return tableNameComparison != 0 ? tableNameComparison : Bytes.compareTo(
|
||||
getStartRow(), split.getStartRow());
|
||||
}
|
||||
|
|
|
@ -460,7 +460,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// its a clean cluster startup, else its a failover.
|
||||
Map<HRegionInfo, ServerName> regions = regionStates.getRegionAssignments();
|
||||
for (Map.Entry<HRegionInfo, ServerName> e: regions.entrySet()) {
|
||||
if (!e.getKey().getTableName().isSystemTable()
|
||||
if (!e.getKey().getTable().isSystemTable()
|
||||
&& e.getValue() != null) {
|
||||
LOG.debug("Found " + e + " out on cluster");
|
||||
failover = true;
|
||||
|
@ -1282,8 +1282,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
if (rs.isOpened()) {
|
||||
ServerName serverName = rs.getServerName();
|
||||
regionOnline(regionInfo, serverName);
|
||||
boolean disabled = getZKTable().isDisablingOrDisabledTable(
|
||||
regionInfo.getTableName());
|
||||
boolean disabled = getZKTable().isDisablingOrDisabledTable(regionInfo.getTable());
|
||||
if (!serverManager.isServerOnline(serverName) && !disabled) {
|
||||
LOG.info("Opened " + regionNameStr
|
||||
+ "but the region server is offline, reassign the region");
|
||||
|
@ -1898,7 +1897,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// When we have a case such as all the regions are added directly into hbase:meta and we call
|
||||
// assignRegion then we need to make the table ENABLED. Hence in such case the table
|
||||
// will not be in ENABLING or ENABLED state.
|
||||
TableName tableName = region.getTableName();
|
||||
TableName tableName = region.getTable();
|
||||
if (!zkTable.isEnablingTable(tableName) && !zkTable.isEnabledTable(tableName)) {
|
||||
LOG.debug("Setting table " + tableName + " to ENABLED state.");
|
||||
setEnabledTable(tableName);
|
||||
|
@ -2085,7 +2084,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
|
||||
TableName tableName = region.getTableName();
|
||||
TableName tableName = region.getTable();
|
||||
boolean disabled = this.zkTable.isDisabledTable(tableName);
|
||||
if (disabled || this.zkTable.isDisablingTable(tableName)) {
|
||||
LOG.info("Table " + tableName + (disabled ? " disabled;" : " disabling;") +
|
||||
|
@ -2584,7 +2583,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
//remove system tables because they would have been assigned earlier
|
||||
for(Iterator<HRegionInfo> iter = allRegions.keySet().iterator();
|
||||
iter.hasNext();) {
|
||||
if (iter.next().getTableName().isSystemTable()) {
|
||||
if (iter.next().getTable().isSystemTable()) {
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
|
@ -2603,7 +2602,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
for (HRegionInfo hri : allRegions.keySet()) {
|
||||
TableName tableName = hri.getTableName();
|
||||
TableName tableName = hri.getTable();
|
||||
if (!zkTable.isEnabledTable(tableName)) {
|
||||
setEnabledTable(tableName);
|
||||
}
|
||||
|
@ -2664,7 +2663,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
HRegionInfo regionInfo = region.getFirst();
|
||||
ServerName regionLocation = region.getSecond();
|
||||
if (regionInfo == null) continue;
|
||||
TableName tableName = regionInfo.getTableName();
|
||||
TableName tableName = regionInfo.getTable();
|
||||
if (tableName.isSystemTable()) continue;
|
||||
regionStates.createRegionState(regionInfo);
|
||||
if (regionStates.isRegionInState(regionInfo, State.SPLIT)) {
|
||||
|
@ -3161,7 +3160,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
} catch (KeeperException ke) {
|
||||
server.abort("Unexpected ZK exception deleting node " + hri, ke);
|
||||
}
|
||||
if (zkTable.isDisablingOrDisabledTable(hri.getTableName())) {
|
||||
if (zkTable.isDisablingOrDisabledTable(hri.getTable())) {
|
||||
it.remove();
|
||||
regionStates.regionOffline(hri);
|
||||
continue;
|
||||
|
@ -3193,8 +3192,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// the master to disable, we need to make sure we close those regions in
|
||||
// that case. This is not racing with the region server itself since RS
|
||||
// report is done after the split transaction completed.
|
||||
if (this.zkTable.isDisablingOrDisabledTable(
|
||||
parent.getTableName())) {
|
||||
if (this.zkTable.isDisablingOrDisabledTable(parent.getTable())) {
|
||||
unassign(a);
|
||||
unassign(b);
|
||||
}
|
||||
|
@ -3217,7 +3215,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// the master to disable, we need to make sure we close those regions in
|
||||
// that case. This is not racing with the region server itself since RS
|
||||
// report is done after the regions merge transaction completed.
|
||||
if (this.zkTable.isDisablingOrDisabledTable(merged.getTableName())) {
|
||||
if (this.zkTable.isDisablingOrDisabledTable(merged.getTable())) {
|
||||
unassign(merged);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -149,7 +149,7 @@ public class CatalogJanitor extends Chore {
|
|||
HRegionInfo info = HRegionInfo.getHRegionInfo(r);
|
||||
if (info == null) return true; // Keep scanning
|
||||
if (isTableSpecified
|
||||
&& info.getTableName().compareTo(tableName) > 0) {
|
||||
&& info.getTable().compareTo(tableName) > 0) {
|
||||
// Another table, stop scanning
|
||||
return false;
|
||||
}
|
||||
|
@ -184,9 +184,8 @@ public class CatalogJanitor extends Chore {
|
|||
final HRegionInfo regionA, final HRegionInfo regionB) throws IOException {
|
||||
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
|
||||
Path rootdir = this.services.getMasterFileSystem().getRootDir();
|
||||
Path tabledir = FSUtils.getTableDir(rootdir,
|
||||
mergedRegion.getTableName());
|
||||
HTableDescriptor htd = getTableDescriptor(mergedRegion.getTableName());
|
||||
Path tabledir = FSUtils.getTableDir(rootdir, mergedRegion.getTable());
|
||||
HTableDescriptor htd = getTableDescriptor(mergedRegion.getTable());
|
||||
HRegionFileSystem regionFs = null;
|
||||
try {
|
||||
regionFs = HRegionFileSystem.openRegionFromFileSystem(
|
||||
|
@ -290,8 +289,7 @@ public class CatalogJanitor extends Chore {
|
|||
if (left == null) return -1;
|
||||
if (right == null) return 1;
|
||||
// Same table name.
|
||||
int result = left.getTableName().compareTo(
|
||||
right.getTableName());
|
||||
int result = left.getTable().compareTo(right.getTable());
|
||||
if (result != 0) return result;
|
||||
// Compare start keys.
|
||||
result = Bytes.compareTo(left.getStartKey(), right.getStartKey());
|
||||
|
@ -375,7 +373,7 @@ public class CatalogJanitor extends Chore {
|
|||
|
||||
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
|
||||
Path rootdir = this.services.getMasterFileSystem().getRootDir();
|
||||
Path tabledir = FSUtils.getTableDir(rootdir, daughter.getTableName());
|
||||
Path tabledir = FSUtils.getTableDir(rootdir, daughter.getTable());
|
||||
|
||||
HRegionFileSystem regionFs = null;
|
||||
try {
|
||||
|
@ -387,7 +385,7 @@ public class CatalogJanitor extends Chore {
|
|||
}
|
||||
|
||||
boolean references = false;
|
||||
HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTableName());
|
||||
HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTable());
|
||||
for (HColumnDescriptor family: parentDescriptor.getFamilies()) {
|
||||
if ((references = regionFs.hasReferences(family.getNameAsString()))) {
|
||||
break;
|
||||
|
|
|
@ -1071,7 +1071,7 @@ MasterServices, Server {
|
|||
MetaReader.fullScan(catalogTracker, disabledOrDisablingOrEnabling, true);
|
||||
for(Iterator<HRegionInfo> iter = allRegions.keySet().iterator();
|
||||
iter.hasNext();) {
|
||||
if (!iter.next().getTableName().isSystemTable()) {
|
||||
if (!iter.next().getTable().isSystemTable()) {
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
|
@ -1111,8 +1111,8 @@ MasterServices, Server {
|
|||
this.assignmentManager.regionOnline(regionInfo, currServer);
|
||||
}
|
||||
|
||||
if (!this.assignmentManager.getZKTable().isEnabledTable(regionInfo.getTableName())) {
|
||||
this.assignmentManager.setEnabledTable(regionInfo.getTableName());
|
||||
if (!this.assignmentManager.getZKTable().isEnabledTable(regionInfo.getTable())) {
|
||||
this.assignmentManager.setEnabledTable(regionInfo.getTable());
|
||||
}
|
||||
LOG.info("System region " + regionInfo.getRegionNameAsString() + " assigned, rit=" + rit +
|
||||
", location=" + catalogTracker.getMetaLocation());
|
||||
|
@ -2097,7 +2097,7 @@ MasterServices, Server {
|
|||
if (pair == null) {
|
||||
return false;
|
||||
}
|
||||
if (!pair.getFirst().getTableName().equals(tableName)) {
|
||||
if (!pair.getFirst().getTable().equals(tableName)) {
|
||||
return false;
|
||||
}
|
||||
result.set(pair);
|
||||
|
|
|
@ -588,7 +588,7 @@ public class MasterFileSystem {
|
|||
public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName)
|
||||
throws IOException {
|
||||
// archive family store files
|
||||
Path tableDir = FSUtils.getTableDir(rootdir, region.getTableName());
|
||||
Path tableDir = FSUtils.getTableDir(rootdir, region.getTable());
|
||||
HFileArchiver.archiveFamily(fs, conf, region, tableDir, familyName);
|
||||
|
||||
// delete the family folder
|
||||
|
|
|
@ -433,7 +433,7 @@ public class RegionStates {
|
|||
// before all table's regions.
|
||||
HRegionInfo boundary = new HRegionInfo(tableName, null, null, false, 0L);
|
||||
for (HRegionInfo hri: regionAssignments.tailMap(boundary).keySet()) {
|
||||
if(!hri.getTableName().equals(tableName)) break;
|
||||
if(!hri.getTable().equals(tableName)) break;
|
||||
tableRegions.add(hri);
|
||||
}
|
||||
return tableRegions;
|
||||
|
@ -529,7 +529,7 @@ public class RegionStates {
|
|||
for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
|
||||
for (HRegionInfo hri: e.getValue()) {
|
||||
if (hri.isMetaRegion()) continue;
|
||||
TableName tablename = hri.getTableName();
|
||||
TableName tablename = hri.getTable();
|
||||
Map<ServerName, List<HRegionInfo>> svrToRegions = result.get(tablename);
|
||||
if (svrToRegions == null) {
|
||||
svrToRegions = new HashMap<ServerName, List<HRegionInfo>>(serverHoldings.size());
|
||||
|
|
|
@ -104,8 +104,8 @@ public class SnapshotOfRegionAssignmentFromMeta {
|
|||
HRegionInfo.getHRegionInfoAndServerName(result);
|
||||
HRegionInfo hri = regionAndServer.getFirst();
|
||||
if (hri == null) return true;
|
||||
if (hri.getTableName() == null) return true;
|
||||
if (disabledTables.contains(hri.getTableName())) {
|
||||
if (hri.getTable() == null) return true;
|
||||
if (disabledTables.contains(hri.getTable())) {
|
||||
return true;
|
||||
}
|
||||
// Are we to include split parents in the list?
|
||||
|
@ -144,7 +144,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
|
|||
regionNameToRegionInfoMap.put(regionInfo.getRegionNameAsString(), regionInfo);
|
||||
|
||||
// Process the table to region map
|
||||
TableName tableName = regionInfo.getTableName();
|
||||
TableName tableName = regionInfo.getTable();
|
||||
List<HRegionInfo> regionList = tableToRegionMap.get(tableName);
|
||||
if (regionList == null) {
|
||||
regionList = new ArrayList<HRegionInfo>();
|
||||
|
|
|
@ -148,7 +148,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
|
|||
regionPerServerIndex = 0;
|
||||
|
||||
for (HRegionInfo region : entry.getValue()) {
|
||||
String tableName = region.getTableName().getNameAsString();
|
||||
String tableName = region.getTable().getNameAsString();
|
||||
Integer idx = tablesToIndex.get(tableName);
|
||||
if (idx == null) {
|
||||
tables.add(tableName);
|
||||
|
|
|
@ -99,7 +99,7 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer {
|
|||
currentServer.getPort(), ServerName.NON_STARTCODE);
|
||||
List<HRegionInfo> list = entry.getValue();
|
||||
for (HRegionInfo region : list) {
|
||||
if(region.getTableName().getNamespaceAsString()
|
||||
if(region.getTable().getNamespaceAsString()
|
||||
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
|
||||
continue;
|
||||
}
|
||||
|
|
|
@ -122,7 +122,7 @@ class RegionLocationFinder {
|
|||
protected List<ServerName> internalGetTopBlockLocation(HRegionInfo region) {
|
||||
List<ServerName> topServerNames = null;
|
||||
try {
|
||||
HTableDescriptor tableDescriptor = getTableDescriptor(region.getTableName());
|
||||
HTableDescriptor tableDescriptor = getTableDescriptor(region.getTable());
|
||||
if (tableDescriptor != null) {
|
||||
HDFSBlocksDistribution blocksDistribution =
|
||||
HRegion.computeHDFSBlocksDistribution(getConf(), tableDescriptor, region);
|
||||
|
|
|
@ -92,7 +92,7 @@ public class ClosedRegionHandler extends EventHandler implements TotesHRegionInf
|
|||
LOG.debug("Handling CLOSED event for " + regionInfo.getEncodedName());
|
||||
// Check if this table is being disabled or not
|
||||
if (this.assignmentManager.getZKTable().
|
||||
isDisablingOrDisabledTable(this.regionInfo.getTableName())) {
|
||||
isDisablingOrDisabledTable(this.regionInfo.getTable())) {
|
||||
assignmentManager.offlineDisabledRegion(regionInfo);
|
||||
return;
|
||||
}
|
||||
|
|
|
@ -230,7 +230,7 @@ public class EnableTableHandler extends EventHandler {
|
|||
} else {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Skipping assign for the region " + hri + " during enable table "
|
||||
+ hri.getTableName() + " because its already in tranition or assigned.");
|
||||
+ hri.getTable() + " because its already in tranition or assigned.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -69,7 +69,7 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
|
|||
this.expectedVersion = expectedVersion;
|
||||
if(regionInfo.isMetaRegion()) {
|
||||
priority = OpenedPriority.META;
|
||||
} else if(regionInfo.getTableName()
|
||||
} else if(regionInfo.getTable()
|
||||
.getNamespaceAsString().equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
|
||||
priority = OpenedPriority.SYSTEM;
|
||||
} else {
|
||||
|
@ -113,8 +113,7 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
|
|||
" because regions is NOT in RIT -- presuming this is because it SPLIT");
|
||||
}
|
||||
if (!openedNodeDeleted) {
|
||||
if (this.assignmentManager.getZKTable().isDisablingOrDisabledTable(
|
||||
regionInfo.getTableName())) {
|
||||
if (this.assignmentManager.getZKTable().isDisablingOrDisabledTable(regionInfo.getTable())) {
|
||||
debugLog(regionInfo, "Opened region "
|
||||
+ regionInfo.getShortNameToLog() + " but "
|
||||
+ "this table is disabled, triggering close of region");
|
||||
|
|
|
@ -251,7 +251,7 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
toAssignRegions.add(hri);
|
||||
} else if (rit != null) {
|
||||
if ((rit.isClosing() || rit.isPendingClose())
|
||||
&& am.getZKTable().isDisablingOrDisabledTable(hri.getTableName())) {
|
||||
&& am.getZKTable().isDisablingOrDisabledTable(hri.getTable())) {
|
||||
// If the table was partially disabled and the RS went down, we should clear the RIT
|
||||
// and remove the node for the region.
|
||||
// The rit that we use may be stale in case the table was in DISABLING state
|
||||
|
@ -324,18 +324,16 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
public static boolean processDeadRegion(HRegionInfo hri, Result result,
|
||||
AssignmentManager assignmentManager, CatalogTracker catalogTracker)
|
||||
throws IOException {
|
||||
boolean tablePresent = assignmentManager.getZKTable().isTablePresent(
|
||||
hri.getTableName());
|
||||
boolean tablePresent = assignmentManager.getZKTable().isTablePresent(hri.getTable());
|
||||
if (!tablePresent) {
|
||||
LOG.info("The table " + hri.getTableName()
|
||||
LOG.info("The table " + hri.getTable()
|
||||
+ " was deleted. Hence not proceeding.");
|
||||
return false;
|
||||
}
|
||||
// If table is not disabled but the region is offlined,
|
||||
boolean disabled = assignmentManager.getZKTable().isDisabledTable(
|
||||
hri.getTableName());
|
||||
boolean disabled = assignmentManager.getZKTable().isDisabledTable(hri.getTable());
|
||||
if (disabled){
|
||||
LOG.info("The table " + hri.getTableName()
|
||||
LOG.info("The table " + hri.getTable()
|
||||
+ " was disabled. Hence not proceeding.");
|
||||
return false;
|
||||
}
|
||||
|
@ -345,10 +343,9 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
//to the dead server. We don't have to do anything.
|
||||
return false;
|
||||
}
|
||||
boolean disabling = assignmentManager.getZKTable().isDisablingTable(
|
||||
hri.getTableName());
|
||||
boolean disabling = assignmentManager.getZKTable().isDisablingTable(hri.getTable());
|
||||
if (disabling) {
|
||||
LOG.info("The table " + hri.getTableName()
|
||||
LOG.info("The table " + hri.getTable()
|
||||
+ " is disabled. Hence not assigning region" + hri.getEncodedName());
|
||||
return false;
|
||||
}
|
||||
|
|
|
@ -3924,11 +3924,11 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
final HLog hlog,
|
||||
final boolean initialize, final boolean ignoreHLog)
|
||||
throws IOException {
|
||||
LOG.info("creating HRegion " + info.getTableName().getNameAsString()
|
||||
LOG.info("creating HRegion " + info.getTable().getNameAsString()
|
||||
+ " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
|
||||
" Table name == " + info.getTableName().getNameAsString());
|
||||
" Table name == " + info.getTable().getNameAsString());
|
||||
|
||||
Path tableDir = FSUtils.getTableDir(rootDir, info.getTableName());
|
||||
Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
HRegionFileSystem rfs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info);
|
||||
HLog effectiveHLog = hlog;
|
||||
|
@ -4088,7 +4088,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Opening region: " + info);
|
||||
}
|
||||
Path dir = FSUtils.getTableDir(rootDir, info.getTableName());
|
||||
Path dir = FSUtils.getTableDir(rootDir, info.getTable());
|
||||
HRegion r = HRegion.newHRegion(dir, wal, fs, conf, info, htd, rsServices);
|
||||
return r.openHRegion(reporter);
|
||||
}
|
||||
|
@ -4217,8 +4217,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
@Deprecated
|
||||
public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
|
||||
return new Path(
|
||||
FSUtils.getTableDir(rootdir, info.getTableName()),
|
||||
info.getEncodedName());
|
||||
FSUtils.getTableDir(rootdir, info.getTable()), info.getEncodedName());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -4277,8 +4276,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
* @throws IOException
|
||||
*/
|
||||
public static HRegion merge(final HRegion a, final HRegion b) throws IOException {
|
||||
if (!a.getRegionInfo().getTableName().equals(
|
||||
b.getRegionInfo().getTableName())) {
|
||||
if (!a.getRegionInfo().getTable().equals(b.getRegionInfo().getTable())) {
|
||||
throw new IOException("Regions do not belong to the same table");
|
||||
}
|
||||
|
||||
|
@ -5197,7 +5195,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
public byte[] checkSplit() {
|
||||
// Can't split META
|
||||
if (this.getRegionInfo().isMetaTable() ||
|
||||
TableName.NAMESPACE_TABLE_NAME.equals(this.getRegionInfo().getTableName())) {
|
||||
TableName.NAMESPACE_TABLE_NAME.equals(this.getRegionInfo().getTable())) {
|
||||
if (shouldForceSplit()) {
|
||||
LOG.warn("Cannot split meta region in HBase 0.20 and above");
|
||||
}
|
||||
|
|
|
@ -2347,7 +2347,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
synchronized (this.onlineRegions) {
|
||||
for (HRegion region: this.onlineRegions.values()) {
|
||||
HRegionInfo regionInfo = region.getRegionInfo();
|
||||
if(regionInfo.getTableName().equals(tableName)) {
|
||||
if(regionInfo.getTable().equals(tableName)) {
|
||||
tableRegions.add(region);
|
||||
}
|
||||
}
|
||||
|
@ -3532,10 +3532,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
}
|
||||
}
|
||||
LOG.info("Open " + region.getRegionNameAsString());
|
||||
htd = htds.get(region.getTableName());
|
||||
htd = htds.get(region.getTable());
|
||||
if (htd == null) {
|
||||
htd = this.tableDescriptors.get(region.getTableName());
|
||||
htds.put(region.getTableName(), htd);
|
||||
htd = this.tableDescriptors.get(region.getTable());
|
||||
htds.put(region.getTable(), htd);
|
||||
}
|
||||
|
||||
final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(
|
||||
|
|
|
@ -281,7 +281,7 @@ public class HStore implements Store {
|
|||
|
||||
@Override
|
||||
public TableName getTableName() {
|
||||
return this.getRegionInfo().getTableName();
|
||||
return this.getRegionInfo().getTable();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -146,7 +146,7 @@ public class RegionCoprocessorHost
|
|||
loadSystemCoprocessors(conf, REGION_COPROCESSOR_CONF_KEY);
|
||||
|
||||
// load system default cp's for user tables from configuration.
|
||||
if (!region.getRegionInfo().getTableName().isSystemTable()) {
|
||||
if (!region.getRegionInfo().getTable().isSystemTable()) {
|
||||
loadSystemCoprocessors(conf, USER_REGION_COPROCESSOR_CONF_KEY);
|
||||
}
|
||||
|
||||
|
|
|
@ -433,7 +433,7 @@ public class RegionMergeTransaction {
|
|||
}
|
||||
|
||||
// Merged region is sorted between two merging regions in META
|
||||
HRegionInfo mergedRegionInfo = new HRegionInfo(a.getTableName(), startKey,
|
||||
HRegionInfo mergedRegionInfo = new HRegionInfo(a.getTable(), startKey,
|
||||
endKey, false, rid);
|
||||
return mergedRegionInfo;
|
||||
}
|
||||
|
|
|
@ -168,8 +168,8 @@ public class SplitTransaction {
|
|||
return false;
|
||||
}
|
||||
long rid = getDaughterRegionIdTimestamp(hri);
|
||||
this.hri_a = new HRegionInfo(hri.getTableName(), startKey, this.splitrow, false, rid);
|
||||
this.hri_b = new HRegionInfo(hri.getTableName(), this.splitrow, endKey, false, rid);
|
||||
this.hri_a = new HRegionInfo(hri.getTable(), startKey, this.splitrow, false, rid);
|
||||
this.hri_b = new HRegionInfo(hri.getTable(), this.splitrow, endKey, false, rid);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
|
|
@ -1615,7 +1615,7 @@ public class HLogSplitter {
|
|||
final long pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
|
||||
HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
|
||||
boolean reloadLocation = false;
|
||||
TableName tableName = loc.getRegionInfo().getTableName();
|
||||
TableName tableName = loc.getRegionInfo().getTable();
|
||||
int tries = 0;
|
||||
Throwable cause = null;
|
||||
while (endTime > EnvironmentEdgeManager.currentTimeMillis()) {
|
||||
|
|
|
@ -779,4 +779,10 @@ public class RemoteHTable implements HTableInterface {
|
|||
public void setWriteBufferSize(long writeBufferSize) throws IOException {
|
||||
throw new IOException("setWriteBufferSize not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
|
||||
long amount, boolean writeToWAL) throws IOException {
|
||||
throw new IOException("incrementColumnValue not supported");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -299,7 +299,7 @@ public class AccessControlLists {
|
|||
} else if(userPerm.isGlobal()) {
|
||||
row = ACL_GLOBAL_NAME;
|
||||
} else {
|
||||
row = userPerm.getTable().getName();
|
||||
row = userPerm.getTableName().getName();
|
||||
}
|
||||
return row;
|
||||
}
|
||||
|
@ -502,7 +502,7 @@ public class AccessControlLists {
|
|||
|
||||
for (Map.Entry<String, TablePermission> entry : allPerms.entries()) {
|
||||
UserPermission up = new UserPermission(Bytes.toBytes(entry.getKey()),
|
||||
entry.getValue().getTable(), entry.getValue().getFamily(),
|
||||
entry.getValue().getTableName(), entry.getValue().getFamily(),
|
||||
entry.getValue().getQualifier(), entry.getValue().getActions());
|
||||
perms.add(up);
|
||||
}
|
||||
|
|
|
@ -208,7 +208,7 @@ public class AccessController extends BaseRegionObserver
|
|||
RegionCoprocessorEnvironment e,
|
||||
Map<byte [], ? extends Collection<?>> families) {
|
||||
HRegionInfo hri = e.getRegion().getRegionInfo();
|
||||
TableName tableName = hri.getTableName();
|
||||
TableName tableName = hri.getTable();
|
||||
|
||||
// 1. All users need read access to hbase:meta table.
|
||||
// this is a very common operation, so deal with it quickly.
|
||||
|
@ -470,7 +470,7 @@ public class AccessController extends BaseRegionObserver
|
|||
Map<byte[], ? extends Set<byte[]>> familyMap)
|
||||
throws IOException {
|
||||
HRegionInfo hri = env.getRegion().getRegionInfo();
|
||||
TableName tableName = hri.getTableName();
|
||||
TableName tableName = hri.getTable();
|
||||
|
||||
if (user == null) {
|
||||
return false;
|
||||
|
@ -702,7 +702,7 @@ public class AccessController extends BaseRegionObserver
|
|||
@Override
|
||||
public void preMove(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo region,
|
||||
ServerName srcServer, ServerName destServer) throws IOException {
|
||||
requirePermission("move", region.getTableName(), null, null, Action.ADMIN);
|
||||
requirePermission("move", region.getTable(), null, null, Action.ADMIN);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -713,7 +713,7 @@ public class AccessController extends BaseRegionObserver
|
|||
@Override
|
||||
public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo)
|
||||
throws IOException {
|
||||
requirePermission("assign", regionInfo.getTableName(), null, null, Action.ADMIN);
|
||||
requirePermission("assign", regionInfo.getTable(), null, null, Action.ADMIN);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -723,7 +723,7 @@ public class AccessController extends BaseRegionObserver
|
|||
@Override
|
||||
public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo,
|
||||
boolean force) throws IOException {
|
||||
requirePermission("unassign", regionInfo.getTableName(), null, null, Action.ADMIN);
|
||||
requirePermission("unassign", regionInfo.getTable(), null, null, Action.ADMIN);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -733,7 +733,7 @@ public class AccessController extends BaseRegionObserver
|
|||
@Override
|
||||
public void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> c,
|
||||
HRegionInfo regionInfo) throws IOException {
|
||||
requirePermission("regionOffline", regionInfo.getTableName(), null, null, Action.ADMIN);
|
||||
requirePermission("regionOffline", regionInfo.getTable(), null, null, Action.ADMIN);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1263,7 +1263,7 @@ public class AccessController extends BaseRegionObserver
|
|||
switch(request.getUserPermission().getPermission().getType()) {
|
||||
case Global :
|
||||
case Table :
|
||||
requirePermission("grant", perm.getTable(), perm.getFamily(),
|
||||
requirePermission("grant", perm.getTableName(), perm.getFamily(),
|
||||
perm.getQualifier(), Action.ADMIN);
|
||||
break;
|
||||
case Namespace :
|
||||
|
@ -1303,7 +1303,7 @@ public class AccessController extends BaseRegionObserver
|
|||
switch(request.getUserPermission().getPermission().getType()) {
|
||||
case Global :
|
||||
case Table :
|
||||
requirePermission("revoke", perm.getTable(), perm.getFamily(),
|
||||
requirePermission("revoke", perm.getTableName(), perm.getFamily(),
|
||||
perm.getQualifier(), Action.ADMIN);
|
||||
break;
|
||||
case Namespace :
|
||||
|
@ -1379,11 +1379,11 @@ public class AccessController extends BaseRegionObserver
|
|||
if (permission instanceof TablePermission) {
|
||||
TablePermission tperm = (TablePermission) permission;
|
||||
for (Permission.Action action : permission.getActions()) {
|
||||
if (!tperm.getTable().equals(tableName)) {
|
||||
if (!tperm.getTableName().equals(tableName)) {
|
||||
throw new CoprocessorException(AccessController.class, String.format("This method "
|
||||
+ "can only execute at the table specified in TablePermission. " +
|
||||
"Table of the region:%s , requested table:%s", tableName,
|
||||
tperm.getTable()));
|
||||
tperm.getTableName()));
|
||||
}
|
||||
|
||||
Map<byte[], Set<byte[]>> familyMap = new TreeMap<byte[], Set<byte[]>>(Bytes.BYTES_COMPARATOR);
|
||||
|
@ -1425,7 +1425,7 @@ public class AccessController extends BaseRegionObserver
|
|||
if (region != null) {
|
||||
HRegionInfo regionInfo = region.getRegionInfo();
|
||||
if (regionInfo != null) {
|
||||
tableName = regionInfo.getTableName();
|
||||
tableName = regionInfo.getTable();
|
||||
}
|
||||
}
|
||||
return tableName;
|
||||
|
@ -1457,7 +1457,7 @@ public class AccessController extends BaseRegionObserver
|
|||
}
|
||||
|
||||
private boolean isSpecialTable(HRegionInfo regionInfo) {
|
||||
TableName tableName = regionInfo.getTableName();
|
||||
TableName tableName = regionInfo.getTable();
|
||||
return tableName.equals(AccessControlLists.ACL_TABLE_NAME)
|
||||
|| tableName.equals(TableName.NAMESPACE_TABLE_NAME)
|
||||
|| tableName.equals(TableName.META_TABLE_NAME);
|
||||
|
|
|
@ -1270,7 +1270,7 @@ public class HBaseFsck extends Configured implements Tool {
|
|||
* Check if the specified region's table is disabled.
|
||||
*/
|
||||
private boolean isTableDisabled(HRegionInfo regionInfo) {
|
||||
return disabledTables.contains(regionInfo.getTableName());
|
||||
return disabledTables.contains(regionInfo.getTable());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1820,7 +1820,7 @@ public class HBaseFsck extends Configured implements Tool {
|
|||
if (hbi.deployedOn.size() == 0) continue;
|
||||
|
||||
// We should be safe here
|
||||
TableName tableName = hbi.metaEntry.getTableName();
|
||||
TableName tableName = hbi.metaEntry.getTable();
|
||||
TableInfo modTInfo = tablesInfo.get(tableName);
|
||||
if (modTInfo == null) {
|
||||
modTInfo = new TableInfo(tableName);
|
||||
|
@ -2460,7 +2460,7 @@ public class HBaseFsck extends Configured implements Tool {
|
|||
// pick only those tables that were not modified in the last few milliseconds.
|
||||
if (info != null && info.getStartKey().length == 0 && !info.isMetaRegion()) {
|
||||
if (info.modTime + timelag < now) {
|
||||
tableNames.add(info.getTableName());
|
||||
tableNames.add(info.getTable());
|
||||
} else {
|
||||
numSkipped.incrementAndGet(); // one more in-flux table
|
||||
}
|
||||
|
@ -2584,7 +2584,7 @@ public class HBaseFsck extends Configured implements Tool {
|
|||
sn = pair.getSecond();
|
||||
}
|
||||
HRegionInfo hri = pair.getFirst();
|
||||
if (!(isTableIncluded(hri.getTableName())
|
||||
if (!(isTableIncluded(hri.getTable())
|
||||
|| hri.isMetaRegion())) {
|
||||
return true;
|
||||
}
|
||||
|
@ -2661,7 +2661,7 @@ public class HBaseFsck extends Configured implements Tool {
|
|||
hash ^= Arrays.hashCode(getStartKey());
|
||||
hash ^= Arrays.hashCode(getEndKey());
|
||||
hash ^= Boolean.valueOf(isOffline()).hashCode();
|
||||
hash ^= getTableName().hashCode();
|
||||
hash ^= getTable().hashCode();
|
||||
if (regionServer != null) {
|
||||
hash ^= regionServer.hashCode();
|
||||
}
|
||||
|
@ -2752,7 +2752,7 @@ public class HBaseFsck extends Configured implements Tool {
|
|||
|
||||
public TableName getTableName() {
|
||||
if (this.metaEntry != null) {
|
||||
return this.metaEntry.getTableName();
|
||||
return this.metaEntry.getTable();
|
||||
} else if (this.hdfsEntry != null) {
|
||||
// we are only guaranteed to have a path and not an HRI for hdfsEntry,
|
||||
// so we get the name from the Path
|
||||
|
@ -3108,7 +3108,7 @@ public class HBaseFsck extends Configured implements Tool {
|
|||
List<HRegionInfo> ret = Lists.newArrayList();
|
||||
for (HRegionInfo hri : regions) {
|
||||
if (hri.isMetaTable() || (!hbck.checkMetaOnly
|
||||
&& hbck.isTableIncluded(hri.getTableName()))) {
|
||||
&& hbck.isTableIncluded(hri.getTable()))) {
|
||||
ret.add(hri);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -254,7 +254,7 @@ class HMerge {
|
|||
Bytes.toString(HConstants.CATALOG_FAMILY) + ":" +
|
||||
Bytes.toString(HConstants.REGIONINFO_QUALIFIER));
|
||||
}
|
||||
if (!region.getTableName().equals(this.tableName)) {
|
||||
if (!region.getTable().equals(this.tableName)) {
|
||||
return null;
|
||||
}
|
||||
return region;
|
||||
|
@ -283,7 +283,7 @@ class HMerge {
|
|||
continue;
|
||||
}
|
||||
HRegionInfo region = HRegionInfo.getHRegionInfo(currentRow);
|
||||
if (!region.getTableName().equals(this.tableName)) {
|
||||
if (!region.getTable().equals(this.tableName)) {
|
||||
currentRow = metaScanner.next();
|
||||
continue;
|
||||
}
|
||||
|
|
|
@ -1915,7 +1915,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
continue;
|
||||
}
|
||||
|
||||
if (info.getTableName().equals(tableName)) {
|
||||
if (info.getTable().equals(tableName)) {
|
||||
LOG.info("getMetaTableRows: row -> " +
|
||||
Bytes.toStringBinary(result.getRow()) + info);
|
||||
rows.add(result.getRow());
|
||||
|
@ -2654,7 +2654,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
while ((r = s.next()) != null) {
|
||||
byte [] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
|
||||
HRegionInfo info = HRegionInfo.parseFromOrNull(b);
|
||||
if (info != null && info.getTableName().equals(tableName)) {
|
||||
if (info != null && info.getTable().equals(tableName)) {
|
||||
b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
|
||||
allRegionsAssigned &= (b != null);
|
||||
}
|
||||
|
|
|
@ -1325,7 +1325,7 @@ public class TestAdmin {
|
|||
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
|
||||
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs);
|
||||
for (HRegionInfo regionInfo : onlineRegions) {
|
||||
if (!regionInfo.getTableName().isSystemTable()) {
|
||||
if (!regionInfo.getTable().isSystemTable()) {
|
||||
info = regionInfo;
|
||||
admin.closeRegionWithEncodedRegionName(regionInfo.getEncodedName(), rs
|
||||
.getServerName().getServerName());
|
||||
|
|
|
@ -142,7 +142,7 @@ public class TestMetaScanner {
|
|||
|
||||
//select a random region
|
||||
HRegionInfo parent = regions.get(random.nextInt(regions.size()));
|
||||
if (parent == null || !TABLENAME.equals(parent.getTableName())) {
|
||||
if (parent == null || !TABLENAME.equals(parent.getTable())) {
|
||||
continue;
|
||||
}
|
||||
|
||||
|
|
|
@ -87,7 +87,7 @@ implements WALObserver {
|
|||
HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException {
|
||||
boolean bypass = false;
|
||||
// check table name matches or not.
|
||||
if (!Bytes.equals(info.getTableName().getName(), this.tableName)) {
|
||||
if (!Bytes.equals(info.getTableName(), this.tableName)) {
|
||||
return bypass;
|
||||
}
|
||||
preWALWriteCalled = true;
|
||||
|
|
|
@ -493,7 +493,7 @@ public class TestRegionObserverInterface {
|
|||
try {
|
||||
for (JVMClusterUtil.RegionServerThread t : cluster.getRegionServerThreads()) {
|
||||
for (HRegionInfo r : ProtobufUtil.getOnlineRegions(t.getRegionServer())) {
|
||||
if (!r.getTableName().equals(tableName)) {
|
||||
if (!r.getTable().equals(tableName)) {
|
||||
continue;
|
||||
}
|
||||
RegionCoprocessorHost cph = t.getRegionServer().getOnlineRegion(r.getRegionName()).
|
||||
|
|
|
@ -104,7 +104,7 @@ public class TestRegionServerCoprocessorExceptionWithAbort {
|
|||
final Put put, final WALEdit edit,
|
||||
final Durability durability) {
|
||||
TableName tableName =
|
||||
c.getEnvironment().getRegion().getRegionInfo().getTableName();
|
||||
c.getEnvironment().getRegion().getRegionInfo().getTable();
|
||||
if (TABLE_NAME.equals(tableName) && Bytes.equals(put.getRow(), ROW)) {
|
||||
throw new NullPointerException("Buggy coprocessor: " + put);
|
||||
}
|
||||
|
|
|
@ -53,8 +53,7 @@ public class TestRegionServerCoprocessorExceptionWithRemove {
|
|||
final Put put, final WALEdit edit,
|
||||
final Durability durability) {
|
||||
String tableName =
|
||||
c.getEnvironment().getRegion().getRegionInfo()
|
||||
.getTableName().getNameAsString();
|
||||
c.getEnvironment().getRegion().getRegionInfo().getTable().getNameAsString();
|
||||
if (tableName.equals("observed_table")) {
|
||||
Integer i = null;
|
||||
i = i + 1;
|
||||
|
|
|
@ -186,7 +186,7 @@ public class TestWALObserver {
|
|||
|
||||
// it's where WAL write cp should occur.
|
||||
long now = EnvironmentEdgeManager.currentTimeMillis();
|
||||
log.append(hri, hri.getTableName(), edit, now, htd);
|
||||
log.append(hri, hri.getTable(), edit, now, htd);
|
||||
|
||||
// the edit shall have been change now by the coprocessor.
|
||||
foundFamily0 = false;
|
||||
|
|
|
@ -158,7 +158,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
|
|||
.toBytes(table));
|
||||
|
||||
for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(hrs)) {
|
||||
if (Bytes.equals(hri.getTableName().getName(), Bytes.toBytes(table))) {
|
||||
if (Bytes.equals(hri.getTable().getName(), Bytes.toBytes(table))) {
|
||||
// splitRegion doesn't work if startkey/endkey are null
|
||||
ProtobufUtil.split(hrs, hri, rowkey(ROWCOUNT / 2)); // hard code split
|
||||
}
|
||||
|
@ -169,7 +169,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
|
|||
do {
|
||||
regions = 0;
|
||||
for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(hrs)) {
|
||||
if (Bytes.equals(hri.getTableName().getName(), Bytes.toBytes(table))) {
|
||||
if (Bytes.equals(hri.getTable().getName(), Bytes.toBytes(table))) {
|
||||
regions++;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -501,7 +501,7 @@ public class TestAssignmentManager {
|
|||
// adding region in pending close.
|
||||
am.getRegionStates().updateRegionState(
|
||||
REGIONINFO, State.SPLITTING, SERVERNAME_A);
|
||||
am.getZKTable().setEnabledTable(REGIONINFO.getTableName());
|
||||
am.getZKTable().setEnabledTable(REGIONINFO.getTable());
|
||||
RegionTransition data = RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_SPLITTING,
|
||||
REGIONINFO.getRegionName(), SERVERNAME_A);
|
||||
String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
|
||||
|
@ -550,9 +550,9 @@ public class TestAssignmentManager {
|
|||
// adding region in pending close.
|
||||
am.getRegionStates().updateRegionState(REGIONINFO, State.PENDING_CLOSE);
|
||||
if (state == Table.State.DISABLING) {
|
||||
am.getZKTable().setDisablingTable(REGIONINFO.getTableName());
|
||||
am.getZKTable().setDisablingTable(REGIONINFO.getTable());
|
||||
} else {
|
||||
am.getZKTable().setDisabledTable(REGIONINFO.getTableName());
|
||||
am.getZKTable().setDisabledTable(REGIONINFO.getTable());
|
||||
}
|
||||
RegionTransition data = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_CLOSING,
|
||||
REGIONINFO.getRegionName(), SERVERNAME_A);
|
||||
|
@ -576,7 +576,7 @@ public class TestAssignmentManager {
|
|||
am.getRegionStates().isRegionsInTransition());
|
||||
}
|
||||
} finally {
|
||||
am.setEnabledTable(REGIONINFO.getTableName());
|
||||
am.setEnabledTable(REGIONINFO.getTable());
|
||||
executor.shutdown();
|
||||
am.shutdown();
|
||||
// Clean up all znodes
|
||||
|
@ -888,7 +888,7 @@ public class TestAssignmentManager {
|
|||
}
|
||||
try{
|
||||
// set table in disabling state.
|
||||
am.getZKTable().setDisablingTable(REGIONINFO.getTableName());
|
||||
am.getZKTable().setDisablingTable(REGIONINFO.getTable());
|
||||
am.joinCluster();
|
||||
// should not call retainAssignment if we get empty regions in assignAllUserRegions.
|
||||
assertFalse(
|
||||
|
@ -896,12 +896,12 @@ public class TestAssignmentManager {
|
|||
gate.get());
|
||||
// need to change table state from disabling to disabled.
|
||||
assertTrue("Table should be disabled.",
|
||||
am.getZKTable().isDisabledTable(REGIONINFO.getTableName()));
|
||||
am.getZKTable().isDisabledTable(REGIONINFO.getTable()));
|
||||
} finally {
|
||||
this.server.getConfiguration().setClass(
|
||||
HConstants.HBASE_MASTER_LOADBALANCER_CLASS, DefaultLoadBalancer.class,
|
||||
LoadBalancer.class);
|
||||
am.getZKTable().setEnabledTable(REGIONINFO.getTableName());
|
||||
am.getZKTable().setEnabledTable(REGIONINFO.getTable());
|
||||
am.shutdown();
|
||||
}
|
||||
}
|
||||
|
@ -927,17 +927,17 @@ public class TestAssignmentManager {
|
|||
this.serverManager);
|
||||
try {
|
||||
// set table in enabling state.
|
||||
am.getZKTable().setEnablingTable(REGIONINFO.getTableName());
|
||||
new EnableTableHandler(server, REGIONINFO.getTableName(),
|
||||
am.getZKTable().setEnablingTable(REGIONINFO.getTable());
|
||||
new EnableTableHandler(server, REGIONINFO.getTable(),
|
||||
am.getCatalogTracker(), am, new NullTableLockManager(), true).prepare()
|
||||
.process();
|
||||
assertEquals("Number of assignments should be 1.", 1, assignmentCount);
|
||||
assertTrue("Table should be enabled.",
|
||||
am.getZKTable().isEnabledTable(REGIONINFO.getTableName()));
|
||||
am.getZKTable().isEnabledTable(REGIONINFO.getTable()));
|
||||
} finally {
|
||||
enabling = false;
|
||||
assignmentCount = 0;
|
||||
am.getZKTable().setEnabledTable(REGIONINFO.getTableName());
|
||||
am.getZKTable().setEnabledTable(REGIONINFO.getTable());
|
||||
am.shutdown();
|
||||
ZKAssign.deleteAllNodes(this.watcher);
|
||||
}
|
||||
|
@ -964,7 +964,7 @@ public class TestAssignmentManager {
|
|||
// adding region plan
|
||||
am.regionPlans.put(REGIONINFO.getEncodedName(),
|
||||
new RegionPlan(REGIONINFO, SERVERNAME_B, SERVERNAME_A));
|
||||
am.getZKTable().setEnabledTable(REGIONINFO.getTableName());
|
||||
am.getZKTable().setEnabledTable(REGIONINFO.getTable());
|
||||
|
||||
try {
|
||||
am.assignInvoked = false;
|
||||
|
|
|
@ -833,7 +833,7 @@ public class TestCatalogJanitor {
|
|||
// archive directory. Otherwise, it just seems to pick the first root directory it can find (so
|
||||
// the single test passes, but when the full suite is run, things get borked).
|
||||
FSUtils.setRootDir(fs.getConf(), rootdir);
|
||||
Path tabledir = FSUtils.getTableDir(rootdir, parent.getTableName());
|
||||
Path tabledir = FSUtils.getTableDir(rootdir, parent.getTable());
|
||||
Path storedir = HStore.getStoreHomedir(tabledir, parent, htd.getColumnFamilies()[0].getName());
|
||||
System.out.println("Old root:" + rootdir);
|
||||
System.out.println("Old table:" + tabledir);
|
||||
|
@ -915,7 +915,7 @@ public class TestCatalogJanitor {
|
|||
final HRegionInfo daughter, final byte [] midkey, final boolean top)
|
||||
throws IOException {
|
||||
Path rootdir = services.getMasterFileSystem().getRootDir();
|
||||
Path tabledir = FSUtils.getTableDir(rootdir, parent.getTableName());
|
||||
Path tabledir = FSUtils.getTableDir(rootdir, parent.getTable());
|
||||
Path storedir = HStore.getStoreHomedir(tabledir, daughter,
|
||||
htd.getColumnFamilies()[0].getName());
|
||||
Reference ref =
|
||||
|
|
|
@ -208,7 +208,7 @@ public class TestDistributedLogSplitting {
|
|||
hrs = rsts.get(i).getRegionServer();
|
||||
regions = ProtobufUtil.getOnlineRegions(hrs);
|
||||
for (HRegionInfo region : regions) {
|
||||
if (region.getTableName().getNameAsString().equalsIgnoreCase("table")) {
|
||||
if (region.getTable().getNameAsString().equalsIgnoreCase("table")) {
|
||||
foundRs = true;
|
||||
break;
|
||||
}
|
||||
|
@ -222,7 +222,7 @@ public class TestDistributedLogSplitting {
|
|||
Iterator<HRegionInfo> it = regions.iterator();
|
||||
while (it.hasNext()) {
|
||||
HRegionInfo region = it.next();
|
||||
if (region.getTableName().getNamespaceAsString()
|
||||
if (region.getTable().getNamespaceAsString()
|
||||
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
|
||||
it.remove();
|
||||
}
|
||||
|
@ -664,12 +664,12 @@ public class TestDistributedLogSplitting {
|
|||
break;
|
||||
}
|
||||
if (tableName != null &&
|
||||
!tableName.equalsIgnoreCase(region.getTableName().getNameAsString())) {
|
||||
!tableName.equalsIgnoreCase(region.getTable().getNameAsString())) {
|
||||
// make sure that we find a RS has online regions for both "table" and "disableTable"
|
||||
hasRegionsForBothTables = true;
|
||||
break;
|
||||
} else if (tableName == null) {
|
||||
tableName = region.getTableName().getNameAsString();
|
||||
tableName = region.getTable().getNameAsString();
|
||||
}
|
||||
}
|
||||
if (isCarryingMeta) {
|
||||
|
@ -1123,7 +1123,7 @@ public class TestDistributedLogSplitting {
|
|||
HRegionServer hrs = rst.getRegionServer();
|
||||
List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs);
|
||||
for (HRegionInfo hri : hris) {
|
||||
if (hri.getTableName().isSystemTable()) {
|
||||
if (hri.getTable().isSystemTable()) {
|
||||
continue;
|
||||
}
|
||||
LOG.debug("adding data to rs = " + rst.getName() +
|
||||
|
@ -1148,7 +1148,7 @@ public class TestDistributedLogSplitting {
|
|||
|
||||
for(Iterator<HRegionInfo> iter = regions.iterator(); iter.hasNext(); ) {
|
||||
HRegionInfo regionInfo = iter.next();
|
||||
if(regionInfo.getTableName().isSystemTable()) {
|
||||
if(regionInfo.getTable().isSystemTable()) {
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
|
@ -1157,7 +1157,7 @@ public class TestDistributedLogSplitting {
|
|||
|
||||
List<HRegionInfo> hris = new ArrayList<HRegionInfo>();
|
||||
for (HRegionInfo region : regions) {
|
||||
if (!region.getTableName().getNameAsString().equalsIgnoreCase(tname)) {
|
||||
if (!region.getTable().getNameAsString().equalsIgnoreCase(tname)) {
|
||||
continue;
|
||||
}
|
||||
hris.add(region);
|
||||
|
@ -1330,7 +1330,7 @@ public class TestDistributedLogSplitting {
|
|||
if (region.isMetaRegion()) {
|
||||
isCarryingMeta = true;
|
||||
}
|
||||
if (tableName == null || region.getTableName().getNameAsString().equals(tableName)) {
|
||||
if (tableName == null || region.getTable().getNameAsString().equals(tableName)) {
|
||||
foundTableRegion = true;
|
||||
}
|
||||
if (foundTableRegion && (isCarryingMeta || !hasMetaRegion)) {
|
||||
|
|
|
@ -489,12 +489,12 @@ public class TestMasterTransitions {
|
|||
for (Result r = null; (r = s.next()) != null;) {
|
||||
HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
|
||||
if (hri == null) break;
|
||||
if (!hri.getTableName().getNameAsString().equals(TABLENAME)) {
|
||||
if (!hri.getTable().getNameAsString().equals(TABLENAME)) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// If start key, add 'aaa'.
|
||||
if(!hri.getTableName().getNameAsString().equals(TABLENAME)) {
|
||||
if(!hri.getTable().getNameAsString().equals(TABLENAME)) {
|
||||
continue;
|
||||
}
|
||||
byte [] row = getStartKey(hri);
|
||||
|
@ -511,24 +511,6 @@ public class TestMasterTransitions {
|
|||
return rows;
|
||||
}
|
||||
|
||||
/*
|
||||
* @return Count of rows in TABLENAME
|
||||
* @throws IOException
|
||||
*/
|
||||
private static int count() throws IOException {
|
||||
HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
|
||||
int rows = 0;
|
||||
Scan scan = new Scan();
|
||||
ResultScanner s = t.getScanner(scan);
|
||||
for (Result r = null; (r = s.next()) != null;) {
|
||||
rows++;
|
||||
}
|
||||
s.close();
|
||||
LOG.info("Counted=" + rows);
|
||||
t.close();
|
||||
return rows;
|
||||
}
|
||||
|
||||
/*
|
||||
* @param hri
|
||||
* @return Start key for hri (If start key is '', then return 'aaa'.
|
||||
|
@ -545,6 +527,4 @@ public class TestMasterTransitions {
|
|||
private static byte [] getTestQualifier() {
|
||||
return getTestFamily();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -64,8 +64,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
|
|||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -307,7 +305,7 @@ public class TestRegionPlacement {
|
|||
TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getOnlineRegionsLocalContext();
|
||||
isNamespaceServer = false;
|
||||
for (HRegion r : regs) {
|
||||
if (r.getRegionInfo().getTableName().getNamespaceAsString()
|
||||
if (r.getRegionInfo().getTable().getNamespaceAsString()
|
||||
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
|
||||
isNamespaceServer = true;
|
||||
break;
|
||||
|
@ -606,7 +604,7 @@ public class TestRegionPlacement {
|
|||
public boolean processRow(Result result) throws IOException {
|
||||
try {
|
||||
HRegionInfo info = MetaScanner.getHRegionInfo(result);
|
||||
if(info.getTableName().getNamespaceAsString()
|
||||
if(info.getTable().getNamespaceAsString()
|
||||
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -283,7 +283,7 @@ public class TestZKBasedOpenCloseRegion {
|
|||
for (Result r = null; (r = s.next()) != null;) {
|
||||
HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
|
||||
if (hri == null) break;
|
||||
if(!hri.getTableName().equals(TABLENAME)) {
|
||||
if(!hri.getTable().equals(TABLENAME)) {
|
||||
continue;
|
||||
}
|
||||
// If start key, add 'aaa'.
|
||||
|
|
|
@ -480,7 +480,7 @@ public class TestEndToEndSplitTransaction {
|
|||
throws IOException, InterruptedException {
|
||||
log("blocking until region is opened for reading:" + hri.getRegionNameAsString());
|
||||
long start = System.currentTimeMillis();
|
||||
HTable table = new HTable(conf, hri.getTableName());
|
||||
HTable table = new HTable(conf, hri.getTable());
|
||||
|
||||
try {
|
||||
byte [] row = hri.getStartKey();
|
||||
|
|
|
@ -60,7 +60,7 @@ public class TestHRegionFileSystem {
|
|||
// Create a Region
|
||||
HRegionInfo hri = new HRegionInfo(TableName.valueOf("TestTable"));
|
||||
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs,
|
||||
FSUtils.getTableDir(rootDir, hri.getTableName()), hri);
|
||||
FSUtils.getTableDir(rootDir, hri.getTable()), hri);
|
||||
|
||||
// Verify if the region is on disk
|
||||
Path regionDir = regionFs.getRegionDir();
|
||||
|
@ -72,12 +72,12 @@ public class TestHRegionFileSystem {
|
|||
|
||||
// Open the region
|
||||
regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, fs,
|
||||
FSUtils.getTableDir(rootDir, hri.getTableName()), hri, false);
|
||||
FSUtils.getTableDir(rootDir, hri.getTable()), hri, false);
|
||||
assertEquals(regionDir, regionFs.getRegionDir());
|
||||
|
||||
// Delete the region
|
||||
HRegionFileSystem.deleteRegionFromFileSystem(conf, fs,
|
||||
FSUtils.getTableDir(rootDir, hri.getTableName()), hri);
|
||||
FSUtils.getTableDir(rootDir, hri.getTable()), hri);
|
||||
assertFalse("The region folder should be removed", fs.exists(regionDir));
|
||||
|
||||
fs.delete(rootDir, true);
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
|
@ -28,12 +27,12 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.MD5Hash;
|
||||
import org.junit.Test;
|
||||
|
|
|
@ -185,7 +185,7 @@ public class TestRegionMergeTransactionOnCluster {
|
|||
FileSystem fs = master.getMasterFileSystem().getFileSystem();
|
||||
Path rootDir = master.getMasterFileSystem().getRootDir();
|
||||
|
||||
Path tabledir = FSUtils.getTableDir(rootDir, mergedRegionInfo.getTableName());
|
||||
Path tabledir = FSUtils.getTableDir(rootDir, mergedRegionInfo.getTable());
|
||||
Path regionAdir = new Path(tabledir, regionA.getEncodedName());
|
||||
Path regionBdir = new Path(tabledir, regionB.getEncodedName());
|
||||
assertTrue(fs.exists(regionAdir));
|
||||
|
|
|
@ -307,7 +307,7 @@ public class TestRegionServerNoMaster {
|
|||
hri.getEncodedNameAsBytes()));
|
||||
|
||||
// Let's start the open handler
|
||||
HTableDescriptor htd = getRS().tableDescriptors.get(hri.getTableName());
|
||||
HTableDescriptor htd = getRS().tableDescriptors.get(hri.getTable());
|
||||
getRS().service.submit(new OpenRegionHandler(getRS(), getRS(), hri, htd, 0));
|
||||
|
||||
// The open handler should have removed the region from RIT but kept the region closed
|
||||
|
@ -361,7 +361,7 @@ public class TestRegionServerNoMaster {
|
|||
// 1) There is no ZK node
|
||||
// 2) The region in RIT was changed.
|
||||
// The order is more or less implementation dependant.
|
||||
HTableDescriptor htd = getRS().tableDescriptors.get(hri.getTableName());
|
||||
HTableDescriptor htd = getRS().tableDescriptors.get(hri.getTable());
|
||||
getRS().service.submit(new OpenRegionHandler(getRS(), getRS(), hri, htd, 0));
|
||||
|
||||
// The open handler should have removed the region from RIT but kept the region closed
|
||||
|
|
|
@ -217,11 +217,11 @@ public class TestSplitTransactionOnCluster {
|
|||
RegionStates regionStates = cluster.getMaster().getAssignmentManager().getRegionStates();
|
||||
Map<String, RegionState> rit = regionStates.getRegionsInTransition();
|
||||
|
||||
for (int i=0; rit.containsKey(hri.getTableName()) && i<100; i++) {
|
||||
for (int i=0; rit.containsKey(hri.getTable()) && i<100; i++) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
assertFalse("region still in transition", rit.containsKey(
|
||||
rit.containsKey(hri.getTableName())));
|
||||
rit.containsKey(hri.getTable())));
|
||||
|
||||
List<HRegion> onlineRegions = regionServer.getOnlineRegions(tableName);
|
||||
// Region server side split is successful.
|
||||
|
@ -923,7 +923,7 @@ public class TestSplitTransactionOnCluster {
|
|||
@Override
|
||||
void transitionZKNode(Server server, RegionServerServices services, HRegion a, HRegion b)
|
||||
throws IOException {
|
||||
if (this.currentRegion.getRegionInfo().getTableName().getNameAsString()
|
||||
if (this.currentRegion.getRegionInfo().getTable().getNameAsString()
|
||||
.equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
|
||||
try {
|
||||
if (!secondSplit){
|
||||
|
@ -935,14 +935,14 @@ public class TestSplitTransactionOnCluster {
|
|||
|
||||
}
|
||||
super.transitionZKNode(server, services, a, b);
|
||||
if (this.currentRegion.getRegionInfo().getTableName().getNameAsString()
|
||||
if (this.currentRegion.getRegionInfo().getTable().getNameAsString()
|
||||
.equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
|
||||
firstSplitCompleted = true;
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public boolean rollback(Server server, RegionServerServices services) throws IOException {
|
||||
if (this.currentRegion.getRegionInfo().getTableName().getNameAsString()
|
||||
if (this.currentRegion.getRegionInfo().getTable().getNameAsString()
|
||||
.equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
|
||||
if(secondSplit){
|
||||
super.rollback(server, services);
|
||||
|
|
|
@ -93,7 +93,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
final HRegionInfo hri =
|
||||
new HRegionInfo(TableName.valueOf("testBasicHalfMapFileTb"));
|
||||
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(
|
||||
conf, fs, new Path(this.testDir, hri.getTableName().getNameAsString()), hri);
|
||||
conf, fs, new Path(this.testDir, hri.getTable().getNameAsString()), hri);
|
||||
|
||||
StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs, 2 * 1024)
|
||||
.withFilePath(regionFs.createTempName())
|
||||
|
@ -142,7 +142,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
public void testReference() throws IOException {
|
||||
final HRegionInfo hri = new HRegionInfo(TableName.valueOf("testReferenceTb"));
|
||||
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(
|
||||
conf, fs, new Path(this.testDir, hri.getTableName().getNameAsString()), hri);
|
||||
conf, fs, new Path(this.testDir, hri.getTable().getNameAsString()), hri);
|
||||
|
||||
// Make a store file and write data to it.
|
||||
StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs, 8 * 1024)
|
||||
|
@ -162,7 +162,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
kv = KeyValue.createKeyValueFromKey(reader.getLastKey());
|
||||
byte [] finalRow = kv.getRow();
|
||||
// Make a reference
|
||||
HRegionInfo splitHri = new HRegionInfo(hri.getTableName(), null, midRow);
|
||||
HRegionInfo splitHri = new HRegionInfo(hri.getTable(), null, midRow);
|
||||
Path refPath = splitStoreFile(regionFs, splitHri, TEST_FAMILY, hsf, midRow, true);
|
||||
StoreFile refHsf = new StoreFile(this.fs, refPath, conf, cacheConf,
|
||||
BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
|
||||
|
@ -186,7 +186,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
Configuration testConf = new Configuration(this.conf);
|
||||
FSUtils.setRootDir(testConf, this.testDir);
|
||||
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(
|
||||
testConf, fs, FSUtils.getTableDir(this.testDir, hri.getTableName()), hri);
|
||||
testConf, fs, FSUtils.getTableDir(this.testDir, hri.getTable()), hri);
|
||||
|
||||
// Make a store file and write data to it.
|
||||
StoreFile.Writer writer = new StoreFile.WriterBuilder(testConf, cacheConf, this.fs, 8 * 1024)
|
||||
|
@ -228,7 +228,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
// adding legal table name chars to verify regex handles it.
|
||||
HRegionInfo hri = new HRegionInfo(TableName.valueOf("_original-evil-name"));
|
||||
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(
|
||||
testConf, fs, FSUtils.getTableDir(this.testDir, hri.getTableName()), hri);
|
||||
testConf, fs, FSUtils.getTableDir(this.testDir, hri.getTable()), hri);
|
||||
|
||||
// Make a store file and write data to it. <root>/<tablename>/<rgn>/<cf>/<file>
|
||||
StoreFile.Writer writer = new StoreFile.WriterBuilder(testConf, cacheConf, this.fs, 8 * 1024)
|
||||
|
@ -240,7 +240,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
// create link to store file. <root>/clone/region/<cf>/<hfile>-<region>-<table>
|
||||
HRegionInfo hriClone = new HRegionInfo(TableName.valueOf("clone"));
|
||||
HRegionFileSystem cloneRegionFs = HRegionFileSystem.createRegionOnFileSystem(
|
||||
testConf, fs, FSUtils.getTableDir(this.testDir, hri.getTableName()),
|
||||
testConf, fs, FSUtils.getTableDir(this.testDir, hri.getTable()),
|
||||
hriClone);
|
||||
Path dstPath = cloneRegionFs.getStoreDir(TEST_FAMILY);
|
||||
HFileLink.create(testConf, this.fs, dstPath, hri, storeFilePath.getName());
|
||||
|
@ -250,8 +250,8 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
// create splits of the link.
|
||||
// <root>/clone/splitA/<cf>/<reftohfilelink>,
|
||||
// <root>/clone/splitB/<cf>/<reftohfilelink>
|
||||
HRegionInfo splitHriA = new HRegionInfo(hri.getTableName(), null, SPLITKEY);
|
||||
HRegionInfo splitHriB = new HRegionInfo(hri.getTableName(), SPLITKEY, null);
|
||||
HRegionInfo splitHriA = new HRegionInfo(hri.getTable(), null, SPLITKEY);
|
||||
HRegionInfo splitHriB = new HRegionInfo(hri.getTable(), SPLITKEY, null);
|
||||
StoreFile f = new StoreFile(fs, linkFilePath, testConf, cacheConf, BloomType.NONE,
|
||||
NoOpDataBlockEncoder.INSTANCE);
|
||||
Path pathA = splitStoreFile(cloneRegionFs, splitHriA, TEST_FAMILY, f, SPLITKEY, true); // top
|
||||
|
@ -300,11 +300,11 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
KeyValue midKV = KeyValue.createKeyValueFromKey(midkey);
|
||||
byte [] midRow = midKV.getRow();
|
||||
// Create top split.
|
||||
HRegionInfo topHri = new HRegionInfo(regionFs.getRegionInfo().getTableName(),
|
||||
HRegionInfo topHri = new HRegionInfo(regionFs.getRegionInfo().getTable(),
|
||||
null, midRow);
|
||||
Path topPath = splitStoreFile(regionFs, topHri, TEST_FAMILY, f, midRow, true);
|
||||
// Create bottom split.
|
||||
HRegionInfo bottomHri = new HRegionInfo(regionFs.getRegionInfo().getTableName(),
|
||||
HRegionInfo bottomHri = new HRegionInfo(regionFs.getRegionInfo().getTable(),
|
||||
midRow, null);
|
||||
Path bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, midRow, false);
|
||||
// Make readers on top and bottom.
|
||||
|
|
|
@ -103,10 +103,10 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool
|
|||
addFamilyMapToWALEdit(put.getFamilyCellMap(), walEdit);
|
||||
HRegionInfo hri = region.getRegionInfo();
|
||||
if (this.noSync) {
|
||||
hlog.appendNoSync(hri, hri.getTableName(), walEdit,
|
||||
hlog.appendNoSync(hri, hri.getTable(), walEdit,
|
||||
new ArrayList<UUID>(), now, htd);
|
||||
} else {
|
||||
hlog.append(hri, hri.getTableName(), walEdit, now, htd);
|
||||
hlog.append(hri, hri.getTable(), walEdit, now, htd);
|
||||
}
|
||||
}
|
||||
long totalTime = (System.currentTimeMillis() - startTime);
|
||||
|
|
|
@ -191,7 +191,7 @@ public class TestTablePermissions {
|
|||
assertEquals("Should have 1 permission for george", 1, userPerms.size());
|
||||
TablePermission permission = userPerms.get(0);
|
||||
assertEquals("Permission should be for " + TEST_TABLE,
|
||||
TEST_TABLE, permission.getTable());
|
||||
TEST_TABLE, permission.getTableName());
|
||||
assertNull("Column family should be empty", permission.getFamily());
|
||||
|
||||
// check actions
|
||||
|
@ -206,7 +206,7 @@ public class TestTablePermissions {
|
|||
assertEquals("Should have 1 permission for hubert", 1, userPerms.size());
|
||||
permission = userPerms.get(0);
|
||||
assertEquals("Permission should be for " + TEST_TABLE,
|
||||
TEST_TABLE, permission.getTable());
|
||||
TEST_TABLE, permission.getTableName());
|
||||
assertNull("Column family should be empty", permission.getFamily());
|
||||
|
||||
// check actions
|
||||
|
@ -221,7 +221,7 @@ public class TestTablePermissions {
|
|||
assertEquals("Should have 1 permission for humphrey", 1, userPerms.size());
|
||||
permission = userPerms.get(0);
|
||||
assertEquals("Permission should be for " + TEST_TABLE,
|
||||
TEST_TABLE, permission.getTable());
|
||||
TEST_TABLE, permission.getTableName());
|
||||
assertTrue("Permission should be for family " + TEST_FAMILY,
|
||||
Bytes.equals(TEST_FAMILY, permission.getFamily()));
|
||||
assertTrue("Permission should be for qualifier " + TEST_QUALIFIER,
|
||||
|
@ -249,7 +249,7 @@ public class TestTablePermissions {
|
|||
assertNotNull(userPerms);
|
||||
assertEquals(1, userPerms.size());
|
||||
permission = userPerms.get(0);
|
||||
assertEquals(TEST_TABLE, permission.getTable());
|
||||
assertEquals(TEST_TABLE, permission.getTableName());
|
||||
assertEquals(1, permission.getActions().length);
|
||||
assertEquals(TablePermission.Action.READ, permission.getActions()[0]);
|
||||
|
||||
|
@ -257,7 +257,7 @@ public class TestTablePermissions {
|
|||
assertNotNull(userPerms);
|
||||
assertEquals(1, userPerms.size());
|
||||
permission = userPerms.get(0);
|
||||
assertEquals(TEST_TABLE2, permission.getTable());
|
||||
assertEquals(TEST_TABLE2, permission.getTableName());
|
||||
assertEquals(2, permission.getActions().length);
|
||||
actions = Arrays.asList(permission.getActions());
|
||||
assertTrue(actions.contains(TablePermission.Action.READ));
|
||||
|
|
|
@ -37,17 +37,16 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -202,7 +201,7 @@ public class TestExportSnapshot {
|
|||
// First region, simple with one plain hfile.
|
||||
HRegionInfo hri = new HRegionInfo(tableWithRefsName);
|
||||
HRegionFileSystem r0fs = HRegionFileSystem.createRegionOnFileSystem(conf,
|
||||
fs, FSUtils.getTableDir(archiveDir, hri.getTableName()), hri);
|
||||
fs, FSUtils.getTableDir(archiveDir, hri.getTable()), hri);
|
||||
Path storeFile = new Path(rootDir, TEST_HFILE);
|
||||
FSDataOutputStream out = fs.create(storeFile);
|
||||
out.write(Bytes.toBytes("Test Data"));
|
||||
|
@ -213,7 +212,7 @@ public class TestExportSnapshot {
|
|||
// This region contains a reference to the hfile in the first region.
|
||||
hri = new HRegionInfo(tableWithRefsName);
|
||||
HRegionFileSystem r1fs = HRegionFileSystem.createRegionOnFileSystem(conf,
|
||||
fs, new Path(archiveDir, hri.getTableName().getNameAsString()), hri);
|
||||
fs, new Path(archiveDir, hri.getTable().getNameAsString()), hri);
|
||||
storeFile = new Path(rootDir, TEST_HFILE + '.' + r0fs.getRegionInfo().getEncodedName());
|
||||
out = fs.create(storeFile);
|
||||
out.write(Bytes.toBytes("Test Data"));
|
||||
|
|
|
@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
|||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
|
||||
|
@ -160,7 +159,7 @@ public class TestRestoreSnapshotHelper {
|
|||
// First region, simple with one plain hfile.
|
||||
HRegionInfo hri = new HRegionInfo(htd.getTableName());
|
||||
HRegionFileSystem r0fs = HRegionFileSystem.createRegionOnFileSystem(conf,
|
||||
fs, FSUtils.getTableDir(archiveDir, hri.getTableName()), hri);
|
||||
fs, FSUtils.getTableDir(archiveDir, hri.getTable()), hri);
|
||||
Path storeFile = new Path(rootDir, TEST_HFILE);
|
||||
fs.createNewFile(storeFile);
|
||||
r0fs.commitStoreFile(TEST_FAMILY, storeFile);
|
||||
|
@ -169,7 +168,7 @@ public class TestRestoreSnapshotHelper {
|
|||
// This region contains a reference to the hfile in the first region.
|
||||
hri = new HRegionInfo(htd.getTableName());
|
||||
HRegionFileSystem r1fs = HRegionFileSystem.createRegionOnFileSystem(conf,
|
||||
fs, FSUtils.getTableDir(archiveDir, hri.getTableName()), hri);
|
||||
fs, FSUtils.getTableDir(archiveDir, hri.getTable()), hri);
|
||||
storeFile = new Path(rootDir, TEST_HFILE + '.' + r0fs.getRegionInfo().getEncodedName());
|
||||
fs.createNewFile(storeFile);
|
||||
r1fs.commitStoreFile(TEST_FAMILY, storeFile);
|
||||
|
|
|
@ -1868,7 +1868,7 @@ public class TestHBaseFsck {
|
|||
|
||||
@Override
|
||||
public boolean processRow(Result rowResult) throws IOException {
|
||||
if(!MetaScanner.getHRegionInfo(rowResult).getTableName().isSystemTable()) {
|
||||
if(!MetaScanner.getHRegionInfo(rowResult).getTable().isSystemTable()) {
|
||||
Delete delete = new Delete(rowResult.getRow());
|
||||
delete.deleteColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
|
||||
deletes.add(delete);
|
||||
|
|
|
@ -235,8 +235,7 @@ public class OfflineMetaRebuildTestCore {
|
|||
for (Result r : scanner) {
|
||||
HRegionInfo info =
|
||||
HRegionInfo.getHRegionInfo(r);
|
||||
if(info != null &&
|
||||
!info.getTableName().getNamespaceAsString()
|
||||
if(info != null && !info.getTable().getNamespaceAsString()
|
||||
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
|
||||
Delete d = new Delete(r.getRow());
|
||||
dels.add(d);
|
||||
|
|
Loading…
Reference in New Issue