HBASE-451 Remove HTableDescriptor from HRegionInfo

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1135385 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-06-14 05:40:32 +00:00
parent 34ddfc126d
commit d8f5be74d7
84 changed files with 2599 additions and 616 deletions

View File

@ -10,6 +10,7 @@ Release 0.91.0 - Unreleased
instead of wrapping in RuntimeException (Ted Yu via garyh)
HBASE-3629 Update our thrift to 0.6 (Moaz Reyad)
HBASE-1502 Remove need for heartbeats in HBase
HBASE-451 Remove HTableDescriptor from HRegionInfo (Subbu M Iyer)
BUG FIXES
HBASE-3280 YouAreDeadException being swallowed in HRS getMaster

View File

@ -171,6 +171,13 @@ public final class HConstants {
/** Used to construct the name of the compaction directory during compaction */
public static final String HREGION_COMPACTIONDIR_NAME = "compaction.dir";
/** The file name used to store HTD in HDFS */
public static final String TABLEINFO_NAME = ".tableinfo";
/** The metaupdated column qualifier */
public static final byte [] META_MIGRATION_QUALIFIER = Bytes.toBytes("metamigrated");
/** Default maximum file size */
public static final long DEFAULT_MAX_FILE_SIZE = 256 * 1024 * 1024;

View File

@ -27,6 +27,7 @@ import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.migration.HRegionInfo090x;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JenkinsHash;
import org.apache.hadoop.hbase.util.MD5Hash;
@ -130,11 +131,11 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
/** HRegionInfo for root region */
public static final HRegionInfo ROOT_REGIONINFO =
new HRegionInfo(0L, HTableDescriptor.ROOT_TABLEDESC);
new HRegionInfo(0L, Bytes.toBytes("-ROOT-"));
/** HRegionInfo for first meta region */
public static final HRegionInfo FIRST_META_REGIONINFO =
new HRegionInfo(1L, HTableDescriptor.META_TABLEDESC);
new HRegionInfo(1L, Bytes.toBytes(".META."));
private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
// This flag is in the parent of a split while the parent is still referenced
@ -146,34 +147,37 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
private String regionNameStr = "";
private boolean split = false;
private byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
protected HTableDescriptor tableDesc = null;
private int hashCode = -1;
//TODO: Move NO_HASH to HStoreFile which is really the only place it is used.
public static final String NO_HASH = null;
private volatile String encodedName = NO_HASH;
private byte [] encodedNameAsBytes = null;
// Current TableName
private byte[] tableName = null;
private String tableNameAsString = null;
private void setHashCode() {
int result = Arrays.hashCode(this.regionName);
result ^= this.regionId;
result ^= Arrays.hashCode(this.startKey);
result ^= Arrays.hashCode(this.endKey);
result ^= Boolean.valueOf(this.offLine).hashCode();
result ^= this.tableDesc.hashCode();
result ^= Arrays.hashCode(this.tableName);
this.hashCode = result;
}
/**
* Private constructor used constructing HRegionInfo for the catalog root and
* first meta regions
*/
private HRegionInfo(long regionId, HTableDescriptor tableDesc) {
private HRegionInfo(long regionId, byte[] tableName) {
super();
this.regionId = regionId;
this.tableDesc = tableDesc;
this.tableName = tableName.clone();
// Note: Root & First Meta regions names are still in old format
this.regionName = createRegionName(tableDesc.getName(), null,
this.regionName = createRegionName(tableName, null,
regionId, false);
this.regionNameStr = Bytes.toStringBinary(this.regionName);
setHashCode();
@ -182,43 +186,66 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
/** Default constructor - creates empty object */
public HRegionInfo() {
super();
this.tableDesc = new HTableDescriptor();
}
/**
* Used only for migration
* @param other HRegionInfoForMigration
*/
public HRegionInfo(HRegionInfo090x other) {
super();
this.endKey = other.getEndKey();
this.offLine = other.isOffline();
this.regionId = other.getRegionId();
this.regionName = other.getRegionName();
this.regionNameStr = Bytes.toStringBinary(this.regionName);
this.split = other.isSplit();
this.startKey = other.getStartKey();
this.hashCode = other.hashCode();
this.encodedName = other.getEncodedName();
this.tableName = other.getTableDesc().getName();
}
public HRegionInfo(final byte[] tableName) {
this(tableName, null, null);
}
/**
* Construct HRegionInfo with explicit parameters
*
* @param tableDesc the table descriptor
* @param tableName the table name
* @param startKey first key in region
* @param endKey end of key range
* @throws IllegalArgumentException
*/
public HRegionInfo(final HTableDescriptor tableDesc, final byte [] startKey,
public HRegionInfo(final byte[] tableName, final byte[] startKey,
final byte[] endKey)
throws IllegalArgumentException {
this(tableDesc, startKey, endKey, false);
this(tableName, startKey, endKey, false);
}
/**
* Construct HRegionInfo with explicit parameters
*
* @param tableDesc the table descriptor
* @param tableName the table descriptor
* @param startKey first key in region
* @param endKey end of key range
* @param split true if this region has split and we have daughter regions
* regions that may or may not hold references to this region.
* @throws IllegalArgumentException
*/
public HRegionInfo(HTableDescriptor tableDesc, final byte [] startKey,
public HRegionInfo(final byte[] tableName, final byte[] startKey,
final byte[] endKey, final boolean split)
throws IllegalArgumentException {
this(tableDesc, startKey, endKey, split, System.currentTimeMillis());
this(tableName, startKey, endKey, split, System.currentTimeMillis());
}
/**
* Construct HRegionInfo with explicit parameters
*
* @param tableDesc the table descriptor
* @param tableName the table descriptor
* @param startKey first key in region
* @param endKey end of key range
* @param split true if this region has split and we have daughter regions
@ -226,22 +253,26 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
* @param regionid Region id to use.
* @throws IllegalArgumentException
*/
public HRegionInfo(HTableDescriptor tableDesc, final byte [] startKey,
public HRegionInfo(final byte[] tableName, final byte[] startKey,
final byte[] endKey, final boolean split, final long regionid)
throws IllegalArgumentException {
super();
if (tableDesc == null) {
throw new IllegalArgumentException("tableDesc cannot be null");
if (tableName == null) {
throw new IllegalArgumentException("tableName cannot be null");
}
this.tableName = tableName.clone();
this.offLine = false;
this.regionId = regionid;
this.regionName = createRegionName(tableDesc.getName(), startKey, regionId, true);
this.regionName = createRegionName(this.tableName, startKey, regionId, true);
this.regionNameStr = Bytes.toStringBinary(this.regionName);
this.split = split;
this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone();
this.startKey = startKey == null?
HConstants.EMPTY_START_ROW: startKey.clone();
this.tableDesc = tableDesc;
this.tableName = tableName.clone();
setHashCode();
}
@ -259,11 +290,12 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
this.regionNameStr = Bytes.toStringBinary(this.regionName);
this.split = other.isSplit();
this.startKey = other.getStartKey();
this.tableDesc = other.getTableDesc();
this.hashCode = other.hashCode();
this.encodedName = other.getEncodedName();
this.tableName = other.tableName;
}
/**
* Make a region name of passed parameters.
* @param tableName
@ -457,6 +489,22 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
return endKey;
}
/**
* Get current table name of the region
* @return byte array of table name
*/
public byte[] getTableName() {
return tableName;
}
/**
* Get current table name as string
* @return string representation of current table
*/
public String getTableNameAsString() {
return Bytes.toString(tableName);
}
/**
* Returns true if the given inclusive range of rows is fully contained
* by this region. For example, if the region is foo,a,g and this is
@ -488,32 +536,34 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
}
/** @return the tableDesc */
@Deprecated
public HTableDescriptor getTableDesc(){
return tableDesc;
return null;
}
/**
* @param newDesc new table descriptor to use
*/
@Deprecated
public void setTableDesc(HTableDescriptor newDesc) {
this.tableDesc = newDesc;
// do nothing.
}
/** @return true if this is the root region */
public boolean isRootRegion() {
return this.tableDesc.isRootRegion();
return Bytes.equals(tableName, HRegionInfo.ROOT_REGIONINFO.getTableName());
}
/** @return true if this region is from a table that is a meta table,
* either <code>.META.</code> or <code>-ROOT-</code>
*/
public boolean isMetaTable() {
return this.tableDesc.isMetaTable();
return Bytes.equals(tableName, HRegionInfo.FIRST_META_REGIONINFO.getTableName());
}
/** @return true if this region is a meta region */
public boolean isMetaRegion() {
return this.tableDesc.isMetaRegion();
return isMetaTable();
}
/**
@ -564,14 +614,14 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
@Override
public String toString() {
return "REGION => {" + HConstants.NAME + " => '" +
this.regionNameStr +
"', STARTKEY => '" +
this.regionNameStr
+ " TableName => " + this.tableName
+ "', STARTKEY => '" +
Bytes.toStringBinary(this.startKey) + "', ENDKEY => '" +
Bytes.toStringBinary(this.endKey) +
"', ENCODED => " + getEncodedName() + "," +
(isOffline()? " OFFLINE => true,": "") +
(isSplit()? " SPLIT => true,": "") +
" TABLE => {" + this.tableDesc.toString() + "}";
(isSplit()? " SPLIT => true,": "") + "}";
}
/**
@ -618,7 +668,7 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
Bytes.writeByteArray(out, regionName);
out.writeBoolean(split);
Bytes.writeByteArray(out, startKey);
tableDesc.write(out);
Bytes.writeByteArray(out, tableName);
out.writeInt(hashCode);
}
@ -632,7 +682,7 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
this.regionNameStr = Bytes.toStringBinary(this.regionName);
this.split = in.readBoolean();
this.startKey = Bytes.readByteArray(in);
this.tableDesc.readFields(in);
this.tableName = Bytes.readByteArray(in);
this.hashCode = in.readInt();
}
@ -646,7 +696,7 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
}
// Are regions of same table?
int result = Bytes.compareTo(this.tableDesc.getName(), o.tableDesc.getName());
int result = Bytes.compareTo(this.tableName, o.tableName);
if (result != 0) {
return result;
}

View File

@ -19,20 +19,24 @@
*/
package org.apache.hadoop.hbase.catalog;
import java.util.ArrayList;
import java.util.List;
import java.io.IOException;
import java.net.ConnectException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.migration.HRegionInfo090x;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.catalog.MetaReader.Visitor;
/**
* Writes region and assignment information to <code>.META.</code>.
@ -219,6 +223,82 @@ public class MetaEditor {
LOG.info("Updated region " + regionInfo.getRegionNameAsString() + " in META");
}
public static void updateRootWithMetaMigrationStatus(CatalogTracker catalogTracker) throws IOException {
updateRootWithMetaMigrationStatus(catalogTracker, true);
}
public static void updateRootWithMetaMigrationStatus(CatalogTracker catalogTracker,
boolean metaUpdated)
throws IOException {
Put put = new Put(HRegionInfo.ROOT_REGIONINFO.getRegionName());
addMetaUpdateStatus(put, metaUpdated);
catalogTracker.waitForRootServerConnectionDefault().put(
CatalogTracker.ROOT_REGION, put);
LOG.info("Updated -ROOT- row with metaMigrated status = " + metaUpdated);
}
public static List<HTableDescriptor> updateMetaWithNewRegionInfo(
final MasterServices masterServices)
throws IOException {
final List<HTableDescriptor> htds = new ArrayList<HTableDescriptor>();
Visitor v = new Visitor() {
@Override
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
HRegionInfo090x hrfm = getHRegionInfoForMigration(r);
htds.add(hrfm.getTableDesc());
masterServices.getMasterFileSystem().createTableDescriptor(hrfm.getTableDesc());
HRegionInfo regionInfo = new HRegionInfo(hrfm);
LOG.debug(" MetaEditor.updatemeta RegionInfo = " + regionInfo.toString()
+ " old HRI = " + hrfm.toString());
Put put = new Put(regionInfo.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(regionInfo));
masterServices.getCatalogTracker().waitForMetaServerConnectionDefault().put(
CatalogTracker.META_REGION, put);
LOG.info("Updated region " + regionInfo + " to META");
return true;
}
};
MetaReader.fullScan(masterServices.getCatalogTracker(), v);
updateRootWithMetaMigrationStatus(masterServices.getCatalogTracker());
return htds;
}
public static HRegionInfo090x getHRegionInfoForMigration(
Result data) throws IOException {
byte [] bytes =
data.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
if (bytes == null) return null;
HRegionInfo090x info = Writables.getHRegionInfoForMigration(bytes);
LOG.info("Current INFO from scan results = " + info);
return info;
}
public static HRegionInfo getHRegionInfo(
Result data) throws IOException {
byte [] bytes =
data.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
if (bytes == null) return null;
HRegionInfo info = Writables.getHRegionInfo(bytes);
LOG.info("Current INFO from scan results = " + info);
return info;
}
private static Put addMetaUpdateStatus(final Put p) {
p.add(HConstants.CATALOG_FAMILY, HConstants.META_MIGRATION_QUALIFIER,
Bytes.toBytes("true"));
return p;
}
private static Put addMetaUpdateStatus(final Put p, final boolean metaUpdated) {
p.add(HConstants.CATALOG_FAMILY, HConstants.META_MIGRATION_QUALIFIER,
Bytes.toBytes(metaUpdated));
return p;
}
private static Put addRegionInfo(final Put p, final HRegionInfo hri)
throws IOException {
p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,

View File

@ -28,16 +28,14 @@ import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.migration.HRegionInfo090x;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Writables;
@ -50,6 +48,8 @@ import org.apache.hadoop.ipc.RemoteException;
* catalogs.
*/
public class MetaReader {
private static final Log LOG = LogFactory.getLog(MetaReader.class);
public static final byte [] META_REGION_PREFIX;
static {
// Copy the prefix from FIRST_META_REGIONINFO into META_REGION_PREFIX.
@ -182,7 +182,7 @@ public class MetaReader {
if (region == null) return true;
HRegionInfo hri = region.getFirst();
if (disabledTables.contains(
hri.getTableDesc().getNameAsString())) return true;
hri.getTableNameAsString())) return true;
// Are we to include split parents in the list?
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
regions.put(hri, region.getSecond());
@ -583,6 +583,48 @@ public class MetaReader {
}
}
public static void fullScanMetaAndPrint(
CatalogTracker catalogTracker)
throws IOException {
final List<HRegionInfo090x> regions =
new ArrayList<HRegionInfo090x>();
Visitor v = new Visitor() {
@Override
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r);
HRegionInfo hrim = MetaEditor.getHRegionInfo(r);
LOG.info("fullScanMetaAndPrint.HRI Print= " + hrim);
return true;
}
};
fullScan(catalogTracker, v);
}
public static List<HRegionInfo090x> fullScanMetaAndPrintHRIM(
CatalogTracker catalogTracker)
throws IOException {
final List<HRegionInfo090x> regions =
new ArrayList<HRegionInfo090x>();
Visitor v = new Visitor() {
@Override
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
LOG.info("fullScanMetaAndPrint1.Current Meta Result: " + r);
HRegionInfo090x hrim = MetaEditor.getHRegionInfoForMigration(r);
LOG.info("fullScanMetaAndPrint.HRIM Print= " + hrim);
regions.add(hrim);
return true;
}
};
fullScan(catalogTracker, v);
return regions;
}
/**
* Implementations 'visit' a catalog table row.
*/

View File

@ -106,6 +106,7 @@ public class HBaseAdmin implements Abortable, Closeable {
CatalogTracker ct = null;
try {
ct = new CatalogTracker(this.conf);
ct.start();
} catch (InterruptedException e) {
// Let it out as an IOE for now until we redo all so tolerate IEs
@ -1266,4 +1267,16 @@ public class HBaseAdmin implements Abortable, Closeable {
this.connection.close();
}
}
/**
* Get tableDescriptors
* @param tableNames List of table names
* @return HTD[] the tableDescriptor
* @throws IOException if a remote or network exception occurs
*/
public HTableDescriptor[] getTableDescriptors(List<String> tableNames)
throws IOException {
return this.connection.getHTableDescriptors(tableNames);
}
}

View File

@ -373,4 +373,13 @@ public interface HConnection extends Abortable, Closeable {
* @deprecated This method will be changed from public to package protected.
*/
public int getCurrentNrHRS() throws IOException;
/**
* @param tableNames List of table names
* @return HTD[] table metadata
* @throws IOException if a remote or network exception occurs
*/
public HTableDescriptor[] getHTableDescriptors(List<String> tableNames)
throws IOException;
}

View File

@ -634,33 +634,6 @@ public class HConnectionManager {
return reload? relocateRegion(name, row): locateRegion(name, row);
}
public HTableDescriptor[] listTables() throws IOException {
final TreeSet<HTableDescriptor> uniqueTables =
new TreeSet<HTableDescriptor>();
MetaScannerVisitor visitor = new MetaScannerVisitor() {
public boolean processRow(Result result) throws IOException {
try {
byte[] value = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
HRegionInfo info = null;
if (value != null) {
info = Writables.getHRegionInfo(value);
}
// Only examine the rows where the startKey is zero length
if (info != null && info.getStartKey().length == 0) {
uniqueTables.add(info.getTableDesc());
}
return true;
} catch (RuntimeException e) {
LOG.error("Result=" + result);
throw e;
}
}
};
MetaScanner.metaScan(conf, visitor);
return uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]);
}
public boolean isTableEnabled(byte[] tableName) throws IOException {
return testTableOnlineState(tableName, true);
}
@ -679,7 +652,7 @@ public class HConnectionManager {
HConstants.REGIONINFO_QUALIFIER);
HRegionInfo info = Writables.getHRegionInfoOrNull(value);
if (info != null) {
if (Bytes.equals(tableName, info.getTableDesc().getName())) {
if (Bytes.equals(tableName, info.getTableName())) {
value = row.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if (value == null) {
@ -716,47 +689,6 @@ public class HConnectionManager {
}
}
private static class HTableDescriptorFinder
implements MetaScanner.MetaScannerVisitor {
byte[] tableName;
HTableDescriptor result;
protected HTableDescriptorFinder(byte[] tableName) {
this.tableName = tableName;
}
public boolean processRow(Result rowResult) throws IOException {
HRegionInfo info = Writables.getHRegionInfoOrNull(
rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER));
if (info == null) return true;
HTableDescriptor desc = info.getTableDesc();
if (Bytes.equals(desc.getName(), tableName)) {
result = desc;
return false;
}
return true;
}
HTableDescriptor getResult() {
return result;
}
}
public HTableDescriptor getHTableDescriptor(final byte[] tableName)
throws IOException {
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
return new UnmodifyableHTableDescriptor(HTableDescriptor.ROOT_TABLEDESC);
}
if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
return HTableDescriptor.META_TABLEDESC;
}
HTableDescriptorFinder finder = new HTableDescriptorFinder(tableName);
MetaScanner.metaScan(conf, finder, tableName);
HTableDescriptor result = finder.getResult();
if (result == null) {
throw new TableNotFoundException(Bytes.toString(tableName));
}
return result;
}
@Override
public HRegionLocation locateRegion(final byte [] regionName)
throws IOException {
@ -836,7 +768,7 @@ public class HConnectionManager {
regionInfo = Writables.getHRegionInfo(value);
// possible we got a region of a different table...
if (!Bytes.equals(regionInfo.getTableDesc().getName(),
if (!Bytes.equals(regionInfo.getTableName(),
tableName)) {
return false; // stop scanning
}
@ -956,7 +888,7 @@ public class HConnectionManager {
HRegionInfo regionInfo = (HRegionInfo) Writables.getWritable(
value, new HRegionInfo());
// possible we got a region of a different table...
if (!Bytes.equals(regionInfo.getTableDesc().getName(), tableName)) {
if (!Bytes.equals(regionInfo.getTableName(), tableName)) {
throw new TableNotFoundException(
"Table '" + Bytes.toString(tableName) + "' was not found.");
}
@ -1783,5 +1715,50 @@ public class HConnectionManager {
LOG.debug("The connection to " + this.zooKeeper
+ " was closed by the finalize method.");
}
public HTableDescriptor[] listTables() throws IOException {
if (this.master == null) {
this.master = getMaster();
}
HTableDescriptor[] htd = master.getHTableDescriptors();
return htd;
}
public HTableDescriptor[] getHTableDescriptors(List<String> tableNames) throws IOException {
if (tableNames == null || tableNames.size() == 0) return null;
if (this.master == null) {
this.master = getMaster();
}
return master.getHTableDescriptors(tableNames);
}
public HTableDescriptor getHTableDescriptor(final byte[] tableName)
throws IOException {
if (tableName == null || tableName.length == 0) return null;
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
return new UnmodifyableHTableDescriptor(HTableDescriptor.ROOT_TABLEDESC);
}
if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
return HTableDescriptor.META_TABLEDESC;
}
if (this.master == null) {
this.master = getMaster();
}
HTableDescriptor hTableDescriptor = null;
HTableDescriptor[] htds = master.getHTableDescriptors();
if (htds != null && htds.length > 0) {
for (HTableDescriptor htd: htds) {
if (Bytes.equals(tableName, htd.getName())) {
hTableDescriptor = htd;
}
}
}
//HTableDescriptor htd = master.getHTableDescriptor(tableName);
if (hTableDescriptor == null) {
throw new TableNotFoundException(Bytes.toString(tableName));
}
return hTableDescriptor;
}
}
}

View File

@ -395,7 +395,7 @@ public class HTable implements HTableInterface, Closeable {
return true;
}
HRegionInfo info = Writables.getHRegionInfo(bytes);
if (Bytes.equals(info.getTableDesc().getName(), getTableName())) {
if (Bytes.equals(info.getTableName(), getTableName())) {
if (!(info.isOffline() || info.isSplit())) {
startKeyList.add(info.getStartKey());
endKeyList.add(info.getEndKey());
@ -425,7 +425,7 @@ public class HTable implements HTableInterface, Closeable {
rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER));
if (!(Bytes.equals(info.getTableDesc().getName(), getTableName()))) {
if (!(Bytes.equals(info.getTableName(), getTableName()))) {
return false;
}

View File

@ -288,7 +288,7 @@ public class MetaScanner {
HRegionInfo info = Writables.getHRegionInfo(
rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER));
if (!(Bytes.equals(info.getTableDesc().getName(), tablename))) {
if (!(Bytes.equals(info.getTableName(), tablename))) {
return false;
}
byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,

View File

@ -30,7 +30,6 @@ class UnmodifyableHRegionInfo extends HRegionInfo {
*/
UnmodifyableHRegionInfo(HRegionInfo info) {
super(info);
this.tableDesc = new UnmodifyableHTableDescriptor(info.getTableDesc());
}
/**

View File

@ -163,6 +163,7 @@ public class HbaseObjectWritable implements Writable, WritableWithSize, Configur
addToMap(HServerAddress.class, code++);
addToMap(HServerInfo.class, code++);
addToMap(HTableDescriptor.class, code++);
addToMap(HTableDescriptor[].class, code++);
addToMap(MapWritable.class, code++);
//

View File

@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.ipc;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HColumnDescriptor;
@ -200,4 +201,25 @@ public interface HMasterInterface extends VersionedProtocol {
* @return Previous balancer value
*/
public boolean balanceSwitch(final boolean b);
/**
* Get array of all HTDs.
* @return array of HTableDescriptor
*/
public HTableDescriptor[] getHTableDescriptors();
/**
* Get current HTD for a given tablename
* @param tableName
* @return HTableDescriptor for the table
*/
//public HTableDescriptor getHTableDescriptor(final byte[] tableName);
/**
* Get array of HTDs for requested tables.
* @param tableNames
* @return array of HTableDescriptor
*/
public HTableDescriptor[] getHTableDescriptors(List<String> tableNames);
}

View File

@ -25,6 +25,7 @@ import java.io.IOException;
import java.lang.Thread.UncaughtExceptionHandler;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
@ -63,6 +64,7 @@ import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
import org.apache.hadoop.hbase.master.handler.SplitRegionHandler;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
@ -77,6 +79,7 @@ import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.data.Stat;
import org.apache.hadoop.hbase.client.Get;
/**
* Manages and performs region assignment.
@ -139,6 +142,10 @@ public class AssignmentManager extends ZooKeeperListener {
private final ExecutorService executorService;
private Map<String, HTableDescriptor> tableDescMap =
new HashMap<String, HTableDescriptor>();
/**
* Constructs a new assignment manager.
*
@ -166,6 +173,7 @@ public class AssignmentManager extends ZooKeeperListener {
this.zkTable = new ZKTable(this.master.getZooKeeper());
this.maximumAssignmentAttempts =
this.master.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10);
initHTableDescriptorMap();
}
/**
@ -244,7 +252,9 @@ public class AssignmentManager extends ZooKeeperListener {
// its a clean cluster startup, else its a failover.
boolean regionsToProcess = false;
for (Map.Entry<HRegionInfo, ServerName> e: this.regions.entrySet()) {
if (!e.getKey().isMetaRegion() && e.getValue() != null) {
if (!e.getKey().isMetaRegion()
&& !e.getKey().isRootRegion()
&& e.getValue() != null) {
LOG.debug("Found " + e + " out on cluster");
regionsToProcess = true;
break;
@ -267,6 +277,7 @@ public class AssignmentManager extends ZooKeeperListener {
}
} else {
// Fresh cluster startup.
LOG.info("Clean cluster startup. Assigning userregions");
cleanoutUnassigned();
assignAllUserRegions();
}
@ -919,7 +930,7 @@ public class AssignmentManager extends ZooKeeperListener {
public void assign(HRegionInfo region, boolean setOfflineInZK,
boolean forceNewPlan) {
String tableName = region.getTableDesc().getNameAsString();
String tableName = region.getTableNameAsString();
boolean disabled = this.zkTable.isDisabledTable(tableName);
if (disabled || this.zkTable.isDisablingTable(tableName)) {
LOG.info("Table " + tableName + (disabled? " disabled;": " disabling;") +
@ -978,6 +989,10 @@ public class AssignmentManager extends ZooKeeperListener {
}
// Move on to open regions.
try {
// Update the tableDesc map.
for (HRegionInfo region : regions) {
updateDescMap(region.getTableNameAsString());
}
// Send OPEN RPC. This can fail if the server on other end is is not up.
// If we fail, fail the startup by aborting the server. There is one
// exception we will tolerate: ServerNotRunningException. This is thrown
@ -1811,10 +1826,10 @@ public class AssignmentManager extends ZooKeeperListener {
public List<HRegionInfo> getRegionsOfTable(byte[] tableName) {
List<HRegionInfo> tableRegions = new ArrayList<HRegionInfo>();
HRegionInfo boundary =
new HRegionInfo(new HTableDescriptor(tableName), null, null);
new HRegionInfo(tableName, null, null);
synchronized (this.regions) {
for (HRegionInfo regionInfo: this.regions.tailMap(boundary).keySet()) {
if(Bytes.equals(regionInfo.getTableDesc().getName(), tableName)) {
if(Bytes.equals(regionInfo.getTableName(), tableName)) {
tableRegions.add(regionInfo);
} else {
break;
@ -2057,7 +2072,7 @@ public class AssignmentManager extends ZooKeeperListener {
// 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.getTableDesc().getNameAsString())) {
parent.getTableNameAsString())) {
unassign(a);
unassign(b);
}
@ -2152,6 +2167,140 @@ public class AssignmentManager extends ZooKeeperListener {
LOG.info("Bulk assigning done");
}
private void initHTableDescriptorMap() {
try {
synchronized (this.tableDescMap) {
this.tableDescMap =
FSUtils.getTableDescriptors(this.master.getConfiguration());
}
} catch (IOException e) {
LOG.info("IOException while initializing HTableDescriptor Map");
}
}
private HTableDescriptor readTableDescriptor(String tableName)
throws IOException {
return FSUtils.getHTableDescriptor(
this.master.getConfiguration(), tableName);
}
private boolean isRootOrMetaRegion(String tableName) {
return (
tableName.equals(
HRegionInfo.ROOT_REGIONINFO.getTableNameAsString())
||
tableName.equals(
HRegionInfo.FIRST_META_REGIONINFO.getTableNameAsString()));
}
private void updateDescMap(String tableName) throws IOException {
if (this.tableDescMap == null) {
LOG.error("Table Descriptor cache is null. " +
"Skipping desc map update for table = " + tableName);
return;
}
if (tableName == null || isRootOrMetaRegion(tableName))
return;
if (!this.tableDescMap.containsKey(tableName)) {
HTableDescriptor htd = readTableDescriptor(tableName);
if (htd != null) {
LOG.info("Updating TableDesc Map for tablename = " + tableName
+ "htd == " + htd);
synchronized (this.tableDescMap) {
this.tableDescMap.put(tableName, htd);
}
} else {
LOG.info("HTable Descriptor is NULL for table = " + tableName);
}
}
}
public void updateTableDesc(String tableName, HTableDescriptor htd) {
if (this.tableDescMap == null) {
LOG.error("Table Descriptor cache is null. " +
"Skipping desc map update for table = " + tableName);
return;
}
if (tableName == null || isRootOrMetaRegion(tableName))
return;
if (!this.tableDescMap.containsKey(tableName)) {
LOG.error("Table descriptor missing in DescMap. for tablename = " + tableName);
}
synchronized (this.tableDescMap) {
this.tableDescMap.put(tableName, htd);
}
LOG.info("TableDesc updated successfully for table = " + tableName);
}
public void deleteTableDesc(String tableName) {
if (this.tableDescMap == null) {
LOG.error("Table Descriptor cache is null. " +
"Skipping desc map update for table = " + tableName);
return;
}
if (tableName == null || isRootOrMetaRegion(tableName))
return;
if (!this.tableDescMap.containsKey(tableName)) {
LOG.error("Table descriptor missing in DescMap. for tablename = " + tableName);
}
synchronized (this.tableDescMap) {
this.tableDescMap.remove(tableName);
}
LOG.info("TableDesc removed successfully for table = " + tableName);
}
public HTableDescriptor[] getHTableDescriptors(List<String> tableNames) {
List htdList = null;
HTableDescriptor[] htd = null;
if (tableNames != null && tableNames.size() > 0) {
if (this.tableDescMap != null) {
htd = new HTableDescriptor[tableNames.size()];
htdList = new ArrayList();
synchronized (this.tableDescMap) {
int index = 0;
for (String tableName : tableNames) {
HTableDescriptor htdesc = this.tableDescMap.get(tableName);
htd[index++] = this.tableDescMap.get(tableName);
if (htdesc != null) {
htdList.add(htdesc);
}
}
}
}
}
if (htdList != null && htdList.size() > 0 ) {
return (HTableDescriptor[]) htdList.toArray(new HTableDescriptor[htdList.size()]);
}
return null;
}
public HTableDescriptor[] getHTableDescriptors() {
if (this.tableDescMap != null) {
synchronized (this.tableDescMap) {
Collection<HTableDescriptor> htdc = this.tableDescMap.values();
if (htdc != null) {
return htdc.toArray(new HTableDescriptor[htdc.size()]);
}
}
}
return null;
}
public HTableDescriptor getTableDescriptor(String tableName) {
HTableDescriptor htd = null;
if (tableName != null) {
synchronized (this.tableDescMap) {
htd = this.tableDescMap.get(tableName);
}
}
return htd;
}
/**
* State of a Region while undergoing transitions.
*/

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.Chore;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
@ -41,8 +42,10 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
/**
* A janitor for the catalog tables. Scans the <code>.META.</code> catalog
* table on a period looking for unused regions to garbage collect.
@ -253,8 +256,10 @@ class CatalogJanitor extends Chore {
if (split == null) return result;
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
Path rootdir = this.services.getMasterFileSystem().getRootDir();
Path tabledir = new Path(rootdir, split.getTableDesc().getNameAsString());
for (HColumnDescriptor family: split.getTableDesc().getFamilies()) {
Path tabledir = new Path(rootdir, split.getTableNameAsString());
HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTableName());
for (HColumnDescriptor family: parentDescriptor.getFamilies()) {
Path p = Store.getStoreHomedir(tabledir, split.getEncodedName(),
family.getName());
if (!fs.exists(p)) continue;
@ -277,4 +282,10 @@ class CatalogJanitor extends Chore {
}
return result;
}
private HTableDescriptor getTableDescriptor(byte[] tableName) {
return this.services.getAssignmentManager().getTableDescriptor(
Bytes.toString(tableName));
}
}

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerLoad;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
@ -51,6 +52,7 @@ import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.Result;
@ -81,6 +83,7 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Sleeper;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.zookeeper.ClusterId;
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
@ -411,8 +414,7 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
// publish cluster ID
status.setStatus("Publishing Cluster ID in ZooKeeper");
ClusterId.setClusterId(this.zooKeeper,
fileSystemManager.getClusterId());
ClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
this.executorService = new ExecutorService(getServerName().toString());
@ -447,6 +449,10 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
// Make sure root and meta assigned before proceeding.
assignRootAndMeta(status);
// Update meta with new HRI if required. i.e migrate all HRI with HTD to
// HRI with out HTD in meta and update the status in ROOT. This must happen
// before we assign all user regions or else the assignment will fail.
updateMetaWithNewHRI();
// Fixup assignment manager status
status.setStatus("Starting assignment manager");
@ -464,6 +470,44 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
initialized = true;
}
public boolean isMetaHRIUpdated()
throws IOException {
boolean metaUpdated = false;
Get get = new Get(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
get.addColumn(HConstants.CATALOG_FAMILY,
HConstants.META_MIGRATION_QUALIFIER);
Result r =
catalogTracker.waitForRootServerConnectionDefault().get(
HRegionInfo.ROOT_REGIONINFO.getRegionName(), get);
if (r != null && r.getBytes() != null)
{
byte[] metaMigrated = r.getValue(HConstants.CATALOG_FAMILY,
HConstants.META_MIGRATION_QUALIFIER);
String migrated = Bytes.toString(metaMigrated);
metaUpdated = new Boolean(migrated).booleanValue();
} else {
LOG.info("metaUpdated = NULL.");
}
LOG.info("Meta updated status = " + metaUpdated);
return metaUpdated;
}
boolean updateMetaWithNewHRI() throws IOException {
if (!isMetaHRIUpdated()) {
LOG.info("Meta has HRI with HTDs. Updating meta now.");
try {
MetaEditor.updateMetaWithNewRegionInfo(this);
LOG.info("Meta updated with new HRI.");
return true;
} catch (IOException e) {
throw new RuntimeException("Update Meta with nw HRI failed. Master startup aborted.");
}
}
LOG.info("Meta already up-to date with new HRI.");
return true;
}
/**
* Check <code>-ROOT-</code> and <code>.META.</code> are assigned. If not,
* assign them.
@ -850,29 +894,23 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
createTable(desc, splitKeys, false);
}
public void createTable(HTableDescriptor desc, byte [][] splitKeys,
public void createTable(HTableDescriptor hTableDescriptor,
byte [][] splitKeys,
boolean sync)
throws IOException {
if (!isMasterRunning()) {
throw new MasterNotRunningException();
}
String tableName = hTableDescriptor.getNameAsString();
if(MetaReader.tableExists(catalogTracker, tableName)) {
throw new TableExistsException(tableName);
}
if (cpHost != null) {
cpHost.preCreateTable(desc, splitKeys);
}
HRegionInfo [] newRegions = null;
if(splitKeys == null || splitKeys.length == 0) {
newRegions = new HRegionInfo [] { new HRegionInfo(desc, null, null) };
} else {
int numRegions = splitKeys.length + 1;
newRegions = new HRegionInfo[numRegions];
byte [] startKey = null;
byte [] endKey = null;
for(int i=0;i<numRegions;i++) {
endKey = (i == splitKeys.length) ? null : splitKeys[i];
newRegions[i] = new HRegionInfo(desc, startKey, endKey);
startKey = endKey;
}
cpHost.preCreateTable(hTableDescriptor, splitKeys);
}
HRegionInfo [] newRegions = getHRegionInfos(hTableDescriptor, splitKeys);
storeTableDescriptor(hTableDescriptor);
int timeout = conf.getInt("hbase.client.catalog.timeout", 10000);
// Need META availability to create a table
try {
@ -883,13 +921,40 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
LOG.warn("Interrupted waiting for meta availability", e);
throw new IOException(e);
}
createTable(newRegions, sync);
createTable(hTableDescriptor ,newRegions, sync);
}
private synchronized void createTable(final HRegionInfo [] newRegions,
private HRegionInfo[] getHRegionInfos(HTableDescriptor hTableDescriptor,
byte[][] splitKeys) {
HRegionInfo[] hRegionInfos = null;
if (splitKeys == null || splitKeys.length == 0) {
hRegionInfos = new HRegionInfo[]{
new HRegionInfo(hTableDescriptor.getName(), null, null)};
} else {
int numRegions = splitKeys.length + 1;
hRegionInfos = new HRegionInfo[numRegions];
byte[] startKey = null;
byte[] endKey = null;
for (int i = 0; i < numRegions; i++) {
endKey = (i == splitKeys.length) ? null : splitKeys[i];
hRegionInfos[i] =
new HRegionInfo(hTableDescriptor.getName(), startKey, endKey);
startKey = endKey;
}
}
return hRegionInfos;
}
private void storeTableDescriptor(HTableDescriptor hTableDescriptor) {
FSUtils.createTableDescriptor(hTableDescriptor, conf);
//fileSystemManager.createTableDescriptor(hTableDescriptor);
}
private synchronized void createTable(final HTableDescriptor hTableDescriptor,
final HRegionInfo [] newRegions,
final boolean sync)
throws IOException {
String tableName = newRegions[0].getTableDesc().getNameAsString();
String tableName = newRegions[0].getTableNameAsString();
if(MetaReader.tableExists(catalogTracker, tableName)) {
throw new TableExistsException(tableName);
}
@ -904,7 +969,7 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
// 2. Create HRegion
HRegion region = HRegion.createHRegion(newRegion,
fileSystemManager.getRootDir(), conf);
fileSystemManager.getRootDir(), conf, hTableDescriptor);
// 3. Insert into META
MetaEditor.addRegionToMeta(catalogTracker, region.getRegionInfo());
@ -914,9 +979,6 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
region.getLog().closeAndDelete();
}
if (newRegions.length == 1) {
this.assignmentManager.assign(newRegions[0], true);
} else {
// 5. Trigger immediate assignment of the regions in round-robin fashion
List<ServerName> servers = serverManager.getOnlineServersList();
try {
@ -925,7 +987,6 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
LOG.error("Caught " + ie + " during round-robin assignment");
throw new IOException(ie);
}
}
// 6. If sync, wait for assignment of regions
if (sync) {
@ -1040,7 +1101,7 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
if (pair == null) {
return false;
}
if (!Bytes.equals(pair.getFirst().getTableDesc().getName(), tableName)) {
if (!Bytes.equals(pair.getFirst().getTableName(), tableName)) {
return false;
}
result.set(pair);
@ -1295,7 +1356,7 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
}
Pair<HRegionInfo, ServerName> pair =
MetaReader.getRegion(this.catalogTracker, regionName);
if (pair == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName));
HRegionInfo hri = pair.getFirst();
if (force) this.assignmentManager.clearRegionFromTransition(hri);
this.assignmentManager.unassign(hri, force);
@ -1304,6 +1365,39 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
}
}
/**
* Get HTD array for given tables
* @param tableNames
* @return HTableDescriptor[]
*/
public HTableDescriptor[] getHTableDescriptors(List<String> tableNames) {
return this.assignmentManager.getHTableDescriptors(tableNames);
}
/**
* Get all table descriptors
* @return HTableDescriptor[]
*/
public HTableDescriptor[] getHTableDescriptors() {
return this.assignmentManager.getHTableDescriptors();
}
/**
* Get a HTD for a given table name
* @param tableName
* @return HTableDescriptor
*/
/*
public HTableDescriptor getHTableDescriptor(byte[] tableName) {
if (tableName != null && tableName.length > 0) {
return this.assignmentManager.getTableDescriptor(
Bytes.toString(tableName));
}
return null;
}
*/
/**
* Compute the average load across all region servers.
* Currently, this uses a very naive computation - just uses the number of

View File

@ -33,12 +33,16 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Store;
@ -332,13 +336,15 @@ public class MasterFileSystem {
// not make it in first place. Turn off block caching for bootstrap.
// Enable after.
HRegionInfo rootHRI = new HRegionInfo(HRegionInfo.ROOT_REGIONINFO);
setInfoFamilyCaching(rootHRI, false);
setInfoFamilyCachingForRoot(false);
HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
setInfoFamilyCaching(metaHRI, false);
HRegion root = HRegion.createHRegion(rootHRI, rd, c);
HRegion meta = HRegion.createHRegion(metaHRI, rd, c);
setInfoFamilyCaching(rootHRI, true);
setInfoFamilyCaching(metaHRI, true);
setInfoFamilyCachingForMeta(false);
HRegion root = HRegion.createHRegion(rootHRI, rd, c,
HTableDescriptor.ROOT_TABLEDESC);
HRegion meta = HRegion.createHRegion(metaHRI, rd, c,
HTableDescriptor.META_TABLEDESC);
setInfoFamilyCachingForRoot(true);
setInfoFamilyCachingForMeta(true);
// Add first region from the META table to the ROOT region.
HRegion.addRegionToMETA(root, meta);
root.close();
@ -352,12 +358,9 @@ public class MasterFileSystem {
}
}
/**
* @param hri Set all family block caching to <code>b</code>
* @param b
*/
private static void setInfoFamilyCaching(final HRegionInfo hri, final boolean b) {
for (HColumnDescriptor hcd: hri.getTableDesc().families.values()) {
private static void setInfoFamilyCachingForRoot(final boolean b) {
for (HColumnDescriptor hcd:
HTableDescriptor.ROOT_TABLEDESC.families.values()) {
if (Bytes.equals(hcd.getName(), HConstants.CATALOG_FAMILY)) {
hcd.setBlockCacheEnabled(b);
hcd.setInMemory(b);
@ -365,6 +368,17 @@ public class MasterFileSystem {
}
}
private static void setInfoFamilyCachingForMeta(final boolean b) {
for (HColumnDescriptor hcd:
HTableDescriptor.META_TABLEDESC.families.values()) {
if (Bytes.equals(hcd.getName(), HConstants.CATALOG_FAMILY)) {
hcd.setBlockCacheEnabled(b);
hcd.setInMemory(b);
}
}
}
public void deleteRegion(HRegionInfo region) throws IOException {
fs.delete(HRegion.getRegionDir(rootdir, region), true);
}
@ -379,16 +393,126 @@ public class MasterFileSystem {
// @see HRegion.checkRegioninfoOnFilesystem()
}
public void deleteFamily(HRegionInfo region, byte[] familyName)
throws IOException {
fs.delete(Store.getStoreHomedir(
new Path(rootdir, region.getTableDesc().getNameAsString()),
region.getEncodedName(), familyName), true);
}
public void stop() {
if (splitLogManager != null) {
this.splitLogManager.stop();
}
}
/**
* Get table info path for a table.
* @param tableName
* @return Table info path
*/
private Path getTableInfoPath(byte[] tableName) {
Path tablePath = new Path(this.rootdir, Bytes.toString(tableName));
Path tableInfoPath = new Path(tablePath, HConstants.TABLEINFO_NAME);
return tableInfoPath;
}
/**
* Get table info path for a table.
* @param tableName
* @return Table info path
*/
private Path getTablePath(byte[] tableName) {
return new Path(this.rootdir, Bytes.toString(tableName));
}
/**
* Get a HTableDescriptor of a table.
* @param tableName
* @return HTableDescriptor
*/
public HTableDescriptor getTableDescriptor(byte[] tableName) {
try {
return FSUtils.getTableDescriptor(fs, this.rootdir, tableName);
} catch (IOException ioe) {
LOG.info("Exception during readTableDecriptor ", ioe);
}
return null;
}
/**
* Create new HTableDescriptor in HDFS.
* @param htableDescriptor
*/
public void createTableDescriptor(HTableDescriptor htableDescriptor) {
FSUtils.createTableDescriptor(htableDescriptor, conf);
}
/**
* Update a table descriptor.
* @param htableDescriptor
* @return updated HTableDescriptor
* @throws IOException
*/
public HTableDescriptor updateTableDescriptor(HTableDescriptor htableDescriptor)
throws IOException {
LOG.info("Update Table Descriptor. Current HTD = " + htableDescriptor);
FSUtils.updateHTableDescriptor(fs, conf, htableDescriptor);
return htableDescriptor;
}
/**
* Delete column of a table
* @param tableName
* @param familyName
* @return Modified HTableDescriptor with requested column deleted.
* @throws IOException
*/
public HTableDescriptor deleteColumn(byte[] tableName, byte[] familyName)
throws IOException {
LOG.info("DeleteColumn. Table = " + Bytes.toString(tableName)
+ " family = " + Bytes.toString(familyName));
HTableDescriptor htd = getTableDescriptor(tableName);
htd.removeFamily(familyName);
updateTableDescriptor(htd);
return htd;
}
/**
* Modify Column of a table
* @param tableName
* @param hcd HColumnDesciptor
* @return Modified HTableDescriptor with the column modified.
* @throws IOException
*/
public HTableDescriptor modifyColumn(byte[] tableName, HColumnDescriptor hcd)
throws IOException {
LOG.info("AddModifyColumn. Table = " + Bytes.toString(tableName)
+ " HCD = " + hcd.toString());
HTableDescriptor htd = getTableDescriptor(tableName);
byte [] familyName = hcd.getName();
if(!htd.hasFamily(familyName)) {
throw new InvalidFamilyOperationException("Family '" +
Bytes.toString(familyName) + "' doesn't exists so cannot be modified");
}
htd.addFamily(hcd);
updateTableDescriptor(htd);
return htd;
}
/**
* Add column to a table
* @param tableName
* @param hcd
* @return Modified HTableDescriptor with new column added.
* @throws IOException
*/
public HTableDescriptor addColumn(byte[] tableName, HColumnDescriptor hcd)
throws IOException {
LOG.info("AddColumn. Table = " + Bytes.toString(tableName)
+ " HCD = " + hcd.toString());
HTableDescriptor htd = getTableDescriptor(tableName);
if(htd == null) {
throw new InvalidFamilyOperationException("Family '" +
hcd.getNameAsString() + "' cannot be modified as HTD is null");
}
htd.addFamily(hcd);
updateTableDescriptor(htd);
return htd;
}
}

View File

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

View File

@ -70,6 +70,8 @@ public class DeleteTableHandler extends TableEventHandler {
}
// Delete table from FS
this.masterServices.getMasterFileSystem().deleteTable(tableName);
// Update table descriptor cache
am.deleteTableDesc(Bytes.toString(tableName));
// If entry for this table in zk, and up in AssignmentManager, remove it.
// Call to undisableTable does this. TODO: Make a more formal purge table.

View File

@ -26,7 +26,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.util.Bytes;
public class ModifyTableHandler extends TableEventHandler {
private final HTableDescriptor htd;
@ -41,13 +43,19 @@ public class ModifyTableHandler extends TableEventHandler {
@Override
protected void handleTableOperation(List<HRegionInfo> hris)
throws IOException {
for (HRegionInfo hri : hris) {
// Update region info in META
hri.setTableDesc(this.htd);
MetaEditor.updateRegionInfo(this.server.getCatalogTracker(), hri);
// Update region info in FS
this.masterServices.getMasterFileSystem().updateRegionInfo(hri);
AssignmentManager am = this.masterServices.getAssignmentManager();
HTableDescriptor htd = am.getTableDescriptor(Bytes.toString(tableName));
if (htd == null) {
throw new IOException("Modify Table operation could not be completed as " +
"HTableDescritor is missing for table = "
+ Bytes.toString(tableName));
}
// Update table descriptor in HDFS
HTableDescriptor updatedHTD = this.masterServices.getMasterFileSystem()
.updateTableDescriptor(this.htd);
// Update in-memory descriptor cache
am.updateTableDesc(Bytes.toString(tableName), updatedHTD);
}
@Override
public String toString() {

View File

@ -109,7 +109,7 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
" because regions is NOT in RIT -- presuming this is because it SPLIT");
}
if (this.assignmentManager.getZKTable().isDisablingOrDisabledTable(
regionInfo.getTableDesc().getNameAsString())) {
regionInfo.getTableNameAsString())) {
LOG.debug("Opened region " + regionInfo.getRegionNameAsString() + " but "
+ "this table is disabled, triggering close of region");
assignmentManager.unassign(regionInfo);

View File

@ -217,7 +217,7 @@ public class ServerShutdownHandler extends EventHandler {
throws IOException {
// If table is not disabled but the region is offlined,
boolean disabled = assignmentManager.getZKTable().isDisabledTable(
hri.getTableDesc().getNameAsString());
hri.getTableNameAsString());
if (disabled) return false;
if (hri.isOffline() && hri.isSplit()) {
LOG.debug("Offlined and split region " + hri.getRegionNameAsString() +
@ -328,8 +328,8 @@ public class ServerShutdownHandler extends EventHandler {
return true;
}
// Now see if we have gone beyond the daughter's startrow.
if (!Bytes.equals(daughter.getTableDesc().getName(),
hri.getTableDesc().getName())) {
if (!Bytes.equals(daughter.getTableName(),
hri.getTableName())) {
// We fell into another table. Stop scanning.
return false;
}

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.util.Bytes;
@ -47,21 +48,24 @@ public class TableAddFamilyHandler extends TableEventHandler {
@Override
protected void handleTableOperation(List<HRegionInfo> hris)
throws IOException {
HTableDescriptor htd = hris.get(0).getTableDesc();
AssignmentManager am = this.masterServices.getAssignmentManager();
HTableDescriptor htd = am.getTableDescriptor(Bytes.toString(tableName));
byte [] familyName = familyDesc.getName();
if (htd == null) {
throw new IOException("Add Family operation could not be completed as " +
"HTableDescritor is missing for table = "
+ Bytes.toString(tableName));
}
if(htd.hasFamily(familyName)) {
throw new InvalidFamilyOperationException(
"Family '" + Bytes.toString(familyName) + "' already exists so " +
"cannot be added");
}
for(HRegionInfo hri : hris) {
// Update the HTD
hri.getTableDesc().addFamily(familyDesc);
// Update region in META
MetaEditor.updateRegionInfo(this.server.getCatalogTracker(), hri);
// Update region info in FS
this.masterServices.getMasterFileSystem().updateRegionInfo(hri);
}
// Update table descriptor in HDFS
htd = this.masterServices.getMasterFileSystem()
.addColumn(tableName, familyDesc);
// Update in-memory descriptor cache
am.updateTableDesc(Bytes.toString(tableName), htd);
}
@Override
public String toString() {
@ -75,4 +79,5 @@ public class TableAddFamilyHandler extends TableEventHandler {
}
return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" + tableNameStr + "-" + family;
}
}

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.util.Bytes;
@ -46,26 +47,25 @@ public class TableDeleteFamilyHandler extends TableEventHandler {
@Override
protected void handleTableOperation(List<HRegionInfo> hris) throws IOException {
HTableDescriptor htd = hris.get(0).getTableDesc();
AssignmentManager am = this.masterServices.getAssignmentManager();
HTableDescriptor htd = am.getTableDescriptor(Bytes.toString(tableName));
if (htd == null) {
throw new IOException("Add Family operation could not be completed as " +
"HTableDescritor is missing for table = "
+ Bytes.toString(tableName));
}
if(!htd.hasFamily(familyName)) {
throw new InvalidFamilyOperationException(
"Family '" + Bytes.toString(familyName) + "' does not exist so " +
"cannot be deleted");
}
for (HRegionInfo hri : hris) {
// Update the HTD
hri.getTableDesc().removeFamily(familyName);
// Update region in META
MetaEditor.updateRegionInfo(this.server.getCatalogTracker(), hri);
MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
// Update region info in FS
mfs.updateRegionInfo(hri);
// Delete directory in FS
mfs.deleteFamily(hri, familyName);
// Update region info in FS
this.masterServices.getMasterFileSystem().updateRegionInfo(hri);
}
// Update table descriptor in HDFS
htd = this.masterServices.getMasterFileSystem()
.deleteColumn(tableName, familyName);
// Update in-memory descriptor cache
am.updateTableDesc(Bytes.toString(tableName), htd);
}
@Override
public String toString() {
String name = "UnknownServerName";

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.util.Bytes;
@ -47,20 +48,23 @@ public class TableModifyFamilyHandler extends TableEventHandler {
@Override
protected void handleTableOperation(List<HRegionInfo> regions) throws IOException {
HTableDescriptor htd = regions.get(0).getTableDesc();
AssignmentManager am = this.masterServices.getAssignmentManager();
HTableDescriptor htd = am.getTableDescriptor(Bytes.toString(tableName));
byte [] familyName = familyDesc.getName();
if (htd == null) {
throw new IOException("Modify Family operation could not be completed as " +
"HTableDescritor is missing for table = "
+ Bytes.toString(tableName));
}
if(!htd.hasFamily(familyName)) {
throw new InvalidFamilyOperationException("Family '" +
Bytes.toString(familyName) + "' doesn't exists so cannot be modified");
}
for(HRegionInfo hri : regions) {
// Update the HTD
hri.getTableDesc().addFamily(familyDesc);
// Update region in META
MetaEditor.updateRegionInfo(this.server.getCatalogTracker(), hri);
// Update region info in FS
this.masterServices.getMasterFileSystem().updateRegionInfo(hri);
}
// Update table descriptor in HDFS
htd = this.masterServices.getMasterFileSystem()
.modifyColumn(tableName, familyDesc);
// Update in-memory descriptor cache
am.updateTableDesc(Bytes.toString(tableName), htd);
}
@Override
public String toString() {
@ -74,4 +78,5 @@ public class TableModifyFamilyHandler extends TableEventHandler {
}
return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" + tableNameStr + "-" + family;
}
}

View File

@ -0,0 +1,680 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.migration;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.EOFException;
import java.io.IOException;
import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JenkinsHash;
import org.apache.hadoop.hbase.util.MD5Hash;
import org.apache.hadoop.io.VersionedWritable;
import org.apache.hadoop.io.WritableComparable;
/**
* HRegion information.
* Contains HRegion id, start and end keys, a reference to this
* HRegions' table descriptor, etc.
*/
public class HRegionInfo090x extends VersionedWritable implements
WritableComparable<HRegionInfo090x>{
private static final byte VERSION = 0;
private static final Log LOG = LogFactory.getLog(HRegionInfo090x.class);
/**
* The new format for a region name contains its encodedName at the end.
* The encoded name also serves as the directory name for the region
* in the filesystem.
*
* New region name format:
* &lt;tablename>,,&lt;startkey>,&lt;regionIdTimestamp>.&lt;encodedName>.
* where,
* &lt;encodedName> is a hex version of the MD5 hash of
* &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
*
* The old region name format:
* &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
* For region names in the old format, the encoded name is a 32-bit
* JenkinsHash integer value (in its decimal notation, string form).
*<p>
* **NOTE**
*
* ROOT, the first META region, and regions created by an older
* version of HBase (0.20 or prior) will continue to use the
* old region name format.
*/
/** Separator used to demarcate the encodedName in a region name
* in the new format. See description on new format above.
*/
private static final int ENC_SEPARATOR = '.';
public static final int MD5_HEX_LENGTH = 32;
/**
* Does region name contain its encoded name?
* @param regionName region name
* @return boolean indicating if this a new format region
* name which contains its encoded name.
*/
private static boolean hasEncodedName(final byte[] regionName) {
// check if region name ends in ENC_SEPARATOR
if ((regionName.length >= 1)
&& (regionName[regionName.length - 1] == ENC_SEPARATOR)) {
// region name is new format. it contains the encoded name.
return true;
}
return false;
}
/**
* @param regionName
* @return the encodedName
*/
public static String encodeRegionName(final byte [] regionName) {
String encodedName;
if (hasEncodedName(regionName)) {
// region is in new format:
// <tableName>,<startKey>,<regionIdTimeStamp>/encodedName/
encodedName = Bytes.toString(regionName,
regionName.length - MD5_HEX_LENGTH - 1,
MD5_HEX_LENGTH);
} else {
// old format region name. ROOT and first META region also
// use this format.EncodedName is the JenkinsHash value.
int hashVal = Math.abs(JenkinsHash.getInstance().hash(regionName,
regionName.length, 0));
encodedName = String.valueOf(hashVal);
}
return encodedName;
}
/**
* Use logging.
* @param encodedRegionName The encoded regionname.
* @return <code>-ROOT-</code> if passed <code>70236052</code> or
* <code>.META.</code> if passed </code>1028785192</code> else returns
* <code>encodedRegionName</code>
*/
public static String prettyPrint(final String encodedRegionName) {
if (encodedRegionName.equals("70236052")) {
return encodedRegionName + "/-ROOT-";
} else if (encodedRegionName.equals("1028785192")) {
return encodedRegionName + "/.META.";
}
return encodedRegionName;
}
/** delimiter used between portions of a region name */
public static final int DELIMITER = ',';
/** HRegionInfo for root region */
public static final HRegionInfo090x ROOT_REGIONINFO =
new HRegionInfo090x(0L, HTableDescriptor.ROOT_TABLEDESC);
/** HRegionInfo for first meta region */
public static final HRegionInfo090x FIRST_META_REGIONINFO =
new HRegionInfo090x(1L, HTableDescriptor.META_TABLEDESC);
private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
// This flag is in the parent of a split while the parent is still referenced
// by daughter regions. We USED to set this flag when we disabled a table
// but now table state is kept up in zookeeper as of 0.90.0 HBase.
private boolean offLine = false;
private long regionId = -1;
private transient byte [] regionName = HConstants.EMPTY_BYTE_ARRAY;
private String regionNameStr = "";
private boolean split = false;
private byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
protected HTableDescriptor tableDesc = null;
private int hashCode = -1;
//TODO: Move NO_HASH to HStoreFile which is really the only place it is used.
public static final String NO_HASH = null;
private volatile String encodedName = NO_HASH;
private byte [] encodedNameAsBytes = null;
private void setHashCode() {
int result = Arrays.hashCode(this.regionName);
result ^= this.regionId;
result ^= Arrays.hashCode(this.startKey);
result ^= Arrays.hashCode(this.endKey);
result ^= Boolean.valueOf(this.offLine).hashCode();
result ^= this.tableDesc.hashCode();
this.hashCode = result;
}
/**
* Private constructor used constructing HRegionInfo for the catalog root and
* first meta regions
*/
private HRegionInfo090x(long regionId, HTableDescriptor tableDesc) {
super();
this.regionId = regionId;
this.tableDesc = tableDesc;
// Note: Root & First Meta regions names are still in old format
this.regionName = createRegionName(tableDesc.getName(), null,
regionId, false);
this.regionNameStr = Bytes.toStringBinary(this.regionName);
setHashCode();
}
/** Default constructor - creates empty object */
public HRegionInfo090x() {
super();
this.tableDesc = new HTableDescriptor();
}
/**
* Construct HRegionInfo with explicit parameters
*
* @param tableDesc the table descriptor
* @param startKey first key in region
* @param endKey end of key range
* @throws IllegalArgumentException
*/
public HRegionInfo090x(final HTableDescriptor tableDesc, final byte[] startKey,
final byte[] endKey)
throws IllegalArgumentException {
this(tableDesc, startKey, endKey, false);
}
/**
* Construct HRegionInfo with explicit parameters
*
* @param tableDesc the table descriptor
* @param startKey first key in region
* @param endKey end of key range
* @param split true if this region has split and we have daughter regions
* regions that may or may not hold references to this region.
* @throws IllegalArgumentException
*/
public HRegionInfo090x(HTableDescriptor tableDesc, final byte[] startKey,
final byte[] endKey, final boolean split)
throws IllegalArgumentException {
this(tableDesc, startKey, endKey, split, System.currentTimeMillis());
}
/**
* Construct HRegionInfo with explicit parameters
*
* @param tableDesc the table descriptor
* @param startKey first key in region
* @param endKey end of key range
* @param split true if this region has split and we have daughter regions
* regions that may or may not hold references to this region.
* @param regionid Region id to use.
* @throws IllegalArgumentException
*/
public HRegionInfo090x(HTableDescriptor tableDesc, final byte[] startKey,
final byte[] endKey, final boolean split, final long regionid)
throws IllegalArgumentException {
super();
if (tableDesc == null) {
throw new IllegalArgumentException("tableDesc cannot be null");
}
this.offLine = false;
this.regionId = regionid;
this.regionName = createRegionName(tableDesc.getName(), startKey, regionId, true);
this.regionNameStr = Bytes.toStringBinary(this.regionName);
this.split = split;
this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone();
this.startKey = startKey == null?
HConstants.EMPTY_START_ROW: startKey.clone();
this.tableDesc = tableDesc;
setHashCode();
}
/**
* Costruct a copy of another HRegionInfo
*
* @param other
*/
public HRegionInfo090x(HRegionInfo090x other) {
super();
this.endKey = other.getEndKey();
this.offLine = other.isOffline();
this.regionId = other.getRegionId();
this.regionName = other.getRegionName();
this.regionNameStr = Bytes.toStringBinary(this.regionName);
this.split = other.isSplit();
this.startKey = other.getStartKey();
this.tableDesc = other.getTableDesc();
this.hashCode = other.hashCode();
this.encodedName = other.getEncodedName();
}
/**
* Make a region name of passed parameters.
* @param tableName
* @param startKey Can be null
* @param regionid Region id (Usually timestamp from when region was created).
* @param newFormat should we create the region name in the new format
* (such that it contains its encoded name?).
* @return Region name made of passed tableName, startKey and id
*/
public static byte [] createRegionName(final byte [] tableName,
final byte [] startKey, final long regionid, boolean newFormat) {
return createRegionName(tableName, startKey, Long.toString(regionid), newFormat);
}
/**
* Make a region name of passed parameters.
* @param tableName
* @param startKey Can be null
* @param id Region id (Usually timestamp from when region was created).
* @param newFormat should we create the region name in the new format
* (such that it contains its encoded name?).
* @return Region name made of passed tableName, startKey and id
*/
public static byte [] createRegionName(final byte [] tableName,
final byte [] startKey, final String id, boolean newFormat) {
return createRegionName(tableName, startKey, Bytes.toBytes(id), newFormat);
}
/**
* Make a region name of passed parameters.
* @param tableName
* @param startKey Can be null
* @param id Region id (Usually timestamp from when region was created).
* @param newFormat should we create the region name in the new format
* (such that it contains its encoded name?).
* @return Region name made of passed tableName, startKey and id
*/
public static byte [] createRegionName(final byte [] tableName,
final byte [] startKey, final byte [] id, boolean newFormat) {
byte [] b = new byte [tableName.length + 2 + id.length +
(startKey == null? 0: startKey.length) +
(newFormat ? (MD5_HEX_LENGTH + 2) : 0)];
int offset = tableName.length;
System.arraycopy(tableName, 0, b, 0, offset);
b[offset++] = DELIMITER;
if (startKey != null && startKey.length > 0) {
System.arraycopy(startKey, 0, b, offset, startKey.length);
offset += startKey.length;
}
b[offset++] = DELIMITER;
System.arraycopy(id, 0, b, offset, id.length);
offset += id.length;
if (newFormat) {
//
// Encoded name should be built into the region name.
//
// Use the region name thus far (namely, <tablename>,<startKey>,<id>)
// to compute a MD5 hash to be used as the encoded name, and append
// it to the byte buffer.
//
String md5Hash = MD5Hash.getMD5AsHex(b, 0, offset);
byte [] md5HashBytes = Bytes.toBytes(md5Hash);
if (md5HashBytes.length != MD5_HEX_LENGTH) {
LOG.error("MD5-hash length mismatch: Expected=" + MD5_HEX_LENGTH +
"; Got=" + md5HashBytes.length);
}
// now append the bytes '.<encodedName>.' to the end
b[offset++] = ENC_SEPARATOR;
System.arraycopy(md5HashBytes, 0, b, offset, MD5_HEX_LENGTH);
offset += MD5_HEX_LENGTH;
b[offset++] = ENC_SEPARATOR;
}
return b;
}
/**
* Gets the table name from the specified region name.
* @param regionName
* @return Table name.
*/
public static byte [] getTableName(byte [] regionName) {
int offset = -1;
for (int i = 0; i < regionName.length; i++) {
if (regionName[i] == DELIMITER) {
offset = i;
break;
}
}
byte [] tableName = new byte[offset];
System.arraycopy(regionName, 0, tableName, 0, offset);
return tableName;
}
/**
* Separate elements of a regionName.
* @param regionName
* @return Array of byte[] containing tableName, startKey and id
* @throws IOException
*/
public static byte [][] parseRegionName(final byte [] regionName)
throws IOException {
int offset = -1;
for (int i = 0; i < regionName.length; i++) {
if (regionName[i] == DELIMITER) {
offset = i;
break;
}
}
if(offset == -1) throw new IOException("Invalid regionName format");
byte [] tableName = new byte[offset];
System.arraycopy(regionName, 0, tableName, 0, offset);
offset = -1;
for (int i = regionName.length - 1; i > 0; i--) {
if(regionName[i] == DELIMITER) {
offset = i;
break;
}
}
if(offset == -1) throw new IOException("Invalid regionName format");
byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
if(offset != tableName.length + 1) {
startKey = new byte[offset - tableName.length - 1];
System.arraycopy(regionName, tableName.length + 1, startKey, 0,
offset - tableName.length - 1);
}
byte [] id = new byte[regionName.length - offset - 1];
System.arraycopy(regionName, offset + 1, id, 0,
regionName.length - offset - 1);
byte [][] elements = new byte[3][];
elements[0] = tableName;
elements[1] = startKey;
elements[2] = id;
return elements;
}
/** @return the regionId */
public long getRegionId(){
return regionId;
}
/**
* @return the regionName as an array of bytes.
* @see #getRegionNameAsString()
*/
public byte [] getRegionName(){
return regionName;
}
/**
* @return Region name as a String for use in logging, etc.
*/
public String getRegionNameAsString() {
if (hasEncodedName(this.regionName)) {
// new format region names already have their encoded name.
return this.regionNameStr;
}
// old format. regionNameStr doesn't have the region name.
//
//
return this.regionNameStr + "." + this.getEncodedName();
}
/** @return the encoded region name */
public synchronized String getEncodedName() {
if (this.encodedName == NO_HASH) {
this.encodedName = encodeRegionName(this.regionName);
}
return this.encodedName;
}
public synchronized byte [] getEncodedNameAsBytes() {
if (this.encodedNameAsBytes == null) {
this.encodedNameAsBytes = Bytes.toBytes(getEncodedName());
}
return this.encodedNameAsBytes;
}
/** @return the startKey */
public byte [] getStartKey(){
return startKey;
}
/** @return the endKey */
public byte [] getEndKey(){
return endKey;
}
/**
* Returns true if the given inclusive range of rows is fully contained
* by this region. For example, if the region is foo,a,g and this is
* passed ["b","c"] or ["a","c"] it will return true, but if this is passed
* ["b","z"] it will return false.
* @throws IllegalArgumentException if the range passed is invalid (ie end < start)
*/
public boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey) {
if (Bytes.compareTo(rangeStartKey, rangeEndKey) > 0) {
throw new IllegalArgumentException(
"Invalid range: " + Bytes.toStringBinary(rangeStartKey) +
" > " + Bytes.toStringBinary(rangeEndKey));
}
boolean firstKeyInRange = Bytes.compareTo(rangeStartKey, startKey) >= 0;
boolean lastKeyInRange =
Bytes.compareTo(rangeEndKey, endKey) < 0 ||
Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY);
return firstKeyInRange && lastKeyInRange;
}
/**
* Return true if the given row falls in this region.
*/
public boolean containsRow(byte[] row) {
return Bytes.compareTo(row, startKey) >= 0 &&
(Bytes.compareTo(row, endKey) < 0 ||
Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY));
}
/** @return the tableDesc */
public HTableDescriptor getTableDesc(){
return tableDesc;
}
/**
* @param newDesc new table descriptor to use
*/
public void setTableDesc(HTableDescriptor newDesc) {
this.tableDesc = newDesc;
}
/** @return true if this is the root region */
public boolean isRootRegion() {
return this.tableDesc.isRootRegion();
}
/** @return true if this region is from a table that is a meta table,
* either <code>.META.</code> or <code>-ROOT-</code>
*/
public boolean isMetaTable() {
return this.tableDesc.isMetaTable();
}
/** @return true if this region is a meta region */
public boolean isMetaRegion() {
return this.tableDesc.isMetaRegion();
}
/**
* @return True if has been split and has daughters.
*/
public boolean isSplit() {
return this.split;
}
/**
* @param split set split status
*/
public void setSplit(boolean split) {
this.split = split;
}
/**
* @return True if this region is offline.
*/
public boolean isOffline() {
return this.offLine;
}
/**
* The parent of a region split is offline while split daughters hold
* references to the parent. Offlined regions are closed.
* @param offLine Set online/offline status.
*/
public void setOffline(boolean offLine) {
this.offLine = offLine;
}
/**
* @return True if this is a split parent region.
*/
public boolean isSplitParent() {
if (!isSplit()) return false;
if (!isOffline()) {
LOG.warn("Region is split but NOT offline: " + getRegionNameAsString());
}
return true;
}
/**
* @see java.lang.Object#toString()
*/
@Override
public String toString() {
return "REGION => {" + HConstants.NAME + " => '" +
this.regionNameStr +
"', STARTKEY => '" +
Bytes.toStringBinary(this.startKey) + "', ENDKEY => '" +
Bytes.toStringBinary(this.endKey) +
"', ENCODED => " + getEncodedName() + "," +
(isOffline()? " OFFLINE => true,": "") +
(isSplit()? " SPLIT => true,": "") +
" TABLE => {" + this.tableDesc.toString() + "}";
}
/**
* @see java.lang.Object#equals(java.lang.Object)
*/
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null) {
return false;
}
if (!(o instanceof HRegionInfo090x)) {
return false;
}
return this.compareTo((HRegionInfo090x)o) == 0;
}
/**
* @see java.lang.Object#hashCode()
*/
@Override
public int hashCode() {
return this.hashCode;
}
/** @return the object version number */
@Override
public byte getVersion() {
return VERSION;
}
//
// Writable
//
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
Bytes.writeByteArray(out, endKey);
out.writeBoolean(offLine);
out.writeLong(regionId);
Bytes.writeByteArray(out, regionName);
out.writeBoolean(split);
Bytes.writeByteArray(out, startKey);
tableDesc.write(out);
out.writeInt(hashCode);
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
this.endKey = Bytes.readByteArray(in);
this.offLine = in.readBoolean();
this.regionId = in.readLong();
this.regionName = Bytes.readByteArray(in);
this.regionNameStr = Bytes.toStringBinary(this.regionName);
this.split = in.readBoolean();
this.startKey = Bytes.readByteArray(in);
try {
this.tableDesc.readFields(in);
} catch(EOFException eofe) {
throw new IOException("HTD not found in input buffer");
}
this.hashCode = in.readInt();
}
//
// Comparable
//
public int compareTo(HRegionInfo090x o) {
if (o == null) {
return 1;
}
// Are regions of same table?
int result = Bytes.compareTo(this.tableDesc.getName(), o.tableDesc.getName());
if (result != 0) {
return result;
}
// Compare start keys.
result = Bytes.compareTo(this.startKey, o.startKey);
if (result != 0) {
return result;
}
// Compare end keys.
return Bytes.compareTo(this.endKey, o.endKey);
}
/**
* @return Comparator to use comparing {@link org.apache.hadoop.hbase.KeyValue}s.
*/
public KVComparator getComparator() {
return isRootRegion()? KeyValue.ROOT_COMPARATOR: isMetaRegion()?
KeyValue.META_COMPARATOR: KeyValue.COMPARATOR;
}
}

View File

@ -199,6 +199,8 @@ public class HRegion implements HeapSize { // , Writable{
final Path regiondir;
KeyValue.KVComparator comparator;
private Pair<Long,Long> lastCompactInfo = null;
/*
* Data structure of write state flags used coordinating flushes,
* compactions and closes.
@ -239,11 +241,11 @@ public class HRegion implements HeapSize { // , Writable{
final WriteState writestate = new WriteState();
final long memstoreFlushSize;
long memstoreFlushSize;
private volatile long lastFlushTime;
final RegionServerServices rsServices;
private List<Pair<Long, Long>> recentFlushes = new ArrayList<Pair<Long,Long>>();
private final long blockingMemStoreSize;
private long blockingMemStoreSize;
final long threadWakeFrequency;
// Used to guard closes
final ReentrantReadWriteLock lock =
@ -265,6 +267,7 @@ public class HRegion implements HeapSize { // , Writable{
* Name of the region info file that resides just under the region directory.
*/
public final static String REGIONINFO_FILE = ".regioninfo";
private HTableDescriptor htableDescriptor = null;
/**
* Should only be used for testing purposes
@ -319,14 +322,14 @@ public class HRegion implements HeapSize { // , Writable{
10 * 1000);
String encodedNameStr = this.regionInfo.getEncodedName();
this.regiondir = getRegionDir(this.tableDir, encodedNameStr);
long flushSize = regionInfo.getTableDesc().getMemStoreFlushSize();
if (flushSize == HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE) {
flushSize = conf.getLong("hbase.hregion.memstore.flush.size",
HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
try {
LOG.info("Setting table desc from HDFS. Region = "
+ this.regionInfo.getTableNameAsString());
loadHTableDescriptor(tableDir);
LOG.info(" This HTD from HDFS == " + this.htableDescriptor);
} catch (IOException ioe) {
LOG.error("Could not instantiate region as error loading HTableDescriptor");
}
this.memstoreFlushSize = flushSize;
this.blockingMemStoreSize = this.memstoreFlushSize *
conf.getLong("hbase.hregion.memstore.block.multiplier", 2);
// don't initialize coprocessors if not running within a regionserver
// TODO: revisit if coprocessors should load in other cases
if (rsServices != null) {
@ -338,6 +341,40 @@ public class HRegion implements HeapSize { // , Writable{
}
}
private void loadHTableDescriptor(Path tableDir) throws IOException {
LOG.debug("Assigning tabledesc from .tableinfo for region = "
+ this.regionInfo.getRegionNameAsString());
// load HTableDescriptor
this.htableDescriptor = FSUtils.getTableDescriptor(tableDir, fs);
if (this.htableDescriptor != null) {
setHTableSpecificConf();
} else {
throw new IOException("Table description missing in " +
".tableinfo. Cannot create new region."
+ " current region is == " + this.regionInfo.toString());
}
}
private void setHTableSpecificConf() {
if (this.htableDescriptor != null) {
LOG.info("Setting up tabledescriptor config now ...");
long flushSize = this.htableDescriptor.getMemStoreFlushSize();
if (flushSize == HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE) {
flushSize = conf.getLong("hbase.hregion.memstore.flush.size",
HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
}
this.memstoreFlushSize = flushSize;
this.blockingMemStoreSize = this.memstoreFlushSize *
conf.getLong("hbase.hregion.memstore.block.multiplier", 2);
}
}
public void setHtableDescriptor(HTableDescriptor htableDescriptor) {
this.htableDescriptor = htableDescriptor;
}
/**
* Initialize this region.
* @return What the next sequence (edit) id should be.
@ -378,7 +415,7 @@ public class HRegion implements HeapSize { // , Writable{
// Load in all the HStores. Get maximum seqid.
long maxSeqId = -1;
for (HColumnDescriptor c : this.regionInfo.getTableDesc().getFamilies()) {
for (HColumnDescriptor c : this.htableDescriptor.getFamilies()) {
status.setStatus("Instantiating store for column family " + c);
Store store = instantiateHStore(this.tableDir, c);
this.stores.put(c.getName(), store);
@ -398,7 +435,7 @@ public class HRegion implements HeapSize { // , Writable{
SplitTransaction.cleanupAnySplitDetritus(this);
FSUtils.deleteDirectory(this.fs, new Path(regiondir, MERGEDIR));
this.writestate.setReadOnly(this.regionInfo.getTableDesc().isReadOnly());
this.writestate.setReadOnly(this.htableDescriptor.isReadOnly());
this.writestate.compacting = 0;
this.lastFlushTime = EnvironmentEdgeManager.currentTimeMillis();
@ -703,7 +740,7 @@ public class HRegion implements HeapSize { // , Writable{
/** @return HTableDescriptor for this region */
public HTableDescriptor getTableDesc() {
return this.regionInfo.getTableDesc();
return this.htableDescriptor;
}
/** @return HLog in use for this region */
@ -1156,7 +1193,7 @@ public class HRegion implements HeapSize { // , Writable{
// log-sequence-ids can be safely ignored.
if (wal != null) {
wal.completeCacheFlush(this.regionInfo.getEncodedNameAsBytes(),
regionInfo.getTableDesc().getName(), completeSequenceId,
regionInfo.getTableName(), completeSequenceId,
this.getRegionInfo().isMetaRegion());
}
@ -1268,7 +1305,7 @@ public class HRegion implements HeapSize { // , Writable{
void prepareScanner(Scan scan) throws IOException {
if(!scan.hasFamilies()) {
// Adding all families to scanner
for(byte[] family: regionInfo.getTableDesc().getFamiliesKeys()){
for(byte[] family: this.htableDescriptor.getFamiliesKeys()){
scan.addFamily(family);
}
}
@ -1303,7 +1340,7 @@ public class HRegion implements HeapSize { // , Writable{
private void prepareDelete(Delete delete) throws IOException {
// Check to see if this is a deleteRow insert
if(delete.getFamilyMap().isEmpty()){
for(byte [] family : regionInfo.getTableDesc().getFamiliesKeys()){
for(byte [] family : this.htableDescriptor.getFamiliesKeys()){
// Don't eat the timestamp
delete.deleteFamily(family, delete.getTimeStamp());
}
@ -1424,8 +1461,8 @@ public class HRegion implements HeapSize { // , Writable{
// single WALEdit.
WALEdit walEdit = new WALEdit();
addFamilyMapToWALEdit(familyMap, walEdit);
this.log.append(regionInfo, regionInfo.getTableDesc().getName(),
walEdit, now);
this.log.append(regionInfo, this.htableDescriptor.getName(),
walEdit, now, this.htableDescriptor);
}
// Now make changes to the memstore.
@ -1683,8 +1720,8 @@ public class HRegion implements HeapSize { // , Writable{
}
// Append the edit to WAL
this.log.append(regionInfo, regionInfo.getTableDesc().getName(),
walEdit, now);
this.log.append(regionInfo, this.htableDescriptor.getName(),
walEdit, now, this.htableDescriptor);
// ------------------------------------
// STEP 4. Write back to memstore
@ -1938,8 +1975,8 @@ public class HRegion implements HeapSize { // , Writable{
if (writeToWAL) {
WALEdit walEdit = new WALEdit();
addFamilyMapToWALEdit(familyMap, walEdit);
this.log.append(regionInfo, regionInfo.getTableDesc().getName(),
walEdit, now);
this.log.append(regionInfo, this.htableDescriptor.getName(),
walEdit, now, this.htableDescriptor);
}
long addedSize = applyFamilyMapToMemstore(familyMap);
@ -2757,18 +2794,25 @@ public class HRegion implements HeapSize { // , Writable{
* @param info Info for region to create.
* @param rootDir Root directory for HBase instance
* @param conf
* @param hTableDescriptor
* @return new HRegion
*
* @throws IOException
*/
public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
final Configuration conf)
final Configuration conf,
final HTableDescriptor hTableDescriptor)
throws IOException {
LOG.info("creating HRegion " + info.getTableNameAsString()
+ " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
" Table name == " + info.getTableNameAsString());
Path tableDir =
HTableDescriptor.getTableDir(rootDir, info.getTableDesc().getName());
HTableDescriptor.getTableDir(rootDir, info.getTableName());
Path regionDir = HRegion.getRegionDir(tableDir, info.getEncodedName());
FileSystem fs = FileSystem.get(conf);
fs.mkdirs(regionDir);
FSUtils.createTableDescriptor(fs, hTableDescriptor, tableDir);
HRegion region = HRegion.newHRegion(tableDir,
new HLog(fs, new Path(regionDir, HConstants.HREGION_LOGDIR_NAME),
new Path(regionDir, HConstants.HREGION_OLDLOGDIR_NAME), conf),
@ -2820,12 +2864,52 @@ public class HRegion implements HeapSize { // , Writable{
throw new NullPointerException("Passed region info is null");
}
Path dir = HTableDescriptor.getTableDir(FSUtils.getRootDir(conf),
info.getTableDesc().getName());
info.getTableName());
HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info,
rsServices);
return r.openHRegion(reporter);
}
public static HRegion openHRegion(Path tableDir, final HRegionInfo info,
final HLog wal, final Configuration conf)
throws IOException {
return openHRegion(tableDir, info, wal, conf, null, null);
}
/**
* Open a Region.
* @param tableDir Table directory
* @param info Info for region to be opened.
* @param wal HLog for region to use. This method will call
* HLog#setSequenceNumber(long) passing the result of the call to
* HRegion#getMinSequenceId() to ensure the log id is properly kept
* up. HRegionStore does this every time it opens a new region.
* @param conf
* @param reporter An interface we can report progress against.
* @return new HRegion
*
* @throws IOException
*/
public static HRegion openHRegion(final Path tableDir, final HRegionInfo info,
final HLog wal, final Configuration conf,
final RegionServerServices rsServices,
final CancelableProgressable reporter)
throws IOException {
LOG.info("HRegion.openHRegion Region name ==" + info.getRegionNameAsString());
if (LOG.isDebugEnabled()) {
LOG.debug("Opening region: " + info);
}
if (info == null) {
throw new NullPointerException("Passed region info is null");
}
Path dir = HTableDescriptor.getTableDir(tableDir,
info.getTableName());
HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info,
rsServices);
return r.openHRegion(reporter);
}
/**
* Open HRegion.
* Calls initialize and sets sequenceid.
@ -2845,7 +2929,7 @@ public class HRegion implements HeapSize { // , Writable{
}
private void checkCompressionCodecs() throws IOException {
for (HColumnDescriptor fam: regionInfo.getTableDesc().getColumnFamilies()) {
for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
CompressionTest.testCompression(fam.getCompression());
CompressionTest.testCompression(fam.getCompactionCompression());
}
@ -2873,6 +2957,11 @@ public class HRegion implements HeapSize { // , Writable{
HConstants.REGIONINFO_QUALIFIER,
EnvironmentEdgeManager.currentTimeMillis(),
Writables.getBytes(r.getRegionInfo())));
edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
HConstants.META_MIGRATION_QUALIFIER,
EnvironmentEdgeManager.currentTimeMillis(),
Bytes.toBytes("true")));
meta.put(HConstants.CATALOG_FAMILY, edits);
} finally {
meta.releaseRowLock(lid);
@ -2911,7 +3000,7 @@ public class HRegion implements HeapSize { // , Writable{
*/
public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
return new Path(
HTableDescriptor.getTableDir(rootdir, info.getTableDesc().getName()),
HTableDescriptor.getTableDir(rootdir, info.getTableName()),
info.getEncodedName());
}
@ -2989,8 +3078,8 @@ public class HRegion implements HeapSize { // , Writable{
* @throws IOException
*/
public static HRegion merge(HRegion a, HRegion b) throws IOException {
if (!a.getRegionInfo().getTableDesc().getNameAsString().equals(
b.getRegionInfo().getTableDesc().getNameAsString())) {
if (!a.getRegionInfo().getTableNameAsString().equals(
b.getRegionInfo().getTableNameAsString())) {
throw new IOException("Regions do not belong to the same table");
}
@ -3043,7 +3132,8 @@ public class HRegion implements HeapSize { // , Writable{
? b.getEndKey()
: a.getEndKey());
HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey);
HRegionInfo newRegionInfo =
new HRegionInfo(tabledesc.getName(), startKey, endKey);
LOG.info("Creating new region " + newRegionInfo.toString());
String encodedName = newRegionInfo.getEncodedName();
Path newRegionDir = HRegion.getRegionDir(a.getTableDir(), encodedName);
@ -3182,7 +3272,7 @@ public class HRegion implements HeapSize { // , Writable{
checkFamily(family);
}
} else { // Adding all families to scanner
for (byte[] family: regionInfo.getTableDesc().getFamiliesKeys()) {
for (byte[] family: this.htableDescriptor.getFamiliesKeys()) {
get.addFamily(family);
}
}
@ -3388,8 +3478,8 @@ public class HRegion implements HeapSize { // , Writable{
// Actually write to WAL now
if (writeToWAL) {
this.log.append(regionInfo, regionInfo.getTableDesc().getName(),
walEdits, now);
this.log.append(regionInfo, this.htableDescriptor.getName(),
walEdits, now, this.htableDescriptor);
}
size = this.addAndGetGlobalMemstoreSize(size);
@ -3459,8 +3549,8 @@ public class HRegion implements HeapSize { // , Writable{
long now = EnvironmentEdgeManager.currentTimeMillis();
WALEdit walEdit = new WALEdit();
walEdit.add(newKv);
this.log.append(regionInfo, regionInfo.getTableDesc().getName(),
walEdit, now);
this.log.append(regionInfo, this.htableDescriptor.getName(),
walEdit, now, this.htableDescriptor);
}
// Now request the ICV to the store, this will set the timestamp
@ -3493,19 +3583,17 @@ public class HRegion implements HeapSize { // , Writable{
private void checkFamily(final byte [] family)
throws NoSuchColumnFamilyException {
if(!regionInfo.getTableDesc().hasFamily(family)) {
if (!this.htableDescriptor.hasFamily(family)) {
throw new NoSuchColumnFamilyException("Column family " +
Bytes.toStringBinary(family) + " does not exist in region " + this
+ " in table " + regionInfo.getTableDesc());
Bytes.toString(family) + " does not exist in region " + this
+ " in table " + this.htableDescriptor);
}
}
public static final long FIXED_OVERHEAD = ClassSize.align(
ClassSize.OBJECT + // this
(4 * Bytes.SIZEOF_LONG) + // memstoreFlushSize, lastFlushTime, blockingMemStoreSize, threadWakeFrequency
Bytes.SIZEOF_BOOLEAN + // splitRequest
ClassSize.ARRAY + // splitPoint
(26 * ClassSize.REFERENCE));
(4 * Bytes.SIZEOF_LONG) + ClassSize.ARRAY +
ClassSize.align(28 * ClassSize.REFERENCE) + ClassSize.OBJECT +
ClassSize.align(Bytes.SIZEOF_INT));
public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
ClassSize.OBJECT + // closeLock

View File

@ -1348,6 +1348,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
public void postOpenDeployTasks(final HRegion r, final CatalogTracker ct,
final boolean daughter)
throws KeeperException, IOException {
LOG.info("HRS.PostOpenDeployTasks");
// Do checks to see if we need to compact (references or too many files)
for (Store s : r.getStores().values()) {
if (s.hasReferences() || s.needsCompaction()) {
@ -1357,24 +1358,36 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
// Add to online regions if all above was successful.
addToOnlineRegions(r);
LOG.info("addToOnlineRegions is done" + r.getRegionInfo());
// Update ZK, ROOT or META
if (r.getRegionInfo().isRootRegion()) {
LOG.info("setRootLocation");
RootLocationEditor.setRootLocation(getZooKeeper(),
this.serverNameFromMasterPOV);
} else if (r.getRegionInfo().isMetaRegion()) {
LOG.info("updateMetaLocation");
MetaEditor.updateMetaLocation(ct, r.getRegionInfo(),
this.serverNameFromMasterPOV);
} else {
LOG.info("updateMetaLocation 111");
if (daughter) {
LOG.info("updateMetaLocation 22");
// If daughter of a split, update whole row, not just location.
MetaEditor.addDaughter(ct, r.getRegionInfo(),
this.serverNameFromMasterPOV);
} else {
LOG.info("updateMetaLocation 33");
MetaEditor.updateRegionLocation(ct, r.getRegionInfo(),
this.serverNameFromMasterPOV);
}
}
LOG.info("END HRS.PostOpenDeployTasks");
}
/**

View File

@ -22,10 +22,7 @@ package org.apache.hadoop.hbase.regionserver;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@ -165,6 +162,12 @@ class LogRoller extends Thread implements WALObserver {
// Not interested.
}
@Override
public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey,
WALEdit logEdit) {
//Not interested
}
@Override
public void logCloseRequested() {
// not interested

View File

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

View File

@ -39,11 +39,7 @@ 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.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
@ -195,9 +191,15 @@ public class Store implements HeapSize {
// Check if this is in-memory store
this.inMemory = family.isInMemory();
long maxFileSize = 0L;
HTableDescriptor hTableDescriptor = region.getTableDesc();
if (hTableDescriptor != null) {
maxFileSize = hTableDescriptor.getMaxFileSize();
} else {
maxFileSize = HConstants.DEFAULT_MAX_FILE_SIZE;
}
// By default we split region if a file > HConstants.DEFAULT_MAX_FILE_SIZE.
long maxFileSize = info.getTableDesc().getMaxFileSize();
if (maxFileSize == HConstants.DEFAULT_MAX_FILE_SIZE) {
maxFileSize = conf.getLong("hbase.hregion.max.filesize",
HConstants.DEFAULT_MAX_FILE_SIZE);

View File

@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.executor.EventHandler;
@ -71,9 +72,7 @@ public class OpenRegionHandler extends EventHandler {
public void process() throws IOException {
try {
final String name = regionInfo.getRegionNameAsString();
LOG.debug("Processing open of " + name);
if (this.server.isStopped() || this.rsServices.isStopping()) {
LOG.info("Server stopping or stopped, skipping open of " + name);
return;
}
final String encodedName = regionInfo.getEncodedName();
@ -182,6 +181,7 @@ public class OpenRegionHandler extends EventHandler {
Thread.currentThread().interrupt();
}
}
// Was there an exception opening the region? This should trigger on
// InterruptedException too. If so, we failed.
return !t.interrupted() && t.getException() == null;
@ -261,6 +261,33 @@ public class OpenRegionHandler extends EventHandler {
return result;
}
/**
* @return Instance of HRegion if successful open else null.
*/
HRegion openRegion(Path tableDir) {
HRegion region = null;
try {
// Instantiate the region. This also periodically tickles our zk OPENING
// state so master doesn't timeout this region in transition.
region = HRegion.openHRegion(tableDir, this.regionInfo, this.rsServices.getWAL(),
this.server.getConfiguration(), this.rsServices,
new CancelableProgressable() {
public boolean progress() {
// We may lose the znode ownership during the open. Currently its
// too hard interrupting ongoing region open. Just let it complete
// and check we still have the znode after region open.
return tickleOpening("open_region_progress");
}
});
} catch (IOException e) {
// We failed open. Let our znode expire in regions-in-transition and
// Master will assign elsewhere. Presumes nothing to close.
LOG.error("Failed open of region=" +
this.regionInfo.getRegionNameAsString(), e);
}
return region;
}
/**
* @return Instance of HRegion if successful open else null.
*/

View File

@ -54,11 +54,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.FSUtils;
@ -832,22 +828,6 @@ public class HLog implements Syncable {
}
}
/** Append an entry to the log.
*
* @param regionInfo
* @param logEdit
* @param now Time of this edit write.
* @throws IOException
*/
public void append(HRegionInfo regionInfo, WALEdit logEdit,
final long now,
final boolean isMetaRegion)
throws IOException {
byte [] regionName = regionInfo.getEncodedNameAsBytes();
byte [] tableName = regionInfo.getTableDesc().getName();
this.append(regionInfo, makeKey(regionName, tableName, -1, now), logEdit);
}
/**
* @param now
* @param regionName
@ -867,7 +847,8 @@ public class HLog implements Syncable {
* @param logKey
* @throws IOException
*/
public void append(HRegionInfo regionInfo, HLogKey logKey, WALEdit logEdit)
public void append(HRegionInfo regionInfo, HLogKey logKey, WALEdit logEdit,
HTableDescriptor htd)
throws IOException {
if (this.closed) {
throw new IOException("Cannot append; log is closed");
@ -882,14 +863,14 @@ public class HLog implements Syncable {
// is greater than or equal to the value in lastSeqWritten.
this.lastSeqWritten.putIfAbsent(regionInfo.getEncodedNameAsBytes(),
Long.valueOf(seqNum));
doWrite(regionInfo, logKey, logEdit);
doWrite(regionInfo, logKey, logEdit, htd);
this.numEntries.incrementAndGet();
}
// Sync if catalog region, and if not then check if that table supports
// deferred log flushing
if (regionInfo.isMetaRegion() ||
!regionInfo.getTableDesc().isDeferredLogFlush()) {
!htd.isDeferredLogFlush()) {
// sync txn to file system
this.sync();
}
@ -919,7 +900,7 @@ public class HLog implements Syncable {
* @throws IOException
*/
public void append(HRegionInfo info, byte [] tableName, WALEdit edits,
final long now)
final long now, HTableDescriptor htd)
throws IOException {
if (edits.isEmpty()) return;
if (this.closed) {
@ -937,13 +918,13 @@ public class HLog implements Syncable {
byte [] hriKey = info.getEncodedNameAsBytes();
this.lastSeqWritten.putIfAbsent(hriKey, seqNum);
HLogKey logKey = makeKey(hriKey, tableName, seqNum, now);
doWrite(info, logKey, edits);
doWrite(info, logKey, edits, htd);
this.numEntries.incrementAndGet();
}
// Sync if catalog region, and if not then check if that table supports
// deferred log flushing
if (info.isMetaRegion() ||
!info.getTableDesc().isDeferredLogFlush()) {
!htd.isDeferredLogFlush()) {
// sync txn to file system
this.sync();
}
@ -1077,14 +1058,15 @@ public class HLog implements Syncable {
}
}
protected void doWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit)
protected void doWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit,
HTableDescriptor htd)
throws IOException {
if (!this.enabled) {
return;
}
if (!this.listeners.isEmpty()) {
for (WALObserver i: this.listeners) {
i.visitLogEntryBeforeWrite(info, logKey, logEdit);
i.visitLogEntryBeforeWrite(htd, logKey, logEdit);
}
}
try {
@ -1115,6 +1097,7 @@ public class HLog implements Syncable {
}
}
/** @return How many items have been added to the log */
int getNumEntries() {
return numEntries.get();

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.regionserver.wal;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
/**
* Get notification of {@link HLog}/WAL log events. The invocations are inline
@ -51,4 +52,14 @@ public interface WALObserver {
*/
public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey,
WALEdit logEdit);
/**
*
* @param htd
* @param logKey
* @param logEdit
*/
public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey,
WALEdit logEdit);
}

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
@ -132,12 +133,18 @@ public class Replication implements WALObserver {
@Override
public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey,
WALEdit logEdit) {
// Not interested
}
@Override
public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey,
WALEdit logEdit) {
NavigableMap<byte[], Integer> scopes =
new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
byte[] family;
for (KeyValue kv : logEdit.getKeyValues()) {
family = kv.getFamily();
int scope = info.getTableDesc().getFamily(family).getScope();
int scope = htd.getFamily(family).getScope();
if (scope != REPLICATION_SCOPE_LOCAL &&
!scopes.containsKey(family)) {
scopes.put(family, scope);

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -833,4 +834,225 @@ public class FSUtils {
}
LOG.info("Finished lease recover attempt for " + p);
}
public static Map<String, HTableDescriptor> getTableDescriptors(
final Configuration config)
throws IOException {
Path path = getRootDir(config);
// since HMaster.getFileSystem() is package private
FileSystem fs = path.getFileSystem(config);
return getTableDescriptors(fs, path);
}
public static Map<String, HTableDescriptor> getTableDescriptors(
final FileSystem fs, final Path hbaseRootDir)
throws IOException {
Map<String, HTableDescriptor> desc =
new HashMap<String, HTableDescriptor>();
DirFilter df = new DirFilter(fs);
// presumes any directory under hbase.rootdir is a table
FileStatus [] tableDirs = fs.listStatus(hbaseRootDir, df);
for (FileStatus tableDir : tableDirs) {
Path d = tableDir.getPath();
String tableName = d.getName();
if (tableName.equals(HConstants.HREGION_LOGDIR_NAME)
|| tableName.equals(Bytes.toString(HConstants.ROOT_TABLE_NAME))
|| tableName.equals(Bytes.toString(HConstants.META_TABLE_NAME))
|| tableName.equals(HConstants.HREGION_OLDLOGDIR_NAME)
) {
continue;
}
LOG.info("Adding tabledescriptor for table = " + tableName);
HTableDescriptor htd = readTableDescriptor(fs, hbaseRootDir,
tableName);
if (htd != null) {
if (!desc.containsKey(tableName)) {
desc.put(tableName, htd);
}
}
}
return desc;
}
private static Path getTableInfoPath(Path hbaseRootDir, String tableName) {
Path tablePath = new Path(hbaseRootDir, tableName);
return new Path(tablePath, HConstants.TABLEINFO_NAME);
}
/**
* Get table info path for a table.
* @param tableName
* @return Table info path
*/
private static Path getTableInfoPath(byte[] tableName, Configuration conf) throws IOException {
Path tablePath = new Path(getRootDir(conf), Bytes.toString(tableName));
Path tableInfoPath = new Path(tablePath, HConstants.TABLEINFO_NAME);
return tableInfoPath;
}
private static Path getTablePath(byte[] tableName, Configuration conf) throws IOException {
return new Path(getRootDir(conf), Bytes.toString(tableName));
}
private static FileSystem getCurrentFileSystem(Configuration conf) throws IOException {
return getRootDir(conf).getFileSystem(conf);
}
/**
* Get HTableDescriptor
* @param config
* @param tableName
* @return HTableDescriptor for table
* @throws IOException
*/
public static HTableDescriptor getHTableDescriptor(Configuration config,
String tableName)
throws IOException {
Path path = getRootDir(config);
FileSystem fs = path.getFileSystem(config);
return readTableDescriptor(fs, path, tableName);
}
private static HTableDescriptor readTableDescriptor(FileSystem fs,
Path hbaseRootDir,
String tableName) {
try {
FSDataInputStream fsDataInputStream =
fs.open(getTableInfoPath(hbaseRootDir, tableName));
HTableDescriptor hTableDescriptor = new HTableDescriptor();
hTableDescriptor.readFields(fsDataInputStream);
fsDataInputStream.close();
return hTableDescriptor;
} catch (IOException ioe) {
LOG.info("Exception during readTableDecriptor. Current table name = " + tableName , ioe);
}
return null;
}
/**
* Get HTD from HDFS.
* @param fs
* @param hbaseRootDir
* @param tableName
* @return
* @throws IOException
*/
public static HTableDescriptor getTableDescriptor(FileSystem fs,
Path hbaseRootDir,
byte[] tableName)
throws IOException {
return readTableDescriptor(fs, hbaseRootDir, Bytes.toString(tableName));
}
public static HTableDescriptor getTableDescriptor(Path tableDir, FileSystem fs) {
try {
LOG.info("Reading table descriptor from .tableinfo. current path = "
+ tableDir);
if (tableDir == null) {
LOG.info("Reading table descriptor from .tableinfo current tablename is NULL ");
return null;
}
FSDataInputStream fsDataInputStream =
fs.open(new Path(tableDir, HConstants.TABLEINFO_NAME));
HTableDescriptor hTableDescriptor = new HTableDescriptor();
hTableDescriptor.readFields(fsDataInputStream);
LOG.info("Current tabledescriptor from .tableinfo is " + hTableDescriptor.toString());
fsDataInputStream.close();
return hTableDescriptor;
} catch (IOException ioe) {
LOG.info("Exception during getTableDescriptor ", ioe);
}
return null;
}
/**
* Create new HTableDescriptor in HDFS.
* @param htableDescriptor
*/
public static void createTableDescriptor(HTableDescriptor htableDescriptor,
Configuration conf) {
try {
Path tableDir = getTablePath(htableDescriptor.getName(), conf);
FileSystem fs = getCurrentFileSystem(conf);
createTableDescriptor(fs, htableDescriptor, tableDir);
} catch(IOException ioe) {
LOG.info("IOException while trying to create tableInfo in HDFS", ioe);
}
}
public static void createTableDescriptor(FileSystem fs,
HTableDescriptor htableDescriptor,
Path tableDir) {
try {
Path tableInfoPath = new Path(tableDir, HConstants.TABLEINFO_NAME);
LOG.info("Current tableInfoPath = " + tableInfoPath
+ " tableDir = " + tableDir) ;
if (fs.exists(tableInfoPath) &&
fs.getFileStatus(tableInfoPath).getLen() > 0) {
LOG.info("TableInfo already exists.. Skipping creation");
return;
}
writeTableDescriptor(fs, htableDescriptor, tableDir);
} catch(IOException ioe) {
LOG.info("IOException while trying to create tableInfo in HDFS", ioe);
}
}
private static void writeTableDescriptor(FileSystem fs,
HTableDescriptor hTableDescriptor,
Path tableDir) throws IOException {
// Create in tmpdir and then move into place in case we crash after
// create but before close. If we don't successfully close the file,
// subsequent region reopens will fail the below because create is
// registered in NN.
Path tableInfoPath = new Path(tableDir, HConstants.TABLEINFO_NAME);
Path tmpPath = new Path(new Path(tableDir,".tmp"),
HConstants.TABLEINFO_NAME);
LOG.info("TableInfoPath = " + tableInfoPath + " tmpPath = " + tmpPath);
FSDataOutputStream out = fs.create(tmpPath, true);
try {
hTableDescriptor.write(out);
out.write('\n');
out.write('\n');
out.write(Bytes.toBytes(hTableDescriptor.toString()));
} finally {
out.close();
}
if (!fs.rename(tmpPath, tableInfoPath)) {
throw new IOException("Unable to rename " + tmpPath + " to " +
tableInfoPath);
} else {
LOG.info("TableDescriptor stored. TableInfoPath = " + tableInfoPath);
}
}
public static void updateHTableDescriptor(FileSystem fs,
Configuration conf,
HTableDescriptor hTableDescriptor) throws IOException
{
Path tableInfoPath = getTableInfoPath(hTableDescriptor.getName(), conf);
FSDataOutputStream out = fs.create(tableInfoPath, true);
try {
hTableDescriptor.write(out);
out.write('\n');
out.write('\n');
out.write(Bytes.toBytes(hTableDescriptor.toString()));
LOG.info("updateHTableDescriptor. Updated tableinfo in HDFS under "
+ tableInfoPath + " For HTD => "
+ hTableDescriptor.toString());
} finally {
out.close();
}
}
private static Path getTmpDir(HTableDescriptor htableDescriptor, Configuration configuration)
throws IOException {
return new Path(getTablePath(htableDescriptor.getName(), configuration), ".tmp");
}
}

View File

@ -156,11 +156,13 @@ public class HBaseFsck {
// get a list of all tables that have not changed recently.
AtomicInteger numSkipped = new AtomicInteger(0);
HTableDescriptor[] allTables = getTables(numSkipped);
errors.print("Number of Tables: " + allTables.length);
errors.print("Number of Tables: " +
(allTables == null ? 0 : allTables.length));
if (details) {
if (numSkipped.get() > 0) {
errors.detail("Number of Tables in flux: " + numSkipped.get());
}
if (allTables != null && allTables.length > 0) {
for (HTableDescriptor td : allTables) {
String tableName = td.getNameAsString();
errors.detail(" Table: " + tableName + "\t" +
@ -169,6 +171,9 @@ public class HBaseFsck {
(td.isMetaRegion() ? "META" : " ")) + "\t" +
" families: " + td.getFamilies().size());
}
}
}
// From the master, get a list of all known live region servers
@ -255,7 +260,7 @@ public class HBaseFsck {
* @throws KeeperException
*/
private boolean isTableDisabled(HRegionInfo regionInfo) {
return disabledTables.contains(regionInfo.getTableDesc().getName());
return disabledTables.contains(regionInfo.getTableName());
}
/**
@ -521,7 +526,7 @@ public class HBaseFsck {
if (hbi.deployedOn.size() == 0) continue;
// We should be safe here
String tableName = hbi.metaEntry.getTableDesc().getNameAsString();
String tableName = hbi.metaEntry.getTableNameAsString();
TInfo modTInfo = tablesInfo.get(tableName);
if (modTInfo == null) {
modTInfo = new TInfo(tableName);
@ -653,7 +658,7 @@ public class HBaseFsck {
* @throws IOException if an error is encountered
*/
HTableDescriptor[] getTables(AtomicInteger numSkipped) {
TreeSet<HTableDescriptor> uniqueTables = new TreeSet<HTableDescriptor>();
List<String> tableNames = new ArrayList<String>();
long now = System.currentTimeMillis();
for (HbckInfo hbi : regionInfo.values()) {
@ -663,15 +668,27 @@ public class HBaseFsck {
// 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) {
uniqueTables.add(info.getTableDesc());
tableNames.add(info.getTableNameAsString());
} else {
numSkipped.incrementAndGet(); // one more in-flux table
}
}
}
return uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]);
return getHTableDescriptors(tableNames);
}
HTableDescriptor[] getHTableDescriptors(List<String> tableNames) {
HTableDescriptor[] htd = null;
try {
LOG.info("getHTableDescriptors == tableNames => " + tableNames);
htd = new HBaseAdmin(conf).getTableDescriptors(tableNames);
} catch (IOException e) {
LOG.debug("Exception getting table descriptors", e);
}
return htd;
}
/**
* Gets the entry in regionInfo corresponding to the the given encoded
* region name. If the region has not been seen yet, a new entry is added

View File

@ -261,7 +261,7 @@ class HMerge {
Bytes.toString(HConstants.REGIONINFO_QUALIFIER));
}
HRegionInfo region = Writables.getHRegionInfo(regionInfoValue);
if (!Bytes.equals(region.getTableDesc().getName(), this.tableName)) {
if (!Bytes.equals(region.getTableName(), this.tableName)) {
return null;
}
return region;

View File

@ -321,78 +321,6 @@ public class MetaUtils {
t.delete(delete);
}
/**
* Offline version of the online TableOperation,
* org.apache.hadoop.hbase.master.AddColumn.
* @param tableName table name
* @param hcd Add this column to <code>tableName</code>
* @throws IOException e
*/
public void addColumn(final byte [] tableName,
final HColumnDescriptor hcd)
throws IOException {
List<HRegionInfo> metas = getMETARows(tableName);
for (HRegionInfo hri: metas) {
final HRegion m = getMetaRegion(hri);
scanMetaRegion(m, new ScannerListener() {
private boolean inTable = true;
@SuppressWarnings("synthetic-access")
public boolean processRow(HRegionInfo info) throws IOException {
LOG.debug("Testing " + Bytes.toString(tableName) + " against " +
Bytes.toString(info.getTableDesc().getName()));
if (Bytes.equals(info.getTableDesc().getName(), tableName)) {
this.inTable = false;
info.getTableDesc().addFamily(hcd);
updateMETARegionInfo(m, info);
return true;
}
// If we got here and we have not yet encountered the table yet,
// inTable will be false. Otherwise, we've passed out the table.
// Stop the scanner.
return this.inTable;
}});
}
}
/**
* Offline version of the online TableOperation,
* org.apache.hadoop.hbase.master.DeleteColumn.
* @param tableName table name
* @param columnFamily Name of column name to remove.
* @throws IOException e
*/
public void deleteColumn(final byte [] tableName,
final byte [] columnFamily) throws IOException {
List<HRegionInfo> metas = getMETARows(tableName);
for (HRegionInfo hri: metas) {
final HRegion m = getMetaRegion(hri);
scanMetaRegion(m, new ScannerListener() {
private boolean inTable = true;
@SuppressWarnings("synthetic-access")
public boolean processRow(HRegionInfo info) throws IOException {
if (Bytes.equals(info.getTableDesc().getName(), tableName)) {
this.inTable = false;
info.getTableDesc().removeFamily(columnFamily);
updateMETARegionInfo(m, info);
Path tabledir = new Path(rootdir,
info.getTableDesc().getNameAsString());
Path p = Store.getStoreHomedir(tabledir, info.getEncodedName(),
columnFamily);
if (!fs.delete(p, true)) {
LOG.warn("Failed delete of " + p);
}
return false;
}
// If we got here and we have not yet encountered the table yet,
// inTable will be false. Otherwise, we've passed out the table.
// Stop the scanner.
return this.inTable;
}});
}
}
/**
* Update COL_REGIONINFO in meta region r with HRegionInfo hri
*
@ -466,7 +394,7 @@ public class MetaUtils {
public boolean processRow(HRegionInfo info) throws IOException {
SL_LOG.debug("Testing " + info);
if (Bytes.equals(info.getTableDesc().getName(),
if (Bytes.equals(info.getTableName(),
HConstants.META_TABLE_NAME)) {
result.add(info);
return false;

View File

@ -560,8 +560,9 @@ public class RegionSplitter {
if (sk.length == 0)
sk = splitAlgo.firstRow();
String startKey = splitAlgo.rowToStr(sk);
HTableDescriptor htd = table.getTableDescriptor();
// check every Column Family for that region
for (HColumnDescriptor c : hri.getTableDesc().getFamilies()) {
for (HColumnDescriptor c : htd.getFamilies()) {
Path cfDir = Store.getStoreHomedir(tableDir, hri.getEncodedName(),
c.getName());
if (fs.exists(cfDir)) {

View File

@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.util;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.migration.HRegionInfo090x;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.Writable;
@ -62,7 +63,7 @@ public class Writables {
/**
* Put a bunch of Writables as bytes all into the one byte array.
* @param w writable
* @param ws writable
* @return The bytes of <code>w</code> gotten by running its
* {@link Writable#write(java.io.DataOutput)} method.
* @throws IOException e
@ -215,4 +216,16 @@ public class Writables {
}
return tgt;
}
/**
* Get HREgionInfoForMigration serialized from bytes.
* @param bytes serialized bytes
* @return HRegionInfoForMigration
* @throws IOException
*/
public static HRegionInfo090x getHRegionInfoForMigration(final byte [] bytes)
throws IOException {
return (HRegionInfo090x)getWritable(bytes, new HRegionInfo090x());
}
}

View File

@ -57,9 +57,15 @@ public abstract class HBaseTestCase extends TestCase {
/** configuration parameter name for test directory */
public static final String TEST_DIRECTORY_KEY = "test.build.data";
/*
protected final static byte [] fam1 = Bytes.toBytes("colfamily1");
protected final static byte [] fam2 = Bytes.toBytes("colfamily2");
protected final static byte [] fam3 = Bytes.toBytes("colfamily3");
*/
protected final static byte [] fam1 = Bytes.toBytes("colfamily11");
protected final static byte [] fam2 = Bytes.toBytes("colfamily21");
protected final static byte [] fam3 = Bytes.toBytes("colfamily31");
protected static final byte [][] COLUMNS = {fam1, fam2, fam3};
private boolean localfs = false;
@ -159,9 +165,8 @@ public abstract class HBaseTestCase extends TestCase {
Path rootdir = filesystem.makeQualified(
new Path(conf.get(HConstants.HBASE_DIR)));
filesystem.mkdirs(rootdir);
return HRegion.createHRegion(new HRegionInfo(desc, startKey, endKey),
rootdir, conf);
HRegionInfo hri = new HRegionInfo(desc.getName(), startKey, endKey);
return HRegion.createHRegion(hri, rootdir, conf, desc);
}
protected HRegion openClosedRegion(final HRegion closedRegion)
@ -653,9 +658,10 @@ public abstract class HBaseTestCase extends TestCase {
}
protected void createRootAndMetaRegions() throws IOException {
root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, testDir, conf);
root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, testDir,
conf, HTableDescriptor.ROOT_TABLEDESC);
meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO, testDir,
conf);
conf, HTableDescriptor.META_TABLEDESC);
HRegion.addRegionToMETA(root, meta);
}

View File

@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.migration.HRegionInfo090x;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
@ -831,7 +832,7 @@ public class HBaseTestingUtility {
int count = 0;
for (int i = 0; i < startKeys.length; i++) {
int j = (i + 1) % startKeys.length;
HRegionInfo hri = new HRegionInfo(table.getTableDescriptor(),
HRegionInfo hri = new HRegionInfo(table.getTableName(),
startKeys[i], startKeys[j]);
Put put = new Put(hri.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
@ -859,6 +860,65 @@ public class HBaseTestingUtility {
return count;
}
public int createMultiRegionsWithLegacyHRI(final Configuration c,
final HTableDescriptor htd,
final byte[] columnFamily, byte [][] startKeys)
throws IOException {
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
HTable meta = new HTable(c, HConstants.META_TABLE_NAME);
if(!htd.hasFamily(columnFamily)) {
HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
htd.addFamily(hcd);
}
List<HRegionInfo090x> newRegions
= new ArrayList<HRegionInfo090x>(startKeys.length);
int count = 0;
for (int i = 0; i < startKeys.length; i++) {
int j = (i + 1) % startKeys.length;
HRegionInfo090x hri = new HRegionInfo090x(htd,
startKeys[i], startKeys[j]);
Put put = new Put(hri.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
meta.put(put);
LOG.info("createMultiRegions: PUT inserted " + hri.toString());
newRegions.add(hri);
count++;
}
return count;
}
public int createMultiRegionsWithNewHRI(final Configuration c, final HTableDescriptor htd,
final byte[] columnFamily, byte [][] startKeys)
throws IOException {
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
HTable meta = new HTable(c, HConstants.META_TABLE_NAME);
if(!htd.hasFamily(columnFamily)) {
HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
htd.addFamily(hcd);
}
List<HRegionInfo> newRegions
= new ArrayList<HRegionInfo>(startKeys.length);
int count = 0;
for (int i = 0; i < startKeys.length; i++) {
int j = (i + 1) % startKeys.length;
HRegionInfo hri = new HRegionInfo(htd.getName(),
startKeys[i], startKeys[j]);
Put put = new Put(hri.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
meta.put(put);
LOG.info("createMultiRegions: PUT inserted " + hri.toString());
newRegions.add(hri);
count++;
}
return count;
}
/**
* Create rows in META for regions of the specified table with the specified
* start keys. The first startKey should be a 0 length byte array if you
@ -879,7 +939,8 @@ public class HBaseTestingUtility {
int count = 0;
for (int i = 0; i < startKeys.length; i++) {
int j = (i + 1) % startKeys.length;
HRegionInfo hri = new HRegionInfo(htd, startKeys[i], startKeys[j]);
HRegionInfo hri = new HRegionInfo(htd.getName(), startKeys[i],
startKeys[j]);
Put put = new Put(hri.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
@ -923,8 +984,7 @@ public class HBaseTestingUtility {
for (Result result : s) {
HRegionInfo info = Writables.getHRegionInfo(
result.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER));
HTableDescriptor desc = info.getTableDesc();
if (Bytes.compareTo(desc.getName(), tableName) == 0) {
if (Bytes.compareTo(info.getTableName(), tableName) == 0) {
LOG.info("getMetaTableRows: row -> " +
Bytes.toStringBinary(result.getRow()));
rows.add(result.getRow());

View File

@ -31,25 +31,25 @@ public class TestCompare extends TestCase {
* Sort of HRegionInfo.
*/
public void testHRegionInfo() {
HRegionInfo a = new HRegionInfo(new HTableDescriptor("a"), null, null);
HRegionInfo b = new HRegionInfo(new HTableDescriptor("b"), null, null);
HRegionInfo a = new HRegionInfo(Bytes.toBytes("a"), null, null);
HRegionInfo b = new HRegionInfo(Bytes.toBytes("b"), null, null);
assertTrue(a.compareTo(b) != 0);
HTableDescriptor t = new HTableDescriptor("t");
byte [] midway = Bytes.toBytes("midway");
a = new HRegionInfo(t, null, midway);
b = new HRegionInfo(t, midway, null);
a = new HRegionInfo(t.getName(), null, midway);
b = new HRegionInfo(t.getName(), midway, null);
assertTrue(a.compareTo(b) < 0);
assertTrue(b.compareTo(a) > 0);
assertEquals(a, a);
assertTrue(a.compareTo(a) == 0);
a = new HRegionInfo(t, Bytes.toBytes("a"), Bytes.toBytes("d"));
b = new HRegionInfo(t, Bytes.toBytes("e"), Bytes.toBytes("g"));
a = new HRegionInfo(t.getName(), Bytes.toBytes("a"), Bytes.toBytes("d"));
b = new HRegionInfo(t.getName(), Bytes.toBytes("e"), Bytes.toBytes("g"));
assertTrue(a.compareTo(b) < 0);
a = new HRegionInfo(t, Bytes.toBytes("aaaa"), Bytes.toBytes("dddd"));
b = new HRegionInfo(t, Bytes.toBytes("e"), Bytes.toBytes("g"));
a = new HRegionInfo(t.getName(), Bytes.toBytes("aaaa"), Bytes.toBytes("dddd"));
b = new HRegionInfo(t.getName(), Bytes.toBytes("e"), Bytes.toBytes("g"));
assertTrue(a.compareTo(b) < 0);
a = new HRegionInfo(t, Bytes.toBytes("aaaa"), Bytes.toBytes("dddd"));
b = new HRegionInfo(t, Bytes.toBytes("aaaa"), Bytes.toBytes("eeee"));
a = new HRegionInfo(t.getName(), Bytes.toBytes("aaaa"), Bytes.toBytes("dddd"));
b = new HRegionInfo(t.getName(), Bytes.toBytes("aaaa"), Bytes.toBytes("eeee"));
assertTrue(a.compareTo(b) < 0);
}
}

View File

@ -57,10 +57,10 @@ public class TestScanMultipleVersions extends HBaseClusterTestCase {
this.desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
// Region 0 will contain the key range [,row_0500)
INFOS[0] = new HRegionInfo(this.desc, HConstants.EMPTY_START_ROW,
INFOS[0] = new HRegionInfo(desc.getName(), HConstants.EMPTY_START_ROW,
Bytes.toBytes("row_0500"));
// Region 1 will contain the key range [row_0500,)
INFOS[1] = new HRegionInfo(this.desc, Bytes.toBytes("row_0500"),
INFOS[1] = new HRegionInfo(desc.getName(), Bytes.toBytes("row_0500"),
HConstants.EMPTY_END_ROW);
// Create root and meta regions
@ -68,7 +68,8 @@ public class TestScanMultipleVersions extends HBaseClusterTestCase {
// Create the regions
for (int i = 0; i < REGIONS.length; i++) {
REGIONS[i] =
HRegion.createHRegion(this.INFOS[i], this.testDir, this.conf);
HRegion.createHRegion(this.INFOS[i], this.testDir, this.conf,
this.desc);
// Insert data
for (int j = 0; j < TIMESTAMPS.length; j++) {
Put put = new Put(ROWS[i], TIMESTAMPS[j], null);

View File

@ -103,8 +103,8 @@ public class TestSerialization {
HRegionInfo deserializedHri =
(HRegionInfo)Writables.getWritable(hrib, new HRegionInfo());
assertEquals(hri.getEncodedName(), deserializedHri.getEncodedName());
assertEquals(hri.getTableDesc().getFamilies().size(),
deserializedHri.getTableDesc().getFamilies().size());
//assertEquals(hri.getTableDesc().getFamilies().size(),
// deserializedHri.getTableDesc().getFamilies().size());
}
@Test public void testRegionInfos() throws Exception {
@ -126,7 +126,7 @@ public class TestSerialization {
for (int i = 0; i < families.length; i++) {
htd.addFamily(new HColumnDescriptor(families[i]));
}
return new HRegionInfo(htd, HConstants.EMPTY_START_ROW,
return new HRegionInfo(htd.getName(), HConstants.EMPTY_START_ROW,
HConstants.EMPTY_END_ROW);
}

View File

@ -36,7 +36,7 @@ public class TimestampTestBase extends HBaseTestCase {
private static final long T1 = 100L;
private static final long T2 = 200L;
private static final byte [] FAMILY_NAME = Bytes.toBytes("colfamily1");
private static final byte [] FAMILY_NAME = Bytes.toBytes("colfamily11");
private static final byte [] QUALIFIER_NAME = Bytes.toBytes("contents");
private static final byte [] ROW = Bytes.toBytes("row");

View File

@ -181,7 +181,7 @@ public class TestFromClientSide {
for (Map.Entry<HRegionInfo, HServerAddress> e: loadedRegions.entrySet()) {
HRegionInfo hri = e.getKey();
assertTrue(HConnectionManager.isRegionCached(conf,
hri.getTableDesc().getName(), hri.getStartKey()));
hri.getTableName(), hri.getStartKey()));
}
// delete the temp file

View File

@ -0,0 +1,155 @@
/**
* Copyright 2010 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import junit.framework.AssertionFailedError;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.migration.HRegionInfo090x;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.apache.hadoop.hbase.util.Writables;
import java.util.List;
public class TestMetaMigration {
final Log LOG = LogFactory.getLog(getClass());
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static MiniHBaseCluster miniHBaseCluster = null;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
miniHBaseCluster = TEST_UTIL.startMiniCluster(1);
}
/**
* @throws java.lang.Exception
*/
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Test
public void testHRegionInfoForMigration() throws Exception {
LOG.info("Starting testHRegionInfoForMigration");
HTableDescriptor htd = new HTableDescriptor("testMetaMigration");
htd.addFamily(new HColumnDescriptor("family"));
HRegionInfo090x hrim = new HRegionInfo090x(htd, HConstants.EMPTY_START_ROW,
HConstants.EMPTY_END_ROW);
LOG.info("INFO 1 = " + hrim);
byte[] bytes = Writables.getBytes(hrim);
LOG.info(" BYtes.toString = " + Bytes.toString(bytes));
LOG.info(" HTD bytes = " + Bytes.toString(Writables.getBytes(hrim.getTableDesc())));
HRegionInfo090x info = Writables.getHRegionInfoForMigration(bytes);
LOG.info("info = " + info);
LOG.info("END testHRegionInfoForMigration");
}
@Test
public void testMetaUpdatedFlagInROOT() throws Exception {
LOG.info("Starting testMetaUpdatedFlagInROOT");
boolean metaUpdated = miniHBaseCluster.getMaster().isMetaHRIUpdated();
assertEquals(true, metaUpdated);
LOG.info("END testMetaUpdatedFlagInROOT");
}
@Test
public void testUpdatesOnMetaWithLegacyHRI() throws Exception {
LOG.info("Starting testMetaWithLegacyHRI");
final byte[] FAMILY = Bytes.toBytes("family");
HTableDescriptor htd = new HTableDescriptor("testMetaMigration");
HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
htd.addFamily(hcd);
Configuration conf = TEST_UTIL.getConfiguration();
TEST_UTIL.createMultiRegionsWithLegacyHRI(conf, htd, FAMILY,
new byte[][]{
HConstants.EMPTY_START_ROW,
Bytes.toBytes("region_a"),
Bytes.toBytes("region_b")});
CatalogTracker ct = miniHBaseCluster.getMaster().getCatalogTracker();
// just for this test set it to false.
MetaEditor.updateRootWithMetaMigrationStatus(ct, false);
MetaReader.fullScanMetaAndPrint(ct);
LOG.info("MEta Print completed.testUpdatesOnMetaWithLegacyHRI");
List<HTableDescriptor> htds = MetaEditor.updateMetaWithNewRegionInfo(
TEST_UTIL.getHBaseCluster().getMaster());
assertEquals(3, htds.size());
// Assert that the flag in ROOT is updated to reflect the correct status
boolean metaUpdated = miniHBaseCluster.getMaster().isMetaHRIUpdated();
assertEquals(true, metaUpdated);
LOG.info("END testMetaWithLegacyHRI");
}
//@Test
public void dtestUpdatesOnMetaWithNewHRI() throws Exception {
LOG.info("Starting testMetaWithLegacyHRI");
final byte[] FAMILY = Bytes.toBytes("family");
HTableDescriptor htd = new HTableDescriptor("testMetaMigration");
HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
htd.addFamily(hcd);
Configuration conf = TEST_UTIL.getConfiguration();
TEST_UTIL.createMultiRegionsWithNewHRI(conf, htd, FAMILY,
new byte[][]{
HConstants.EMPTY_START_ROW,
Bytes.toBytes("region_a"),
Bytes.toBytes("region_b")});
List<HTableDescriptor> htds = MetaEditor.updateMetaWithNewRegionInfo(
TEST_UTIL.getHBaseCluster().getMaster());
assertEquals(3, htds.size());
}
public static void assertEquals(int expected,
int actual) {
if (expected != actual) {
throw new AssertionFailedError("expected:<" +
expected + "> but was:<" +
actual + ">");
}
}
public static void assertEquals(boolean expected,
boolean actual) {
if (expected != actual) {
throw new AssertionFailedError("expected:<" +
expected + "> but was:<" +
actual + ">");
}
}
}

View File

@ -118,6 +118,7 @@ public class TestMultipleTimestamps {
@Test
public void testReseeksWithMultipleColumnOneTimestamp() throws IOException {
LOG.info("testReseeksWithMultipleColumnOneTimestamp");
byte [] TABLE = Bytes.toBytes("testReseeksWithMultiple" +
"ColumnOneTimestamps");
byte [] FAMILY = Bytes.toBytes("event_log");
@ -155,6 +156,8 @@ public class TestMultipleTimestamps {
@Test
public void testReseeksWithMultipleColumnMultipleTimestamp() throws
IOException {
LOG.info("testReseeksWithMultipleColumnMultipleTimestamp");
byte [] TABLE = Bytes.toBytes("testReseeksWithMultiple" +
"ColumnMiltipleTimestamps");
byte [] FAMILY = Bytes.toBytes("event_log");
@ -197,6 +200,7 @@ public class TestMultipleTimestamps {
@Test
public void testReseeksWithMultipleFiles() throws IOException {
LOG.info("testReseeksWithMultipleFiles");
byte [] TABLE = Bytes.toBytes("testReseeksWithMultipleFiles");
byte [] FAMILY = Bytes.toBytes("event_log");
byte [][] FAMILIES = new byte[][] { FAMILY };
@ -262,8 +266,12 @@ public class TestMultipleTimestamps {
}
public void testWithVersionDeletes(boolean flushTables) throws IOException {
LOG.info("testWithVersionDeletes_"+
(flushTables ? "flush" : "noflush"));
byte [] TABLE = Bytes.toBytes("testWithVersionDeletes_" +
(flushTables ? "flush" : "noflush"));
byte [] FAMILY = Bytes.toBytes("event_log");
byte [][] FAMILIES = new byte[][] { FAMILY };
@ -292,6 +300,8 @@ public class TestMultipleTimestamps {
@Test
public void testWithMultipleVersionDeletes() throws IOException {
LOG.info("testWithMultipleVersionDeletes");
byte [] TABLE = Bytes.toBytes("testWithMultipleVersionDeletes");
byte [] FAMILY = Bytes.toBytes("event_log");
byte [][] FAMILIES = new byte[][] { FAMILY };

View File

@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
@ -89,6 +90,7 @@ public class TestScannerTimeout {
*/
@Test
public void test2481() throws Exception {
LOG.info("START ************ test2481");
Scan scan = new Scan();
HTable table =
new HTable(new Configuration(TEST_UTIL.getConfiguration()), TABLE_NAME);
@ -109,6 +111,7 @@ public class TestScannerTimeout {
return;
}
fail("We should be timing out");
LOG.info("END ************ test2481");
}
/**
@ -118,6 +121,7 @@ public class TestScannerTimeout {
*/
@Test
public void test2772() throws Exception {
LOG.info("START************ test2772");
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME);
Scan scan = new Scan();
// Set a very high timeout, we want to test what happens when a RS
@ -134,6 +138,8 @@ public class TestScannerTimeout {
Result[] results = r.next(NB_ROWS);
assertEquals(NB_ROWS, results.length);
r.close();
LOG.info("END ************ test2772");
}
/**
@ -143,14 +149,24 @@ public class TestScannerTimeout {
*/
@Test
public void test3686a() throws Exception {
LOG.info("START ************ TEST3686A---1");
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME);
LOG.info("START ************ TEST3686A---1111");
Scan scan = new Scan();
scan.setCaching(SCANNER_CACHING);
LOG.info("************ TEST3686A");
MetaReader.fullScanMetaAndPrint(TEST_UTIL.getHBaseCluster().getMaster().getCatalogTracker());
HTable table = new HTable(TABLE_NAME);
LOG.info("START ************ TEST3686A---22");
ResultScanner r = table.getScanner(scan);
LOG.info("START ************ TEST3686A---33");
int count = 1;
r.next();
LOG.info("START ************ TEST3686A---44");
// Kill after one call to next(), which got 5 rows.
rs.abort("die!");
while(r.next() != null) {
@ -158,6 +174,7 @@ public class TestScannerTimeout {
}
assertEquals(NB_ROWS, count);
r.close();
LOG.info("************ END TEST3686A");
}
/**
@ -168,6 +185,7 @@ public class TestScannerTimeout {
*/
@Test
public void test3686b() throws Exception {
LOG.info("START ************ test3686b");
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME);
Scan scan = new Scan();
scan.setCaching(SCANNER_CACHING);
@ -189,5 +207,7 @@ public class TestScannerTimeout {
}
assertEquals(NB_ROWS, count);
r.close();
LOG.info("END ************ END test3686b");
}
}

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.TimestampTestBase;
* run against an HRegion and against an HTable: i.e. both local and remote.
*/
public class TestTimestamp extends HBaseClusterTestCase {
public static String COLUMN_NAME = "colfamily1";
public static String COLUMN_NAME = "colfamily11";
/** constructor */
public TestTimestamp() {

View File

@ -180,7 +180,8 @@ public class TestCoprocessorInterface extends HBaseTestCase {
HRegion reopenRegion(final HRegion closedRegion, Class<?> implClass)
throws IOException {
HRegion r = new HRegion(closedRegion.getRegionDir(), closedRegion.getLog(),
//HRegionInfo info = new HRegionInfo(tableName, null, null, false);
HRegion r = new HRegion(closedRegion.getTableDir(), closedRegion.getLog(),
closedRegion.getFilesystem(), closedRegion.getConf(),
closedRegion.getRegionInfo(), null);
r.initialize();
@ -211,9 +212,9 @@ public class TestCoprocessorInterface extends HBaseTestCase {
for(byte [] family : families) {
htd.addFamily(new HColumnDescriptor(family));
}
HRegionInfo info = new HRegionInfo(htd, null, null, false);
HRegionInfo info = new HRegionInfo(tableName, null, null, false);
Path path = new Path(DIR + callingMethod);
HRegion r = HRegion.createHRegion(info, path, conf);
HRegion r = HRegion.createHRegion(info, path, conf, htd);
// this following piece is a hack.
RegionCoprocessorHost host = new RegionCoprocessorHost(r, null, conf);

View File

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

View File

@ -91,9 +91,9 @@ public class TestRegionObserverStacking extends TestCase {
for(byte [] family : families) {
htd.addFamily(new HColumnDescriptor(family));
}
HRegionInfo info = new HRegionInfo(htd, null, null, false);
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
Path path = new Path(DIR + callingMethod);
HRegion r = HRegion.createHRegion(info, path, conf);
HRegion r = HRegion.createHRegion(info, path, conf, htd);
// this following piece is a hack. currently a coprocessorHost
// is secretly loaded at OpenRegionHandler. we don't really
// start a region server here, so just manually create cphost

View File

@ -138,7 +138,12 @@ public class TestWALObserver {
*/
@Test
public void testWALObserverWriteToWAL() throws Exception {
HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(TEST_TABLE));
final HTableDescriptor htd = createBasic3FamilyHTD(Bytes.toString(TEST_TABLE));
HRegion region2 = HRegion.createHRegion(hri,
hbaseRootDir, this.conf, htd);
Path basedir = new Path(this.hbaseRootDir, Bytes.toString(TEST_TABLE));
deleteDir(basedir);
fs.mkdirs(new Path(basedir, hri.getEncodedName()));
@ -190,7 +195,7 @@ public class TestWALObserver {
// it's where WAL write cp should occur.
long now = EnvironmentEdgeManager.currentTimeMillis();
log.append(hri, hri.getTableDesc().getName(), edit, now);
log.append(hri, hri.getTableName(), edit, now, htd);
// the edit shall have been change now by the coprocessor.
foundFamily0 = false;
@ -221,16 +226,25 @@ public class TestWALObserver {
* Test WAL replay behavior with WALObserver.
*/
@Test
public void testWALObserverReplay() throws Exception {
public void testWALCoprocessorReplay() throws Exception {
// WAL replay is handled at HRegion::replayRecoveredEdits(), which is
// ultimately called by HRegion::initialize()
byte[] tableName = Bytes.toBytes("testWALCoprocessorReplay");
final HTableDescriptor htd = getBasic3FamilyHTableDescriptor(Bytes.toString(tableName));
//final HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(tableName));
//final HRegionInfo hri1 = createBasic3FamilyHRegionInfo(Bytes.toString(tableName));
final HRegionInfo hri = new HRegionInfo(tableName, null, null);
final HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(tableName));
final Path basedir = new Path(this.hbaseRootDir, Bytes.toString(tableName));
deleteDir(basedir);
fs.mkdirs(new Path(basedir, hri.getEncodedName()));
final Configuration newConf = HBaseConfiguration.create(this.conf);
HRegion region2 = HRegion.createHRegion(hri,
hbaseRootDir, newConf,htd);
//HLog wal = new HLog(this.fs, this.dir, this.oldLogDir, this.conf);
HLog wal = createWAL(this.conf);
//Put p = creatPutWith2Families(TEST_ROW);
@ -238,40 +252,46 @@ public class TestWALObserver {
long now = EnvironmentEdgeManager.currentTimeMillis();
//addFamilyMapToWALEdit(p.getFamilyMap(), edit);
final int countPerFamily = 1000;
for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
//for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
for (HColumnDescriptor hcd: htd.getFamilies()) {
//addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily,
//EnvironmentEdgeManager.getDelegate(), wal);
addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily,
EnvironmentEdgeManager.getDelegate(), wal);
EnvironmentEdgeManager.getDelegate(), wal, htd);
}
wal.append(hri, tableName, edit, now);
wal.append(hri, tableName, edit, now, htd);
// sync to fs.
wal.sync();
final Configuration newConf = HBaseConfiguration.create(this.conf);
User user = HBaseTestingUtility.getDifferentUser(newConf,
".replay.wal.secondtime");
user.runAs(new PrivilegedExceptionAction() {
public Object run() throws Exception {
runWALSplit(newConf);
Path p = runWALSplit(newConf);
LOG.info("WALSplit path == " + p);
FileSystem newFS = FileSystem.get(newConf);
// Make a new wal for new region open.
HLog wal2 = createWAL(newConf);
HRegion region2 = new HRegion(basedir, wal2, FileSystem.get(newConf),
Path tableDir =
HTableDescriptor.getTableDir(hbaseRootDir, hri.getTableName());
HRegion region = new HRegion(tableDir, wal2, FileSystem.get(newConf),
newConf, hri, TEST_UTIL.getHBaseCluster().getRegionServer(0));
long seqid2 = region2.initialize();
long seqid2 = region.initialize();
SampleRegionWALObserver cp2 =
(SampleRegionWALObserver)region2.getCoprocessorHost().findCoprocessor(
(SampleRegionWALObserver)region.getCoprocessorHost().findCoprocessor(
SampleRegionWALObserver.class.getName());
// TODO: asserting here is problematic.
assertNotNull(cp2);
assertTrue(cp2.isPreWALRestoreCalled());
assertTrue(cp2.isPostWALRestoreCalled());
region2.close();
region.close();
wal2.closeAndDelete();
return null;
}
});
}
/**
* Test to see CP loaded successfully or not. There is a duplication
* at TestHLog, but the purpose of that one is to see whether the loaded
@ -301,7 +321,7 @@ public class TestWALObserver {
HColumnDescriptor a = new HColumnDescriptor(TEST_FAMILY[i]);
htd.addFamily(a);
}
return new HRegionInfo(htd, null, null, false);
return new HRegionInfo(htd.getName(), null, null, false);
}
/*
@ -356,7 +376,7 @@ public class TestWALObserver {
}
private void addWALEdits (final byte [] tableName, final HRegionInfo hri,
final byte [] rowName, final byte [] family,
final int count, EnvironmentEdge ee, final HLog wal)
final int count, EnvironmentEdge ee, final HLog wal, final HTableDescriptor htd)
throws IOException {
String familyStr = Bytes.toString(family);
for (int j = 0; j < count; j++) {
@ -365,8 +385,30 @@ public class TestWALObserver {
WALEdit edit = new WALEdit();
edit.add(new KeyValue(rowName, family, qualifierBytes,
ee.currentTimeMillis(), columnBytes));
wal.append(hri, tableName, edit, ee.currentTimeMillis());
wal.append(hri, tableName, edit, ee.currentTimeMillis(), htd);
}
}
private HTableDescriptor getBasic3FamilyHTableDescriptor(
final String tableName) {
HTableDescriptor htd = new HTableDescriptor(tableName);
for (int i = 0; i < TEST_FAMILY.length; i++ ) {
HColumnDescriptor a = new HColumnDescriptor(TEST_FAMILY[i]);
htd.addFamily(a);
}
return htd;
}
private HTableDescriptor createBasic3FamilyHTD(final String tableName) {
HTableDescriptor htd = new HTableDescriptor(tableName);
HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
htd.addFamily(a);
HColumnDescriptor b = new HColumnDescriptor(Bytes.toBytes("b"));
htd.addFamily(b);
HColumnDescriptor c = new HColumnDescriptor(Bytes.toBytes("c"));
htd.addFamily(c);
return htd;
}
}

View File

@ -33,9 +33,9 @@ public class TestColumnPrefixFilter {
String family = "Family";
HTableDescriptor htd = new HTableDescriptor("TestColumnPrefixFilter");
htd.addFamily(new HColumnDescriptor(family));
HRegionInfo info = new HRegionInfo(htd, null, null, false);
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
HRegion region = HRegion.createHRegion(info, HBaseTestingUtility.
getTestDir(), TEST_UTIL.getConfiguration());
getTestDir(), TEST_UTIL.getConfiguration(), htd);
List<String> rows = generateRandomWords(100, "row");
List<String> columns = generateRandomWords(10000, "column");
@ -89,9 +89,9 @@ public class TestColumnPrefixFilter {
String family = "Family";
HTableDescriptor htd = new HTableDescriptor("TestColumnPrefixFilter");
htd.addFamily(new HColumnDescriptor(family));
HRegionInfo info = new HRegionInfo(htd, null, null, false);
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
HRegion region = HRegion.createHRegion(info, HBaseTestingUtility.
getTestDir(), TEST_UTIL.getConfiguration());
getTestDir(), TEST_UTIL.getConfiguration(), htd);
List<String> rows = generateRandomWords(100, "row");
List<String> columns = generateRandomWords(10000, "column");

View File

@ -75,8 +75,9 @@ public class TestDependentColumnFilter extends TestCase {
HTableDescriptor htd = new HTableDescriptor(getName());
htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
htd.addFamily(new HColumnDescriptor(FAMILIES[1]));
HRegionInfo info = new HRegionInfo(htd, null, null, false);
this.region = HRegion.createHRegion(info, testUtil.getTestDir(), testUtil.getConfiguration());
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
this.region = HRegion.createHRegion(info, testUtil.getTestDir(),
testUtil.getConfiguration(), htd);
addData();
}

View File

@ -92,8 +92,8 @@ public class TestFilter extends HBaseTestCase {
HTableDescriptor htd = new HTableDescriptor(getName());
htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
htd.addFamily(new HColumnDescriptor(FAMILIES[1]));
HRegionInfo info = new HRegionInfo(htd, null, null, false);
this.region = HRegion.createHRegion(info, this.testDir, this.conf);
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
this.region = HRegion.createHRegion(info, this.testDir, this.conf, htd);
// Insert first half
for(byte [] ROW : ROWS_ONE) {

View File

@ -116,9 +116,14 @@ public class TestCatalogJanitor {
*/
class MockMasterServices implements MasterServices {
private final MasterFileSystem mfs;
private final AssignmentManager asm;
MockMasterServices(final Server server) throws IOException {
this.mfs = new MasterFileSystem(server, null);
HTableDescriptor htd = new HTableDescriptor("table");
htd.addFamily(new HColumnDescriptor("family"));
this.asm = Mockito.mock(AssignmentManager.class);
Mockito.when(asm.getTableDescriptor("table")).thenReturn(htd);
}
@Override
@ -128,7 +133,7 @@ public class TestCatalogJanitor {
@Override
public AssignmentManager getAssignmentManager() {
return null;
return this.asm;
}
@Override
@ -219,11 +224,14 @@ public class TestCatalogJanitor {
HTableDescriptor htd = new HTableDescriptor("table");
htd.addFamily(new HColumnDescriptor("family"));
HRegionInfo parent =
new HRegionInfo(htd, Bytes.toBytes("aaa"), Bytes.toBytes("eee"));
new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
Bytes.toBytes("eee"));
HRegionInfo splita =
new HRegionInfo(htd, Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
Bytes.toBytes("ccc"));
HRegionInfo splitb =
new HRegionInfo(htd, Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
Bytes.toBytes("eee"));
// Test that when both daughter regions are in place, that we do not
// remove the parent.
List<KeyValue> kvs = new ArrayList<KeyValue>();

View File

@ -353,6 +353,7 @@ public class TestDistributedLogSplitting {
int num_edits, int edit_size) throws IOException {
byte[] table = Bytes.toBytes(tname);
HTableDescriptor htd = new HTableDescriptor(tname);
byte[] value = new byte[edit_size];
for (int i = 0; i < edit_size; i++) {
value[i] = (byte)('a' + (i % 26));
@ -369,7 +370,7 @@ public class TestDistributedLogSplitting {
System.currentTimeMillis(), value));
// LOG.info("Region " + i + ": " + e);
j++;
log.append(hris.get(j % n), table, e, System.currentTimeMillis());
log.append(hris.get(j % n), table, e, System.currentTimeMillis(), htd);
counts[j % n] += 1;
// if ((i % 8096) == 0) {
// log.sync();

View File

@ -486,7 +486,7 @@ public class TestLoadBalancer {
Bytes.putInt(start, 0, numRegions << 1);
Bytes.putInt(end, 0, (numRegions << 1) + 1);
HRegionInfo hri = new HRegionInfo(
new HTableDescriptor(Bytes.toBytes("table" + i)), start, end,
Bytes.toBytes("table" + i), start, end,
false, regionId++);
regions.add(hri);
}

View File

@ -31,18 +31,14 @@ import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.executor.RegionTransitionData;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
import org.apache.hadoop.hbase.master.LoadBalancer.RegionPlan;
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.JVMClusterUtil;
@ -262,12 +258,21 @@ public class TestMasterFailover {
byte [] enabledTable = Bytes.toBytes("enabledTable");
HTableDescriptor htdEnabled = new HTableDescriptor(enabledTable);
htdEnabled.addFamily(new HColumnDescriptor(FAMILY));
FileSystem filesystem = FileSystem.get(conf);
Path rootdir = filesystem.makeQualified(
new Path(conf.get(HConstants.HBASE_DIR)));
HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getName(), null, null);
HRegion.createHRegion(hriEnabled, rootdir, conf, htdEnabled);
List<HRegionInfo> enabledRegions = TEST_UTIL.createMultiRegionsInMeta(
TEST_UTIL.getConfiguration(), htdEnabled, SPLIT_KEYS);
byte [] disabledTable = Bytes.toBytes("disabledTable");
HTableDescriptor htdDisabled = new HTableDescriptor(disabledTable);
htdDisabled.addFamily(new HColumnDescriptor(FAMILY));
HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getName(), null, null);
HRegion.createHRegion(hriDisabled, rootdir, conf, htdDisabled);
List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
@ -566,12 +571,23 @@ public class TestMasterFailover {
byte [] enabledTable = Bytes.toBytes("enabledTable");
HTableDescriptor htdEnabled = new HTableDescriptor(enabledTable);
htdEnabled.addFamily(new HColumnDescriptor(FAMILY));
FileSystem filesystem = FileSystem.get(conf);
Path rootdir = filesystem.makeQualified(
new Path(conf.get(HConstants.HBASE_DIR)));
HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getName(),
null, null);
HRegion.createHRegion(hriEnabled, rootdir, conf, htdEnabled);
List<HRegionInfo> enabledRegions = TEST_UTIL.createMultiRegionsInMeta(
TEST_UTIL.getConfiguration(), htdEnabled, SPLIT_KEYS);
byte [] disabledTable = Bytes.toBytes("disabledTable");
HTableDescriptor htdDisabled = new HTableDescriptor(disabledTable);
htdDisabled.addFamily(new HColumnDescriptor(FAMILY));
HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getName(), null, null);
HRegion.createHRegion(hriDisabled, rootdir, conf, htdDisabled);
List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);

View File

@ -25,14 +25,12 @@ import java.util.List;
import java.util.NavigableMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hbase.tmpl.master.MasterStatusTmpl;
@ -56,11 +54,20 @@ public class TestMasterStatusServlet {
new ServerName("fakehost", 12345, 1234567890);
static final HTableDescriptor FAKE_TABLE =
new HTableDescriptor("mytable");
static final HRegionInfo FAKE_REGION =
new HRegionInfo(FAKE_TABLE, Bytes.toBytes("a"), Bytes.toBytes("b"));
static final HRegionInfo FAKE_HRI =
new HRegionInfo(FAKE_TABLE.getName(), Bytes.toBytes("a"), Bytes.toBytes("b"));
// static final HRegionInfo FAKE_REGION = null;
@Before
public void setupBasicMocks() {
try {
HRegion.createHRegion(FAKE_HRI, HBaseTestingUtility.getTestDir(),
HBaseConfiguration.create(), FAKE_TABLE);
} catch(IOException ioe) {
}
conf = HBaseConfiguration.create();
master = Mockito.mock(HMaster.class);
@ -77,7 +84,7 @@ public class TestMasterStatusServlet {
NavigableMap<String, RegionState> regionsInTransition =
Maps.newTreeMap();
regionsInTransition.put("r1",
new RegionState(FAKE_REGION, RegionState.State.CLOSING, 12345L, FAKE_HOST));
new RegionState(FAKE_HRI, RegionState.State.CLOSING, 12345L, FAKE_HOST));
Mockito.doReturn(regionsInTransition).when(am).getRegionsInTransition();
Mockito.doReturn(am).when(master).getAssignmentManager();

View File

@ -68,10 +68,10 @@ public class TestColumnSeeking {
HColumnDescriptor.DEFAULT_BLOOMFILTER);
HTableDescriptor htd = new HTableDescriptor(table);
htd.addFamily(hcd);
HRegionInfo info = new HRegionInfo(htd, null, null, false);
HRegionInfo info = new HRegionInfo(Bytes.toBytes(table), null, null, false);
HRegion region =
HRegion.createHRegion(info, HBaseTestingUtility.getTestDir(), TEST_UTIL
.getConfiguration());
.getConfiguration(), htd);
List<String> rows = generateRandomWords(10, "row");
List<String> allColumns = generateRandomWords(10, "column");
@ -172,10 +172,11 @@ public class TestColumnSeeking {
HTableDescriptor htd = new HTableDescriptor(table);
htd.addFamily(new HColumnDescriptor(family));
HRegionInfo info = new HRegionInfo(htd, null, null, false);
HRegionInfo info = new HRegionInfo(Bytes.toBytes(table), null, null, false);
HRegion region =
HRegion.createHRegion(info, HBaseTestingUtility.getTestDir(), TEST_UTIL
.getConfiguration());
.getConfiguration(), htd);
List<String> rows = generateRandomWords(10, "row");
List<String> allColumns = generateRandomWords(100, "column");

View File

@ -82,9 +82,12 @@ public class TestCompactSelection extends TestCase {
HTableDescriptor htd = new HTableDescriptor(Bytes.toBytes("table"));
htd.addFamily(hcd);
HRegionInfo info = new HRegionInfo(htd, null, null, false);
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
HLog hlog = new HLog(fs, logdir, oldLogDir, conf);
HRegion region = new HRegion(basedir, hlog, fs, conf, info, null);
HRegion.createHRegion(info, basedir, conf, htd);
Path tableDir = new Path(basedir, Bytes.toString(htd.getName()));
HRegion region = new HRegion(tableDir, hlog, fs, conf, info, null);
store = new Store(basedir, region, hcd, fs, conf);
TEST_FILE = StoreFile.getRandomFilename(fs, store.getHomedir());

View File

@ -72,17 +72,17 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase {
Path rootdir = filesystem.makeQualified(new Path(conf.get(HConstants.HBASE_DIR)));
filesystem.mkdirs(rootdir);
// Up flush size else we bind up when we use default catalog flush of 16k.
HRegionInfo.FIRST_META_REGIONINFO.getTableDesc().
setMemStoreFlushSize(64 * 1024 * 1024);
HTableDescriptor.META_TABLEDESC.setMemStoreFlushSize(64 * 1024 * 1024);
HRegion mr = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO,
rootdir, this.conf);
rootdir, this.conf, HTableDescriptor.META_TABLEDESC);
// Write rows for three tables 'A', 'B', and 'C'.
for (char c = 'A'; c < 'D'; c++) {
HTableDescriptor htd = new HTableDescriptor("" + c);
final int last = 128;
final int interval = 2;
for (int i = 0; i <= last; i += interval) {
HRegionInfo hri = new HRegionInfo(htd,
HRegionInfo hri = new HRegionInfo(htd.getName(),
i == 0? HConstants.EMPTY_BYTE_ARRAY: Bytes.toBytes((byte)i),
i == last? HConstants.EMPTY_BYTE_ARRAY: Bytes.toBytes((byte)i + interval));
Put put = new Put(hri.getRegionName());

View File

@ -2794,9 +2794,9 @@ public class TestHRegion extends HBaseTestCase {
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(hcd);
HRegionInfo info = new HRegionInfo(htd, null, null, false);
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
Path path = new Path(DIR + "testBloomFilterSize");
region = HRegion.createHRegion(info, path, conf);
region = HRegion.createHRegion(info, path, conf, htd);
int num_unique_rows = 10;
int duplicate_multiplier =2;
@ -2852,9 +2852,9 @@ public class TestHRegion extends HBaseTestCase {
HColumnDescriptor.DEFAULT_REPLICATION_SCOPE);
HTableDescriptor htd = new HTableDescriptor(TABLE);
htd.addFamily(hcd);
HRegionInfo info = new HRegionInfo(htd, null, null, false);
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
Path path = new Path(DIR + "testAllColumnsWithBloomFilter");
region = HRegion.createHRegion(info, path, conf);
region = HRegion.createHRegion(info, path, conf, htd);
// For row:0, col:0: insert versions 1 through 5.
byte row[] = Bytes.toBytes("row:" + 0);
@ -2897,9 +2897,9 @@ public class TestHRegion extends HBaseTestCase {
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(hcd);
HRegionInfo info = new HRegionInfo(htd, null, null, false);
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
Path path = new Path(DIR + "TestDeleteRowWithBloomFilter");
region = HRegion.createHRegion(info, path, conf);
region = HRegion.createHRegion(info, path, conf, htd);
// Insert some data
byte row[] = Bytes.toBytes("row1");
@ -3033,14 +3033,14 @@ public class TestHRegion extends HBaseTestCase {
for(byte [] family : families) {
htd.addFamily(new HColumnDescriptor(family));
}
HRegionInfo info = new HRegionInfo(htd, null, null, false);
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
Path path = new Path(DIR + callingMethod);
if (fs.exists(path)) {
if (!fs.delete(path, true)) {
throw new IOException("Failed delete of " + path);
}
}
region = HRegion.createHRegion(info, path, conf);
region = HRegion.createHRegion(info, path, conf, htd);
}
/**

View File

@ -56,7 +56,7 @@ public class TestHRegionInfo {
public void testContainsRange() {
HTableDescriptor tableDesc = new HTableDescriptor("testtable");
HRegionInfo hri = new HRegionInfo(
tableDesc, Bytes.toBytes("a"), Bytes.toBytes("g"));
tableDesc.getName(), Bytes.toBytes("a"), Bytes.toBytes("g"));
// Single row range at start of region
assertTrue(hri.containsRange(Bytes.toBytes("a"), Bytes.toBytes("a")));
// Fully contained range

View File

@ -82,8 +82,8 @@ public class TestRSStatusServlet {
public void testWithRegions() throws IOException {
HTableDescriptor htd = new HTableDescriptor("mytable");
List<HRegionInfo> regions = Lists.newArrayList(
new HRegionInfo(htd, Bytes.toBytes("a"), Bytes.toBytes("d")),
new HRegionInfo(htd, Bytes.toBytes("d"), Bytes.toBytes("z"))
new HRegionInfo(htd.getName(), Bytes.toBytes("a"), Bytes.toBytes("d")),
new HRegionInfo(htd.getName(), Bytes.toBytes("d"), Bytes.toBytes("z"))
);
Mockito.doReturn(regions).when(rs).getOnlineRegions();

View File

@ -61,7 +61,7 @@ public class TestResettingCounters {
HTableDescriptor htd = new HTableDescriptor(table);
for (byte [] family : families) htd.addFamily(new HColumnDescriptor(family));
HRegionInfo hri = new HRegionInfo(htd, null, null, false);
HRegionInfo hri = new HRegionInfo(htd.getName(), null, null, false);
String testDir = HBaseTestingUtility.getTestDir() + "/TestResettingCounters/";
Path path = new Path(testDir);
if (fs.exists(path)) {
@ -69,7 +69,7 @@ public class TestResettingCounters {
throw new IOException("Failed delete of " + path);
}
}
HRegion region = HRegion.createHRegion(hri, path, conf);
HRegion region = HRegion.createHRegion(hri, path, conf, htd);
Increment odd = new Increment(rows[0]);
Increment even = new Increment(rows[0]);

View File

@ -72,7 +72,7 @@ public class TestScanner extends HBaseTestCase {
}
/** HRegionInfo for root region */
public static final HRegionInfo REGION_INFO =
new HRegionInfo(TESTTABLEDESC, HConstants.EMPTY_BYTE_ARRAY,
new HRegionInfo(TESTTABLEDESC.getName(), HConstants.EMPTY_BYTE_ARRAY,
HConstants.EMPTY_BYTE_ARRAY);
private static final byte [] ROW_KEY = REGION_INFO.getRegionName();
@ -101,7 +101,7 @@ public class TestScanner extends HBaseTestCase {
byte [] startrow = Bytes.toBytes("bbb");
byte [] stoprow = Bytes.toBytes("ccc");
try {
this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
this.r = createNewHRegion(TESTTABLEDESC, null, null);
addContent(this.r, HConstants.CATALOG_FAMILY);
List<KeyValue> results = new ArrayList<KeyValue>();
// Do simple test of getting one row only first.
@ -175,7 +175,7 @@ public class TestScanner extends HBaseTestCase {
public void testFilters() throws IOException {
try {
this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
this.r = createNewHRegion(TESTTABLEDESC, null, null);
addContent(this.r, HConstants.CATALOG_FAMILY);
byte [] prefix = Bytes.toBytes("ab");
Filter newFilter = new PrefixFilter(prefix);
@ -203,7 +203,7 @@ public class TestScanner extends HBaseTestCase {
*/
public void testRaceBetweenClientAndTimeout() throws Exception {
try {
this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
this.r = createNewHRegion(TESTTABLEDESC, null, null);
addContent(this.r, HConstants.CATALOG_FAMILY);
Scan scan = new Scan();
InternalScanner s = r.getScanner(scan);
@ -352,7 +352,7 @@ public class TestScanner extends HBaseTestCase {
assertEquals(0, info.getStartKey().length);
assertEquals(0, info.getEndKey().length);
assertEquals(0, Bytes.compareTo(info.getRegionName(), REGION_INFO.getRegionName()));
assertEquals(0, info.getTableDesc().compareTo(REGION_INFO.getTableDesc()));
//assertEquals(0, info.getTableDesc().compareTo(REGION_INFO.getTableDesc()));
}
/** Use a scanner to get the region info and then validate the results */
@ -448,7 +448,7 @@ public class TestScanner extends HBaseTestCase {
* @throws Exception
*/
public void testScanAndSyncFlush() throws Exception {
this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
this.r = createNewHRegion(TESTTABLEDESC, null, null);
HRegionIncommon hri = new HRegionIncommon(r);
try {
LOG.info("Added: " + addContent(hri, Bytes.toString(HConstants.CATALOG_FAMILY),
@ -472,7 +472,7 @@ public class TestScanner extends HBaseTestCase {
* @throws Exception
*/
public void testScanAndRealConcurrentFlush() throws Exception {
this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
this.r = createNewHRegion(TESTTABLEDESC, null, null);
HRegionIncommon hri = new HRegionIncommon(r);
try {
LOG.info("Added: " + addContent(hri, Bytes.toString(HConstants.CATALOG_FAMILY),

View File

@ -153,7 +153,7 @@ public class TestSplitTransaction {
int daughtersRowCount = 0;
for (HRegion r: daughters) {
// Open so can count its content.
HRegion openRegion = HRegion.openHRegion(r.getRegionInfo(),
HRegion openRegion = HRegion.openHRegion(this.testdir, r.getRegionInfo(),
r.getLog(), r.getConf());
try {
int count = countRows(openRegion);
@ -208,7 +208,7 @@ public class TestSplitTransaction {
int daughtersRowCount = 0;
for (HRegion r: daughters) {
// Open so can count its content.
HRegion openRegion = HRegion.openHRegion(r.getRegionInfo(),
HRegion openRegion = HRegion.openHRegion(this.testdir, r.getRegionInfo(),
r.getLog(), r.getConf());
try {
int count = countRows(openRegion);
@ -252,7 +252,8 @@ public class TestSplitTransaction {
HTableDescriptor htd = new HTableDescriptor("table");
HColumnDescriptor hcd = new HColumnDescriptor(CF);
htd.addFamily(hcd);
HRegionInfo hri = new HRegionInfo(htd, STARTROW, ENDROW);
return HRegion.openHRegion(hri, wal, TEST_UTIL.getConfiguration());
HRegionInfo hri = new HRegionInfo(htd.getName(), STARTROW, ENDROW);
HRegion.createHRegion(hri, testdir, TEST_UTIL.getConfiguration(), htd);
return HRegion.openHRegion(testdir, hri, wal, TEST_UTIL.getConfiguration());
}
}

View File

@ -127,7 +127,7 @@ public class TestStore extends TestCase {
HTableDescriptor htd = new HTableDescriptor(table);
htd.addFamily(hcd);
HRegionInfo info = new HRegionInfo(htd, null, null, false);
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
HLog hlog = new HLog(fs, logdir, oldLogDir, conf);
HRegion region = new HRegion(basedir, hlog, fs, conf, info, null);

View File

@ -69,10 +69,6 @@ public class TestWideScanner extends HBaseTestCase {
}
/** HRegionInfo for root region */
public static final HRegionInfo REGION_INFO =
new HRegionInfo(TESTTABLEDESC, HConstants.EMPTY_BYTE_ARRAY,
HConstants.EMPTY_BYTE_ARRAY);
MiniDFSCluster cluster = null;
HRegion r;
@ -107,7 +103,7 @@ public class TestWideScanner extends HBaseTestCase {
public void testWideScanBatching() throws IOException {
final int batch = 256;
try {
this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
this.r = createNewHRegion(TESTTABLEDESC, null, null);
int inserted = addWideContent(this.r);
List<KeyValue> results = new ArrayList<KeyValue>();
Scan scan = new Scan();

View File

@ -233,11 +233,17 @@ public class TestOpenRegionHandler {
HTableDescriptor htd =
new HTableDescriptor("testOpenRegionHandlerYankingRegionFromUnderIt");
final HRegionInfo hri =
new HRegionInfo(htd, HConstants.EMPTY_END_ROW, HConstants.EMPTY_END_ROW);
new HRegionInfo(htd.getName(), HConstants.EMPTY_END_ROW,
HConstants.EMPTY_END_ROW);
HRegion region =
HRegion.createHRegion(hri, HBaseTestingUtility.getTestDir(), HTU
.getConfiguration(), htd);
OpenRegionHandler handler = new OpenRegionHandler(server, rss, hri) {
HRegion openRegion() {
// Open region first, then remove znode as though it'd been hijacked.
HRegion region = super.openRegion();
//HRegion region = super.openRegion();
HRegion region = super.openRegion(HBaseTestingUtility.getTestDir());
// Don't actually open region BUT remove the znode as though it'd
// been hijacked on us.
ZooKeeperWatcher zkw = this.server.getZooKeeper();

View File

@ -38,11 +38,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
@ -145,11 +141,14 @@ public class TestHLog {
Path tabledir = new Path(hbaseDir, getName());
fs.mkdirs(tabledir);
for(int i = 0; i < howmany; i++) {
infos[i] = new HRegionInfo(new HTableDescriptor(tableName),
infos[i] = new HRegionInfo(tableName,
Bytes.toBytes("" + i), Bytes.toBytes("" + (i+1)), false);
fs.mkdirs(new Path(tabledir, infos[i].getEncodedName()));
LOG.info("allo " + new Path(tabledir, infos[i].getEncodedName()).toString());
}
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(new HColumnDescriptor("column"));
// Add edits for three regions.
try {
for (int ii = 0; ii < howmany; ii++) {
@ -164,7 +163,7 @@ public class TestHLog {
System.currentTimeMillis(), column));
LOG.info("Region " + i + ": " + edit);
log.append(infos[i], tableName, edit,
System.currentTimeMillis());
System.currentTimeMillis(), htd);
}
}
log.rollWriter();
@ -206,13 +205,15 @@ public class TestHLog {
HLog wal = new HLog(fs, subdir, oldLogDir, conf);
final int total = 20;
HRegionInfo info = new HRegionInfo(new HTableDescriptor(bytes),
HRegionInfo info = new HRegionInfo(bytes,
null,null, false);
HTableDescriptor htd = new HTableDescriptor();
htd.addFamily(new HColumnDescriptor(bytes));
for (int i = 0; i < total; i++) {
WALEdit kvs = new WALEdit();
kvs.add(new KeyValue(Bytes.toBytes(i), bytes, bytes));
wal.append(info, bytes, kvs, System.currentTimeMillis());
wal.append(info, bytes, kvs, System.currentTimeMillis(), htd);
}
// Now call sync and try reading. Opening a Reader before you sync just
// gives you EOFE.
@ -230,7 +231,7 @@ public class TestHLog {
for (int i = 0; i < total; i++) {
WALEdit kvs = new WALEdit();
kvs.add(new KeyValue(Bytes.toBytes(i), bytes, bytes));
wal.append(info, bytes, kvs, System.currentTimeMillis());
wal.append(info, bytes, kvs, System.currentTimeMillis(), htd);
}
reader = HLog.getReader(fs, walPath, conf);
count = 0;
@ -249,7 +250,7 @@ public class TestHLog {
for (int i = 0; i < total; i++) {
WALEdit kvs = new WALEdit();
kvs.add(new KeyValue(Bytes.toBytes(i), bytes, value));
wal.append(info, bytes, kvs, System.currentTimeMillis());
wal.append(info, bytes, kvs, System.currentTimeMillis(), htd);
}
// Now I should have written out lots of blocks. Sync then read.
wal.sync();
@ -334,17 +335,19 @@ public class TestHLog {
@Test
public void testAppendClose() throws Exception {
byte [] tableName = Bytes.toBytes(getName());
HRegionInfo regioninfo = new HRegionInfo(new HTableDescriptor(tableName),
HRegionInfo regioninfo = new HRegionInfo(tableName,
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
Path subdir = new Path(dir, "hlogdir");
Path archdir = new Path(dir, "hlogdir_archive");
HLog wal = new HLog(fs, subdir, archdir, conf);
final int total = 20;
HTableDescriptor htd = new HTableDescriptor();
htd.addFamily(new HColumnDescriptor(tableName));
for (int i = 0; i < total; i++) {
WALEdit kvs = new WALEdit();
kvs.add(new KeyValue(Bytes.toBytes(i), tableName, tableName));
wal.append(regioninfo, tableName, kvs, System.currentTimeMillis());
wal.append(regioninfo, tableName, kvs, System.currentTimeMillis(), htd);
}
// Now call sync to send the data to HDFS datanodes
wal.sync();
@ -460,11 +463,15 @@ public class TestHLog {
Bytes.toBytes(Integer.toString(i)),
timestamp, new byte[] { (byte)(i + '0') }));
}
HRegionInfo info = new HRegionInfo(new HTableDescriptor(tableName),
HRegionInfo info = new HRegionInfo(tableName,
row,Bytes.toBytes(Bytes.toString(row) + "1"), false);
log.append(info, tableName, cols, System.currentTimeMillis());
HTableDescriptor htd = new HTableDescriptor();
htd.addFamily(new HColumnDescriptor("column"));
log.append(info, tableName, cols, System.currentTimeMillis(), htd);
long logSeqId = log.startCacheFlush();
log.completeCacheFlush(info.getEncodedNameAsBytes(), tableName, logSeqId, info.isMetaRegion());
log.completeCacheFlush(info.getEncodedNameAsBytes(), tableName, logSeqId,
info.isMetaRegion());
log.close();
Path filename = log.computeFilename();
log = null;
@ -528,9 +535,11 @@ public class TestHLog {
Bytes.toBytes(Integer.toString(i)),
timestamp, new byte[] { (byte)(i + '0') }));
}
HRegionInfo hri = new HRegionInfo(new HTableDescriptor(tableName),
HRegionInfo hri = new HRegionInfo(tableName,
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
log.append(hri, tableName, cols, System.currentTimeMillis());
HTableDescriptor htd = new HTableDescriptor();
htd.addFamily(new HColumnDescriptor("column"));
log.append(hri, tableName, cols, System.currentTimeMillis(), htd);
long logSeqId = log.startCacheFlush();
log.completeCacheFlush(hri.getEncodedNameAsBytes(), tableName, logSeqId, false);
log.close();
@ -587,14 +596,17 @@ public class TestHLog {
DumbWALObserver visitor = new DumbWALObserver();
log.registerWALActionsListener(visitor);
long timestamp = System.currentTimeMillis();
HRegionInfo hri = new HRegionInfo(new HTableDescriptor(tableName),
HTableDescriptor htd = new HTableDescriptor();
htd.addFamily(new HColumnDescriptor("column"));
HRegionInfo hri = new HRegionInfo(tableName,
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
for (int i = 0; i < COL_COUNT; i++) {
WALEdit cols = new WALEdit();
cols.add(new KeyValue(row, Bytes.toBytes("column"),
Bytes.toBytes(Integer.toString(i)),
timestamp, new byte[]{(byte) (i + '0')}));
log.append(hri, tableName, cols, System.currentTimeMillis());
log.append(hri, tableName, cols, System.currentTimeMillis(), htd);
}
assertEquals(COL_COUNT, visitor.increments);
log.unregisterWALActionsListener(visitor);
@ -602,7 +614,7 @@ public class TestHLog {
cols.add(new KeyValue(row, Bytes.toBytes("column"),
Bytes.toBytes(Integer.toString(11)),
timestamp, new byte[]{(byte) (11 + '0')}));
log.append(hri, tableName, cols, System.currentTimeMillis());
log.append(hri, tableName, cols, System.currentTimeMillis(), htd);
assertEquals(COL_COUNT, visitor.increments);
}
@ -613,9 +625,9 @@ public class TestHLog {
final byte [] tableName2 = Bytes.toBytes("testLogCleaning2");
HLog log = new HLog(fs, dir, oldLogDir, conf);
HRegionInfo hri = new HRegionInfo(new HTableDescriptor(tableName),
HRegionInfo hri = new HRegionInfo(tableName,
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
HRegionInfo hri2 = new HRegionInfo(new HTableDescriptor(tableName2),
HRegionInfo hri2 = new HRegionInfo(tableName2,
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
// Add a single edit and make sure that rolling won't remove the file
@ -667,12 +679,15 @@ public class TestHLog {
private void addEdits(HLog log, HRegionInfo hri, byte [] tableName,
int times) throws IOException {
HTableDescriptor htd = new HTableDescriptor();
htd.addFamily(new HColumnDescriptor("row"));
final byte [] row = Bytes.toBytes("row");
for (int i = 0; i < times; i++) {
long timestamp = System.currentTimeMillis();
WALEdit cols = new WALEdit();
cols.add(new KeyValue(row, row, row, timestamp, row));
log.append(hri, tableName, cols, timestamp);
log.append(hri, tableName, cols, timestamp, htd);
}
}
@ -685,6 +700,12 @@ public class TestHLog {
increments++;
}
@Override
public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, WALEdit logEdit) {
//To change body of implemented methods use File | Settings | File Templates.
increments++;
}
@Override
public void logRolled(Path newFile) {
// TODO Auto-generated method stub

View File

@ -27,11 +27,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.Before;
@ -88,7 +84,7 @@ public class TestWALObserver {
list.add(observer);
DummyWALObserver laterobserver = new DummyWALObserver();
HLog hlog = new HLog(fs, logDir, oldLogDir, conf, list, null);
HRegionInfo hri = new HRegionInfo(new HTableDescriptor(SOME_BYTES),
HRegionInfo hri = new HRegionInfo(SOME_BYTES,
SOME_BYTES, SOME_BYTES, false);
for (int i = 0; i < 20; i++) {
@ -96,8 +92,11 @@ public class TestWALObserver {
KeyValue kv = new KeyValue(b,b,b);
WALEdit edit = new WALEdit();
edit.add(kv);
HTableDescriptor htd = new HTableDescriptor();
htd.addFamily(new HColumnDescriptor(b));
HLogKey key = new HLogKey(b,b, 0, 0);
hlog.append(hri, key, edit);
hlog.append(hri, key, edit, htd);
if (i == 10) {
hlog.registerWALActionsListener(laterobserver);
}
@ -114,6 +113,7 @@ public class TestWALObserver {
assertEquals(2, observer.closedCount);
}
/**
* Just counts when methods are called
*/
@ -142,5 +142,10 @@ public class TestWALObserver {
public void logCloseRequested() {
closedCount++;
}
public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, WALEdit logEdit) {
//To change body of implemented methods use File | Settings | File Templates.
}
}
}

View File

@ -133,14 +133,19 @@ public class TestWALReplay {
deleteDir(basedir);
fs.mkdirs(new Path(basedir, hri.getEncodedName()));
HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
HRegion region2 = HRegion.createHRegion(hri,
hbaseRootDir, this.conf, htd);
final byte [] tableName = Bytes.toBytes(tableNameStr);
final byte [] rowName = tableName;
HLog wal1 = createWAL(this.conf);
// Add 1k to each family.
final int countPerFamily = 1000;
for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, wal1);
for (HColumnDescriptor hcd: htd.getFamilies()) {
addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee,
wal1, htd);
}
wal1.close();
runWALSplit(this.conf);
@ -149,8 +154,9 @@ public class TestWALReplay {
// Up the sequenceid so that these edits are after the ones added above.
wal2.setSequenceNumber(wal1.getSequenceNumber());
// Add 1k to each family.
for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, wal2);
for (HColumnDescriptor hcd: htd.getFamilies()) {
addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily,
ee, wal2, htd);
}
wal2.close();
runWALSplit(this.conf);
@ -187,11 +193,14 @@ public class TestWALReplay {
final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
deleteDir(basedir);
HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
HRegion region2 = HRegion.createHRegion(hri,
hbaseRootDir, this.conf, htd);
HLog wal = createWAL(this.conf);
HRegion region = HRegion.openHRegion(hri, wal, this.conf);
Path f = new Path(basedir, "hfile");
HFile.Writer writer = new HFile.Writer(this.fs, f);
byte [] family = hri.getTableDesc().getFamilies().iterator().next().getName();
byte [] family = htd.getFamilies().iterator().next().getName();
byte [] row = Bytes.toBytes(tableNameStr);
writer.append(new KeyValue(row, family, family, row));
writer.close();
@ -240,6 +249,9 @@ public class TestWALReplay {
deleteDir(basedir);
final byte[] rowName = Bytes.toBytes(tableNameStr);
final int countPerFamily = 10;
final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
HRegion region3 = HRegion.createHRegion(hri,
hbaseRootDir, this.conf, htd);
// Write countPerFamily edits into the three families. Do a flush on one
// of the families during the load of edits so its seqid is not same as
@ -250,7 +262,7 @@ public class TestWALReplay {
// HRegionServer usually does this. It knows the largest seqid across all regions.
wal.setSequenceNumber(seqid);
boolean first = true;
for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
for (HColumnDescriptor hcd: htd.getFamilies()) {
addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
if (first ) {
// If first, so we have at least one family w/ different seqid to rest.
@ -261,7 +273,7 @@ public class TestWALReplay {
// Now assert edits made it in.
final Get g = new Get(rowName);
Result result = region.get(g, null);
assertEquals(countPerFamily * hri.getTableDesc().getFamilies().size(),
assertEquals(countPerFamily * htd.getFamilies().size(),
result.size());
// Now close the region, split the log, reopen the region and assert that
// replay of log has no effect, that our seqids are calculated correctly so
@ -285,7 +297,7 @@ public class TestWALReplay {
// Next test. Add more edits, then 'crash' this region by stealing its wal
// out from under it and assert that replay of the log adds the edits back
// correctly when region is opened again.
for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
for (HColumnDescriptor hcd: htd.getFamilies()) {
addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region2, "y");
}
// Get count of edits.
@ -319,7 +331,7 @@ public class TestWALReplay {
Result result3 = region3.get(g, null);
// Assert that count of cells is same as before crash.
assertEquals(result2.size(), result3.size());
assertEquals(hri.getTableDesc().getFamilies().size() * countPerFamily,
assertEquals(htd.getFamilies().size() * countPerFamily,
countOfRestoredEdits.get());
// I can't close wal1. Its been appropriated when we split.
@ -342,6 +354,10 @@ public class TestWALReplay {
final Path basedir = new Path(hbaseRootDir, tableNameStr);
deleteDir(basedir);
fs.mkdirs(new Path(basedir, hri.getEncodedName()));
final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
HRegion region2 = HRegion.createHRegion(hri,
hbaseRootDir, this.conf, htd);
final HLog wal = createWAL(this.conf);
final byte[] tableName = Bytes.toBytes(tableNameStr);
final byte[] rowName = tableName;
@ -349,8 +365,9 @@ public class TestWALReplay {
// Add 1k to each family.
final int countPerFamily = 1000;
for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, wal);
for (HColumnDescriptor hcd: htd.getFamilies()) {
addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily,
ee, wal, htd);
}
// Add a cache flush, shouldn't have any effect
@ -362,14 +379,14 @@ public class TestWALReplay {
long now = ee.currentTimeMillis();
edit.add(new KeyValue(rowName, Bytes.toBytes("another family"), rowName,
now, rowName));
wal.append(hri, tableName, edit, now);
wal.append(hri, tableName, edit, now, htd);
// Delete the c family to verify deletes make it over.
edit = new WALEdit();
now = ee.currentTimeMillis();
edit.add(new KeyValue(rowName, Bytes.toBytes("c"), null, now,
KeyValue.Type.DeleteFamily));
wal.append(hri, tableName, edit, now);
wal.append(hri, tableName, edit, now, htd);
// Sync.
wal.sync();
@ -411,7 +428,7 @@ public class TestWALReplay {
Get get = new Get(rowName);
Result result = region.get(get, -1);
// Make sure we only see the good edits
assertEquals(countPerFamily * (hri.getTableDesc().getFamilies().size() - 1),
assertEquals(countPerFamily * (htd.getFamilies().size() - 1),
result.size());
region.close();
} finally {
@ -441,7 +458,7 @@ public class TestWALReplay {
private void addWALEdits (final byte [] tableName, final HRegionInfo hri,
final byte [] rowName, final byte [] family,
final int count, EnvironmentEdge ee, final HLog wal)
final int count, EnvironmentEdge ee, final HLog wal, final HTableDescriptor htd)
throws IOException {
String familyStr = Bytes.toString(family);
for (int j = 0; j < count; j++) {
@ -450,7 +467,7 @@ public class TestWALReplay {
WALEdit edit = new WALEdit();
edit.add(new KeyValue(rowName, family, qualifierBytes,
ee.currentTimeMillis(), columnBytes));
wal.append(hri, tableName, edit, ee.currentTimeMillis());
wal.append(hri, tableName, edit, ee.currentTimeMillis(), htd);
}
}
@ -472,17 +489,9 @@ public class TestWALReplay {
* @param tableName Name of table to use when we create HTableDescriptor.
*/
private HRegionInfo createBasic3FamilyHRegionInfo(final String tableName) {
HTableDescriptor htd = new HTableDescriptor(tableName);
HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
htd.addFamily(a);
HColumnDescriptor b = new HColumnDescriptor(Bytes.toBytes("b"));
htd.addFamily(b);
HColumnDescriptor c = new HColumnDescriptor(Bytes.toBytes("c"));
htd.addFamily(c);
return new HRegionInfo(htd, null, null, false);
return new HRegionInfo(Bytes.toBytes(tableName), null, null, false);
}
/*
* Run the split. Verify only single split file made.
* @param c
@ -514,4 +523,15 @@ public class TestWALReplay {
HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
return wal;
}
private HTableDescriptor createBasic3FamilyHTD(final String tableName) {
HTableDescriptor htd = new HTableDescriptor(tableName);
HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
htd.addFamily(a);
HColumnDescriptor b = new HColumnDescriptor(Bytes.toBytes("b"));
htd.addFamily(b);
HColumnDescriptor c = new HColumnDescriptor(Bytes.toBytes("c"));
htd.addFamily(c);
return htd;
}
}

View File

@ -125,7 +125,7 @@ public class TestReplicationSourceManager {
col.setScope(HConstants.REPLICATION_SCOPE_LOCAL);
htd.addFamily(col);
hri = new HRegionInfo(htd, r1, r2);
hri = new HRegionInfo(htd.getName(), r1, r2);
}
@ -162,7 +162,8 @@ public class TestReplicationSourceManager {
URLEncoder.encode("regionserver:60020", "UTF8"));
manager.init();
HTableDescriptor htd = new HTableDescriptor();
htd.addFamily(new HColumnDescriptor(f1));
// Testing normal log rolling every 20
for(long i = 1; i < 101; i++) {
if(i > 1 && i % 20 == 0) {
@ -171,7 +172,7 @@ public class TestReplicationSourceManager {
LOG.info(i);
HLogKey key = new HLogKey(hri.getRegionName(),
test, seq++, System.currentTimeMillis());
hlog.append(hri, key, edit);
hlog.append(hri, key, edit, htd);
}
// Simulate a rapid insert that's followed
@ -184,7 +185,7 @@ public class TestReplicationSourceManager {
for (int i = 0; i < 3; i++) {
HLogKey key = new HLogKey(hri.getRegionName(),
test, seq++, System.currentTimeMillis());
hlog.append(hri, key, edit);
hlog.append(hri, key, edit, htd);
}
assertEquals(6, manager.getHLogs().size());
@ -196,7 +197,7 @@ public class TestReplicationSourceManager {
HLogKey key = new HLogKey(hri.getRegionName(),
test, seq++, System.currentTimeMillis());
hlog.append(hri, key, edit);
hlog.append(hri, key, edit, htd);
assertEquals(1, manager.getHLogs().size());

View File

@ -88,7 +88,7 @@ public class TestTableRegionModel extends TestCase {
public void testGetName() {
TableRegionModel model = buildTestModel();
String modelName = model.getName();
HRegionInfo hri = new HRegionInfo(new HTableDescriptor(TABLE),
HRegionInfo hri = new HRegionInfo(Bytes.toBytes(TABLE),
START_KEY, END_KEY, false, ID);
assertEquals(modelName, hri.getRegionNameAsString());
}

View File

@ -131,7 +131,7 @@ public class TestHBaseFsck {
htd, byte[] startKey, byte[] endKey)
throws IOException {
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
HRegionInfo hri = new HRegionInfo(htd, startKey, endKey);
HRegionInfo hri = new HRegionInfo(htd.getName(), startKey, endKey);
Put put = new Put(hri.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));

View File

@ -136,8 +136,8 @@ public class TestMergeTable {
private HRegion createRegion(final HTableDescriptor desc,
byte [] startKey, byte [] endKey, int firstRow, int nrows, Path rootdir)
throws IOException {
HRegionInfo hri = new HRegionInfo(desc, startKey, endKey);
HRegion region = HRegion.createHRegion(hri, rootdir, UTIL.getConfiguration());
HRegionInfo hri = new HRegionInfo(desc.getName(), startKey, endKey);
HRegion region = HRegion.createHRegion(hri, rootdir, UTIL.getConfiguration(), desc);
LOG.info("Created region " + region.getRegionNameAsString());
for(int i = firstRow; i < firstRow + nrows; i++) {
Put put = new Put(Bytes.toBytes("row_" + String.format("%1$05d", i)));
@ -156,10 +156,11 @@ public class TestMergeTable {
protected void setupROOTAndMeta(Path rootdir, final HRegion [] regions)
throws IOException {
HRegion root =
HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, rootdir, UTIL.getConfiguration());
HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, rootdir,
UTIL.getConfiguration(), HTableDescriptor.ROOT_TABLEDESC);
HRegion meta =
HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO, rootdir,
UTIL.getConfiguration());
UTIL.getConfiguration(), HTableDescriptor.META_TABLEDESC);
HRegion.addRegionToMETA(root, meta);
for (HRegion r: regions) {
HRegion.addRegionToMETA(meta, r);

View File

@ -71,31 +71,36 @@ public class TestMergeTool extends HBaseTestCase {
* Create the HRegionInfos for the regions.
*/
// Region 0 will contain the key range [row_0200,row_0300)
sourceRegions[0] = new HRegionInfo(this.desc, Bytes.toBytes("row_0200"),
sourceRegions[0] = new HRegionInfo(this.desc.getName(),
Bytes.toBytes("row_0200"),
Bytes.toBytes("row_0300"));
// Region 1 will contain the key range [row_0250,row_0400) and overlaps
// with Region 0
sourceRegions[1] =
new HRegionInfo(this.desc, Bytes.toBytes("row_0250"),
new HRegionInfo(this.desc.getName(),
Bytes.toBytes("row_0250"),
Bytes.toBytes("row_0400"));
// Region 2 will contain the key range [row_0100,row_0200) and is adjacent
// to Region 0 or the region resulting from the merge of Regions 0 and 1
sourceRegions[2] =
new HRegionInfo(this.desc, Bytes.toBytes("row_0100"),
new HRegionInfo(this.desc.getName(),
Bytes.toBytes("row_0100"),
Bytes.toBytes("row_0200"));
// Region 3 will contain the key range [row_0500,row_0600) and is not
// adjacent to any of Regions 0, 1, 2 or the merged result of any or all
// of those regions
sourceRegions[3] =
new HRegionInfo(this.desc, Bytes.toBytes("row_0500"),
new HRegionInfo(this.desc.getName(),
Bytes.toBytes("row_0500"),
Bytes.toBytes("row_0600"));
// Region 4 will have empty start and end keys and overlaps all regions.
sourceRegions[4] =
new HRegionInfo(this.desc, HConstants.EMPTY_BYTE_ARRAY,
new HRegionInfo(this.desc.getName(),
HConstants.EMPTY_BYTE_ARRAY,
HConstants.EMPTY_BYTE_ARRAY);
/*
@ -134,7 +139,8 @@ public class TestMergeTool extends HBaseTestCase {
*/
for (int i = 0; i < sourceRegions.length; i++) {
regions[i] =
HRegion.createHRegion(this.sourceRegions[i], this.testDir, this.conf);
HRegion.createHRegion(this.sourceRegions[i], this.testDir, this.conf,
this.desc);
/*
* Insert data
*/