HBASE-9413 WebUI says ".META." table but table got renames to "hbase:meta". Meed to update the UI

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1520463 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2013-09-06 00:29:48 +00:00
parent b0af10a526
commit df49eca1eb
114 changed files with 486 additions and 486 deletions

View File

@ -60,20 +60,20 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* There are two versions associated with HRegionInfo: HRegionInfo.VERSION and
* HConstants.META_VERSION. HRegionInfo.VERSION indicates the data structure's versioning
* while HConstants.META_VERSION indicates the versioning of the serialized HRIs stored in
* the META table.
* the hbase:meta table.
*
* Pre-0.92:
* HRI.VERSION == 0 and HConstants.META_VERSION does not exist (is not stored at META table)
* HRI.VERSION == 0 and HConstants.META_VERSION does not exist (is not stored at hbase:meta table)
* HRegionInfo had an HTableDescriptor reference inside it.
* HRegionInfo is serialized as Writable to META table.
* HRegionInfo is serialized as Writable to hbase:meta table.
* For 0.92.x and 0.94.x:
* HRI.VERSION == 1 and HConstants.META_VERSION == 0
* HRI no longer has HTableDescriptor in it.
* HRI is serialized as Writable to META table.
* HRI is serialized as Writable to hbase:meta table.
* For 0.96.x:
* HRI.VERSION == 1 and HConstants.META_VERSION == 1
* HRI data structure is the same as 0.92 and 0.94
* HRI is serialized as PB to META table.
* HRI is serialized as PB to hbase:meta table.
*
* Versioning of HRegionInfo is deprecated. HRegionInfo does protobuf
* serialization using RegionInfo class, which has it's own versioning.
@ -100,7 +100,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
*<p>
* **NOTE**
*
* The first META region, and regions created by an older
* The first hbase:meta region, and regions created by an older
* version of HBase (0.20 or prior) will continue to use the
* old region name format.
*/
@ -143,7 +143,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
regionName.length - MD5_HEX_LENGTH - 1,
MD5_HEX_LENGTH);
} else {
// old format region name. First META region also
// old format region name. First hbase:meta region also
// use this format.EncodedName is the JenkinsHash value.
int hashVal = Math.abs(JenkinsHash.getInstance().hash(regionName,
regionName.length, 0));
@ -162,12 +162,12 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
/**
* Use logging.
* @param encodedRegionName The encoded regionname.
* @return <code>.META.</code> if passed <code>1028785192</code> else returns
* @return <code>hbase:meta</code> if passed <code>1028785192</code> else returns
* <code>encodedRegionName</code>
*/
public static String prettyPrint(final String encodedRegionName) {
if (encodedRegionName.equals("1028785192")) {
return encodedRegionName + "/.META.";
return encodedRegionName + "/hbase:meta";
}
return encodedRegionName;
}
@ -562,7 +562,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
}
/**
* @return true if this region is from .META.
* @return true if this region is from hbase:meta
*/
public boolean isMetaTable() {
return isMetaRegion();
@ -927,7 +927,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* Extract a HRegionInfo and ServerName from catalog table {@link Result}.
* @param r Result to pull from
* @return A pair of the {@link HRegionInfo} and the {@link ServerName}
* (or null for server address if no address set in .META.).
* (or null for server address if no address set in hbase:meta).
* @throws IOException
*/
public static Pair<HRegionInfo, ServerName> getHRegionInfoAndServerName(final Result r) {

View File

@ -59,7 +59,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
/**
* HTableDescriptor contains the details about an HBase table such as the descriptors of
* all the column families, is the table a catalog table, <code> -ROOT- </code> or
* <code> .META. </code>, if the table is read only, the maximum size of the memstore,
* <code> hbase:meta </code>, if the table is read only, the maximum size of the memstore,
* when the region split should occur, coprocessors associated with it etc...
*/
@InterfaceAudience.Public
@ -156,7 +156,7 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
/**
* <em>INTERNAL</em> Used by rest interface to access this metadata
* attribute which denotes if it is a catalog table, either
* <code> .META. </code> or <code> -ROOT- </code>
* <code> hbase:meta </code> or <code> -ROOT- </code>
*
* @see #isMetaRegion()
*/
@ -256,7 +256,7 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
/**
* <em> INTERNAL </em> Private constructor used internally creating table descriptors for
* catalog tables, <code>.META.</code> and <code>-ROOT-</code>.
* catalog tables, <code>hbase:meta</code> and <code>-ROOT-</code>.
*/
protected HTableDescriptor(final TableName name, HColumnDescriptor[] families) {
setName(name);
@ -267,7 +267,7 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
/**
* <em> INTERNAL </em>Private constructor used internally creating table descriptors for
* catalog tables, <code>.META.</code> and <code>-ROOT-</code>.
* catalog tables, <code>hbase:meta</code> and <code>-ROOT-</code>.
*/
protected HTableDescriptor(final TableName name, HColumnDescriptor[] families,
Map<ImmutableBytesWritable,ImmutableBytesWritable> values) {
@ -347,7 +347,7 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
/*
* Set meta flags on this table.
* IS_ROOT_KEY is set if its a -ROOT- table
* IS_META_KEY is set either if its a -ROOT- or a .META. table
* IS_META_KEY is set either if its a -ROOT- or a hbase:meta table
* Called by constructors.
* @param name
*/
@ -381,10 +381,10 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
}
/**
* Checks if this table is <code> .META. </code>
* Checks if this table is <code> hbase:meta </code>
* region.
*
* @return true if this table is <code> .META. </code>
* @return true if this table is <code> hbase:meta </code>
* region
*/
public boolean isMetaRegion() {
@ -410,20 +410,20 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
/**
* <em> INTERNAL </em> Used to denote if the current table represents
* <code> -ROOT- </code> or <code> .META. </code> region. This is used
* <code> -ROOT- </code> or <code> hbase:meta </code> region. This is used
* internally by the HTableDescriptor constructors
*
* @param isMeta true if its either <code> -ROOT- </code> or
* <code> .META. </code> region
* <code> hbase:meta </code> region
*/
protected void setMetaRegion(boolean isMeta) {
setValue(IS_META_KEY, isMeta? TRUE: FALSE);
}
/**
* Checks if the table is a <code>.META.</code> table
* Checks if the table is a <code>hbase:meta</code> table
*
* @return true if table is <code> .META. </code> region.
* @return true if table is <code> hbase:meta </code> region.
*/
public boolean isMetaTable() {
return isMetaRegion() && !isRootRegion();
@ -1319,7 +1319,7 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
new Path(name.getNamespaceAsString(), new Path(name.getQualifierAsString()))));
}
/** Table descriptor for <code>.META.</code> catalog table */
/** Table descriptor for <code>hbase:meta</code> catalog table */
public static final HTableDescriptor META_TABLEDESC = new HTableDescriptor(
TableName.META_TABLE_NAME,
new HColumnDescriptor[] {
@ -1339,7 +1339,7 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
"org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint",
null, Coprocessor.PRIORITY_SYSTEM, null);
} catch (IOException ex) {
//LOG.warn("exception in loading coprocessor for the META table");
//LOG.warn("exception in loading coprocessor for the hbase:meta table");
throw new RuntimeException(ex);
}
}
@ -1381,7 +1381,7 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
return Bytes.toString(getValue(OWNER_KEY));
}
// Note that every table should have an owner (i.e. should have OWNER_KEY set).
// .META. and -ROOT- should return system user as owner, not null (see
// hbase:meta and -ROOT- should return system user as owner, not null (see
// MasterFileSystem.java:bootstrap()).
return null;
}

View File

@ -47,11 +47,11 @@ import java.net.UnknownHostException;
/**
* Tracks the availability of the catalog tables
* <code>.META.</code>.
* <code>hbase:meta</code>.
*
* This class is "read-only" in that the locations of the catalog tables cannot
* be explicitly set. Instead, ZooKeeper is used to learn of the availability
* and location of <code>.META.</code>.
* and location of <code>hbase:meta</code>.
*
* <p>Call {@link #start()} to start up operation. Call {@link #stop()}} to
* interrupt waits and close up shop.
@ -67,7 +67,7 @@ public class CatalogTracker {
// locations on fault, the client would instead get notifications out of zk.
//
// But this original intent is frustrated by the fact that this class has to
// read an hbase table, the -ROOT- table, to figure out the .META. region
// read an hbase table, the -ROOT- table, to figure out the hbase:meta region
// location which means we depend on an HConnection. HConnection will do
// retrying but also, it has its own mechanism for finding root and meta
// locations (and for 'verifying'; it tries the location and if it fails, does
@ -224,9 +224,9 @@ public class CatalogTracker {
}
/**
* Gets the current location for <code>.META.</code> or null if location is
* Gets the current location for <code>hbase:meta</code> or null if location is
* not currently available.
* @return {@link ServerName} for server hosting <code>.META.</code> or null
* @return {@link ServerName} for server hosting <code>hbase:meta</code> or null
* if none available
* @throws InterruptedException
*/
@ -242,11 +242,11 @@ public class CatalogTracker {
return this.metaRegionTracker.isLocationAvailable();
}
/**
* Gets the current location for <code>.META.</code> if available and waits
* Gets the current location for <code>hbase:meta</code> if available and waits
* for up to the specified timeout if not immediately available. Returns null
* if the timeout elapses before root is available.
* @param timeout maximum time to wait for root availability, in milliseconds
* @return {@link ServerName} for server hosting <code>.META.</code> or null
* @return {@link ServerName} for server hosting <code>hbase:meta</code> or null
* if none available
* @throws InterruptedException if interrupted while waiting
* @throws NotAllMetaRegionsOnlineException if meta not available before
@ -294,7 +294,7 @@ public class CatalogTracker {
}
/**
* Waits indefinitely for availability of <code>.META.</code>. Used during
* Waits indefinitely for availability of <code>hbase:meta</code>. Used during
* cluster startup. Does not verify meta, just that something has been
* set up in zk.
* @see #waitForMeta(long)
@ -306,7 +306,7 @@ public class CatalogTracker {
if (waitForMeta(100) != null) break;
} catch (NotAllMetaRegionsOnlineException e) {
if (LOG.isTraceEnabled()) {
LOG.info(".META. still not available, sleeping and retrying." +
LOG.info("hbase:meta still not available, sleeping and retrying." +
" Reason: " + e.getMessage());
}
}
@ -409,10 +409,10 @@ public class CatalogTracker {
}
/**
* Verify <code>.META.</code> is deployed and accessible.
* Verify <code>hbase:meta</code> is deployed and accessible.
* @param timeout How long to wait on zk for meta address (passed through to
* the internal call to {@link #waitForMetaServerConnection(long)}.
* @return True if the <code>.META.</code> location is healthy.
* @return True if the <code>hbase:meta</code> location is healthy.
* @throws IOException
* @throws InterruptedException
*/

View File

@ -43,7 +43,7 @@ import java.util.Set;
import java.util.TreeMap;
/**
* Reads region and assignment information from <code>.META.</code>.
* Reads region and assignment information from <code>hbase:meta</code>.
*/
@InterfaceAudience.Private
public class MetaReader {
@ -55,7 +55,7 @@ public class MetaReader {
static final byte [] META_REGION_PREFIX;
static {
// Copy the prefix from FIRST_META_REGIONINFO into META_REGION_PREFIX.
// FIRST_META_REGIONINFO == '.META.,,1'. META_REGION_PREFIX == '.META.,'
// FIRST_META_REGIONINFO == 'hbase:meta,,1'. META_REGION_PREFIX == 'hbase:meta,'
int len = HRegionInfo.FIRST_META_REGIONINFO.getRegionName().length - 2;
META_REGION_PREFIX = new byte [len];
System.arraycopy(HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), 0,
@ -63,7 +63,7 @@ public class MetaReader {
}
/**
* Performs a full scan of <code>.META.</code>, skipping regions from any
* Performs a full scan of <code>hbase:meta</code>, skipping regions from any
* tables in the specified set of disabled tables.
* @param catalogTracker
* @param disabledTables set of disabled tables that will not be returned
@ -79,7 +79,7 @@ public class MetaReader {
}
/**
* Performs a full scan of <code>.META.</code>, skipping regions from any
* Performs a full scan of <code>hbase:meta</code>, skipping regions from any
* tables in the specified set of disabled tables.
* @param catalogTracker
* @param disabledTables set of disabled tables that will not be returned
@ -117,7 +117,7 @@ public class MetaReader {
}
/**
* Performs a full scan of <code>.META.</code>.
* Performs a full scan of <code>hbase:meta</code>.
* @return List of {@link Result}
* @throws IOException
*/
@ -129,7 +129,7 @@ public class MetaReader {
}
/**
* Performs a full scan of a <code>.META.</code> table.
* Performs a full scan of a <code>hbase:meta</code> table.
* @return List of {@link Result}
* @throws IOException
*/
@ -141,7 +141,7 @@ public class MetaReader {
}
/**
* Performs a full scan of <code>.META.</code>.
* Performs a full scan of <code>hbase:meta</code>.
* @param catalogTracker
* @param visitor Visitor invoked against each row.
* @throws IOException
@ -183,7 +183,7 @@ public class MetaReader {
/**
* Callers should call close on the returned {@link HTable} instance.
* @param ct
* @return An {@link HTable} for <code>.META.</code>
* @return An {@link HTable} for <code>hbase:meta</code>
* @throws IOException
*/
static HTable getMetaHTable(final CatalogTracker ct)
@ -235,7 +235,7 @@ public class MetaReader {
}
/**
* Gets the result in META for the specified region.
* Gets the result in hbase:meta for the specified region.
* @param catalogTracker
* @param regionName
* @return result of the specified region
@ -267,7 +267,7 @@ public class MetaReader {
}
/**
* Checks if the specified table exists. Looks at the META table hosted on
* Checks if the specified table exists. Looks at the hbase:meta table hosted on
* the specified server.
* @param catalogTracker
* @param tableName table to check
@ -367,7 +367,7 @@ public class MetaReader {
/**
* @param tableName
* @return Place to start Scan in <code>.META.</code> when passed a
* @return Place to start Scan in <code>hbase:meta</code> when passed a
* <code>tableName</code>; returns &lt;tableName&rt; &lt;,&rt; &lt;,&rt;
*/
static byte [] getTableStartRowForMeta(TableName tableName) {
@ -475,7 +475,7 @@ public class MetaReader {
getServerUserRegions(CatalogTracker catalogTracker, final ServerName serverName)
throws IOException {
final NavigableMap<HRegionInfo, Result> hris = new TreeMap<HRegionInfo, Result>();
// Fill the above hris map with entries from .META. that have the passed
// Fill the above hris map with entries from hbase:meta that have the passed
// servername.
CollectingVisitor<Result> v = new CollectingVisitor<Result>() {
@Override
@ -518,7 +518,7 @@ public class MetaReader {
* @param visitor Visitor invoked against each row.
* @param startrow Where to start the scan. Pass null if want to begin scan
* at first row.
* <code>.META.</code>, the default (pass false to scan .META.)
* <code>hbase:meta</code>, the default (pass false to scan hbase:meta)
* @throws IOException
*/
public static void fullScan(CatalogTracker catalogTracker,
@ -595,7 +595,7 @@ public class MetaReader {
}
/**
* Count regions in <code>.META.</code> for passed table.
* Count regions in <code>hbase:meta</code> for passed table.
* @param c
* @param tableName
* @return Count or regions in table <code>tableName</code>

View File

@ -277,11 +277,11 @@ public class HBaseAdmin implements Abortable, Closeable {
}
/**
* List all the userspace tables. In other words, scan the META table.
* List all the userspace tables. In other words, scan the hbase:meta table.
*
* If we wanted this to be really fast, we could implement a special
* catalog table that just contains table names and their descriptors.
* Right now, it only exists as part of the META table's region info.
* Right now, it only exists as part of the hbase:meta table's region info.
*
* @return - returns an array of HTableDescriptors
* @throws IOException if a remote or network exception occurs
@ -641,7 +641,7 @@ public class HBaseAdmin implements Abortable, Closeable {
throw ProtobufUtil.getRemoteException(se);
}
// let us wait until .META. table is updated and
// let us wait until hbase:meta table is updated and
// HMaster removes the table from its HTableDescriptors
if (values == null || values.length == 0) {
tableExists = false;
@ -1270,7 +1270,7 @@ public class HBaseAdmin implements Abortable, Closeable {
* master will not be informed of the close.
* @param regionname region name to close
* @param serverName If supplied, we'll use this location rather than
* the one currently in <code>.META.</code>
* the one currently in <code>hbase:meta</code>
* @throws IOException if a remote or network exception occurs
*/
public void closeRegion(final String regionname, final String serverName)
@ -1283,7 +1283,7 @@ public class HBaseAdmin implements Abortable, Closeable {
* master will not be informed of the close.
* @param regionname region name to close
* @param serverName The servername of the regionserver. If passed null we
* will use servername found in the .META. table. A server name
* will use servername found in the hbase:meta table. A server name
* is made of host, port and startcode. Here is an example:
* <code> host187.example.com,60020,1289493121758</code>
* @throws IOException if a remote or network exception occurs

View File

@ -209,11 +209,11 @@ public interface HConnection extends Abortable, Closeable {
IOException;
/**
* List all the userspace tables. In other words, scan the META table.
* List all the userspace tables. In other words, scan the hbase:meta table.
*
* If we wanted this to be really fast, we could implement a special
* catalog table that just contains table names and their descriptors.
* Right now, it only exists as part of the META table's region info.
* Right now, it only exists as part of the hbase:meta table's region info.
*
* @return - returns an array of HTableDescriptors
* @throws IOException if a remote or network exception occurs

View File

@ -1089,14 +1089,14 @@ public class HConnectionManager {
}
/*
* Search .META. for the HRegionLocation info that contains the table and
* Search hbase:meta for the HRegionLocation info that contains the table and
* row we're seeking. It will prefetch certain number of regions info and
* save them to the global region cache.
*/
private void prefetchRegionCache(final TableName tableName,
final byte[] row) {
// Implement a new visitor for MetaScanner, and use it to walk through
// the .META.
// the hbase:meta
MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
public boolean processRow(Result result) throws IOException {
try {
@ -1134,12 +1134,12 @@ public class HConnectionManager {
MetaScanner.metaScan(conf, this, visitor, tableName, row,
this.prefetchRegionLimit, TableName.META_TABLE_NAME);
} catch (IOException e) {
LOG.warn("Encountered problems when prefetch META table: ", e);
LOG.warn("Encountered problems when prefetch hbase:meta table: ", e);
}
}
/*
* Search the .META. table for the HRegionLocation
* Search the hbase:meta table for the HRegionLocation
* info that contains the table and row we're seeking.
*/
private HRegionLocation locateRegionInMeta(final TableName parentTable,
@ -1245,7 +1245,7 @@ public class HConnectionManager {
}
if (isDeadServer(serverName)){
throw new RegionServerStoppedException(".META. says the region "+
throw new RegionServerStoppedException("hbase:meta says the region "+
regionInfo.getRegionNameAsString()+" is managed by the server " + serverName +
", but it is dead.");
}

View File

@ -186,8 +186,8 @@ public interface HTableInterface extends Closeable {
*
* @deprecated As of version 0.92 this method is deprecated without
* replacement.
* getRowOrBefore is used internally to find entries in .META. and makes
* various assumptions about the table (which are true for .META. but not
* getRowOrBefore is used internally to find entries in hbase:meta and makes
* various assumptions about the table (which are true for hbase:meta but not
* in general) to be efficient.
*/
Result getRowOrBefore(byte[] row, byte[] family) throws IOException;

View File

@ -38,14 +38,14 @@ import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Scanner class that contains the <code>.META.</code> table scanning logic.
* Scanner class that contains the <code>hbase:meta</code> table scanning logic.
* Provided visitors will be called for each row.
*
* Although public visibility, this is not a public-facing API and may evolve in
* minor releases.
*
* <p> Note that during concurrent region splits, the scanner might not see
* META changes across rows (for parent and daughter entries) consistently.
* hbase:meta changes across rows (for parent and daughter entries) consistently.
* see HBASE-5986, and {@link DefaultMetaScannerVisitor} for details. </p>
*/
@InterfaceAudience.Private
@ -155,10 +155,10 @@ public class MetaScanner {
byte[] rowBefore = regionInfo.getStartKey();
startRow = HRegionInfo.createRegionName(tableName, rowBefore, HConstants.ZEROES, false);
} else if (tableName == null || tableName.getName().length == 0) {
// Full META scan
// Full hbase:meta scan
startRow = HConstants.EMPTY_START_ROW;
} else {
// Scan META for an entire table
// Scan hbase:meta for an entire table
startRow = HRegionInfo.createRegionName(tableName, HConstants.EMPTY_START_ROW,
HConstants.ZEROES, false);
}
@ -287,7 +287,7 @@ public class MetaScanner {
}
/**
* Visitor class called to process each row of the .META. table
* Visitor class called to process each row of the hbase:meta table
*/
public interface MetaScannerVisitor extends Closeable {
/**
@ -337,9 +337,9 @@ public class MetaScanner {
/**
* A MetaScannerVisitor for a table. Provides a consistent view of the table's
* META entries during concurrent splits (see HBASE-5986 for details). This class
* hbase:meta entries during concurrent splits (see HBASE-5986 for details). This class
* does not guarantee ordered traversal of meta entries, and can block until the
* META entries for daughters are available during splits.
* hbase:meta entries for daughters are available during splits.
*/
public static abstract class TableMetaScannerVisitor extends DefaultMetaScannerVisitor {
private TableName tableName;

View File

@ -117,12 +117,12 @@ public abstract class RegionServerCallable<T> implements RetryingCallable<T> {
(location != null && getConnection().isDeadServer(location.getServerName()))) {
// if thrown these exceptions, we clear all the cache entries that
// map to that slow/dead server; otherwise, let cache miss and ask
// .META. again to find the new location
// hbase:meta again to find the new location
getConnection().clearCaches(location.getServerName());
} else if (t instanceof RegionMovedException) {
getConnection().updateCachedLocations(tableName, row, t, location);
} else if (t instanceof NotServingRegionException && !retrying) {
// Purge cache entries for this specific region from META cache
// Purge cache entries for this specific region from hbase:meta cache
// since we don't call connect(true) when number of retries is 1.
getConnection().deleteCachedRegionLocation(location);
}

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.hbase.HRegionLocation;
/**
* Cluster registry.
* Implemenations hold cluster information such as this cluster's id, location of .META., etc.
* Implemenations hold cluster information such as this cluster's id, location of hbase:meta, etc.
*/
interface Registry {
/**

View File

@ -213,7 +213,7 @@ public enum EventType {
/**
* Master controlled events to be executed on the master.<br>
* M_META_SERVER_SHUTDOWN <br>
* Master is processing shutdown of RS hosting a meta region (-ROOT- or .META.).
* Master is processing shutdown of RS hosting a meta region (-ROOT- or hbase:meta).
*/
M_META_SERVER_SHUTDOWN (72, ExecutorType.MASTER_META_SERVER_OPERATIONS),
/**

View File

@ -111,16 +111,16 @@ public class MetaRegionTracker extends ZooKeeperNodeTracker {
}
/**
* Sets the location of <code>.META.</code> in ZooKeeper to the
* Sets the location of <code>hbase:meta</code> in ZooKeeper to the
* specified server address.
* @param zookeeper zookeeper reference
* @param location The server hosting <code>.META.</code>
* @param location The server hosting <code>hbase:meta</code>
* @throws KeeperException unexpected zookeeper exception
*/
public static void setMetaLocation(ZooKeeperWatcher zookeeper,
final ServerName location)
throws KeeperException {
LOG.info("Setting META region location in ZooKeeper as " + location);
LOG.info("Setting hbase:meta region location in ZooKeeper as " + location);
// Make the MetaRegionServer pb and then get its bytes and save this as
// the znode content.
byte [] data = toByteArray(location);
@ -155,13 +155,13 @@ public class MetaRegionTracker extends ZooKeeperNodeTracker {
}
/**
* Deletes the location of <code>.META.</code> in ZooKeeper.
* Deletes the location of <code>hbase:meta</code> in ZooKeeper.
* @param zookeeper zookeeper reference
* @throws KeeperException unexpected zookeeper exception
*/
public static void deleteMetaLocation(ZooKeeperWatcher zookeeper)
throws KeeperException {
LOG.info("Unsetting META region location in ZooKeeper");
LOG.info("Unsetting hbase:meta region location in ZooKeeper");
try {
// Just delete the node. Don't need any watches.
ZKUtil.deleteNode(zookeeper, zookeeper.metaServerZNode);

View File

@ -1587,7 +1587,7 @@ public class ZKUtil {
zkw.backupMasterAddressesZNode)) {
sb.append("\n ").append(child);
}
sb.append("\nRegion server holding .META.: " + MetaRegionTracker.getMetaRegionLocation(zkw));
sb.append("\nRegion server holding hbase:meta: " + MetaRegionTracker.getMetaRegionLocation(zkw));
sb.append("\nRegion servers:");
for (String child : listChildrenNoWatch(zkw, zkw.rsZNode)) {
sb.append("\n ").append(child);

View File

@ -47,7 +47,7 @@ import org.apache.hadoop.classification.InterfaceStability;
* include the costly helper methods marked as deprecated.
* <p/>
* Cell implements Comparable<Cell> which is only meaningful when comparing to other keys in the
* same table. It uses CellComparator which does not work on the -ROOT- and .META. tables.
* same table. It uses CellComparator which does not work on the -ROOT- and hbase:meta tables.
* <p/>
* In the future, we may consider adding a boolean isOnHeap() method and a getValueBuffer() method
* that can be used to pass a value directly from an off-heap ByteBuffer to the network without

View File

@ -30,9 +30,9 @@ import com.google.common.primitives.Longs;
/**
* Compare two HBase cells. Do not use this method comparing <code>-ROOT-</code> or
* <code>.META.</code> cells. Cells from these tables need a specialized comparator, one that
* <code>hbase:meta</code> cells. Cells from these tables need a specialized comparator, one that
* takes account of the special formatting of the row where we have commas to delimit table from
* regionname, from row. See KeyValue for how it has a special comparator to do .META. cells
* regionname, from row. See KeyValue for how it has a special comparator to do hbase:meta cells
* and yet another for -ROOT-.
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(

View File

@ -356,7 +356,7 @@ public final class HConstants {
// should go down.
/** The META table's name. */
/** The hbase:meta table's name. */
@Deprecated // for compat from 0.94 -> 0.96.
public static final byte[] META_TABLE_NAME = TableName.META_TABLE_NAME.getName();

View File

@ -85,7 +85,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
*/
public static final KVComparator COMPARATOR = new KVComparator();
/**
* A {@link KVComparator} for <code>.META.</code> catalog table
* A {@link KVComparator} for <code>hbase:meta</code> catalog table
* {@link KeyValue}s.
*/
public static final KVComparator META_COMPARATOR = new MetaComparator();
@ -1342,7 +1342,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
final int offset, final int length, final int delimiter) {
int index = getDelimiterInReverse(b, offset, length, delimiter);
if (index < 0) {
throw new IllegalArgumentException(".META. key must have two '" + (char)delimiter + "' "
throw new IllegalArgumentException("hbase:meta key must have two '" + (char)delimiter + "' "
+ "delimiters and have the following format: '<table>,<key>,<etc>'");
}
return index;
@ -1391,12 +1391,12 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
}
/**
* A {@link KVComparator} for <code>.META.</code> catalog table
* A {@link KVComparator} for <code>hbase:meta</code> catalog table
* {@link KeyValue}s.
*/
public static class MetaComparator extends KVComparator {
/**
* Compare key portion of a {@link KeyValue} for keys in <code>.META.</code>
* Compare key portion of a {@link KeyValue} for keys in <code>hbase:meta</code>
* table.
*/
@Override
@ -1407,7 +1407,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
int rightDelimiter = getDelimiter(right, roffset, rlength,
HConstants.DELIMITER);
if (leftDelimiter < 0 && rightDelimiter >= 0) {
// Nothing between .META. and regionid. Its first key.
// Nothing between hbase:meta and regionid. Its first key.
return -1;
} else if (rightDelimiter < 0 && leftDelimiter >= 0) {
return 1;

View File

@ -64,7 +64,7 @@ public final class TableName implements Comparable<TableName> {
"(?:(?:(?:"+VALID_NAMESPACE_REGEX+"\\"+NAMESPACE_DELIM+")?)" +
"(?:"+VALID_TABLE_QUALIFIER_REGEX+"))";
/** The META table's name. */
/** The hbase:meta table's name. */
public static final TableName META_TABLE_NAME =
valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta");

View File

@ -153,7 +153,7 @@ public class TestKeyValue extends TestCase {
try {
c.compare(a, b);
} catch (IllegalArgumentException iae) {
assertEquals(".META. key must have two ',' delimiters and have the following" +
assertEquals("hbase:meta key must have two ',' delimiters and have the following" +
" format: '<table>,<key>,<etc>'", iae.getMessage());
return;
}

View File

@ -48,7 +48,7 @@ public interface MetricsMasterFileSystemSource extends BaseSource {
String SPLIT_SIZE_NAME = "hlogSplitSize";
String META_SPLIT_TIME_DESC = "Time it takes to finish splitMetaLog()";
String META_SPLIT_SIZE_DESC = "Size of META HLog files being split";
String META_SPLIT_SIZE_DESC = "Size of hbase:meta HLog files being split";
String SPLIT_TIME_DESC = "Time it takes to finish HLog.splitLog()";
String SPLIT_SIZE_DESC = "Size of HLog files being split";

View File

@ -32,7 +32,7 @@ public class RestartRsHoldingMetaAction extends RestartActionBaseAction {
LOG.info("Performing action: Restart region server holding META");
ServerName server = cluster.getServerHoldingMeta();
if (server == null) {
LOG.warn("No server is holding .META. right now.");
LOG.warn("No server is holding hbase:meta right now.");
return;
}
restartRs(server, sleepTime);

View File

@ -194,7 +194,7 @@ public class PrefixTreeCodec implements DataBlockEncoder{
if (comparator instanceof RawBytesComparator){
throw new IllegalArgumentException("comparator must be KeyValue.KeyComparator");
} else if (comparator instanceof MetaComparator){
throw new IllegalArgumentException("DataBlockEncoding.PREFIX_TREE not compatible with META "
throw new IllegalArgumentException("DataBlockEncoding.PREFIX_TREE not compatible with hbase:meta "
+"table");
}

View File

@ -273,7 +273,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
<tr>
<td>Fragmentation</td>
<td><% frags.get("-TOTAL-") != null ? frags.get("-TOTAL-").intValue() + "%" : "n/a" %></td>
<td>Overall fragmentation of all tables, including .META.</td>
<td>Overall fragmentation of all tables, including hbase:meta</td>
</tr>
</%if>
<tr>
@ -318,7 +318,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
</%if>
<%java>String description = null;
if (tableName.equals(TableName.META_TABLE_NAME)){
description = "The .META. table holds references to all User Table regions";
description = "The hbase:meta table holds references to all User Table regions";
} else {
description = "The .NAMESPACE. table holds information about namespaces.";
}

View File

@ -68,9 +68,9 @@
the region named
<em>domains,apache.org,5464829424211263407</em> is party to the table
<em>domains</em>, has an id of <em>5464829424211263407</em> and the first key
in the region is <em>apache.org</em>. The <em>.META.</em> 'table' is an internal
in the region is <em>apache.org</em>. The <em>hbase:meta</em> 'table' is an internal
system table (or 'catalog' tables in db-speak).
The .META. table keeps a list of all regions in the system. The empty key is used to denote
The hbase:meta table keeps a list of all regions in the system. The empty key is used to denote
table start and table end. A region with an empty start key is the first region in a table.
If region has both an empty start and an empty end key, its the only region in the table. See
<a href="http://hbase.org">HBase Home</a> for further explication.<p>

View File

@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.ServiceException;
/**
* Writes region and assignment information to <code>.META.</code>.
* Writes region and assignment information to <code>hbase:meta</code>.
* TODO: Put MetaReader and MetaEditor together; doesn't make sense having
* them distinct. see HBASE-3475.
*/
@ -93,9 +93,9 @@ public class MetaEditor {
}
/**
* Put the passed <code>p</code> to the <code>.META.</code> table.
* Put the passed <code>p</code> to the <code>hbase:meta</code> table.
* @param ct CatalogTracker on whose back we will ride the edit.
* @param p Put to add to .META.
* @param p Put to add to hbase:meta
* @throws IOException
*/
static void putToMetaTable(final CatalogTracker ct, final Put p)
@ -128,9 +128,9 @@ public class MetaEditor {
}
/**
* Put the passed <code>ps</code> to the <code>.META.</code> table.
* Put the passed <code>ps</code> to the <code>hbase:meta</code> table.
* @param ct CatalogTracker on whose back we will ride the edit.
* @param ps Put to add to .META.
* @param ps Put to add to hbase:meta
* @throws IOException
*/
public static void putsToMetaTable(final CatalogTracker ct, final List<Put> ps)
@ -144,9 +144,9 @@ public class MetaEditor {
}
/**
* Delete the passed <code>d</code> from the <code>.META.</code> table.
* Delete the passed <code>d</code> from the <code>hbase:meta</code> table.
* @param ct CatalogTracker on whose back we will ride the edit.
* @param d Delete to add to .META.
* @param d Delete to add to hbase:meta
* @throws IOException
*/
static void deleteFromMetaTable(final CatalogTracker ct, final Delete d)
@ -157,9 +157,9 @@ public class MetaEditor {
}
/**
* Delete the passed <code>deletes</code> from the <code>.META.</code> table.
* Delete the passed <code>deletes</code> from the <code>hbase:meta</code> table.
* @param ct CatalogTracker on whose back we will ride the edit.
* @param deletes Deletes to add to .META. This list should support #remove.
* @param deletes Deletes to add to hbase:meta This list should support #remove.
* @throws IOException
*/
public static void deleteFromMetaTable(final CatalogTracker ct, final List<Delete> deletes)
@ -173,9 +173,9 @@ public class MetaEditor {
}
/**
* Execute the passed <code>mutations</code> against <code>.META.</code> table.
* Execute the passed <code>mutations</code> against <code>hbase:meta</code> table.
* @param ct CatalogTracker on whose back we will ride the edit.
* @param mutations Puts and Deletes to execute on .META.
* @param mutations Puts and Deletes to execute on hbase:meta
* @throws IOException
*/
static void mutateMetaTable(final CatalogTracker ct, final List<Mutation> mutations)
@ -193,7 +193,7 @@ public class MetaEditor {
}
/**
* Adds a META row for the specified new region.
* Adds a hbase:meta row for the specified new region.
* @param regionInfo region information
* @throws IOException if problem connecting or updating meta
*/
@ -205,7 +205,7 @@ public class MetaEditor {
}
/**
* Adds a META row for the specified new region to the given catalog table. The
* Adds a hbase:meta row for the specified new region to the given catalog table. The
* HTable is not flushed or closed.
* @param meta the HTable for META
* @param regionInfo region information
@ -216,7 +216,7 @@ public class MetaEditor {
}
/**
* Adds a (single) META row for the specified new region and its daughters. Note that this does
* Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this does
* not add its daughter's as different rows, but adds information about the daughters
* in the same row as the parent. Use
* {@link #splitRegion(CatalogTracker, HRegionInfo, HRegionInfo, HRegionInfo, ServerName)}
@ -238,7 +238,7 @@ public class MetaEditor {
}
/**
* Adds a (single) META row for the specified new region and its daughters. Note that this does
* Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this does
* not add its daughter's as different rows, but adds information about the daughters
* in the same row as the parent. Use
* {@link #splitRegion(CatalogTracker, HRegionInfo, HRegionInfo, HRegionInfo, ServerName)}
@ -260,7 +260,7 @@ public class MetaEditor {
}
/**
* Adds a META row for each of the specified new regions.
* Adds a hbase:meta row for each of the specified new regions.
* @param catalogTracker CatalogTracker
* @param regionInfos region information list
* @throws IOException if problem connecting or updating meta
@ -297,7 +297,7 @@ public class MetaEditor {
/**
* Merge the two regions into one in an atomic operation. Deletes the two
* merging regions in META and adds the merged region with the information of
* merging regions in hbase:meta and adds the merged region with the information of
* two merging regions.
* @param catalogTracker the catalog tracker
* @param mergedRegion the merged region
@ -401,7 +401,7 @@ public class MetaEditor {
/**
* Updates the location of the specified META region in ROOT to be the
* Updates the location of the specified hbase:meta region in ROOT to be the
* specified server hostname and startcode.
* <p>
* Uses passed catalog tracker to get a connection to the server hosting
@ -412,7 +412,7 @@ public class MetaEditor {
* @param sn Server name
* @param openSeqNum the latest sequence number obtained when the region was open
* @throws IOException
* @throws ConnectException Usually because the regionserver carrying .META.
* @throws ConnectException Usually because the regionserver carrying hbase:meta
* is down.
* @throws NullPointerException Because no -ROOT- server connection
*/
@ -423,11 +423,11 @@ public class MetaEditor {
}
/**
* Updates the location of the specified region in META to be the specified
* Updates the location of the specified region in hbase:meta to be the specified
* server hostname and startcode.
* <p>
* Uses passed catalog tracker to get a connection to the server hosting
* META and makes edits to that region.
* hbase:meta and makes edits to that region.
*
* @param catalogTracker catalog tracker
* @param regionInfo region to update location of
@ -494,7 +494,7 @@ public class MetaEditor {
}
/**
* Adds and Removes the specified regions from .META.
* Adds and Removes the specified regions from hbase:meta
* @param catalogTracker
* @param regionsToRemove list of regions to be deleted from META
* @param regionsToAdd list of regions to be added to META

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
/**
* A tool to migrate the data stored in META table to pbuf serialization.
* A tool to migrate the data stored in hbase:meta table to pbuf serialization.
* Supports migrating from 0.92.x and 0.94.x to 0.96.x for the catalog table.
* @deprecated will be removed for the major release after 0.96.
*/
@ -132,19 +132,19 @@ public class MetaMigrationConvertingToPB {
LOG.info("META already up-to date with PB serialization");
return 0;
}
LOG.info("META has Writable serializations, migrating META to PB serialization");
LOG.info("META has Writable serializations, migrating hbase:meta to PB serialization");
try {
long rows = updateMeta(services);
LOG.info("META updated with PB serialization. Total rows updated: " + rows);
return rows;
} catch (IOException e) {
LOG.warn("Update META with PB serialization failed." + "Master startup aborted.");
LOG.warn("Update hbase:meta with PB serialization failed." + "Master startup aborted.");
throw e;
}
}
/**
* Update META rows, converting writable serialization to PB
* Update hbase:meta rows, converting writable serialization to PB
* @return num migrated rows
*/
static long updateMeta(final MasterServices masterServices) throws IOException {
@ -163,7 +163,7 @@ public class MetaMigrationConvertingToPB {
static boolean isMetaTableUpdated(final CatalogTracker catalogTracker) throws IOException {
List<Result> results = MetaReader.fullScanOfMeta(catalogTracker);
if (results == null || results.isEmpty()) {
LOG.info(".META. doesn't have any entries to update.");
LOG.info("hbase:meta doesn't have any entries to update.");
return true;
}
for (Result r : results) {

View File

@ -101,11 +101,11 @@ public class HFilePrettyPrinter {
"Enable row order check; looks for out-of-order keys");
options.addOption("a", "checkfamily", false, "Enable family check");
options.addOption("f", "file", true,
"File to scan. Pass full-path; e.g. hdfs://a:9000/hbase/.META./12/34");
"File to scan. Pass full-path; e.g. hdfs://a:9000/hbase/hbase:meta/12/34");
options.addOption("w", "seekToRow", true,
"Seek to this row and print all the kvs for this row only");
options.addOption("r", "region", true,
"Region to scan. Pass region name; e.g. '.META.,,1'");
"Region to scan. Pass region name; e.g. 'hbase:meta,,1'");
options.addOption("s", "stats", false, "Print statistics");
}

View File

@ -270,7 +270,7 @@ public class WALPlayer extends Configured implements Tool {
System.err.println("Usage: " + NAME + " [options] <wal inputdir> <tables> [<tableMappings>]");
System.err.println("Read all WAL entries for <tables>.");
System.err.println("If no tables (\"\") are specific, all tables are imported.");
System.err.println("(Careful, even -ROOT- and .META. entries will be imported in that case.)");
System.err.println("(Careful, even -ROOT- and hbase:meta entries will be imported in that case.)");
System.err.println("Otherwise <tables> is a comma separated list of tables.\n");
System.err.println("The WAL entries can be mapped to new set of tables via <tableMapping>.");
System.err.println("<tableMapping> is a command separated list of targettables.");

View File

@ -148,7 +148,7 @@ public class AssignmentManager extends ZooKeeperListener {
private final int maximumAttempts;
/**
* The sleep time for which the assignment will wait before retrying in case of META assignment
* The sleep time for which the assignment will wait before retrying in case of hbase:meta assignment
* failure due to lack of availability of region plan
*/
private final long sleepTimeBeforeRetryingMetaAssignment;
@ -416,7 +416,7 @@ public class AssignmentManager extends ZooKeeperListener {
// TODO: Regions that have a null location and are not in regionsInTransitions
// need to be handled.
// Scan META to build list of existing regions, servers, and assignment
// Scan hbase:meta to build list of existing regions, servers, and assignment
// Returns servers who have not checked in (assumed dead) and their regions
Map<ServerName, List<HRegionInfo>> deadServers = rebuildUserRegions();
@ -1141,7 +1141,7 @@ public class AssignmentManager extends ZooKeeperListener {
Pair<HRegionInfo, ServerName> p = MetaReader.getRegion(catalogTracker, name);
regionInfo = p.getFirst();
} catch (IOException e) {
LOG.info("Exception reading META doing HBCK repair operation", e);
LOG.info("Exception reading hbase:meta doing HBCK repair operation", e);
return;
}
}
@ -1874,9 +1874,9 @@ public class AssignmentManager extends ZooKeeperListener {
continue;
}
// TODO : Ensure HBCK fixes this
LOG.error("Unable to determine a plan to assign META even after repeated attempts. Run HBCK to fix this");
LOG.error("Unable to determine a plan to assign hbase:meta even after repeated attempts. Run HBCK to fix this");
} catch (InterruptedException e) {
LOG.error("Got exception while waiting for META assignment");
LOG.error("Got exception while waiting for hbase:meta assignment");
Thread.currentThread().interrupt();
}
}
@ -1895,7 +1895,7 @@ public class AssignmentManager extends ZooKeeperListener {
// In case of assignment from EnableTableHandler table state is ENABLING. Any how
// EnableTableHandler will set ENABLED after assigning all the table regions. If we
// try to set to ENABLED directly then client API may think table is enabled.
// When we have a case such as all the regions are added directly into .META. and we call
// When we have a case such as all the regions are added directly into hbase:meta and we call
// assignRegion then we need to make the table ENABLED. Hence in such case the table
// will not be in ENABLING or ENABLED state.
TableName tableName = region.getTableName();
@ -2453,13 +2453,13 @@ public class AssignmentManager extends ZooKeeperListener {
}
/**
* Assigns the META region.
* Assigns the hbase:meta region.
* <p>
* Assumes that META is currently closed and is not being actively served by
* Assumes that hbase:meta is currently closed and is not being actively served by
* any RegionServer.
* <p>
* Forcibly unsets the current meta region location in ZooKeeper and assigns
* META to a random RegionServer.
* hbase:meta to a random RegionServer.
* @throws KeeperException
*/
public void assignMeta() throws KeeperException {
@ -2573,7 +2573,7 @@ public class AssignmentManager extends ZooKeeperListener {
// See HBASE-6281.
Set<TableName> disabledOrDisablingOrEnabling = ZKTable.getDisabledOrDisablingTables(watcher);
disabledOrDisablingOrEnabling.addAll(ZKTable.getEnablingTables(watcher));
// Scan META for all user regions, skipping any disabled tables
// Scan hbase:meta for all user regions, skipping any disabled tables
Map<HRegionInfo, ServerName> allRegions;
SnapshotOfRegionAssignmentFromMeta snapshotOfRegionAssignment =
new SnapshotOfRegionAssignmentFromMeta(catalogTracker, disabledOrDisablingOrEnabling, true);
@ -2677,7 +2677,7 @@ public class AssignmentManager extends ZooKeeperListener {
if (regionLocation == null) {
// regionLocation could be null if createTable didn't finish properly.
// When createTable is in progress, HMaster restarts.
// Some regions have been added to .META., but have not been assigned.
// Some regions have been added to hbase:meta, but have not been assigned.
// When this happens, the region's table must be in ENABLING state.
// It can't be in ENABLED state as that is set when all regions are
// assigned.
@ -2770,7 +2770,7 @@ public class AssignmentManager extends ZooKeeperListener {
}
/**
* Processes list of dead servers from result of META scan and regions in RIT
* Processes list of dead servers from result of hbase:meta scan and regions in RIT
* <p>
* This is used for failover to recover the lost regions that belonged to
* RegionServers which failed while there was no active master or regions

View File

@ -53,7 +53,7 @@ import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.hbase.util.Triple;
/**
* A janitor for the catalog tables. Scans the <code>.META.</code> catalog
* A janitor for the catalog tables. Scans the <code>hbase:meta</code> catalog
* table on a period looking for unused regions to garbage collect.
*/
@InterfaceAudience.Private
@ -108,7 +108,7 @@ public class CatalogJanitor extends Chore {
}
/**
* Scans META and returns a number of scanned rows, and a map of merged
* Scans hbase:meta and returns a number of scanned rows, and a map of merged
* regions, and an ordered map of split parents.
* @return triple of scanned rows, map of merged regions and map of split
* parent regioninfos
@ -120,7 +120,7 @@ public class CatalogJanitor extends Chore {
}
/**
* Scans META and returns a number of scanned rows, and a map of merged
* Scans hbase:meta and returns a number of scanned rows, and a map of merged
* regions, and an ordered map of split parents. if the given table name is
* null, return merged regions and split parents of all tables, else only the
* specified table
@ -132,14 +132,14 @@ public class CatalogJanitor extends Chore {
Triple<Integer, Map<HRegionInfo, Result>, Map<HRegionInfo, Result>> getMergedRegionsAndSplitParents(
final TableName tableName) throws IOException {
final boolean isTableSpecified = (tableName != null);
// TODO: Only works with single .META. region currently. Fix.
// TODO: Only works with single hbase:meta region currently. Fix.
final AtomicInteger count = new AtomicInteger(0);
// Keep Map of found split parents. There are candidates for cleanup.
// Use a comparator that has split parents come before its daughters.
final Map<HRegionInfo, Result> splitParents =
new TreeMap<HRegionInfo, Result>(new SplitParentFirstComparator());
final Map<HRegionInfo, Result> mergedRegions = new TreeMap<HRegionInfo, Result>();
// This visitor collects split parents and counts rows in the .META. table
// This visitor collects split parents and counts rows in the hbase:meta table
MetaScannerVisitor visitor = new MetaScanner.MetaScannerVisitorBase() {
@Override
@ -162,7 +162,7 @@ public class CatalogJanitor extends Chore {
}
};
// Run full scan of .META. catalog table passing in our custom visitor with
// Run full scan of hbase:meta catalog table passing in our custom visitor with
// the start row
MetaScanner.metaScan(server.getConfiguration(), null, visitor, tableName);
@ -172,11 +172,11 @@ public class CatalogJanitor extends Chore {
/**
* If merged region no longer holds reference to the merge regions, archive
* merge region on hdfs and perform deleting references in .META.
* merge region on hdfs and perform deleting references in hbase:meta
* @param mergedRegion
* @param regionA
* @param regionB
* @return true if we delete references in merged region on .META. and archive
* @return true if we delete references in merged region on hbase:meta and archive
* the files on the file system
* @throws IOException
*/
@ -207,7 +207,7 @@ public class CatalogJanitor extends Chore {
}
/**
* Run janitorial scan of catalog <code>.META.</code> table looking for
* Run janitorial scan of catalog <code>hbase:meta</code> table looking for
* garbage to collect.
* @return number of cleaned regions
* @throws IOException

View File

@ -147,7 +147,7 @@ public class ClusterStatusPublisher extends Chore {
// We're reusing an existing protobuf message, but we don't send everything.
// This could be extended in the future, for example if we want to send stuff like the
// META server name.
// hbase:meta server name.
ClusterStatus cs = new ClusterStatus(VersionInfo.getVersion(),
master.getMasterFileSystem().getClusterId().toString(),
null,

View File

@ -826,7 +826,7 @@ MasterServices, Server {
}
// get a list for previously failed RS which need log splitting work
// we recover .META. region servers inside master initialization and
// we recover hbase:meta region servers inside master initialization and
// handle other failed servers in SSH in order to start up master node ASAP
Set<ServerName> previouslyFailedServers = this.fileSystemManager
.getFailedServersFromLogFolders();
@ -834,7 +834,7 @@ MasterServices, Server {
// remove stale recovering regions from previous run
this.fileSystemManager.removeStaleRecoveringRegionsFromZK(previouslyFailedServers);
// log splitting for .META. server
// log splitting for hbase:meta server
ServerName oldMetaServerLocation = this.catalogTracker.getMetaLocation();
if (oldMetaServerLocation != null && previouslyFailedServers.contains(oldMetaServerLocation)) {
splitMetaLogBeforeAssignment(oldMetaServerLocation);
@ -853,20 +853,20 @@ MasterServices, Server {
// Make sure meta assigned before proceeding.
status.setStatus("Assigning Meta Region");
assignMeta(status);
// check if master is shutting down because above assignMeta could return even META isn't
// check if master is shutting down because above assignMeta could return even hbase:meta isn't
// assigned when master is shutting down
if(this.stopped) return;
if (this.distributedLogReplay && (!previouslyFailedMetaRSs.isEmpty())) {
// replay WAL edits mode need new .META. RS is assigned firstly
// replay WAL edits mode need new hbase:meta RS is assigned firstly
status.setStatus("replaying log for Meta Region");
// need to use union of previouslyFailedMetaRSs recorded in ZK and previouslyFailedServers
// instead of oldMetaServerLocation to address the following two situations:
// 1) the chained failure situation(recovery failed multiple times in a row).
// 2) master get killed right before it could delete the recovering META from ZK while the
// 2) master get killed right before it could delete the recovering hbase:meta from ZK while the
// same server still has non-meta wals to be replayed so that
// removeStaleRecoveringRegionsFromZK can't delete the stale META region
// Passing more servers into splitMetaLog is all right. If a server doesn't have .META. wal,
// removeStaleRecoveringRegionsFromZK can't delete the stale hbase:meta region
// Passing more servers into splitMetaLog is all right. If a server doesn't have hbase:meta wal,
// there is no op for the server.
previouslyFailedMetaRSs.addAll(previouslyFailedServers);
this.fileSystemManager.splitMetaLog(previouslyFailedMetaRSs);
@ -879,7 +879,7 @@ MasterServices, Server {
enableServerShutdownHandler();
status.setStatus("Submitting log splitting work for previously failed region servers");
// Master has recovered META region server and we put
// Master has recovered hbase:meta region server and we put
// other failed region servers in a queue to be handled later by SSH
for (ServerName tmpServer : previouslyFailedServers) {
this.serverManager.processDeadServer(tmpServer, true);
@ -974,7 +974,7 @@ MasterServices, Server {
}
/**
* Check <code>.META.</code> is assigned. If not, assign it.
* Check <code>hbase:meta</code> is assigned. If not, assign it.
* @param status MonitoredTask
* @throws InterruptedException
* @throws IOException
@ -987,7 +987,7 @@ MasterServices, Server {
long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
boolean beingExpired = false;
status.setStatus("Assigning META region");
status.setStatus("Assigning hbase:meta region");
assignmentManager.getRegionStates().createRegionState(HRegionInfo.FIRST_META_REGIONINFO);
boolean rit = this.assignmentManager
@ -1002,15 +1002,15 @@ MasterServices, Server {
splitMetaLogBeforeAssignment(currentMetaServer);
}
assignmentManager.assignMeta();
// Make sure a .META. location is set.
// Make sure a hbase:meta location is set.
enableSSHandWaitForMeta();
assigned++;
if (beingExpired && this.distributedLogReplay) {
// In Replay WAL Mode, we need the new .META. server online
// In Replay WAL Mode, we need the new hbase:meta server online
this.fileSystemManager.splitMetaLog(currentMetaServer);
}
} else if (rit && !metaRegionLocation) {
// Make sure a .META. location is set.
// Make sure a hbase:meta location is set.
enableSSHandWaitForMeta();
assigned++;
} else {
@ -1020,19 +1020,19 @@ MasterServices, Server {
}
enableMeta(TableName.META_TABLE_NAME);
LOG.info(".META. assigned=" + assigned + ", rit=" + rit +
LOG.info("hbase:meta assigned=" + assigned + ", rit=" + rit +
", location=" + catalogTracker.getMetaLocation());
status.setStatus("META assigned.");
}
private void splitMetaLogBeforeAssignment(ServerName currentMetaServer) throws IOException {
if (this.distributedLogReplay) {
// In log replay mode, we mark META region as recovering in ZK
// In log replay mode, we mark hbase:meta region as recovering in ZK
Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
regions.add(HRegionInfo.FIRST_META_REGIONINFO);
this.fileSystemManager.prepareLogReplay(currentMetaServer, regions);
} else {
// In recovered.edits mode: create recovered edits file for .META. server
// In recovered.edits mode: create recovered edits file for hbase:meta server
this.fileSystemManager.splitMetaLog(currentMetaServer);
}
}
@ -1054,7 +1054,7 @@ MasterServices, Server {
// See HBASE-6281.
Set<TableName> disabledOrDisablingOrEnabling = ZKTable.getDisabledOrDisablingTables(zooKeeper);
disabledOrDisablingOrEnabling.addAll(ZKTable.getEnablingTables(zooKeeper));
// Scan META for all system regions, skipping any disabled tables
// Scan hbase:meta for all system regions, skipping any disabled tables
Map<HRegionInfo, ServerName> allRegions =
MetaReader.fullScan(catalogTracker, disabledOrDisablingOrEnabling, true);
for(Iterator<HRegionInfo> iter = allRegions.keySet().iterator();
@ -1172,7 +1172,7 @@ MasterServices, Server {
}
/**
* This function returns a set of region server names under .META. recovering region ZK node
* This function returns a set of region server names under hbase:meta recovering region ZK node
* @return Set of meta server names which were recorded in ZK
* @throws KeeperException
*/
@ -2454,7 +2454,7 @@ MasterServices, Server {
this.activeMasterManager.clusterHasActiveMaster.notifyAll();
}
}
// If no region server is online then master may stuck waiting on .META. to come on line.
// If no region server is online then master may stuck waiting on hbase:meta to come on line.
// See HBASE-8422.
if (this.catalogTracker != null && this.serverManager.getOnlineServers().isEmpty()) {
this.catalogTracker.stop();
@ -2512,7 +2512,7 @@ MasterServices, Server {
/**
* Report whether this master has started initialization and is about to do meta region assignment
* @return true if master is in initialization & about to assign META regions
* @return true if master is in initialization & about to assign hbase:meta regions
*/
public boolean isInitializationStartsMetaRegionAssignment() {
return this.initializationBeforeMetaAssignment;

View File

@ -135,7 +135,7 @@ public class MasterFileSystem {
* Create initial layout in filesystem.
* <ol>
* <li>Check if the meta region exists and is readable, if not create it.
* Create hbase.version and the .META. directory if not one.
* Create hbase.version and the hbase:meta directory if not one.
* </li>
* <li>Create a log archive directory for RS to put archived logs</li>
* </ol>
@ -482,7 +482,7 @@ public class MasterFileSystem {
.migrateFSTableDescriptorsIfNecessary(fs, rd);
}
// Create tableinfo-s for META if not already there.
// Create tableinfo-s for hbase:meta if not already there.
new FSTableDescriptors(fs, rd).createTableDescriptor(HTableDescriptor.META_TABLEDESC);
return rd;
@ -516,7 +516,7 @@ public class MasterFileSystem {
private static void bootstrap(final Path rd, final Configuration c)
throws IOException {
LOG.info("BOOTSTRAP: creating META region");
LOG.info("BOOTSTRAP: creating hbase:meta region");
try {
// Bootstrapping, make sure blockcache is off. Else, one will be
// created here in bootstrap and it'll need to be cleaned up. Better to
@ -536,7 +536,7 @@ public class MasterFileSystem {
}
/**
* Enable in memory caching for .META.
* Enable in memory caching for hbase:meta
*/
public static void setInfoFamilyCachingForMeta(final boolean b) {
for (HColumnDescriptor hcd:

View File

@ -631,20 +631,20 @@ public class RegionPlacementMaintainer {
}
/**
* Update the assignment plan into .META.
* @param plan the assignments plan to be updated into .META.
* @throws IOException if cannot update assignment plan in .META.
* Update the assignment plan into hbase:meta
* @param plan the assignments plan to be updated into hbase:meta
* @throws IOException if cannot update assignment plan in hbase:meta
*/
public void updateAssignmentPlanToMeta(FavoredNodesPlan plan)
throws IOException {
try {
LOG.info("Start to update the META with the new assignment plan");
LOG.info("Start to update the hbase:meta with the new assignment plan");
Map<HRegionInfo, List<ServerName>> assignmentMap =
plan.getAssignmentMap();
FavoredNodeAssignmentHelper.updateMetaWithFavoredNodesInfo(assignmentMap, conf);
LOG.info("Updated the META with the new assignment plan");
LOG.info("Updated the hbase:meta with the new assignment plan");
} catch (Exception e) {
LOG.error("Failed to update META with the new assignment" +
LOG.error("Failed to update hbase:meta with the new assignment" +
"plan because " + e.getMessage());
}
}
@ -727,13 +727,13 @@ public class RegionPlacementMaintainer {
public void updateAssignmentPlan(FavoredNodesPlan plan)
throws IOException {
LOG.info("Start to update the new assignment plan for the META table and" +
LOG.info("Start to update the new assignment plan for the hbase:meta table and" +
" the region servers");
// Update the new assignment plan to META
updateAssignmentPlanToMeta(plan);
// Update the new assignment plan to Region Servers
updateAssignmentPlanToRegionServers(plan);
LOG.info("Finish to update the new assignment plan for the META table and" +
LOG.info("Finish to update the new assignment plan for the hbase:meta table and" +
" the region servers");
}
@ -950,9 +950,9 @@ public class RegionPlacementMaintainer {
public static void main(String args[]) throws IOException {
Options opt = new Options();
opt.addOption("w", "write", false, "write the assignments to META only");
opt.addOption("w", "write", false, "write the assignments to hbase:meta only");
opt.addOption("u", "update", false,
"update the assignments to META and RegionServers together");
"update the assignments to hbase:meta and RegionServers together");
opt.addOption("n", "dry-run", false, "do not write assignments to META");
opt.addOption("v", "verify", false, "verify current assignments against META");
opt.addOption("p", "print", false, "print the current assignment plan in META");
@ -1047,7 +1047,7 @@ public class RegionPlacementMaintainer {
// Verify the region placement.
rp.verifyRegionPlacement(verificationDetails);
} else if (cmd.hasOption("n") || cmd.hasOption("dry-run")) {
// Generate the assignment plan only without updating the META and RS
// Generate the assignment plan only without updating the hbase:meta and RS
FavoredNodesPlan plan = rp.getNewAssignmentPlan();
printAssignmentPlan(plan);
} else if (cmd.hasOption("w") || cmd.hasOption("write")) {
@ -1062,7 +1062,7 @@ public class RegionPlacementMaintainer {
FavoredNodesPlan plan = rp.getNewAssignmentPlan();
// Print the new assignment plan
printAssignmentPlan(plan);
// Update the assignment to META and Region Servers
// Update the assignment to hbase:meta and Region Servers
rp.updateAssignmentPlan(plan);
} else if (cmd.hasOption("diff")) {
FavoredNodesPlan newPlan = rp.getNewAssignmentPlan();

View File

@ -413,7 +413,7 @@ public class RegionStates {
/**
* Gets the online regions of the specified table.
* This method looks at the in-memory state. It does not go to <code>.META.</code>.
* This method looks at the in-memory state. It does not go to <code>hbase:meta</code>.
* Only returns <em>online</em> regions. If a region on this table has been
* closed during a disable, etc., it will be included in the returned list.
* So, the returned list may not necessarily be ALL regions in this table, its
@ -562,7 +562,7 @@ public class RegionStates {
}
/**
* Get the HRegionInfo from cache, if not there, from the META table
* Get the HRegionInfo from cache, if not there, from the hbase:meta table
* @param regionName
* @return HRegionInfo for the region
*/
@ -583,7 +583,7 @@ public class RegionStates {
return hri;
} catch (IOException e) {
server.abort("Aborting because error occoured while reading "
+ Bytes.toStringBinary(regionName) + " from .META.", e);
+ Bytes.toStringBinary(regionName) + " from hbase:meta", e);
return null;
}
}

View File

@ -66,7 +66,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
/** the regionServer to region map */
private final Map<ServerName, List<HRegionInfo>> regionServerToRegionMap;
/** the existing assignment plan in the META region */
/** the existing assignment plan in the hbase:meta region */
private final FavoredNodesPlan existingAssignmentPlan;
private final Set<TableName> disabledTables;
private final boolean excludeOfflinedSplitParents;
@ -88,11 +88,11 @@ public class SnapshotOfRegionAssignmentFromMeta {
}
/**
* Initialize the region assignment snapshot by scanning the META table
* Initialize the region assignment snapshot by scanning the hbase:meta table
* @throws IOException
*/
public void initialize() throws IOException {
LOG.info("Start to scan the META for the current region assignment " +
LOG.info("Start to scan the hbase:meta for the current region assignment " +
"snappshot");
// TODO: at some point this code could live in the MetaReader
Visitor v = new Visitor() {
@ -132,10 +132,10 @@ public class SnapshotOfRegionAssignmentFromMeta {
}
}
};
// Scan .META. to pick up user regions
// Scan hbase:meta to pick up user regions
MetaReader.fullScan(tracker, v);
//regionToRegionServerMap = regions;
LOG.info("Finished to scan the META for the current region assignment" +
LOG.info("Finished to scan the hbase:meta for the current region assignment" +
"snapshot");
}

View File

@ -307,7 +307,7 @@ public class SplitLogManager extends ZooKeeperListener {
}
/**
* The caller will block until all the META log files of the given region server
* The caller will block until all the hbase:meta log files of the given region server
* have been processed - successfully split or an error is encountered - by an
* available worker region server. This method must only be called after the
* region servers have been brought online.

View File

@ -672,9 +672,9 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
return 1000000; // return a number much greater than any of the other cost
}
// META region is special
// hbase:meta region is special
if (cluster.numMovedMetaRegions > 0) {
// assume each META region move costs 10 times
// assume each hbase:meta region move costs 10 times
moveCost += META_MOVE_COST_MULT * cluster.numMovedMetaRegions;
}

View File

@ -86,7 +86,7 @@ public class ClosedRegionHandler extends EventHandler implements TotesHRegionInf
}
return getClass().getSimpleName() + "-" + name + "-" + getSeqid();
}
@Override
public void process() {
LOG.debug("Handling CLOSED event for " + regionInfo.getEncodedName());

View File

@ -86,7 +86,7 @@ public class CreateTableHandler extends EventHandler {
public CreateTableHandler prepare()
throws NotAllMetaRegionsOnlineException, TableExistsException, IOException {
int timeout = conf.getInt("hbase.client.catalog.timeout", 10000);
// Need META availability to create a table
// Need hbase:meta availability to create a table
try {
if(catalogTracker.waitForMeta(timeout) == null) {
throw new NotAllMetaRegionsOnlineException();
@ -109,7 +109,7 @@ public class CreateTableHandler extends EventHandler {
// If we have multiple client threads trying to create the table at the
// same time, given the async nature of the operation, the table
// could be in a state where .META. table hasn't been updated yet in
// could be in a state where hbase:meta table hasn't been updated yet in
// the process() function.
// Use enabling state to tell if there is already a request for the same
// table in progress. This will introduce a new zookeeper call. Given
@ -267,7 +267,7 @@ public class CreateTableHandler extends EventHandler {
}
/**
* Add the specified set of regions to the META table.
* Add the specified set of regions to the hbase:meta table.
*/
protected void addRegionsToMeta(final CatalogTracker ct, final List<HRegionInfo> regionInfos)
throws IOException {

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.zookeeper.KeeperException;
/**
* Shutdown handler for the server hosting <code>.META.</code>
* Shutdown handler for the server hosting <code>hbase:meta</code>
*/
@InterfaceAudience.Private
public class MetaServerShutdownHandler extends ServerShutdownHandler {
@ -56,7 +56,7 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
AssignmentManager am = this.services.getAssignmentManager();
try {
if (this.shouldSplitHlog) {
LOG.info("Splitting META logs for " + serverName);
LOG.info("Splitting hbase:meta logs for " + serverName);
if (this.distributedLogReplay) {
Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
regions.add(HRegionInfo.FIRST_META_REGIONINFO);
@ -125,12 +125,12 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
}
/**
* Before assign the META region, ensure it haven't
* Before assign the hbase:meta region, ensure it haven't
* been assigned by other place
* <p>
* Under some scenarios, the META region can be opened twice, so it seemed online
* Under some scenarios, the hbase:meta region can be opened twice, so it seemed online
* in two regionserver at the same time.
* If the META region has been assigned, so the operation can be canceled.
* If the hbase:meta region has been assigned, so the operation can be canceled.
* @throws InterruptedException
* @throws IOException
* @throws KeeperException
@ -142,10 +142,10 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
if (!this.server.getCatalogTracker().verifyMetaRegionLocation(timeout)) {
this.services.getAssignmentManager().assignMeta();
} else if (serverName.equals(server.getCatalogTracker().getMetaLocation())) {
throw new IOException(".META. is onlined on the dead server "
throw new IOException("hbase:meta is onlined on the dead server "
+ serverName);
} else {
LOG.info("Skip assigning .META., because it is online on the "
LOG.info("Skip assigning hbase:meta, because it is online on the "
+ server.getCatalogTracker().getMetaLocation());
}
}

View File

@ -86,7 +86,7 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
public HRegionInfo getHRegionInfo() {
return this.regionInfo;
}
@Override
public String toString() {
String name = "UnknownServerName";

View File

@ -99,7 +99,7 @@ public class ServerShutdownHandler extends EventHandler {
}
/**
* @return True if the server we are processing was carrying <code>.META.</code>
* @return True if the server we are processing was carrying <code>hbase:meta</code>
*/
boolean isCarryingMeta() {
return false;
@ -121,16 +121,16 @@ public class ServerShutdownHandler extends EventHandler {
try {
// We don't want worker thread in the MetaServerShutdownHandler
// executor pool to block by waiting availability of .META.
// executor pool to block by waiting availability of hbase:meta
// Otherwise, it could run into the following issue:
// 1. The current MetaServerShutdownHandler instance For RS1 waits for the .META.
// 1. The current MetaServerShutdownHandler instance For RS1 waits for the hbase:meta
// to come online.
// 2. The newly assigned .META. region server RS2 was shutdown right after
// it opens the .META. region. So the MetaServerShutdownHandler
// 2. The newly assigned hbase:meta region server RS2 was shutdown right after
// it opens the hbase:meta region. So the MetaServerShutdownHandler
// instance For RS1 will still be blocked.
// 3. The new instance of MetaServerShutdownHandler for RS2 is queued.
// 4. The newly assigned .META. region server RS3 was shutdown right after
// it opens the .META. region. So the MetaServerShutdownHandler
// 4. The newly assigned hbase:meta region server RS3 was shutdown right after
// it opens the hbase:meta region. So the MetaServerShutdownHandler
// instance For RS1 and RS2 will still be blocked.
// 5. The new instance of MetaServerShutdownHandler for RS3 is queued.
// 6. Repeat until we run out of MetaServerShutdownHandler worker threads
@ -141,7 +141,7 @@ public class ServerShutdownHandler extends EventHandler {
// If AssignmentManager hasn't finished rebuilding user regions,
// we are not ready to assign dead regions either. So we re-queue up
// the dead server for further processing too.
if (isCarryingMeta() // .META.
if (isCarryingMeta() // hbase:meta
|| !services.getAssignmentManager().isFailoverCleanupDone()) {
this.services.getServerManager().processDeadServer(serverName, this.shouldSplitHlog);
return;
@ -150,18 +150,18 @@ public class ServerShutdownHandler extends EventHandler {
// Wait on meta to come online; we need it to progress.
// TODO: Best way to hold strictly here? We should build this retry logic
// into the MetaReader operations themselves.
// TODO: Is the reading of .META. necessary when the Master has state of
// cluster in its head? It should be possible to do without reading .META.
// in all but one case. On split, the RS updates the .META.
// TODO: Is the reading of hbase:meta necessary when the Master has state of
// cluster in its head? It should be possible to do without reading hbase:meta
// in all but one case. On split, the RS updates the hbase:meta
// table and THEN informs the master of the split via zk nodes in
// 'unassigned' dir. Currently the RS puts ephemeral nodes into zk so if
// the regionserver dies, these nodes do not stick around and this server
// shutdown processing does fixup (see the fixupDaughters method below).
// If we wanted to skip the .META. scan, we'd have to change at least the
// If we wanted to skip the hbase:meta scan, we'd have to change at least the
// final SPLIT message to be permanent in zk so in here we'd know a SPLIT
// completed (zk is updated after edits to .META. have gone in). See
// completed (zk is updated after edits to hbase:meta have gone in). See
// {@link SplitTransaction}. We'd also have to be figure another way for
// doing the below .META. daughters fixup.
// doing the below hbase:meta daughters fixup.
NavigableMap<HRegionInfo, Result> hris = null;
while (!this.server.isStopped()) {
try {
@ -173,8 +173,8 @@ public class ServerShutdownHandler extends EventHandler {
Thread.currentThread().interrupt();
throw new IOException("Interrupted", e);
} catch (IOException ioe) {
LOG.info("Received exception accessing META during server shutdown of " +
serverName + ", retrying META read", ioe);
LOG.info("Received exception accessing hbase:meta during server shutdown of " +
serverName + ", retrying hbase:meta read", ioe);
}
}
if (this.server.isStopped()) {
@ -340,7 +340,7 @@ public class ServerShutdownHandler extends EventHandler {
return false;
}
if (hri.isOffline() && hri.isSplit()) {
//HBASE-7721: Split parent and daughters are inserted into META as an atomic operation.
//HBASE-7721: Split parent and daughters are inserted into hbase:meta as an atomic operation.
//If the meta scanner saw the parent split, then it should see the daughters as assigned
//to the dead server. We don't have to do anything.
return false;

View File

@ -95,7 +95,7 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
/**
* Create the on-disk regions, using the tableRootDir provided by the CreateTableHandler.
* The cloned table will be created in a temp directory, and then the CreateTableHandler
* will be responsible to add the regions returned by this method to META and do the assignment.
* will be responsible to add the regions returned by this method to hbase:meta and do the assignment.
*/
@Override
protected List<HRegionInfo> handleCreateHdfsRegions(final Path tableRootDir,

View File

@ -143,7 +143,7 @@ public final class MasterSnapshotVerifier {
/**
* Check that all the regions in the snapshot are valid, and accounted for.
* @param snapshotDir snapshot directory to check
* @throws IOException if we can't reach .META. or read the files from the FS
* @throws IOException if we can't reach hbase:meta or read the files from the FS
*/
private void verifyRegions(Path snapshotDir) throws IOException {
List<HRegionInfo> regions = MetaReader.getTableRegions(this.services.getCatalogTracker(),

View File

@ -100,8 +100,8 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
* The restore table is executed in place.
* - The on-disk data will be restored - reference files are put in place without moving data
* - [if something fail here: you need to delete the table and re-run the restore]
* - META will be updated
* - [if something fail here: you need to run hbck to fix META entries]
* - hbase:meta will be updated
* - [if something fail here: you need to run hbck to fix hbase:meta entries]
* The passed in list gets changed in this method
*/
@Override
@ -133,7 +133,7 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
// which is the same state that the regions will be after a delete table.
forceRegionsOffline(metaChanges);
// 4. Applies changes to .META.
// 4. Applies changes to hbase:meta
status.setStatus("Preparing to restore each region");
// 4.1 Removes the current set of regions from META
@ -152,7 +152,7 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
//
// At this point the old regions are no longer present in META.
// and the set of regions present in the snapshot will be written to META.
// All the information in META are coming from the .regioninfo of each region present
// All the information in hbase:meta are coming from the .regioninfo of each region present
// in the snapshot folder.
hris.clear();
if (metaChanges.hasRegionsToAdd()) hris.addAll(metaChanges.getRegionsToAdd());

View File

@ -62,7 +62,7 @@ import org.apache.zookeeper.KeeperException;
/**
* A handler for taking snapshots from the master.
*
* This is not a subclass of TableEventHandler because using that would incur an extra META scan.
* This is not a subclass of TableEventHandler because using that would incur an extra hbase:meta scan.
*
* The {@link #snapshotRegions(List)} call should get implemented for each snapshot flavor.
*/

View File

@ -56,7 +56,7 @@ class GetClosestRowBeforeTracker {
* @param kv Presume first on row: i.e. empty column, maximum timestamp and
* a type of Type.Maximum
* @param ttl Time to live in ms for this Store
* @param metaregion True if this is .META. or -ROOT- region.
* @param metaregion True if this is hbase:meta or -ROOT- region.
*/
GetClosestRowBeforeTracker(final KVComparator c, final KeyValue kv,
final long ttl, final boolean metaregion) {

View File

@ -599,7 +599,7 @@ public class HRegion implements HeapSize { // , Writable{
coprocessorHost.preOpen();
}
// Write HRI to a file in case we need to recover .META.
// Write HRI to a file in case we need to recover hbase:meta
status.setStatus("Writing region info on filesystem");
fs.checkRegionInfoOnFilesystem();
@ -4171,9 +4171,9 @@ public class HRegion implements HeapSize { // , Writable{
/**
* Inserts a new region's meta information into the passed
* <code>meta</code> region. Used by the HMaster bootstrap code adding
* new table to META table.
* new table to hbase:meta table.
*
* @param meta META HRegion to be updated
* @param meta hbase:meta HRegion to be updated
* @param r HRegion to add to <code>meta</code>
*
* @throws IOException

View File

@ -742,7 +742,7 @@ public class HRegionFileSystem {
LOG.warn(REGION_INFO_FILE + " file not found for region: " + regionInfo.getEncodedName());
}
// Write HRI to a file in case we need to recover .META.
// Write HRI to a file in case we need to recover hbase:meta
writeRegionInfoOnFilesystem(content, true);
}
@ -780,7 +780,7 @@ public class HRegionFileSystem {
FSUtils.delete(fs, tmpPath, true);
}
// Write HRI to a file in case we need to recover .META.
// Write HRI to a file in case we need to recover hbase:meta
writeRegionInfoFileContent(conf, fs, tmpPath, regionInfoContent);
// Move the created file to the original path
@ -788,7 +788,7 @@ public class HRegionFileSystem {
throw new IOException("Unable to rename " + tmpPath + " to " + regionInfoFile);
}
} else {
// Write HRI to a file in case we need to recover .META.
// Write HRI to a file in case we need to recover hbase:meta
writeRegionInfoFileContent(conf, fs, regionInfoFile, regionInfoContent);
}
}
@ -817,7 +817,7 @@ public class HRegionFileSystem {
throw new IOException("Unable to create region directory: " + regionDir);
}
// Write HRI to a file in case we need to recover .META.
// Write HRI to a file in case we need to recover hbase:meta
regionFs.writeRegionInfoOnFilesystem(false);
return regionFs;
}
@ -848,7 +848,7 @@ public class HRegionFileSystem {
regionFs.cleanupSplitsDir();
regionFs.cleanupMergesDir();
// if it doesn't exists, Write HRI to a file, in case we need to recover .META.
// if it doesn't exists, Write HRI to a file, in case we need to recover hbase:meta
regionFs.checkRegionInfoOnFilesystem();
}

View File

@ -1654,7 +1654,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
//TODO: at some point this should delegate to the HLogFactory
//currently, we don't care about the region as much as we care about the
//table.. (hence checking the tablename below)
//_ROOT_ and .META. regions have separate WAL.
//_ROOT_ and hbase:meta regions have separate WAL.
if (regionInfo != null && regionInfo.isMetaTable()) {
return getMetaWAL();
}
@ -2322,7 +2322,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
/**
* Gets the online regions of the specified table.
* This method looks at the in-memory onlineRegions. It does not go to <code>.META.</code>.
* This method looks at the in-memory onlineRegions. It does not go to <code>hbase:meta</code>.
* Only returns <em>online</em> regions. If a region on this table has been
* closed during a disable, etc., it will not be included in the returned list.
* So, the returned list may not necessarily be ALL regions in this table, its
@ -3496,7 +3496,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
if (onlineRegion.getCoprocessorHost() != null) {
onlineRegion.getCoprocessorHost().preOpen();
}
// See HBASE-5094. Cross check with META if still this RS is owning
// See HBASE-5094. Cross check with hbase:meta if still this RS is owning
// the region.
Pair<HRegionInfo, ServerName> p = MetaReader.getRegion(
this.catalogTracker, region.getRegionName());
@ -3516,7 +3516,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
}
} else {
LOG.warn("The region " + region.getEncodedName() + " is online on this server" +
" but META does not have this server - continue opening.");
" but hbase:meta does not have this server - continue opening.");
removeFromOnlineRegions(onlineRegion, null);
}
}

View File

@ -48,7 +48,7 @@ import com.google.protobuf.TextFormat;
/**
* A guava function that will return a priority for use by QoS facility in regionserver; e.g.
* rpcs to .META. and -ROOT-, etc., get priority.
* rpcs to hbase:meta and -ROOT-, etc., get priority.
*/
// TODO: Remove. This is doing way too much work just to figure a priority. Do as Elliott
// suggests and just have the client specify a priority.

View File

@ -112,7 +112,7 @@ class RegionMergeRequest implements Runnable {
}
return;
}
LOG.info("Regions merged, META updated, and report to master. region_a="
LOG.info("Regions merged, hbase:meta updated, and report to master. region_a="
+ region_a + ", region_b=" + region_b + ",merged region="
+ mt.getMergedRegionInfo().getRegionNameAsString()
+ ". Region merge took "

View File

@ -202,7 +202,7 @@ public class RegionMergeTransaction {
}
// WARN: make sure there is no parent region of the two merging regions in
// .META. If exists, fixing up daughters would cause daughter regions(we
// hbase:meta If exists, fixing up daughters would cause daughter regions(we
// have merged one) online again when we restart master, so we should clear
// the parent region to prevent the above case
// Since HBASE-7721, we don't need fix up daughters any more. so here do
@ -327,7 +327,7 @@ public class RegionMergeTransaction {
this.journal.add(JournalEntry.PONR);
// Add merged region and delete region_a and region_b
// as an atomic update. See HBASE-7721. This update to META makes the region
// as an atomic update. See HBASE-7721. This update to hbase:meta makes the region
// will determine whether the region is merged or not in case of failures.
// If it is successful, master will roll-forward, if not, master will
// rollback
@ -408,7 +408,7 @@ public class RegionMergeTransaction {
final HRegionInfo b) {
long rid = EnvironmentEdgeManager.currentTimeMillis();
// Regionid is timestamp. Merged region's id can't be less than that of
// merging regions else will insert at wrong location in .META.
// merging regions else will insert at wrong location in hbase:meta
if (rid < a.getRegionId() || rid < b.getRegionId()) {
LOG.warn("Clock skew; merging regions id are " + a.getRegionId()
+ " and " + b.getRegionId() + ", but current time here is " + rid);
@ -772,7 +772,7 @@ public class RegionMergeTransaction {
}
/**
* Checks if the given region has merge qualifier in .META.
* Checks if the given region has merge qualifier in hbase:meta
* @param services
* @param regionName name of specified region
* @return true if the given region has merge qualifier in META.(It will be

View File

@ -106,7 +106,7 @@ class SplitRequest implements Runnable {
}
return;
}
LOG.info("Region split, META updated, and report to master. Parent="
LOG.info("Region split, hbase:meta updated, and report to master. Parent="
+ parent.getRegionNameAsString() + ", new regions: "
+ st.getFirstDaughter().getRegionNameAsString() + ", "
+ st.getSecondDaughter().getRegionNameAsString() + ". Split took "

View File

@ -181,7 +181,7 @@ public class SplitTransaction {
private static long getDaughterRegionIdTimestamp(final HRegionInfo hri) {
long rid = EnvironmentEdgeManager.currentTimeMillis();
// Regionid is timestamp. Can't be less than that of parent else will insert
// at wrong location in .META. (See HBASE-710).
// at wrong location in hbase:meta (See HBASE-710).
if (rid < hri.getRegionId()) {
LOG.warn("Clock skew; parent regions id is " + hri.getRegionId() +
" but current time here is " + rid);
@ -306,8 +306,8 @@ public class SplitTransaction {
this.journal.add(JournalEntry.STARTED_REGION_B_CREATION);
HRegion b = this.parent.createDaughterRegionFromSplits(this.hri_b);
// This is the point of no return. Adding subsequent edits to .META. as we
// do below when we do the daughter opens adding each to .META. can fail in
// This is the point of no return. Adding subsequent edits to hbase:meta as we
// do below when we do the daughter opens adding each to hbase:meta can fail in
// various interesting ways the most interesting of which is a timeout
// BUT the edits all go through (See HBASE-3872). IF we reach the PONR
// then subsequent failures need to crash out this regionserver; the
@ -315,7 +315,7 @@ public class SplitTransaction {
// The offlined parent will have the daughters as extra columns. If
// we leave the daughter regions in place and do not remove them when we
// crash out, then they will have their references to the parent in place
// still and the server shutdown fixup of .META. will point to these
// still and the server shutdown fixup of hbase:meta will point to these
// regions.
// We should add PONR JournalEntry before offlineParentInMeta,so even if
// OfflineParentInMeta timeout,this will cause regionserver exit,and then
@ -324,7 +324,7 @@ public class SplitTransaction {
this.journal.add(JournalEntry.PONR);
// Edit parent in meta. Offlines parent region and adds splita and splitb
// as an atomic update. See HBASE-7721. This update to META makes the region
// as an atomic update. See HBASE-7721. This update to hbase:meta makes the region
// will determine whether the region is split or not in case of failures.
// If it is successful, master will roll-forward, if not, master will rollback
// and assign the parent region.

View File

@ -240,7 +240,7 @@ public class OpenRegionHandler extends EventHandler {
/**
* Update ZK or META. This can take a while if for example the
* .META. is not available -- if server hosting .META. crashed and we are
* hbase:meta is not available -- if server hosting hbase:meta crashed and we are
* waiting on it to come back -- so run in a thread and keep updating znode
* state meantime so master doesn't timeout our region-in-transition.
* Caller must cleanup region if this fails.

View File

@ -48,7 +48,7 @@ public interface HLog {
/** File Extension used while splitting an HLog into regions (HBASE-2312) */
String SPLITTING_EXT = "-splitting";
boolean SPLIT_SKIP_ERRORS_DEFAULT = false;
/** The META region's HLog filename extension */
/** The hbase:meta region's HLog filename extension */
String META_HLOG_FILE_EXTN = ".meta";
/**

View File

@ -410,7 +410,7 @@ public class HLogKey implements WritableComparable<HLogKey> {
} catch (IllegalArgumentException iae) {
if (Bytes.toString(tablenameBytes).equals(TableName.OLD_META_STR)) {
// It is a pre-namespace meta table edit, continue with new format.
LOG.info("Got an old META edit, continuing with new format ");
LOG.info("Got an old .META. edit, continuing with new format ");
this.tablename = TableName.META_TABLE_NAME;
this.encodedRegionName = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
} else if (Bytes.toString(tablenameBytes).equals(TableName.OLD_ROOT_STR)) {

View File

@ -325,7 +325,7 @@ public class HLogPrettyPrinter {
options.addOption("j", "json", false, "Output JSON");
options.addOption("p", "printvals", false, "Print values");
options.addOption("r", "region", true,
"Region to filter by. Pass region name; e.g. '.META.,,1'");
"Region to filter by. Pass region name; e.g. 'hbase:meta,,1'");
options.addOption("s", "sequence", true,
"Sequence to filter by. Pass sequence number.");
options.addOption("w", "row", true, "Row to filter by. Pass row name.");

View File

@ -1528,7 +1528,7 @@ public class HLogSplitter {
// fetch location from cache
HRegionLocation loc = onlineRegions.get(originalEncodedRegionName);
if(loc != null) return loc;
// fetch location from .META. directly without using cache to avoid hit old dead server
// fetch location from hbase:meta directly without using cache to avoid hit old dead server
loc = hconn.getRegionLocation(table, row, true);
if (loc == null) {
throw new IOException("Can't locate location for row:" + Bytes.toString(row)

View File

@ -387,7 +387,7 @@ Content-Type: text/plain
urls,http|www.legacy.com|80|site=Legacy|aamsz=300x250||position=1|prod
=1,1244851990859
urls,http|weather.boston.com|80|LYNX.js,1244851990859
.META.,,1
hbase:meta,,1
content,601292a839b95e50200d8f8767859864,1244869158156
content,9d7f3aeb2a5c1e2b45d690a91de3f23c,1244879698031
content,7f6d48830ef51d635e9a5b672e79a083,1244879698031

View File

@ -210,7 +210,7 @@ public class AccessController extends BaseRegionObserver
HRegionInfo hri = e.getRegion().getRegionInfo();
TableName tableName = hri.getTableName();
// 1. All users need read access to .META. table.
// 1. All users need read access to hbase:meta table.
// this is a very common operation, so deal with it quickly.
if (hri.isMetaRegion()) {
if (permRequest == Permission.Action.READ) {
@ -224,10 +224,10 @@ public class AccessController extends BaseRegionObserver
permRequest, tableName, families);
}
// Users with CREATE/ADMIN rights need to modify .META. and _acl_ table
// e.g. When a new table is created a new entry in .META. is added,
// Users with CREATE/ADMIN rights need to modify hbase:meta and _acl_ table
// e.g. When a new table is created a new entry in hbase:meta is added,
// so the user need to be allowed to write on it.
// e.g. When a table is removed an entry is removed from .META. and _acl_
// e.g. When a table is removed an entry is removed from hbase:meta and _acl_
// and the user need to be allowed to write on both tables.
if (permRequest == Permission.Action.WRITE &&
(hri.isMetaRegion() ||

View File

@ -219,7 +219,7 @@ public class RestoreSnapshotHelper {
}
/**
* Describe the set of operations needed to update META after restore.
* Describe the set of operations needed to update hbase:meta after restore.
*/
public static class RestoreMetaChanges {
private final Map<String, Pair<String, String> > parentsMap;
@ -258,7 +258,7 @@ public class RestoreSnapshotHelper {
/**
* Returns the list of 'restored regions' during the on-disk restore.
* The caller is responsible to add the regions to META if not present.
* The caller is responsible to add the regions to hbase:meta if not present.
* @return the list of regions restored
*/
public List<HRegionInfo> getRegionsToRestore() {

View File

@ -50,7 +50,7 @@ public class FSTableDescriptorMigrationToSubdir {
}
/**
* Determines if migration is required by checking to see whether the META table has been
* Determines if migration is required by checking to see whether the hbase:meta table has been
* migrated.
*/
private static boolean needsMigration(FileSystem fs, Path rootDir) throws IOException {
@ -66,7 +66,7 @@ public class FSTableDescriptorMigrationToSubdir {
* First migrates snapshots.
* Then migrates each user table in order,
* then attempts ROOT (should be gone)
* Migrates META last to indicate migration is complete.
* Migrates hbase:meta last to indicate migration is complete.
*/
private static void migrateFsTableDescriptors(FileSystem fs, Path rootDir) throws IOException {
// First migrate snapshots - will migrate any snapshot dir that contains a table info file

View File

@ -148,7 +148,7 @@ public class FSTableDescriptors implements TableDescriptors {
cachehits++;
return HTableDescriptor.META_TABLEDESC;
}
// .META. is already handled. If some one tries to get the descriptor for
// hbase:meta is already handled. If some one tries to get the descriptor for
// .logs, .oldlogs or .corrupt throw an exception.
if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(tablename.getNameAsString())) {
throw new IOException("No descriptor found for non table = " + tablename);
@ -490,7 +490,7 @@ public class FSTableDescriptors implements TableDescriptors {
*/
private TableDescriptorAndModtime getTableDescriptorAndModtime(TableName tableName)
throws IOException {
// ignore both -ROOT- and .META. tables
// ignore both -ROOT- and hbase:meta tables
if (tableName.equals(TableName.META_TABLE_NAME)) {
return null;
}

View File

@ -975,7 +975,7 @@ public abstract class FSUtils {
// TODO move this method OUT of FSUtils. No dependencies to HMaster
/**
* Returns the total overall fragmentation percentage. Includes .META. and
* Returns the total overall fragmentation percentage. Includes hbase:meta and
* -ROOT- as well.
*
* @param master The master defining the HBase root and file system.
@ -990,7 +990,7 @@ public abstract class FSUtils {
/**
* Runs through the HBase rootdir and checks how many stores for each table
* have more than one file in them. Checks -ROOT- and .META. too. The total
* have more than one file in them. Checks -ROOT- and hbase:meta too. The total
* percentage across all tables is stored under the special key "-TOTAL-".
*
* @param master The master defining the HBase root and file system.
@ -1009,7 +1009,7 @@ public abstract class FSUtils {
/**
* Runs through the HBase rootdir and checks how many stores for each table
* have more than one file in them. Checks -ROOT- and .META. too. The total
* have more than one file in them. Checks -ROOT- and hbase:meta too. The total
* percentage across all tables is stored under the special key "-TOTAL-".
*
* @param fs The file system to use.

View File

@ -119,7 +119,7 @@ import com.google.protobuf.ServiceException;
* HBaseFsck (hbck) is a tool for checking and repairing region consistency and
* table integrity problems in a corrupted HBase.
* <p>
* Region consistency checks verify that .META., region deployment on region
* Region consistency checks verify that hbase:meta, region deployment on region
* servers and the state of data in HDFS (.regioninfo files) all are in
* accordance.
* <p>
@ -131,7 +131,7 @@ import com.google.protobuf.ServiceException;
* The general repair strategy works in two phases:
* <ol>
* <li> Repair Table Integrity on HDFS. (merge or fabricate regions)
* <li> Repair Region Consistency with .META. and assignments
* <li> Repair Region Consistency with hbase:meta and assignments
* </ol>
* <p>
* For table integrity repairs, the tables' region directories are scanned
@ -143,7 +143,7 @@ import com.google.protobuf.ServiceException;
* <p>
* Table integrity repairs deal solely with HDFS and could potentially be done
* offline -- the hbase region servers or master do not need to be running.
* This phase can eventually be used to completely reconstruct the META table in
* This phase can eventually be used to completely reconstruct the hbase:meta table in
* an offline fashion.
* <p>
* Region consistency requires three conditions -- 1) valid .regioninfo file
@ -203,7 +203,7 @@ public class HBaseFsck extends Configured implements Tool {
private boolean fixTableLocks = false; // fix table locks which are expired
// limit checking/fixes to listed tables, if empty attempt to check/fix all
// .META. are always checked
// hbase:meta are always checked
private Set<TableName> tablesIncluded = new HashSet<TableName>();
private int maxMerge = DEFAULT_MAX_MERGE; // maximum number of overlapping regions to merge
private int maxOverlapsToSideline = DEFAULT_OVERLAPS_TO_SIDELINE; // maximum number of overlapping regions to sideline
@ -229,7 +229,7 @@ public class HBaseFsck extends Configured implements Tool {
private TreeMap<String, HbckInfo> regionInfoMap = new TreeMap<String, HbckInfo>();
private TreeSet<TableName> disabledTables =
new TreeSet<TableName>();
// Empty regioninfo qualifiers in .META.
// Empty regioninfo qualifiers in hbase:meta
private Set<Result> emptyRegionInfoQualifiers = new HashSet<Result>();
/**
@ -385,7 +385,7 @@ public class HBaseFsck extends Configured implements Tool {
/**
* This repair method requires the cluster to be online since it contacts
* region servers and the masters. It makes each region's state in HDFS, in
* .META., and deployments consistent.
* hbase:meta, and deployments consistent.
*
* @return If > 0 , number of errors detected, if < 0 there was an unrecoverable
* error. If 0, we have a clean hbase.
@ -396,32 +396,32 @@ public class HBaseFsck extends Configured implements Tool {
// get regions according to what is online on each RegionServer
loadDeployedRegions();
// check whether .META. is deployed and online
// check whether hbase:meta is deployed and online
if (!recordMetaRegion()) {
// Will remove later if we can fix it
errors.reportError("Fatal error: unable to get .META. region location. Exiting...");
errors.reportError("Fatal error: unable to get hbase:meta region location. Exiting...");
return -2;
}
// Check if .META. is found only once and in the right place
// Check if hbase:meta is found only once and in the right place
if (!checkMetaRegion()) {
String errorMsg = ".META. table is not consistent. ";
String errorMsg = "hbase:meta table is not consistent. ";
if (shouldFixAssignments()) {
errorMsg += "HBCK will try fixing it. Rerun once .META. is back to consistent state.";
errorMsg += "HBCK will try fixing it. Rerun once hbase:meta is back to consistent state.";
} else {
errorMsg += "Run HBCK with proper fix options to fix .META. inconsistency.";
errorMsg += "Run HBCK with proper fix options to fix hbase:meta inconsistency.";
}
errors.reportError(errorMsg + " Exiting...");
return -2;
}
// Not going with further consistency check for tables when META itself is not consistent.
LOG.info("Loading regionsinfo from the .META. table");
// Not going with further consistency check for tables when hbase:meta itself is not consistent.
LOG.info("Loading regionsinfo from the hbase:meta table");
boolean success = loadMetaEntries();
if (!success) return -1;
// Empty cells in .META.?
// Empty cells in hbase:meta?
reportEmptyMetaCells();
// Check if we have to cleanup empty REGIONINFO_QUALIFIER rows from .META.
// Check if we have to cleanup empty REGIONINFO_QUALIFIER rows from hbase:meta
if (shouldFixEmptyMetaCells()) {
fixEmptyMetaCells();
}
@ -648,7 +648,7 @@ public class HBaseFsck extends Configured implements Tool {
isReference = StoreFileInfo.isReference(path);
} catch (Throwable t) {
// Ignore. Some files may not be store files at all.
// For example, files under .oldlogs folder in .META.
// For example, files under .oldlogs folder in hbase:meta
// Warning message is already logged by
// StoreFile#isReference.
}
@ -694,7 +694,7 @@ public class HBaseFsck extends Configured implements Tool {
* TODO -- need to add tests for this.
*/
private void reportEmptyMetaCells() {
errors.print("Number of empty REGIONINFO_QUALIFIER rows in .META.: " +
errors.print("Number of empty REGIONINFO_QUALIFIER rows in hbase:meta: " +
emptyRegionInfoQualifiers.size());
if (details) {
for (Result r: emptyRegionInfoQualifiers) {
@ -806,7 +806,7 @@ public class HBaseFsck extends Configured implements Tool {
// get table name from hdfs, populate various HBaseFsck tables.
TableName tableName = hbi.getTableName();
if (tableName == null) {
// There was an entry in META not in the HDFS?
// There was an entry in hbase:meta not in the HDFS?
LOG.warn("tableName was null for: " + hbi);
continue;
}
@ -878,12 +878,12 @@ public class HBaseFsck extends Configured implements Tool {
}
/**
* To fix the empty REGIONINFO_QUALIFIER rows from .META. <br>
* To fix the empty REGIONINFO_QUALIFIER rows from hbase:meta <br>
* @throws IOException
*/
public void fixEmptyMetaCells() throws IOException {
if (shouldFixEmptyMetaCells() && !emptyRegionInfoQualifiers.isEmpty()) {
LOG.info("Trying to fix empty REGIONINFO_QUALIFIER .META. rows.");
LOG.info("Trying to fix empty REGIONINFO_QUALIFIER hbase:meta rows.");
for (Result region : emptyRegionInfoQualifiers) {
deleteMetaRegion(region.getRow());
errors.getErrorList().remove(ERROR_CODE.EMPTY_META_CELL);
@ -957,7 +957,7 @@ public class HBaseFsck extends Configured implements Tool {
/**
* This borrows code from MasterFileSystem.bootstrap()
*
* @return an open .META. HRegion
* @return an open hbase:meta HRegion
*/
private HRegion createNewMeta() throws IOException {
Path rootdir = FSUtils.getRootDir(getConf());
@ -983,7 +983,7 @@ public class HBaseFsck extends Configured implements Tool {
for (Entry<TableName, TableInfo> e : tablesInfo.entrySet()) {
TableName name = e.getKey();
// skip ".META."
// skip "hbase:meta"
if (name.compareTo(TableName.META_TABLE_NAME) == 0) {
continue;
}
@ -1066,23 +1066,23 @@ public class HBaseFsck extends Configured implements Tool {
}
// we can rebuild, move old meta out of the way and start
LOG.info("HDFS regioninfo's seems good. Sidelining old .META.");
LOG.info("HDFS regioninfo's seems good. Sidelining old hbase:meta");
Path backupDir = sidelineOldMeta();
LOG.info("Creating new .META.");
LOG.info("Creating new hbase:meta");
HRegion meta = createNewMeta();
// populate meta
List<Put> puts = generatePuts(tablesInfo);
if (puts == null) {
LOG.fatal("Problem encountered when creating new .META. entries. " +
"You may need to restore the previously sidelined .META.");
LOG.fatal("Problem encountered when creating new hbase:meta entries. " +
"You may need to restore the previously sidelined hbase:meta");
return false;
}
meta.batchMutate(puts.toArray(new Put[0]));
HRegion.closeHRegion(meta);
LOG.info("Success! .META. table rebuilt.");
LOG.info("Old .META. is moved into " + backupDir);
LOG.info("Success! hbase:meta table rebuilt.");
LOG.info("Old hbase:meta is moved into " + backupDir);
return true;
}
@ -1223,7 +1223,7 @@ public class HBaseFsck extends Configured implements Tool {
* @return Path to backup of original directory
*/
Path sidelineOldMeta() throws IOException {
// put current .META. aside.
// put current hbase:meta aside.
Path hbaseDir = FSUtils.getRootDir(getConf());
FileSystem fs = hbaseDir.getFileSystem(getConf());
Path backupDir = getSidelineDir();
@ -1233,7 +1233,7 @@ public class HBaseFsck extends Configured implements Tool {
sidelineTable(fs, TableName.META_TABLE_NAME, hbaseDir, backupDir);
} catch (IOException e) {
LOG.fatal("... failed to sideline meta. Currently in inconsistent state. To restore "
+ "try to rename .META. in " + backupDir.getName() + " to "
+ "try to rename hbase:meta in " + backupDir.getName() + " to "
+ hbaseDir.getName() + ".", e);
throw e; // throw original exception
}
@ -1333,7 +1333,7 @@ public class HBaseFsck extends Configured implements Tool {
}
/**
* Record the location of the META region as found in ZooKeeper.
* Record the location of the hbase:meta region as found in ZooKeeper.
*/
private boolean recordMetaRegion() throws IOException {
HRegionLocation metaLocation = connection.locateRegion(
@ -1575,7 +1575,7 @@ public class HBaseFsck extends Configured implements Tool {
HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
if (hri == null) {
LOG.warn("Unable to close region " + hi.getRegionNameAsString()
+ " because META had invalid or missing "
+ " because hbase:meta had invalid or missing "
+ HConstants.CATALOG_FAMILY_STR + ":"
+ Bytes.toString(HConstants.REGIONINFO_QUALIFIER)
+ " qualifier value.");
@ -1638,13 +1638,13 @@ public class HBaseFsck extends Configured implements Tool {
LOG.warn("Region " + descriptiveName + " was recently modified -- skipping");
return;
}
// ========== Cases where the region is not in META =============
// ========== Cases where the region is not in hbase:meta =============
else if (!inMeta && !inHdfs && !isDeployed) {
// We shouldn't have record of this region at all then!
assert false : "Entry for region with no data";
} else if (!inMeta && !inHdfs && isDeployed) {
errors.reportError(ERROR_CODE.NOT_IN_META_HDFS, "Region "
+ descriptiveName + ", key=" + key + ", not on HDFS or in META but " +
+ descriptiveName + ", key=" + key + ", not on HDFS or in hbase:meta but " +
"deployed on " + Joiner.on(", ").join(hbi.deployedOn));
if (shouldFixAssignments()) {
undeployRegions(hbi);
@ -1652,7 +1652,7 @@ public class HBaseFsck extends Configured implements Tool {
} else if (!inMeta && inHdfs && !isDeployed) {
errors.reportError(ERROR_CODE.NOT_IN_META_OR_DEPLOYED, "Region "
+ descriptiveName + " on HDFS, but not listed in META " +
+ descriptiveName + " on HDFS, but not listed in hbase:meta " +
"or deployed on any region server");
// restore region consistency of an adopted orphan
if (shouldFixMeta()) {
@ -1663,7 +1663,7 @@ public class HBaseFsck extends Configured implements Tool {
return;
}
LOG.info("Patching .META. with .regioninfo: " + hbi.getHdfsHRI());
LOG.info("Patching hbase:meta with .regioninfo: " + hbi.getHdfsHRI());
HBaseFsckRepair.fixMetaHoleOnline(getConf(), hbi.getHdfsHRI());
tryAssignmentRepair(hbi, "Trying to reassign region...");
@ -1679,13 +1679,13 @@ public class HBaseFsck extends Configured implements Tool {
return;
}
LOG.info("Patching .META. with with .regioninfo: " + hbi.getHdfsHRI());
LOG.info("Patching hbase:meta with with .regioninfo: " + hbi.getHdfsHRI());
HBaseFsckRepair.fixMetaHoleOnline(getConf(), hbi.getHdfsHRI());
tryAssignmentRepair(hbi, "Trying to fix unassigned region...");
}
// ========== Cases where the region is in META =============
// ========== Cases where the region is in hbase:meta =============
} else if (inMeta && inHdfs && !isDeployed && splitParent) {
// check whether this is an actual error, or just transient state where parent
// is not cleaned
@ -1743,7 +1743,7 @@ public class HBaseFsck extends Configured implements Tool {
}
} else if (inMeta && inHdfs && isMultiplyDeployed) {
errors.reportError(ERROR_CODE.MULTI_DEPLOYED, "Region " + descriptiveName
+ " is listed in META on region server " + hbi.metaEntry.regionServer
+ " is listed in hbase:meta on region server " + hbi.metaEntry.regionServer
+ " but is multiply assigned to region servers " +
Joiner.on(", ").join(hbi.deployedOn));
// If we are trying to fix the errors
@ -1754,7 +1754,7 @@ public class HBaseFsck extends Configured implements Tool {
}
} else if (inMeta && inHdfs && isDeployed && !deploymentMatchesMeta) {
errors.reportError(ERROR_CODE.SERVER_DOES_NOT_MATCH_META, "Region "
+ descriptiveName + " listed in META on region server " +
+ descriptiveName + " listed in hbase:meta on region server " +
hbi.metaEntry.regionServer + " but found on region server " +
hbi.deployedOn.get(0));
// If we are trying to fix the errors
@ -2505,10 +2505,10 @@ public class HBaseFsck extends Configured implements Tool {
}
/**
* Check values in regionInfo for .META.
* Check if zero or more than one regions with META are found.
* Check values in regionInfo for hbase:meta
* Check if zero or more than one regions with hbase:meta are found.
* If there are inconsistencies (i.e. zero or more than one regions
* pretend to be holding the .META.) try to fix that and report an error.
* pretend to be holding the hbase:meta) try to fix that and report an error.
* @throws IOException from HBaseFsckRepair functions
* @throws KeeperException
* @throws InterruptedException
@ -2521,15 +2521,15 @@ public class HBaseFsck extends Configured implements Tool {
}
}
// There will be always one entry in regionInfoMap corresponding to .META.
// There will be always one entry in regionInfoMap corresponding to hbase:meta
// Check the deployed servers. It should be exactly one server.
HbckInfo metaHbckInfo = metaRegions.get(0);
List<ServerName> servers = metaHbckInfo.deployedOn;
if (servers.size() != 1) {
if (servers.size() == 0) {
errors.reportError(ERROR_CODE.NO_META_REGION, ".META. is not found on any region.");
errors.reportError(ERROR_CODE.NO_META_REGION, "hbase:meta is not found on any region.");
if (shouldFixAssignments()) {
errors.print("Trying to fix a problem with .META...");
errors.print("Trying to fix a problem with hbase:meta..");
setShouldRerun();
// try to fix it (treat it as unassigned region)
HBaseFsckRepair.fixUnassigned(admin, metaHbckInfo.metaEntry);
@ -2537,9 +2537,9 @@ public class HBaseFsck extends Configured implements Tool {
}
} else if (servers.size() > 1) {
errors
.reportError(ERROR_CODE.MULTI_META_REGION, ".META. is found on more than one region.");
.reportError(ERROR_CODE.MULTI_META_REGION, "hbase:meta is found on more than one region.");
if (shouldFixAssignments()) {
errors.print("Trying to fix a problem with .META...");
errors.print("Trying to fix a problem with hbase:meta..");
setShouldRerun();
// try fix it (treat is a dupe assignment)
HBaseFsckRepair.fixMultiAssignment(admin, metaHbckInfo.metaEntry, servers);
@ -2553,7 +2553,7 @@ public class HBaseFsck extends Configured implements Tool {
}
/**
* Scan .META., adding all regions found to the regionInfo map.
* Scan hbase:meta, adding all regions found to the regionInfo map.
* @throws IOException if an error is encountered
*/
boolean loadMetaEntries() throws IOException {
@ -2570,13 +2570,13 @@ public class HBaseFsck extends Configured implements Tool {
public boolean processRow(Result result) throws IOException {
try {
// record the latest modification of this META record
// record the latest modification of this hbase:meta record
long ts = Collections.max(result.list(), comp).getTimestamp();
Pair<HRegionInfo, ServerName> pair = HRegionInfo.getHRegionInfoAndServerName(result);
if (pair == null || pair.getFirst() == null) {
emptyRegionInfoQualifiers.add(result);
errors.reportError(ERROR_CODE.EMPTY_META_CELL,
"Empty REGIONINFO_QUALIFIER found in .META.");
"Empty REGIONINFO_QUALIFIER found in hbase:meta");
return true;
}
ServerName sn = null;
@ -2596,7 +2596,7 @@ public class HBaseFsck extends Configured implements Tool {
} else if (previous.metaEntry == null) {
previous.metaEntry = m;
} else {
throw new IOException("Two entries in META are same " + previous);
throw new IOException("Two entries in hbase:meta are same " + previous);
}
// show proof of progress to the user, once for every 100 records.
@ -2612,7 +2612,7 @@ public class HBaseFsck extends Configured implements Tool {
}
};
if (!checkMetaOnly) {
// Scan .META. to pick up user regions
// Scan hbase:meta to pick up user regions
MetaScanner.metaScan(getConf(), visitor);
}
@ -3246,8 +3246,8 @@ public class HBaseFsck extends Configured implements Tool {
}
/**
* Set META check mode.
* Print only info about META table deployment/state
* Set hbase:meta check mode.
* Print only info about hbase:meta table deployment/state
*/
void setCheckMetaOnly() {
checkMetaOnly = true;
@ -3420,7 +3420,7 @@ public class HBaseFsck extends Configured implements Tool {
/**
* We are interested in only those tables that have not changed their state in
* META during the last few seconds specified by hbase.admin.fsck.timelag
* hbase:meta during the last few seconds specified by hbase.admin.fsck.timelag
* @param seconds - the time in seconds
*/
public void setTimeLag(long seconds) {
@ -3468,7 +3468,7 @@ public class HBaseFsck extends Configured implements Tool {
out.println(" -sleepBeforeRerun <timeInSeconds> Sleep this many seconds" +
" before checking if the fix worked if run with -fix");
out.println(" -summary Print only summary of the tables and status.");
out.println(" -metaonly Only check the state of the .META. table.");
out.println(" -metaonly Only check the state of the hbase:meta table.");
out.println(" -sidelineDir <hdfs://> HDFS path to backup existing meta.");
out.println("");
@ -3477,7 +3477,7 @@ public class HBaseFsck extends Configured implements Tool {
out.println(" -fixAssignments Try to fix region assignments. Replaces the old -fix");
out.println(" -fixMeta Try to fix meta problems. This assumes HDFS region info is good.");
out.println(" -noHdfsChecking Don't load/check region info from HDFS."
+ " Assumes META region info is good. Won't check/fix any HDFS issue, e.g. hole, orphan, or overlap");
+ " Assumes hbase:meta region info is good. Won't check/fix any HDFS issue, e.g. hole, orphan, or overlap");
out.println(" -fixHdfsHoles Try to fix region holes in hdfs.");
out.println(" -fixHdfsOrphans Try to fix region dirs with no .regioninfo file in hdfs");
out.println(" -fixTableOrphans Try to fix table dirs with no .tableinfo file in hdfs (online mode only)");
@ -3489,7 +3489,7 @@ public class HBaseFsck extends Configured implements Tool {
out.println(" -fixSplitParents Try to force offline split parents to be online.");
out.println(" -ignorePreCheckPermission ignore filesystem permission pre-check");
out.println(" -fixReferenceFiles Try to offline lingering reference store files");
out.println(" -fixEmptyMetaCells Try to fix .META. entries not referencing any region"
out.println(" -fixEmptyMetaCells Try to fix hbase:meta entries not referencing any region"
+ " (empty REGIONINFO_QUALIFIER rows)");
out.println("");

View File

@ -338,7 +338,7 @@ public class HFileV1Detector extends Configured implements Tool {
}
private static boolean isTableDir(final FileSystem fs, final Path path) throws IOException {
// check for old format, of having /table/.tableinfo; .META. doesn't has .tableinfo,
// check for old format, of having /table/.tableinfo; hbase:meta doesn't has .tableinfo,
// include it.
return (FSTableDescriptors.getTableInfoPath(fs, path) != null || FSTableDescriptors
.getCurrentTableInfoStatus(fs, path, false) != null) || path.toString().endsWith(".META.");

View File

@ -70,7 +70,7 @@ class HMerge {
* Scans the table and merges two adjacent regions if they are small. This
* only happens when a lot of rows are deleted.
*
* When merging the META region, the HBase instance must be offline.
* When merging the hbase:meta region, the HBase instance must be offline.
* When merging a normal table, the HBase instance must be online, but the
* table must be disabled.
*
@ -89,7 +89,7 @@ class HMerge {
* Scans the table and merges two adjacent regions if they are small. This
* only happens when a lot of rows are deleted.
*
* When merging the META region, the HBase instance must be offline.
* When merging the hbase:meta region, the HBase instance must be offline.
* When merging a normal table, the HBase instance must be online, but the
* table must be disabled.
*
@ -116,7 +116,7 @@ class HMerge {
if (tableName.equals(TableName.META_TABLE_NAME)) {
if (masterIsRunning) {
throw new IllegalStateException(
"Can not compact META table if instance is on-line");
"Can not compact hbase:meta table if instance is on-line");
}
// TODO reenable new OfflineMerger(conf, fs).process();
} else {

View File

@ -211,7 +211,7 @@ public class Merge extends Configured implements Tool {
* Removes a region's meta information from the passed <code>meta</code>
* region.
*
* @param meta META HRegion to be updated
* @param meta hbase:meta HRegion to be updated
* @param regioninfo HRegionInfo of region to remove from <code>meta</code>
*
* @throws IOException
@ -231,7 +231,7 @@ public class Merge extends Configured implements Tool {
* Adds a region's meta information from the passed <code>meta</code>
* region.
*
* @param metainfo META HRegionInfo to be updated
* @param metainfo hbase:meta HRegionInfo to be updated
* @param region HRegion to add to <code>meta</code>
*
* @throws IOException

View File

@ -122,7 +122,7 @@ public class MetaUtils {
}
try {
for (HRegion r: metaRegions.values()) {
LOG.info("CLOSING META " + r.toString());
LOG.info("CLOSING hbase:meta " + r.toString());
r.close();
}
} catch (IOException e) {

View File

@ -57,7 +57,7 @@ public abstract class ModifyRegionUtils {
/**
* Create new set of regions on the specified file-system.
* NOTE: that you should add the regions to .META. after this operation.
* NOTE: that you should add the regions to hbase:meta after this operation.
*
* @param conf {@link Configuration}
* @param rootDir Root directory for HBase instance
@ -72,7 +72,7 @@ public abstract class ModifyRegionUtils {
/**
* Create new set of regions on the specified file-system.
* NOTE: that you should add the regions to .META. after this operation.
* NOTE: that you should add the regions to hbase:meta after this operation.
*
* @param conf {@link Configuration}
* @param rootDir Root directory for HBase instance

View File

@ -660,7 +660,7 @@ public class RegionSplitter {
continue;
}
} catch (NoServerForRegionException nsfre) {
// NSFRE will occur if the old META entry has no server assigned
// NSFRE will occur if the old hbase:meta entry has no server assigned
LOG.info(nsfre);
logicalSplitting.add(region);
continue;

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.io.MultipleIOException;
/**
* This code is used to rebuild meta off line from file system data. If there
* are any problem detected, it will fail suggesting actions for the user to do
* to "fix" problems. If it succeeds, it will backup the previous .META. and
* to "fix" problems. If it succeeds, it will backup the previous hbase:meta and
* -ROOT- dirs and write new tables in place.
*
* This is an advanced feature, so is only exposed for use if explicitly

View File

@ -19,7 +19,7 @@
TestMetaMigrationConvertToPB uses the file TestMetaMigrationConvertToPB.tgz for testing
upgrade to 0.96 from 0.92/0.94 cluster data. The files are untarred to the local
filesystem, and copied over to a minidfscluster. However, since the directory
name .META. causes problems on Windows, it has been renamed to -META- inside
name hbase:meta causes problems on Windows, it has been renamed to -META- inside
the .tgz file. After untarring and copying the contents to minidfs,
TestMetaMigrationConvertToPB.setUpBeforeClass() renames the file back to .META.
TestMetaMigrationConvertToPB.setUpBeforeClass() renames the file back to hbase:meta
See https://issues.apache.org/jira/browse/HBASE-6821.

View File

@ -246,7 +246,7 @@ public abstract class HBaseCluster implements Closeable, Configurable {
}
/**
* Get the ServerName of region server serving the first META region
* Get the ServerName of region server serving the first hbase:meta region
*/
public ServerName getServerHoldingMeta() throws IOException {
return getServerHoldingRegion(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());

View File

@ -883,7 +883,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
Configuration c = new Configuration(this.conf);
this.hbaseCluster =
new MiniHBaseCluster(c, numMasters, numSlaves, masterClass, regionserverClass);
// Don't leave here till we've done a successful scan of the .META.
// Don't leave here till we've done a successful scan of the hbase:meta
HTable t = new HTable(c, TableName.META_TABLE_NAME);
ResultScanner s = t.getScanner(new Scan());
while (s.next() != null) {
@ -905,7 +905,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/
public void restartHBaseCluster(int servers) throws IOException, InterruptedException {
this.hbaseCluster = new MiniHBaseCluster(this.conf, servers);
// Don't leave here till we've done a successful scan of the .META.
// Don't leave here till we've done a successful scan of the hbase:meta
HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
ResultScanner s = t.getScanner(new Scan());
while (s.next() != null) {
@ -1146,7 +1146,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
desc.addFamily(hcd);
}
getHBaseAdmin().createTable(desc, startKey, endKey, numRegions);
// HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
return new HTable(getConfiguration(), tableName);
}
@ -1172,7 +1172,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
desc.addFamily(hcd);
}
getHBaseAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
return new HTable(c, tableName);
}
@ -1220,7 +1220,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
desc.addFamily(hcd);
}
getHBaseAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
return new HTable(c, tableName);
}
@ -1304,7 +1304,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
desc.addFamily(hcd);
}
getHBaseAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
return new HTable(new Configuration(getConfiguration()), tableName);
}
@ -1341,7 +1341,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
desc.addFamily(hcd);
}
getHBaseAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
return new HTable(new Configuration(getConfiguration()), tableName);
}
@ -1380,7 +1380,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
i++;
}
getHBaseAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
return new HTable(new Configuration(getConfiguration()), tableName);
}
@ -1412,7 +1412,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
HColumnDescriptor hcd = new HColumnDescriptor(family);
desc.addFamily(hcd);
getHBaseAdmin().createTable(desc, splitRows);
// HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
return new HTable(getConfiguration(), tableName);
}
@ -1433,7 +1433,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
desc.addFamily(hcd);
}
getHBaseAdmin().createTable(desc, splitRows);
// HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(TableName.valueOf(tableName));
return new HTable(getConfiguration(), tableName);
}
@ -1849,7 +1849,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
/**
* Create rows in META for regions of the specified table with the specified
* Create rows in hbase:meta for regions of the specified table with the specified
* start keys. The first startKey should be a 0 length byte array if you
* want to form a proper range of regions.
* @param conf
@ -1878,7 +1878,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
/**
* Returns all rows from the .META. table.
* Returns all rows from the hbase:meta table.
*
* @throws IOException When reading the rows fails.
*/
@ -1898,7 +1898,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
/**
* Returns all rows from the .META. table for a given user table
* Returns all rows from the hbase:meta table for a given user table
*
* @throws IOException When reading the rows fails.
*/
@ -1932,7 +1932,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* It first searches for the meta rows that contain the region of the
* specified table, then gets the index of that RS, and finally retrieves
* the RS's reference.
* @param tableName user table to lookup in .META.
* @param tableName user table to lookup in hbase:meta
* @return region server that holds it, null if the row doesn't exist
* @throws IOException
* @throws InterruptedException
@ -1947,7 +1947,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* It first searches for the meta rows that contain the region of the
* specified table, then gets the index of that RS, and finally retrieves
* the RS's reference.
* @param tableName user table to lookup in .META.
* @param tableName user table to lookup in hbase:meta
* @return region server that holds it, null if the row doesn't exist
* @throws IOException
*/
@ -2496,7 +2496,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
Thread.sleep(200);
}
// Finally make sure all regions are fully open and online out on the cluster. Regions may be
// in the .META. table and almost open on all regionservers but there setting the region
// in the hbase:meta table and almost open on all regionservers but there setting the region
// online in the regionserver is the very last thing done and can take a little while to happen.
// Below we do a get. The get will retry if a NotServeringRegionException or a
// RegionOpeningException. It is crass but when done all will be online.
@ -2618,9 +2618,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
/**
* Wait until all regions for a table in .META. have a non-empty
* Wait until all regions for a table in hbase:meta have a non-empty
* info:server, up to 60 seconds. This means all regions have been deployed,
* master has been informed and updated .META. with the regions deployed
* master has been informed and updated hbase:meta with the regions deployed
* server.
* @param tableName the table name
* @throws IOException
@ -2630,9 +2630,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
/**
* Wait until all regions for a table in .META. have a non-empty
* Wait until all regions for a table in hbase:meta have a non-empty
* info:server, or until timeout. This means all regions have been deployed,
* master has been informed and updated .META. with the regions deployed
* master has been informed and updated hbase:meta with the regions deployed
* server.
* @param tableName the table name
* @param timeout timeout, in milliseconds

View File

@ -627,7 +627,7 @@ public class MiniHBaseCluster extends HBaseCluster {
* Get the location of the specified region
* @param regionName Name of the region in bytes
* @return Index into List of {@link MiniHBaseCluster#getRegionServerThreads()}
* of HRS carrying .META.. Returns -1 if none found.
* of HRS carrying hbase:meta. Returns -1 if none found.
*/
public int getServerWith(byte[] regionName) {
int index = -1;
@ -657,7 +657,7 @@ public class MiniHBaseCluster extends HBaseCluster {
/**
* Counts the total numbers of regions being served by the currently online
* region servers by asking each how many regions they have. Does not look
* at META at all. Count includes catalog tables.
* at hbase:meta at all. Count includes catalog tables.
* @return number of regions being served by all region servers
*/
public long countServedRegions() {

View File

@ -88,11 +88,11 @@ public class TestInfoServers {
new HTable(UTIL.getConfiguration(), TableName.META_TABLE_NAME).close();
int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort();
assertContainsContent(new URL("http://localhost:" + port +
"/master-status"), "META");
"/master-status"), "meta");
port = UTIL.getHBaseCluster().getRegionServerThreads().get(0).getRegionServer().
getInfoServer().getPort();
assertContainsContent(new URL("http://localhost:" + port +
"/rs-status"), "META");
"/rs-status"), "meta");
}
@Test

View File

@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Mocking utility for common META functionality
* Mocking utility for common hbase:meta functionality
*/
public class MetaMockingUtil {
@ -40,7 +40,7 @@ public class MetaMockingUtil {
* Returns a Result object constructed from the given region information simulating
* a catalog table result.
* @param region the HRegionInfo object or null
* @return A mocked up Result that fakes a Get on a row in the <code>.META.</code> table.
* @return A mocked up Result that fakes a Get on a row in the <code>hbase:meta</code> table.
* @throws IOException
*/
public static Result getMetaTableRowResult(final HRegionInfo region)
@ -53,7 +53,7 @@ public class MetaMockingUtil {
* a catalog table result.
* @param region the HRegionInfo object or null
* @param ServerName to use making startcode and server hostname:port in meta or null
* @return A mocked up Result that fakes a Get on a row in the <code>.META.</code> table.
* @return A mocked up Result that fakes a Get on a row in the <code>hbase:meta</code> table.
* @throws IOException
*/
public static Result getMetaTableRowResult(final HRegionInfo region, final ServerName sn)
@ -68,7 +68,7 @@ public class MetaMockingUtil {
* @param ServerName to use making startcode and server hostname:port in meta or null
* @param splita daughter region or null
* @param splitb daughter region or null
* @return A mocked up Result that fakes a Get on a row in the <code>.META.</code> table.
* @return A mocked up Result that fakes a Get on a row in the <code>hbase:meta</code> table.
* @throws IOException
*/
public static Result getMetaTableRowResult(HRegionInfo region, final ServerName sn,
@ -113,7 +113,7 @@ public class MetaMockingUtil {
/**
* @param sn ServerName to use making startcode and server in meta
* @param hri Region to serialize into HRegionInfo
* @return A mocked up Result that fakes a Get on a row in the <code>.META.</code> table.
* @return A mocked up Result that fakes a Get on a row in the <code>hbase:meta</code> table.
* @throws IOException
*/
public static Result getMetaTableRowResultAsSplitRegion(final HRegionInfo hri, final ServerName sn)

View File

@ -108,7 +108,7 @@ public class TestCatalogTracker {
// start fresh in zk.
MetaRegionTracker.deleteMetaLocation(this.watcher);
} catch (KeeperException e) {
LOG.warn("Unable to delete META location", e);
LOG.warn("Unable to delete hbase:meta location", e);
}
// Clear out our doctored connection or could mess up subsequent tests.
@ -126,7 +126,7 @@ public class TestCatalogTracker {
}
/**
* Test that we get notification if .META. moves.
* Test that we get notification if hbase:meta moves.
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
@ -337,7 +337,7 @@ public class TestCatalogTracker {
/**
* @return A mocked up Result that fakes a Get on a row in the
* <code>.META.</code> table.
* <code>hbase:meta</code> table.
* @throws IOException
*/
private Result getMetaTableRowResult() throws IOException {

View File

@ -61,7 +61,7 @@ import org.junit.experimental.categories.Category;
/**
* Test migration that changes HRI serialization into PB. Tests by bringing up a cluster from actual
* data from a 0.92 cluster, as well as manually downgrading and then upgrading the META info.
* data from a 0.92 cluster, as well as manually downgrading and then upgrading the hbase:meta info.
* @deprecated Remove after 0.96
*/
@Category(MediumTests.class)
@ -81,7 +81,7 @@ public class TestMetaMigrationConvertingToPB {
* This test uses a tgz file named "TestMetaMigrationConvertingToPB.tgz" under
* hbase-server/src/test/data which contains file data from a 0.92 cluster.
* The cluster has a table named "TestTable", which has 100 rows. 0.94 has same
* META structure, so it should be the same.
* hbase:meta structure, so it should be the same.
*
* hbase(main):001:0> create 'TestTable', 'f1'
* hbase(main):002:0> for i in 1..100
@ -114,7 +114,7 @@ public class TestMetaMigrationConvertingToPB {
doFsCommand(shell,
new String [] {"-put", untar.toURI().toString(), hbaseRootDir.toString()});
//windows fix: tgz file has .META. directory renamed as -META- since the original is an illegal
//windows fix: tgz file has hbase:meta directory renamed as -META- since the original is an illegal
//name under windows. So we rename it back. See src/test/data//TestMetaMigrationConvertingToPB.README and
//https://issues.apache.org/jira/browse/HBASE-6821
doFsCommand(shell, new String [] {"-mv", new Path(hbaseRootDir, "-META-").toString(),
@ -273,7 +273,7 @@ public class TestMetaMigrationConvertingToPB {
}
/**
* Verify that every META row is updated
* Verify that every hbase:meta row is updated
*/
void verifyMetaRowsAreUpdated(CatalogTracker catalogTracker)
throws IOException {
@ -300,7 +300,7 @@ public class TestMetaMigrationConvertingToPB {
}
}
/** Changes the version of META to 0 to simulate 0.92 and 0.94 clusters*/
/** Changes the version of hbase:meta to 0 to simulate 0.92 and 0.94 clusters*/
private void undoVersionInRoot(CatalogTracker ct) throws IOException {
Put p = new Put(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
@ -312,7 +312,7 @@ public class TestMetaMigrationConvertingToPB {
}
/**
* Inserts multiple regions into META using Writable serialization instead of PB
* Inserts multiple regions into hbase:meta using Writable serialization instead of PB
*/
public int createMultiRegionsWithWritableSerialization(final Configuration c,
final byte[] tableName, int numRegions) throws IOException {
@ -336,7 +336,7 @@ public class TestMetaMigrationConvertingToPB {
}
/**
* Inserts multiple regions into META using Writable serialization instead of PB
* Inserts multiple regions into hbase:meta using Writable serialization instead of PB
*/
public int createMultiRegionsWithWritableSerialization(final Configuration c,
final TableName tableName, byte [][] startKeys)
@ -386,7 +386,7 @@ public class TestMetaMigrationConvertingToPB {
}
/**
* Inserts multiple regions into META using PB serialization
* Inserts multiple regions into hbase:meta using PB serialization
*/
int createMultiRegionsWithPBSerialization(final Configuration c,
final byte[] tableName, int numRegions)
@ -404,7 +404,7 @@ public class TestMetaMigrationConvertingToPB {
}
/**
* Inserts multiple regions into META using PB serialization
* Inserts multiple regions into hbase:meta using PB serialization
*/
int createMultiRegionsWithPBSerialization(final Configuration c, final byte[] tableName,
byte [][] startKeys) throws IOException {

View File

@ -92,7 +92,7 @@ public class TestMetaReaderEditor {
/**
* Does {@link MetaReader#getRegion(CatalogTracker, byte[])} and a write
* against .META. while its hosted server is restarted to prove our retrying
* against hbase:meta while its hosted server is restarted to prove our retrying
* works.
* @throws IOException
* @throws InterruptedException

View File

@ -1619,7 +1619,7 @@ public class TestAdmin {
private HRegionServer startAndWriteData(String tableName, byte[] value)
throws IOException, InterruptedException {
// When the META table can be opened, the region servers are running
// When the hbase:meta table can be opened, the region servers are running
new HTable(
TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME).close();
@ -1701,7 +1701,7 @@ public class TestAdmin {
fail("Expected to throw ConstraintException");
} catch (ConstraintException e) {
}
// Before the fix for HBASE-6146, the below table creation was failing as the META table
// Before the fix for HBASE-6146, the below table creation was failing as the hbase:meta table
// actually getting disabled by the disableTable() call.
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testDisableCatalogTable".getBytes()));
HColumnDescriptor hcd = new HColumnDescriptor("cf1".getBytes());

View File

@ -91,7 +91,7 @@ public class TestMetaScanner {
verify(visitor, times(3)).processRow((Result)anyObject());
// Scanning the table with a specified empty start row should also
// give us three META rows
// give us three hbase:meta rows
reset(visitor);
doReturn(true).when(visitor).processRow((Result)anyObject());
MetaScanner.metaScan(conf, visitor, TABLENAME, HConstants.EMPTY_BYTE_ARRAY, 1000);

View File

@ -113,7 +113,7 @@ public class TestSnapshotFromClient {
}
/**
* Test snapshotting not allowed .META. and -ROOT-
* Test snapshotting not allowed hbase:meta and -ROOT-
* @throws Exception
*/
@Test (timeout=300000)
@ -123,7 +123,7 @@ public class TestSnapshotFromClient {
try {
admin.snapshot(snapshotName, TableName.META_TABLE_NAME);
fail("taking a snapshot of .META. should not be allowed");
fail("taking a snapshot of hbase:meta should not be allowed");
} catch (IllegalArgumentException e) {
// expected
}

View File

@ -268,7 +268,7 @@ public class TestHFileBlock {
public void testReaderV2() throws IOException {
for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
for (boolean pread : new boolean[] { false, true }) {
LOG.info("testReaderV2: Compression algorithm: " + algo +
LOG.info("testReaderV2: Compression algorithm: " + algo +
", pread=" + pread);
Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
+ algo);
@ -295,7 +295,7 @@ public class TestHFileBlock {
b.sanityCheck();
assertEquals(4936, b.getUncompressedSizeWithoutHeader());
assertEquals(algo == GZ ? 2173 : 4936,
assertEquals(algo == GZ ? 2173 : 4936,
b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
String blockStr = b.toString();
@ -393,7 +393,7 @@ public class TestHFileBlock {
static void writeEncodedBlock(Algorithm algo, DataBlockEncoding encoding,
DataOutputStream dos, final List<Integer> encodedSizes,
final List<ByteBuffer> encodedBlocks, int blockId,
final List<ByteBuffer> encodedBlocks, int blockId,
boolean includesMemstoreTS, byte[] dummyHeader) throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DoubleOutputStream doubleOutputStream =
@ -478,7 +478,7 @@ public class TestHFileBlock {
for (boolean pread : BOOLEAN_VALUES) {
for (boolean cacheOnWrite : BOOLEAN_VALUES) {
Random rand = defaultRandom();
LOG.info("testPreviousOffset:Compression algorithm: " + algo +
LOG.info("testPreviousOffset:Compression algorithm: " + algo +
", pread=" + pread +
", cacheOnWrite=" + cacheOnWrite);
Path path = new Path(TEST_UTIL.getDataTestDir(), "prev_offset");
@ -532,7 +532,7 @@ public class TestHFileBlock {
assertEquals(b.getPrevBlockOffset(), b2.getPrevBlockOffset());
assertEquals(curOffset, b2.getOffset());
assertEquals(b.getBytesPerChecksum(), b2.getBytesPerChecksum());
assertEquals(b.getOnDiskDataSizeWithHeader(),
assertEquals(b.getOnDiskDataSizeWithHeader(),
b2.getOnDiskDataSizeWithHeader());
assertEquals(0, HFile.getChecksumFailuresCount());
@ -541,12 +541,12 @@ public class TestHFileBlock {
if (cacheOnWrite) {
// In the cache-on-write mode we store uncompressed bytes so we
// can compare them to what was read by the block reader.
// b's buffer has header + data + checksum while
// b's buffer has header + data + checksum while
// expectedContents have header + data only
ByteBuffer bufRead = b.getBufferWithHeader();
ByteBuffer bufExpected = expectedContents.get(i);
boolean bytesAreCorrect = Bytes.compareTo(bufRead.array(),
bufRead.arrayOffset(),
bufRead.arrayOffset(),
bufRead.limit() - b.totalChecksumBytes(),
bufExpected.array(), bufExpected.arrayOffset(),
bufExpected.limit()) == 0;
@ -565,9 +565,9 @@ public class TestHFileBlock {
+ Bytes.toStringBinary(bufRead.array(),
bufRead.arrayOffset(), Math.min(32, bufRead.limit()));
if (detailedLogging) {
LOG.warn("expected header" +
LOG.warn("expected header" +
HFileBlock.toStringHeader(bufExpected) +
"\nfound header" +
"\nfound header" +
HFileBlock.toStringHeader(bufRead));
LOG.warn("bufread offset " + bufRead.arrayOffset() +
" limit " + bufRead.limit() +
@ -759,7 +759,7 @@ public class TestHFileBlock {
byte[] byteArr = new byte[HConstants.HFILEBLOCK_HEADER_SIZE + size];
ByteBuffer buf = ByteBuffer.wrap(byteArr, 0, size);
HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf,
HFileBlock.FILL_HEADER, -1, includesMemstoreTS,
HFileBlock.FILL_HEADER, -1, includesMemstoreTS,
HFileBlock.MINOR_VERSION_NO_CHECKSUM, 0, ChecksumType.NULL.getCode(),
0);
long byteBufferExpectedSize =

View File

@ -174,7 +174,7 @@ public class TestHFileBlockCompatibility {
public void testReaderV2() throws IOException {
for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
for (boolean pread : new boolean[] { false, true }) {
LOG.info("testReaderV2: Compression algorithm: " + algo +
LOG.info("testReaderV2: Compression algorithm: " + algo +
", pread=" + pread);
Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
+ algo);
@ -199,7 +199,7 @@ public class TestHFileBlockCompatibility {
b.sanityCheck();
assertEquals(4936, b.getUncompressedSizeWithoutHeader());
assertEquals(algo == GZ ? 2173 : 4936,
assertEquals(algo == GZ ? 2173 : 4936,
b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
String blockStr = b.toString();
@ -239,7 +239,7 @@ public class TestHFileBlockCompatibility {
for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
for (boolean pread : new boolean[] { false, true }) {
for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
LOG.info("testDataBlockEncoding algo " + algo +
LOG.info("testDataBlockEncoding algo " + algo +
" pread = " + pread +
" encoding " + encoding);
Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
@ -305,12 +305,12 @@ public class TestHFileBlockCompatibility {
/**
* This is the version of the HFileBlock.Writer that is used to
* create V2 blocks with minor version 0. These blocks do not
* have hbase-level checksums. The code is here to test
* backward compatibility. The reason we do not inherit from
* This is the version of the HFileBlock.Writer that is used to
* create V2 blocks with minor version 0. These blocks do not
* have hbase-level checksums. The code is here to test
* backward compatibility. The reason we do not inherit from
* HFileBlock.Writer is because we never ever want to change the code
* in this class but the code in HFileBlock.Writer will continually
* in this class but the code in HFileBlock.Writer will continually
* evolve.
*/
public static final class Writer {
@ -318,7 +318,7 @@ public class TestHFileBlockCompatibility {
// These constants are as they were in minorVersion 0.
private static final int HEADER_SIZE = HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
private static final boolean DONT_FILL_HEADER = HFileBlock.DONT_FILL_HEADER;
private static final byte[] DUMMY_HEADER =
private static final byte[] DUMMY_HEADER =
HFileBlock.DUMMY_HEADER_NO_CHECKSUM;
private enum State {
@ -711,7 +711,7 @@ public class TestHFileBlockCompatibility {
}
/**
* Creates a new HFileBlock.
* Creates a new HFileBlock.
*/
public HFileBlock getBlockForCaching() {
return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),

View File

@ -189,7 +189,7 @@ public class TestImportExport {
}
/**
* Test export .META. table
* Test export hbase:meta table
*
* @throws Exception
*/

View File

@ -471,7 +471,7 @@ public class TestAssignmentManager {
/**
* To test if the split region is removed from RIT if the region was in SPLITTING state but the RS
* has actually completed the splitting in META but went down. See HBASE-6070 and also HBASE-5806
* has actually completed the splitting in hbase:meta but went down. See HBASE-6070 and also HBASE-5806
*
* @throws KeeperException
* @throws IOException
@ -1067,7 +1067,7 @@ public class TestAssignmentManager {
// it and a get to return the single region, REGIONINFO, this test is
// messing with. Needed when "new master" joins cluster. AM will try and
// rebuild its list of user regions and it will also get the HRI that goes
// with an encoded name by doing a Get on .META.
// with an encoded name by doing a Get on hbase:meta
ClientProtos.ClientService.BlockingInterface ri =
Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
// Get a meta row result that has region up on SERVERNAME_A for REGIONINFO

View File

@ -1314,7 +1314,7 @@ public class TestDistributedLogSplitting {
/**
* Find a RS that has regions of a table.
* @param hasMetaRegion when true, the returned RS has META region as well
* @param hasMetaRegion when true, the returned RS has hbase:meta region as well
* @param tableName
* @return
* @throws Exception

View File

@ -214,7 +214,7 @@ public class TestMasterFailover {
List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
log("Regions in META and namespace have been created");
log("Regions in hbase:meta and namespace have been created");
// at this point we only expect 3 regions to be assigned out (catalogs and namespace)
assertEquals(2, cluster.countServedRegions());
@ -519,7 +519,7 @@ public class TestMasterFailover {
List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
log("Regions in META and Namespace have been created");
log("Regions in hbase:meta and Namespace have been created");
// at this point we only expect 2 regions to be assigned out (catalogs and namespace )
assertEquals(2, cluster.countServedRegions());
@ -870,7 +870,7 @@ public class TestMasterFailover {
}
// TODO: Next test to add is with testing permutations of the RIT or the RS
// killed are hosting ROOT and META regions.
// killed are hosting ROOT and hbase:meta regions.
private void log(String string) {
LOG.info("\n\n" + string + " \n\n");

View File

@ -164,7 +164,7 @@ public class TestMasterStatusServlet {
regionsInTransition.put(hri.getEncodedName(),
new RegionState(hri, RegionState.State.CLOSING, 12345L, FAKE_HOST));
}
// Add META in transition as well
// Add hbase:meta in transition as well
regionsInTransition.put(
HRegionInfo.FIRST_META_REGIONINFO.getEncodedName(),
new RegionState(HRegionInfo.FIRST_META_REGIONINFO,

View File

@ -81,7 +81,7 @@ public class TestMasterTransitions {
/**
* Listener for regionserver events testing hbase-2428 (Infinite loop of
* region closes if META region is offline). In particular, listen
* region closes if hbase:meta region is offline). In particular, listen
* for the close of the 'metaServer' and when it comes in, requeue it with a
* delay as though there were an issue processing the shutdown. As part of
* the requeuing, send over a close of a region on 'otherServer' so it comes
@ -196,7 +196,7 @@ public class TestMasterTransitions {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
final HMaster master = cluster.getMaster();
int metaIndex = cluster.getServerWithMeta();
// Figure the index of the server that is not server the .META.
// Figure the index of the server that is not server the hbase:meta
int otherServerIndex = -1;
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
if (i == metaIndex) continue;
@ -472,7 +472,7 @@ public class TestMasterTransitions {
}
*/
/*
* Add to each of the regions in .META. a value. Key is the startrow of the
* Add to each of the regions in hbase:meta a value. Key is the startrow of the
* region (except its 'aaa' for first region). Actual value is the row name.
* @param expected
* @return

View File

@ -225,7 +225,7 @@ public class TestRegionPlacement {
// Verify all the region server are update with the latest favored nodes
verifyRegionServerUpdated(currentPlan);
// Test Case 2: To verify whether the region placement tools can
// correctly update the new assignment plan to META and Region Server.
// correctly update the new assignment plan to hbase:meta and Region Server.
// The new assignment plan is generated by shuffle the existing assignment
// plan by switching PRIMARY, SECONDARY and TERTIARY nodes.
// Shuffle the plan by switching the secondary region server with
@ -234,7 +234,7 @@ public class TestRegionPlacement {
// Shuffle the secondary with tertiary favored nodes
FavoredNodesPlan shuffledPlan = this.shuffleAssignmentPlan(currentPlan,
FavoredNodesPlan.Position.SECONDARY, FavoredNodesPlan.Position.TERTIARY);
// Let the region placement update the META and Region Servers
// Let the region placement update the hbase:meta and Region Servers
rp.updateAssignmentPlan(shuffledPlan);
// Verify the region assignment. There are supposed to no region reassignment
@ -246,7 +246,7 @@ public class TestRegionPlacement {
shuffledPlan = this.shuffleAssignmentPlan(currentPlan,
FavoredNodesPlan.Position.PRIMARY, FavoredNodesPlan.Position.SECONDARY);
// Let the region placement update the META and Region Servers
// Let the region placement update the hbase:meta and Region Servers
rp.updateAssignmentPlan(shuffledPlan);
verifyRegionAssignment(shuffledPlan, REGION_NUM, REGION_NUM);
@ -417,7 +417,7 @@ public class TestRegionPlacement {
/**
* To verify the region assignment status.
* It will check the assignment plan consistency between META and
* It will check the assignment plan consistency between hbase:meta and
* region servers.
* Also it will verify weather the number of region movement and
* the number regions on the primary region server are expected
@ -431,7 +431,7 @@ public class TestRegionPlacement {
private void verifyRegionAssignment(FavoredNodesPlan plan,
int regionMovementNum, int numRegionsOnPrimaryRS)
throws InterruptedException, IOException {
// Verify the assignment plan in META is consistent with the expected plan.
// Verify the assignment plan in hbase:meta is consistent with the expected plan.
verifyMETAUpdated(plan);
// Verify the number of region movement is expected
@ -541,10 +541,10 @@ public class TestRegionPlacement {
List<ServerName> favoredServerList = plan.getAssignmentMap().get(region.getRegionInfo());
// All regions are supposed to have favored nodes,
// except for META and ROOT
// except for hbase:meta and ROOT
if (favoredServerList == null) {
HTableDescriptor desc = region.getTableDesc();
// Verify they are ROOT and META regions since no favored nodes
// Verify they are ROOT and hbase:meta regions since no favored nodes
assertNull(favoredSocketAddress);
assertTrue("User region " +
region.getTableDesc().getTableName() +
@ -575,7 +575,7 @@ public class TestRegionPlacement {
/**
* Check whether regions are assigned to servers consistent with the explicit
* hints that are persisted in the META table.
* hints that are persisted in the hbase:meta table.
* Also keep track of the number of the regions are assigned to the
* primary region server.
* @return the number of regions are assigned to the primary region server

View File

@ -70,7 +70,7 @@ public class TestRestartCluster {
ZKAssign.createNodeOffline(zooKeeper, HRegionInfo.FIRST_META_REGIONINFO, sn);
LOG.debug("Created UNASSIGNED zNode for ROOT and META regions in state " +
LOG.debug("Created UNASSIGNED zNode for ROOT and hbase:meta regions in state " +
EventType.M_ZK_REGION_OFFLINE);
// start the HB cluster

View File

@ -184,10 +184,10 @@ public class TestRollingRestart {
Thread.sleep(1000);
assertRegionsAssigned(cluster, regions);
// Bring the RS hosting META down
// Bring the RS hosting hbase:meta down
RegionServerThread metaServer = getServerHostingMeta(cluster);
log("Stopping server hosting META #1");
metaServer.getRegionServer().stop("Stopping META server");
log("Stopping server hosting hbase:meta #1");
metaServer.getRegionServer().stop("Stopping hbase:meta server");
cluster.hbaseCluster.waitOnRegionServer(metaServer);
log("Meta server down #1");
expectedNumRS--;
@ -204,10 +204,10 @@ public class TestRollingRestart {
assertRegionsAssigned(cluster, regions);
assertEquals(expectedNumRS, cluster.getRegionServerThreads().size());
// Kill off the server hosting META again
// Kill off the server hosting hbase:meta again
metaServer = getServerHostingMeta(cluster);
log("Stopping server hosting META #2");
metaServer.getRegionServer().stop("Stopping META server");
log("Stopping server hosting hbase:meta #2");
metaServer.getRegionServer().stop("Stopping hbase:meta server");
cluster.hbaseCluster.waitOnRegionServer(metaServer);
log("Meta server down");
expectedNumRS--;
@ -231,8 +231,8 @@ public class TestRollingRestart {
assertRegionsAssigned(cluster, regions);
// Shutdown server hosting META
metaServer = getServerHostingMeta(cluster);
log("Stopping server hosting META (1 of 3)");
metaServer.getRegionServer().stop("Stopping META server");
log("Stopping server hosting hbase:meta (1 of 3)");
metaServer.getRegionServer().stop("Stopping hbase:meta server");
cluster.hbaseCluster.waitOnRegionServer(metaServer);
log("Meta server down (1 of 3)");
log("Waiting for RS shutdown to be handled by master");
@ -243,10 +243,10 @@ public class TestRollingRestart {
log("Verifying there are " + numRegions + " assigned on cluster");
assertRegionsAssigned(cluster, regions);
// Shutdown server hosting META again
// Shutdown server hosting hbase:meta again
metaServer = getServerHostingMeta(cluster);
log("Stopping server hosting META (2 of 3)");
metaServer.getRegionServer().stop("Stopping META server");
log("Stopping server hosting hbase:meta (2 of 3)");
metaServer.getRegionServer().stop("Stopping hbase:meta server");
cluster.hbaseCluster.waitOnRegionServer(metaServer);
log("Meta server down (2 of 3)");
log("Waiting for RS shutdown to be handled by master");
@ -257,10 +257,10 @@ public class TestRollingRestart {
log("Verifying there are " + numRegions + " assigned on cluster");
assertRegionsAssigned(cluster, regions);
// Shutdown server hosting META again
// Shutdown server hosting hbase:meta again
metaServer = getServerHostingMeta(cluster);
log("Stopping server hosting META (3 of 3)");
metaServer.getRegionServer().stop("Stopping META server");
log("Stopping server hosting hbase:meta (3 of 3)");
metaServer.getRegionServer().stop("Stopping hbase:meta server");
cluster.hbaseCluster.waitOnRegionServer(metaServer);
log("Meta server down (3 of 3)");
log("Waiting for RS shutdown to be handled by master");

View File

@ -266,7 +266,7 @@ public class TestZKBasedOpenCloseRegion {
}
/*
* Add to each of the regions in .META. a value. Key is the startrow of the
* Add to each of the regions in hbase:meta a value. Key is the startrow of the
* region (except its 'aaa' for first region). Actual value is the row name.
* @param expected
* @return

Some files were not shown because too many files have changed in this diff Show More