HBASE-2618 Don't inherit from HConstants

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@953790 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2010-06-11 17:49:55 +00:00
parent 296b055f97
commit 4193875580
37 changed files with 493 additions and 420 deletions

View File

@ -681,6 +681,7 @@ Release 0.21.0 - Unreleased
setting (currentTimeMillis) (Daniel Ploeg via Ryan Rawson)
HBASE-2558 Our javadoc overview -- "Getting Started", requirements, etc. --
is not carried across by mvn javadoc:javadoc target
HBASE-2618 Don't inherit from HConstants (Benoit Sigoure via Stack)
NEW FEATURES
HBASE-1961 HBase EC2 scripts

View File

@ -25,17 +25,17 @@ import org.apache.hadoop.hbase.util.Bytes;
/**
* HConstants holds a bunch of HBase-related constants
*/
public interface HConstants {
public final class HConstants {
/** long constant for zero */
static final Long ZERO_L = Long.valueOf(0L);
static final String NINES = "99999999999999";
static final String ZEROES = "00000000000000";
public static final Long ZERO_L = Long.valueOf(0L);
public static final String NINES = "99999999999999";
public static final String ZEROES = "00000000000000";
// For migration
/** name of version file */
static final String VERSION_FILE_NAME = "hbase.version";
public static final String VERSION_FILE_NAME = "hbase.version";
/**
* Current version of file system.
@ -52,98 +52,98 @@ public interface HConstants {
//TODO: Is having HBase homed on port 60k OK?
/** Cluster is in distributed mode or not */
static final String CLUSTER_DISTRIBUTED = "hbase.cluster.distributed";
public static final String CLUSTER_DISTRIBUTED = "hbase.cluster.distributed";
/** Cluster is standalone or pseudo-distributed */
static final String CLUSTER_IS_LOCAL = "false";
public static final String CLUSTER_IS_LOCAL = "false";
/** Cluster is fully-distributed */
static final String CLUSTER_IS_DISTRIBUTED = "true";
public static final String CLUSTER_IS_DISTRIBUTED = "true";
/** default host address */
static final String DEFAULT_HOST = "0.0.0.0";
public static final String DEFAULT_HOST = "0.0.0.0";
/** Parameter name for port master listens on. */
static final String MASTER_PORT = "hbase.master.port";
public static final String MASTER_PORT = "hbase.master.port";
/** default port that the master listens on */
static final int DEFAULT_MASTER_PORT = 60000;
public static final int DEFAULT_MASTER_PORT = 60000;
/** default port for master web api */
static final int DEFAULT_MASTER_INFOPORT = 60010;
public static final int DEFAULT_MASTER_INFOPORT = 60010;
/** Name of ZooKeeper quorum configuration parameter. */
static final String ZOOKEEPER_QUORUM = "hbase.zookeeper.quorum";
public static final String ZOOKEEPER_QUORUM = "hbase.zookeeper.quorum";
/** Name of ZooKeeper config file in conf/ directory. */
static final String ZOOKEEPER_CONFIG_NAME = "zoo.cfg";
public static final String ZOOKEEPER_CONFIG_NAME = "zoo.cfg";
/** Parameter name for number of times to retry writes to ZooKeeper. */
static final String ZOOKEEPER_RETRIES = "zookeeper.retries";
public static final String ZOOKEEPER_RETRIES = "zookeeper.retries";
/** Default number of times to retry writes to ZooKeeper. */
static final int DEFAULT_ZOOKEEPER_RETRIES = 5;
public static final int DEFAULT_ZOOKEEPER_RETRIES = 5;
/** Parameter name for ZooKeeper pause between retries. In milliseconds. */
static final String ZOOKEEPER_PAUSE = "zookeeper.pause";
public static final String ZOOKEEPER_PAUSE = "zookeeper.pause";
/** Default ZooKeeper pause value. In milliseconds. */
static final int DEFAULT_ZOOKEEPER_PAUSE = 2 * 1000;
public static final int DEFAULT_ZOOKEEPER_PAUSE = 2 * 1000;
/** default client port that the zookeeper listens on */
static final int DEFAULT_ZOOKEPER_CLIENT_PORT = 2181;
public static final int DEFAULT_ZOOKEPER_CLIENT_PORT = 2181;
/** Parameter name for the root dir in ZK for this cluster */
static final String ZOOKEEPER_ZNODE_PARENT = "zookeeper.znode.parent";
public static final String ZOOKEEPER_ZNODE_PARENT = "zookeeper.znode.parent";
static final String DEFAULT_ZOOKEEPER_ZNODE_PARENT = "/hbase";
public static final String DEFAULT_ZOOKEEPER_ZNODE_PARENT = "/hbase";
/** Parameter name for port region server listens on. */
static final String REGIONSERVER_PORT = "hbase.regionserver.port";
public static final String REGIONSERVER_PORT = "hbase.regionserver.port";
/** Default port region server listens on. */
static final int DEFAULT_REGIONSERVER_PORT = 60020;
public static final int DEFAULT_REGIONSERVER_PORT = 60020;
/** default port for region server web api */
static final int DEFAULT_REGIONSERVER_INFOPORT = 60030;
public static final int DEFAULT_REGIONSERVER_INFOPORT = 60030;
/** Parameter name for what region server interface to use. */
static final String REGION_SERVER_CLASS = "hbase.regionserver.class";
public static final String REGION_SERVER_CLASS = "hbase.regionserver.class";
/** Parameter name for what region server implementation to use. */
static final String REGION_SERVER_IMPL= "hbase.regionserver.impl";
public static final String REGION_SERVER_IMPL= "hbase.regionserver.impl";
/** Default region server interface class name. */
static final String DEFAULT_REGION_SERVER_CLASS = HRegionInterface.class.getName();
public static final String DEFAULT_REGION_SERVER_CLASS = HRegionInterface.class.getName();
/** Parameter name for how often threads should wake up */
static final String THREAD_WAKE_FREQUENCY = "hbase.server.thread.wakefrequency";
public static final String THREAD_WAKE_FREQUENCY = "hbase.server.thread.wakefrequency";
/** Parameter name for how often a region should should perform a major compaction */
static final String MAJOR_COMPACTION_PERIOD = "hbase.hregion.majorcompaction";
public static final String MAJOR_COMPACTION_PERIOD = "hbase.hregion.majorcompaction";
/** Parameter name for HBase instance root directory */
static final String HBASE_DIR = "hbase.rootdir";
public static final String HBASE_DIR = "hbase.rootdir";
/** Used to construct the name of the log directory for a region server
* Use '.' as a special character to seperate the log files from table data */
static final String HREGION_LOGDIR_NAME = ".logs";
public static final String HREGION_LOGDIR_NAME = ".logs";
/** Like the previous, but for old logs that are about to be deleted */
static final String HREGION_OLDLOGDIR_NAME = ".oldlogs";
public static final String HREGION_OLDLOGDIR_NAME = ".oldlogs";
/** Name of old log file for reconstruction */
static final String HREGION_OLDLOGFILE_NAME = "oldlogfile.log";
public static final String HREGION_OLDLOGFILE_NAME = "oldlogfile.log";
/** Used to construct the name of the compaction directory during compaction */
static final String HREGION_COMPACTIONDIR_NAME = "compaction.dir";
public static final String HREGION_COMPACTIONDIR_NAME = "compaction.dir";
/** Default maximum file size */
static final long DEFAULT_MAX_FILE_SIZE = 256 * 1024 * 1024;
public static final long DEFAULT_MAX_FILE_SIZE = 256 * 1024 * 1024;
/** Default size of a reservation block */
static final int DEFAULT_SIZE_RESERVATION_BLOCK = 1024 * 1024 * 5;
public static final int DEFAULT_SIZE_RESERVATION_BLOCK = 1024 * 1024 * 5;
/** Maximum value length, enforced on KeyValue construction */
static final int MAXIMUM_VALUE_LENGTH = Integer.MAX_VALUE;
public static final int MAXIMUM_VALUE_LENGTH = Integer.MAX_VALUE;
// Always store the location of the root table's HRegion.
// This HRegion is never split.
@ -171,91 +171,91 @@ public interface HConstants {
//
/** The root table's name.*/
static final byte [] ROOT_TABLE_NAME = Bytes.toBytes("-ROOT-");
public static final byte [] ROOT_TABLE_NAME = Bytes.toBytes("-ROOT-");
/** The META table's name. */
static final byte [] META_TABLE_NAME = Bytes.toBytes(".META.");
public static final byte [] META_TABLE_NAME = Bytes.toBytes(".META.");
/** delimiter used between portions of a region name */
public static final int META_ROW_DELIMITER = ',';
/** The catalog family as a string*/
static final String CATALOG_FAMILY_STR = "info";
public static final String CATALOG_FAMILY_STR = "info";
/** The catalog family */
static final byte [] CATALOG_FAMILY = Bytes.toBytes(CATALOG_FAMILY_STR);
public static final byte [] CATALOG_FAMILY = Bytes.toBytes(CATALOG_FAMILY_STR);
/** The catalog historian family */
static final byte [] CATALOG_HISTORIAN_FAMILY = Bytes.toBytes("historian");
public static final byte [] CATALOG_HISTORIAN_FAMILY = Bytes.toBytes("historian");
/** The regioninfo column qualifier */
static final byte [] REGIONINFO_QUALIFIER = Bytes.toBytes("regioninfo");
public static final byte [] REGIONINFO_QUALIFIER = Bytes.toBytes("regioninfo");
/** The server column qualifier */
static final byte [] SERVER_QUALIFIER = Bytes.toBytes("server");
public static final byte [] SERVER_QUALIFIER = Bytes.toBytes("server");
/** The startcode column qualifier */
static final byte [] STARTCODE_QUALIFIER = Bytes.toBytes("serverstartcode");
public static final byte [] STARTCODE_QUALIFIER = Bytes.toBytes("serverstartcode");
/** The lower-half split region column qualifier */
static final byte [] SPLITA_QUALIFIER = Bytes.toBytes("splitA");
public static final byte [] SPLITA_QUALIFIER = Bytes.toBytes("splitA");
/** The upper-half split region column qualifier */
static final byte [] SPLITB_QUALIFIER = Bytes.toBytes("splitB");
public static final byte [] SPLITB_QUALIFIER = Bytes.toBytes("splitB");
// Other constants
/**
* An empty instance.
*/
static final byte [] EMPTY_BYTE_ARRAY = new byte [0];
public static final byte [] EMPTY_BYTE_ARRAY = new byte [0];
/**
* Used by scanners, etc when they want to start at the beginning of a region
*/
static final byte [] EMPTY_START_ROW = EMPTY_BYTE_ARRAY;
public static final byte [] EMPTY_START_ROW = EMPTY_BYTE_ARRAY;
/**
* Last row in a table.
*/
static final byte [] EMPTY_END_ROW = EMPTY_START_ROW;
public static final byte [] EMPTY_END_ROW = EMPTY_START_ROW;
/**
* Used by scanners and others when they're trying to detect the end of a
* table
*/
static final byte [] LAST_ROW = EMPTY_BYTE_ARRAY;
public static final byte [] LAST_ROW = EMPTY_BYTE_ARRAY;
/**
* Max length a row can have because of the limitation in TFile.
*/
static final int MAX_ROW_LENGTH = Short.MAX_VALUE;
public static final int MAX_ROW_LENGTH = Short.MAX_VALUE;
/** When we encode strings, we always specify UTF8 encoding */
static final String UTF8_ENCODING = "UTF-8";
public static final String UTF8_ENCODING = "UTF-8";
/**
* Timestamp to use when we want to refer to the latest cell.
* This is the timestamp sent by clients when no timestamp is specified on
* commit.
*/
static final long LATEST_TIMESTAMP = Long.MAX_VALUE;
public static final long LATEST_TIMESTAMP = Long.MAX_VALUE;
/**
* LATEST_TIMESTAMP in bytes form
*/
static final byte [] LATEST_TIMESTAMP_BYTES = Bytes.toBytes(LATEST_TIMESTAMP);
public static final byte [] LATEST_TIMESTAMP_BYTES = Bytes.toBytes(LATEST_TIMESTAMP);
/**
* Define for 'return-all-versions'.
*/
static final int ALL_VERSIONS = Integer.MAX_VALUE;
public static final int ALL_VERSIONS = Integer.MAX_VALUE;
/**
* Unlimited time-to-live.
*/
// static final int FOREVER = -1;
static final int FOREVER = Integer.MAX_VALUE;
// public static final int FOREVER = -1;
public static final int FOREVER = Integer.MAX_VALUE;
/**
* Seconds in a week
@ -271,9 +271,9 @@ public interface HConstants {
// the shell. They really aren't a part of the public API. It would be
// nice if we could put them somewhere where they did not need to be
// public. They could have package visibility
static final String NAME = "NAME";
static final String VERSIONS = "VERSIONS";
static final String IN_MEMORY = "IN_MEMORY";
public static final String NAME = "NAME";
public static final String VERSIONS = "VERSIONS";
public static final String IN_MEMORY = "IN_MEMORY";
/**
* This is a retry backoff multiplier table similar to the BSD TCP syn
@ -339,4 +339,8 @@ public interface HConstants {
*/
public static long DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD = 60000;
private HConstants() {
// Can't be instantiated with this ctor.
}
}

View File

@ -48,7 +48,7 @@ import java.util.Random;
* A non-instantiable class that has a static method capable of compacting
* a table by merging adjacent regions.
*/
class HMerge implements HConstants {
class HMerge {
static final Log LOG = LogFactory.getLog(HMerge.class);
static final Random rand = new Random();
@ -78,7 +78,7 @@ class HMerge implements HConstants {
HConnection connection = HConnectionManager.getConnection(conf);
boolean masterIsRunning = connection.isMasterRunning();
HConnectionManager.deleteConnectionInfo(conf, false);
if (Bytes.equals(tableName, META_TABLE_NAME)) {
if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
if (masterIsRunning) {
throw new IllegalStateException(
"Can not compact META table if instance is on-line");
@ -106,16 +106,16 @@ class HMerge implements HConstants {
throws IOException {
this.conf = conf;
this.fs = fs;
this.maxFilesize =
conf.getLong("hbase.hregion.max.filesize", DEFAULT_MAX_FILE_SIZE);
this.maxFilesize = conf.getLong("hbase.hregion.max.filesize",
HConstants.DEFAULT_MAX_FILE_SIZE);
this.tabledir = new Path(
fs.makeQualified(new Path(conf.get(HBASE_DIR))),
fs.makeQualified(new Path(conf.get(HConstants.HBASE_DIR))),
Bytes.toString(tableName)
);
Path logdir = new Path(tabledir, "merge_" + System.currentTimeMillis() +
HREGION_LOGDIR_NAME);
Path oldLogDir = new Path(tabledir, HREGION_OLDLOGDIR_NAME);
HConstants.HREGION_LOGDIR_NAME);
Path oldLogDir = new Path(tabledir, HConstants.HREGION_OLDLOGDIR_NAME);
this.hlog =
new HLog(fs, logdir, oldLogDir, conf, null);
}
@ -204,8 +204,9 @@ class HMerge implements HConstants {
throws IOException {
super(conf, fs, tableName);
this.tableName = tableName;
this.table = new HTable(conf, META_TABLE_NAME);
this.metaScanner = table.getScanner(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
this.table = new HTable(conf, HConstants.META_TABLE_NAME);
this.metaScanner = table.getScanner(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
this.latestRegion = null;
}
@ -215,11 +216,12 @@ class HMerge implements HConstants {
if (results == null) {
return null;
}
byte [] regionInfoValue = results.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
byte[] regionInfoValue = results.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
if (regionInfoValue == null || regionInfoValue.length == 0) {
throw new NoSuchElementException("meta region entry missing " +
Bytes.toString(CATALOG_FAMILY) + ":" +
Bytes.toString(REGIONINFO_QUALIFIER));
Bytes.toString(HConstants.CATALOG_FAMILY) + ":" +
Bytes.toString(HConstants.REGIONINFO_QUALIFIER));
}
HRegionInfo region = Writables.getHRegionInfo(regionInfoValue);
if (!Bytes.equals(region.getTableDesc().getName(), this.tableName)) {
@ -253,7 +255,8 @@ class HMerge implements HConstants {
boolean foundResult = false;
while (currentRow != null) {
LOG.info("Row: <" + Bytes.toString(currentRow.getRow()) + ">");
byte [] regionInfoValue = currentRow.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
byte[] regionInfoValue = currentRow.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
if (regionInfoValue == null || regionInfoValue.length == 0) {
currentRow = metaScanner.next();
continue;
@ -299,7 +302,7 @@ class HMerge implements HConstants {
newRegion.getRegionInfo().setOffline(true);
Put put = new Put(newRegion.getRegionName());
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER,
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(newRegion.getRegionInfo()));
table.put(put);
@ -317,12 +320,11 @@ class HMerge implements HConstants {
OfflineMerger(Configuration conf, FileSystem fs)
throws IOException {
super(conf, fs, META_TABLE_NAME);
super(conf, fs, HConstants.META_TABLE_NAME);
Path rootTableDir = HTableDescriptor.getTableDir(
fs.makeQualified(new Path(conf.get(HBASE_DIR))),
ROOT_TABLE_NAME);
fs.makeQualified(new Path(conf.get(HConstants.HBASE_DIR))),
HConstants.ROOT_TABLE_NAME);
// Scan root region to find all the meta regions
@ -331,7 +333,8 @@ class HMerge implements HConstants {
root.initialize(null, null);
Scan scan = new Scan();
scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
scan.addColumn(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
InternalScanner rootScanner =
root.getScanner(scan);

View File

@ -54,7 +54,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
* <code>hbase.master.meta.thread.rescanfrequency</code>, and
* <code>hbase.server.thread.wakefrequency</code> a second or less.
*/
public class LocalHBaseCluster implements HConstants {
public class LocalHBaseCluster {
static final Log LOG = LogFactory.getLog(LocalHBaseCluster.class);
private final HMaster master;
private final List<JVMClusterUtil.RegionServerThread> regionThreads;
@ -113,7 +113,7 @@ public class LocalHBaseCluster implements HConstants {
// Start the HRegionServers. Always have region servers come up on
// port '0' so there won't be clashes over default port as unit tests
// start/stop ports at different times during the life of the test.
conf.set(REGIONSERVER_PORT, "0");
conf.set(HConstants.REGIONSERVER_PORT, "0");
this.regionThreads =
new CopyOnWriteArrayList<JVMClusterUtil.RegionServerThread>();
this.regionServerClass =
@ -239,8 +239,8 @@ public class LocalHBaseCluster implements HConstants {
* @return True if a 'local' address in hbase.master value.
*/
public static boolean isLocal(final Configuration c) {
String mode = c.get(CLUSTER_DISTRIBUTED);
return mode == null || mode.equals(CLUSTER_IS_LOCAL);
final String mode = c.get(HConstants.CLUSTER_DISTRIBUTED);
return mode == null || mode.equals(HConstants.CLUSTER_IS_LOCAL);
}
/**

View File

@ -70,9 +70,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
*
* Used by {@link HTable} and {@link HBaseAdmin}
*/
public class HConnectionManager implements HConstants {
private static final Delete [] DELETE_ARRAY_TYPE = new Delete[]{};
private static final Put [] PUT_ARRAY_TYPE = new Put[]{};
public class HConnectionManager {
private static final Delete[] DELETE_ARRAY_TYPE = new Delete[] {};
private static final Put[] PUT_ARRAY_TYPE = new Put[] {};
// Register a shutdown hook, one that cleans up RPC and closes zk sessions.
static {
@ -245,7 +245,7 @@ public class HConnectionManager implements HConstants {
}
/* Encapsulates finding the servers for an HBase instance */
private static class TableServers implements ServerConnection, HConstants {
private static class TableServers implements ServerConnection {
static final Log LOG = LogFactory.getLog(TableServers.class);
private final Class<? extends HRegionInterface> serverInterfaceClass;
private final long pause;
@ -284,7 +284,8 @@ public class HConnectionManager implements HConstants {
this.conf = conf;
String serverClassName =
conf.get(REGION_SERVER_CLASS, DEFAULT_REGION_SERVER_CLASS);
conf.get(HConstants.REGION_SERVER_CLASS,
HConstants.DEFAULT_REGION_SERVER_CLASS);
this.closed = false;
@ -300,7 +301,9 @@ public class HConnectionManager implements HConstants {
this.pause = conf.getLong("hbase.client.pause", 1000);
this.numRetries = conf.getInt("hbase.client.retries.number", 10);
this.maxRPCAttempts = conf.getInt("hbase.client.rpc.maxattempts", 1);
this.rpcTimeout = conf.getLong(HBASE_REGIONSERVER_LEASE_PERIOD_KEY, DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD);
this.rpcTimeout = conf.getLong(
HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD);
this.master = null;
this.masterChecked = false;
@ -443,7 +446,8 @@ public class HConnectionManager implements HConstants {
MetaScannerVisitor visitor = new MetaScannerVisitor() {
public boolean processRow(Result result) throws IOException {
try {
byte[] value = result.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
byte[] value = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
HRegionInfo info = null;
if (value != null) {
info = Writables.getHRegionInfo(value);
@ -477,11 +481,13 @@ public class HConnectionManager implements HConstants {
MetaScannerVisitor visitor = new MetaScannerVisitor() {
@Override
public boolean processRow(Result row) throws IOException {
byte[] value = row.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
byte[] value = row.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
HRegionInfo info = Writables.getHRegionInfoOrNull(value);
if (info != null) {
if (Bytes.equals(tableName, info.getTableDesc().getName())) {
value = row.getValue(CATALOG_FAMILY, SERVER_QUALIFIER);
value = row.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if (value == null) {
available.set(false);
return false;
@ -519,7 +525,8 @@ public class HConnectionManager implements HConstants {
byte[] endKey;
HRegionInfo currentRegion;
Scan scan = new Scan(startKey);
scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
scan.addColumn(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
int rows = this.conf.getInt("hbase.meta.scanner.caching", 100);
scan.setCaching(rows);
ScannerCallable s = new ScannerCallable(this,
@ -569,7 +576,8 @@ public class HConnectionManager implements HConstants {
}
public boolean processRow(Result rowResult) throws IOException {
HRegionInfo info = Writables.getHRegionInfo(
rowResult.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER));
rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER));
HTableDescriptor desc = info.getTableDesc();
if (Bytes.compareTo(desc.getName(), tableName) == 0) {
result = desc;
@ -619,7 +627,7 @@ public class HConnectionManager implements HConstants {
"table name cannot be null or zero length");
}
if (Bytes.equals(tableName, ROOT_TABLE_NAME)) {
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
synchronized (rootRegionLock) {
// This block guards against two threads trying to find the root
// region at the same time. One will go do the find while the
@ -630,13 +638,13 @@ public class HConnectionManager implements HConstants {
}
return this.rootRegionLocation;
}
} else if (Bytes.equals(tableName, META_TABLE_NAME)) {
return locateRegionInMeta(ROOT_TABLE_NAME, tableName, row, useCache,
metaRegionLock);
} else if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
return locateRegionInMeta(HConstants.ROOT_TABLE_NAME, tableName, row,
useCache, metaRegionLock);
} else {
// Region not in the cache - have to go to the meta RS
return locateRegionInMeta(META_TABLE_NAME, tableName, row, useCache,
userRegionLock);
return locateRegionInMeta(HConstants.META_TABLE_NAME, tableName, row,
useCache, userRegionLock);
}
}
@ -702,8 +710,8 @@ public class HConnectionManager implements HConstants {
if (regionInfoRow == null) {
throw new TableNotFoundException(Bytes.toString(tableName));
}
byte [] value = regionInfoRow.getValue(CATALOG_FAMILY,
REGIONINFO_QUALIFIER);
byte[] value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
if (value == null || value.length == 0) {
throw new IOException("HRegionInfo was null or empty in " +
Bytes.toString(parentTable) + ", row=" + regionInfoRow);
@ -721,7 +729,8 @@ public class HConnectionManager implements HConstants {
regionInfo.getRegionNameAsString());
}
value = regionInfoRow.getValue(CATALOG_FAMILY, SERVER_QUALIFIER);
value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
String serverAddress = "";
if(value != null) {
serverAddress = Bytes.toString(value);
@ -824,9 +833,9 @@ public class HConnectionManager implements HConstants {
// make sure that the end key is greater than the row we're looking
// for, otherwise the row actually belongs in the next region, not
// this one. the exception case is when the endkey is EMPTY_START_ROW,
// signifying that the region we're checking is actually the last
// region in the table.
// this one. the exception case is when the endkey is
// HConstants.EMPTY_START_ROW, signifying that the region we're
// checking is actually the last region in the table.
if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) ||
KeyValue.getRowComparator(tableName).compareRows(endKey, 0, endKey.length,
row, 0, row.length) > 0) {

View File

@ -32,7 +32,7 @@ import java.io.IOException;
* and uses a Retryable scanner. Provided visitors will be called
* for each row.
*/
class MetaScanner implements HConstants {
class MetaScanner {
/**
* Scans the meta table and calls a visitor on each RowResult and uses a empty
@ -45,7 +45,7 @@ class MetaScanner implements HConstants {
public static void metaScan(Configuration configuration,
MetaScannerVisitor visitor)
throws IOException {
metaScan(configuration, visitor, EMPTY_START_ROW);
metaScan(configuration, visitor, HConstants.EMPTY_START_ROW);
}
/**
@ -63,14 +63,16 @@ class MetaScanner implements HConstants {
HConnection connection = HConnectionManager.getConnection(configuration);
byte [] startRow = tableName == null || tableName.length == 0 ?
HConstants.EMPTY_START_ROW :
HRegionInfo.createRegionName(tableName, null, ZEROES, false);
HRegionInfo.createRegionName(tableName, null, HConstants.ZEROES,
false);
// Scan over each meta region
ScannerCallable callable;
int rows = configuration.getInt("hbase.meta.scanner.caching", 100);
do {
Scan scan = new Scan(startRow).addFamily(CATALOG_FAMILY);
callable = new ScannerCallable(connection, META_TABLE_NAME, scan);
final Scan scan = new Scan(startRow).addFamily(HConstants.CATALOG_FAMILY);
callable = new ScannerCallable(connection, HConstants.META_TABLE_NAME,
scan);
// Open scanner
connection.getRegionServerWithRetries(callable);
try {
@ -94,7 +96,7 @@ class MetaScanner implements HConstants {
callable.setClose();
connection.getRegionServerWithRetries(callable);
}
} while (Bytes.compareTo(startRow, LAST_ROW) != 0);
} while (Bytes.compareTo(startRow, HConstants.LAST_ROW) != 0);
}
/**

View File

@ -100,17 +100,17 @@ import java.util.concurrent.atomic.AtomicBoolean;
* <p>A <code>META</code> region is not 'online' until it has been scanned
* once.
*/
abstract class BaseScanner extends Chore implements HConstants {
abstract class BaseScanner extends Chore {
static final Log LOG = LogFactory.getLog(BaseScanner.class.getName());
// These are names of new columns in a meta region offlined parent row. They
// are added by the metascanner after we verify that split daughter made it
// in. Their value is 'true' if present.
private static final byte [] SPLITA_CHECKED =
Bytes.toBytes(Bytes.toString(SPLITA_QUALIFIER) + "_checked");
private static final byte [] SPLITB_CHECKED =
Bytes.toBytes(Bytes.toString(SPLITB_QUALIFIER) + "_checked");
private static final byte[] SPLITA_CHECKED =
Bytes.toBytes(Bytes.toString(HConstants.SPLITA_QUALIFIER) + "_checked");
private static final byte[] SPLITB_CHECKED =
Bytes.toBytes(Bytes.toString(HConstants.SPLITB_QUALIFIER) + "_checked");
// Make the 'true' Writable once only.
private static byte [] TRUE_WRITABLE_AS_BYTES;
private static byte[] TRUE_WRITABLE_AS_BYTES;
static {
try {
TRUE_WRITABLE_AS_BYTES = Writables.getBytes(new BooleanWritable(true));
@ -250,8 +250,9 @@ abstract class BaseScanner extends Chore implements HConstants {
* @return Empty String or server address found in <code>r</code>
*/
static String getServerAddress(final Result r) {
byte [] val = r.getValue(CATALOG_FAMILY, SERVER_QUALIFIER);
return val == null || val.length <= 0? "": Bytes.toString(val);
final byte[] val = r.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
return val == null || val.length <= 0 ? "" : Bytes.toString(val);
}
/*
@ -259,8 +260,9 @@ abstract class BaseScanner extends Chore implements HConstants {
* @return Return 0L or server startcode found in <code>r</code>
*/
static long getStartCode(final Result r) {
byte [] val = r.getValue(CATALOG_FAMILY, STARTCODE_QUALIFIER);
return val == null || val.length <= 0? 0L: Bytes.toLong(val);
final byte[] val = r.getValue(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER);
return val == null || val.length <= 0 ? 0L : Bytes.toLong(val);
}
/*
@ -299,9 +301,9 @@ abstract class BaseScanner extends Chore implements HConstants {
boolean result = false;
// Run checks on each daughter split.
boolean hasReferencesA = checkDaughter(metaRegionName, srvr,
parent, rowContent, SPLITA_QUALIFIER);
parent, rowContent, HConstants.SPLITA_QUALIFIER);
boolean hasReferencesB = checkDaughter(metaRegionName, srvr,
parent, rowContent, SPLITB_QUALIFIER);
parent, rowContent, HConstants.SPLITB_QUALIFIER);
if (!hasReferencesA && !hasReferencesB) {
LOG.info("Deleting region " + parent.getRegionNameAsString() +
" (encoded=" + parent.getEncodedName() +
@ -402,7 +404,7 @@ abstract class BaseScanner extends Chore implements HConstants {
final HRegionInfo split, final byte [] daughter)
throws IOException {
Put p = new Put(parent);
p.add(CATALOG_FAMILY, getNameOfVerifiedDaughterColumn(daughter),
p.add(HConstants.CATALOG_FAMILY, getNameOfVerifiedDaughterColumn(daughter),
TRUE_WRITABLE_AS_BYTES);
srvr.put(metaRegionName, p);
}
@ -415,9 +417,9 @@ abstract class BaseScanner extends Chore implements HConstants {
* @throws IOException
*/
private boolean getDaughterRowChecked(final Result rowContent,
final byte [] which)
final byte[] which)
throws IOException {
byte [] b = rowContent.getValue(CATALOG_FAMILY,
final byte[] b = rowContent.getValue(HConstants.CATALOG_FAMILY,
getNameOfVerifiedDaughterColumn(which));
BooleanWritable bw = null;
if (b != null && b.length > 0) {
@ -432,8 +434,8 @@ abstract class BaseScanner extends Chore implements HConstants {
* <code>daughter</code> is.
*/
private static byte [] getNameOfVerifiedDaughterColumn(final byte [] daughter) {
return Bytes.equals(SPLITA_QUALIFIER, daughter)?
SPLITA_CHECKED: SPLITB_CHECKED;
return (Bytes.equals(HConstants.SPLITA_QUALIFIER, daughter)
? SPLITA_CHECKED : SPLITB_CHECKED);
}
/*
@ -447,7 +449,8 @@ abstract class BaseScanner extends Chore implements HConstants {
private HRegionInfo getDaughterRegionInfo(final Result rowContent,
final byte [] which)
throws IOException {
return Writables.getHRegionInfoOrNull(rowContent.getValue(CATALOG_FAMILY, which));
return Writables.getHRegionInfoOrNull(
rowContent.getValue(HConstants.CATALOG_FAMILY, which));
}
/*

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.master;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
@ -94,8 +95,8 @@ class ChangeTableState extends TableOperation {
Put put = updateRegionInfo(i);
server.put(m.getRegionName(), put);
Delete delete = new Delete(i.getRegionName());
delete.deleteColumns(CATALOG_FAMILY, SERVER_QUALIFIER);
delete.deleteColumns(CATALOG_FAMILY, STARTCODE_QUALIFIER);
delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER);
server.delete(m.getRegionName(), delete);
}
if (LOG.isDebugEnabled()) {
@ -132,7 +133,7 @@ class ChangeTableState extends TableOperation {
for (HRegionInfo i: e.getValue()) {
// The scan we did could be totally staled, get the freshest data
Get get = new Get(i.getRegionName());
get.addColumn(CATALOG_FAMILY, SERVER_QUALIFIER);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
Result values = server.get(m.getRegionName(), get);
String serverAddress = BaseScanner.getServerAddress(values);
// If this region is unassigned, skip!
@ -155,7 +156,7 @@ class ChangeTableState extends TableOperation {
throws IOException {
i.setOffline(!online);
Put put = new Put(i.getRegionName());
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(i));
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, Writables.getBytes(i));
return put;
}
}

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.master;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.client.Put;
@ -48,7 +49,7 @@ abstract class ColumnOperation extends TableOperation {
protected void updateRegionInfo(HRegionInterface server, byte [] regionName,
HRegionInfo i) throws IOException {
Put put = new Put(i.getRegionName());
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(i));
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, Writables.getBytes(i));
server.put(regionName, put);
if (LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: " + i.getRegionNameAsString());

View File

@ -104,7 +104,7 @@ import java.util.concurrent.locks.ReentrantLock;
* @see HMasterRegionInterface
* @see Watcher
*/
public class HMaster extends Thread implements HConstants, HMasterInterface,
public class HMaster extends Thread implements HMasterInterface,
HMasterRegionInterface, Watcher {
// MASTER is name of the webapp and the attribute name used stuffing this
//instance into web context.
@ -174,7 +174,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
checkRootDir(this.rootdir, this.conf, this.fs);
// Make sure the region servers can archive their old logs
this.oldLogDir = new Path(this.rootdir, HREGION_OLDLOGDIR_NAME);
this.oldLogDir = new Path(this.rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
if(!this.fs.exists(this.oldLogDir)) {
this.fs.mkdirs(this.oldLogDir);
}
@ -188,7 +188,8 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
this.address = new HServerAddress(this.rpcServer.getListenerAddress());
this.numRetries = conf.getInt("hbase.client.retries.number", 2);
this.threadWakeFrequency = conf.getInt(THREAD_WAKE_FREQUENCY, 10 * 1000);
this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY,
10 * 1000);
this.sleeper = new Sleeper(this.threadWakeFrequency, this.closed);
this.connection = ServerConnectionManager.getConnection(conf);
@ -227,7 +228,8 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
final FileSystem fs)
throws IOException {
// If FS is in safe mode wait till out of it.
FSUtils.waitOnSafeMode(c, c.getInt(THREAD_WAKE_FREQUENCY, 10 * 1000));
FSUtils.waitOnSafeMode(c, c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
10 * 1000));
// Filesystem is good. Go ahead and check for hbase.rootdir.
if (!fs.exists(rd)) {
fs.mkdirs(rd);
@ -293,7 +295,8 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
// Find out our address up in DNS.
String s = DNS.getDefaultHost(c.get("hbase.master.dns.interface","default"),
c.get("hbase.master.dns.nameserver","default"));
s += ":" + c.get(MASTER_PORT, Integer.toString(DEFAULT_MASTER_PORT));
s += ":" + c.get(HConstants.MASTER_PORT,
Integer.toString(HConstants.DEFAULT_MASTER_PORT));
return s;
}
@ -754,13 +757,14 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
HRegionInterface srvr = this.connection.getHRegionConnection(m.getServer());
byte[] firstRowInTable = Bytes.toBytes(tableName + ",,");
Scan scan = new Scan(firstRowInTable);
scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
long scannerid = srvr.openScanner(metaRegionName, scan);
try {
Result data = srvr.next(scannerid);
if (data != null && data.size() > 0) {
HRegionInfo info = Writables.getHRegionInfo(
data.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER));
data.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER));
if (info.getTableDesc().getNameAsString().equals(tableName)) {
// A region for this table already exists. Ergo table exists.
throw new TableExistsException(tableName);
@ -775,7 +779,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
}
public void deleteTable(final byte [] tableName) throws IOException {
if (Bytes.equals(tableName, ROOT_TABLE_NAME)) {
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
throw new IOException("Can't delete root table");
}
new TableDelete(this, tableName).process();
@ -799,14 +803,14 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
}
public void enableTable(final byte [] tableName) throws IOException {
if (Bytes.equals(tableName, ROOT_TABLE_NAME)) {
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
throw new IOException("Can't enable root table");
}
new ChangeTableState(this, tableName, true).process();
}
public void disableTable(final byte [] tableName) throws IOException {
if (Bytes.equals(tableName, ROOT_TABLE_NAME)) {
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
throw new IOException("Can't disable root table");
}
new ChangeTableState(this, tableName, false).process();
@ -826,8 +830,9 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
HRegionInterface srvr =
this.connection.getHRegionConnection(m.getServer());
Scan scan = new Scan(firstRowInTable);
scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
scan.addColumn(CATALOG_FAMILY, SERVER_QUALIFIER);
scan.addColumn(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
// TODO: Use caching.
long scannerid = srvr.openScanner(metaRegionName, scan);
try {
@ -836,9 +841,11 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
if (data == null || data.size() <= 0)
break;
HRegionInfo info = Writables.getHRegionInfo(
data.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER));
data.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER));
if (Bytes.equals(info.getTableDesc().getName(), tableName)) {
byte [] value = data.getValue(CATALOG_FAMILY, SERVER_QUALIFIER);
final byte[] value = data.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if (value != null && value.length > 0) {
HServerAddress server = new HServerAddress(Bytes.toString(value));
result.add(new Pair<HRegionInfo,HServerAddress>(info, server));
@ -864,8 +871,9 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
byte [] metaRegionName = m.getRegionName();
HRegionInterface srvr = this.connection.getHRegionConnection(m.getServer());
Scan scan = new Scan(firstRowInTable);
scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
scan.addColumn(CATALOG_FAMILY, SERVER_QUALIFIER);
scan.addColumn(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
long scannerid = srvr.openScanner(metaRegionName, scan);
try {
while (true) {
@ -873,11 +881,13 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
if (data == null || data.size() <= 0)
break;
HRegionInfo info = Writables.getHRegionInfo(
data.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER));
data.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER));
if (Bytes.compareTo(info.getTableDesc().getName(), tableName) == 0) {
if ((Bytes.compareTo(info.getStartKey(), rowKey) >= 0) &&
(Bytes.compareTo(info.getEndKey(), rowKey) < 0)) {
byte [] value = data.getValue(CATALOG_FAMILY, SERVER_QUALIFIER);
final byte[] value = data.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if (value != null && value.length > 0) {
HServerAddress server =
new HServerAddress(Bytes.toString(value));
@ -904,13 +914,16 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
byte [] metaRegionName = m.getRegionName();
HRegionInterface srvr = connection.getHRegionConnection(m.getServer());
Get get = new Get(regionName);
get.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
get.addColumn(CATALOG_FAMILY, SERVER_QUALIFIER);
get.addColumn(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
Result data = srvr.get(metaRegionName, get);
if(data == null || data.size() <= 0) continue;
HRegionInfo info = Writables.getHRegionInfo(
data.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER));
byte [] value = data.getValue(CATALOG_FAMILY, SERVER_QUALIFIER);
data.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER));
final byte[] value = data.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if(value != null && value.length > 0) {
HServerAddress server =
new HServerAddress(Bytes.toString(value));
@ -1004,7 +1017,8 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
if (hostnameAndPort == null) {
// Get server from the .META. if it wasn't passed as argument
hostnameAndPort =
Bytes.toString(rr.getValue(CATALOG_FAMILY, SERVER_QUALIFIER));
Bytes.toString(rr.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER));
}
// Take region out of the intransistions in case it got stuck there doing
// an open or whatever.
@ -1012,7 +1026,8 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
// If hostnameAndPort is still null, then none, exit.
if (hostnameAndPort == null) break;
long startCode =
Bytes.toLong(rr.getValue(CATALOG_FAMILY, STARTCODE_QUALIFIER));
Bytes.toLong(rr.getValue(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER));
String name = HServerInfo.getServerName(hostnameAndPort, startCode);
LOG.info("Marking " + hri.getRegionNameAsString() +
" as closing on " + name + "; cleaning SERVER + STARTCODE; " +
@ -1052,19 +1067,23 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
*/
HRegionInfo getHRegionInfo(final byte [] row, final Result res)
throws IOException {
byte [] regioninfo = res.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
byte[] regioninfo = res.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
if (regioninfo == null) {
StringBuilder sb = new StringBuilder();
NavigableMap<byte[], byte[]> infoMap = res.getFamilyMap(CATALOG_FAMILY);
NavigableMap<byte[], byte[]> infoMap =
res.getFamilyMap(HConstants.CATALOG_FAMILY);
for (byte [] e: infoMap.keySet()) {
if (sb.length() > 0) {
sb.append(", ");
}
sb.append(Bytes.toString(CATALOG_FAMILY) + ":" + Bytes.toString(e));
sb.append(Bytes.toString(HConstants.CATALOG_FAMILY) + ":"
+ Bytes.toString(e));
}
LOG.warn(Bytes.toString(CATALOG_FAMILY) + ":" +
Bytes.toString(REGIONINFO_QUALIFIER) + " is empty for row: " +
Bytes.toString(row) + "; has keys: " + sb.toString());
LOG.warn(Bytes.toString(HConstants.CATALOG_FAMILY) + ":" +
Bytes.toString(HConstants.REGIONINFO_QUALIFIER)
+ " is empty for row: " + Bytes.toString(row) + "; has keys: "
+ sb.toString());
return null;
}
return Writables.getHRegionInfo(regioninfo);

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.master;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableNotDisabledException;
@ -51,7 +52,7 @@ class ModifyTableMeta extends TableOperation {
HRegionInfo i)
throws IOException {
Put put = new Put(i.getRegionName());
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(i));
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, Writables.getBytes(i));
server.put(regionName, put);
LOG.debug("updated HTableDescriptor for region " + i.getRegionNameAsString());
}

View File

@ -19,6 +19,7 @@
*/
package org.apache.hadoop.hbase.master;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HServerInfo;
@ -73,9 +74,9 @@ class ProcessRegionOpen extends ProcessRegionStatusChange {
// Register the newly-available Region's location.
Put p = new Put(regionInfo.getRegionName());
p.add(CATALOG_FAMILY, SERVER_QUALIFIER,
p.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(serverInfo.getHostnamePort()));
p.add(CATALOG_FAMILY, STARTCODE_QUALIFIER,
p.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
Bytes.toBytes(serverInfo.getStartCode()));
server.put(metaRegionName, p);
LOG.info("Updated row " + regionInfo.getRegionNameAsString() +

View File

@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.master;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HServerInfo;
@ -253,7 +254,7 @@ class ProcessServerShutdown extends RegionServerOperation {
master.getRegionManager().getRootRegionLocation().getBindAddress());
}
Scan scan = new Scan();
scan.addFamily(CATALOG_FAMILY);
scan.addFamily(HConstants.CATALOG_FAMILY);
long scannerId = server.openScanner(
HRegionInfo.ROOT_REGIONINFO.getRegionName(), scan);
scanMetaRegion(server, scannerId,
@ -273,7 +274,7 @@ class ProcessServerShutdown extends RegionServerOperation {
Bytes.toString(m.getRegionName()) + " on " + m.getServer());
}
Scan scan = new Scan();
scan.addFamily(CATALOG_FAMILY);
scan.addFamily(HConstants.CATALOG_FAMILY);
long scannerId = server.openScanner(
m.getRegionName(), scan);
scanMetaRegion(server, scannerId, m.getRegionName());

View File

@ -59,7 +59,7 @@ import java.util.concurrent.atomic.AtomicReference;
/**
* Class to manage assigning regions to servers, state of root and meta, etc.
*/
public class RegionManager implements HConstants {
public class RegionManager {
protected static final Log LOG = LogFactory.getLog(RegionManager.class);
private AtomicReference<HServerAddress> rootRegionLocation =
@ -137,8 +137,10 @@ public class RegionManager implements HConstants {
// Scans the meta table
metaScannerThread = new MetaScanner(master);
zooKeeperNumRetries = conf.getInt(ZOOKEEPER_RETRIES, DEFAULT_ZOOKEEPER_RETRIES);
zooKeeperPause = conf.getInt(ZOOKEEPER_PAUSE, DEFAULT_ZOOKEEPER_PAUSE);
zooKeeperNumRetries = conf.getInt(HConstants.ZOOKEEPER_RETRIES,
HConstants.DEFAULT_ZOOKEEPER_RETRIES);
zooKeeperPause = conf.getInt(HConstants.ZOOKEEPER_PAUSE,
HConstants.DEFAULT_ZOOKEEPER_PAUSE);
reassignRootRegion();
}
@ -508,13 +510,13 @@ public class RegionManager implements HConstants {
* PathFilter that accepts hbase tables only.
*/
static class TableDirFilter implements PathFilter {
public boolean accept(Path path) {
public boolean accept(final Path path) {
// skip the region servers' log dirs && version file
// HBASE-1112 want to separate the log dirs from table's data dirs by a
// special character.
String pathname = path.getName();
return !pathname.equals(HLog.HREGION_LOGDIR_NAME) &&
!pathname.equals(VERSION_FILE_NAME);
final String pathname = path.getName();
return (!pathname.equals(HConstants.HREGION_LOGDIR_NAME)
&& !pathname.equals(HConstants.VERSION_FILE_NAME));
}
}
@ -524,7 +526,7 @@ public class RegionManager implements HConstants {
*/
static class RegionDirFilter implements PathFilter {
public boolean accept(Path path) {
return !path.getName().equals(HREGION_COMPACTIONDIR_NAME);
return !path.getName().equals(HConstants.HREGION_COMPACTIONDIR_NAME);
}
}
@ -727,7 +729,8 @@ public class RegionManager implements HConstants {
byte [] regionName = region.getRegionName();
Put put = new Put(regionName);
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(info));
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(info));
server.put(metaRegionName, put);
// 4. Close the new region to flush it to disk. Close its log file too.
@ -1175,10 +1178,10 @@ public class RegionManager implements HConstants {
private long getPauseTime(int tries) {
int attempt = tries;
if (attempt >= RETRY_BACKOFF.length) {
attempt = RETRY_BACKOFF.length - 1;
if (attempt >= HConstants.RETRY_BACKOFF.length) {
attempt = HConstants.RETRY_BACKOFF.length - 1;
}
return this.zooKeeperPause * RETRY_BACKOFF[attempt];
return this.zooKeeperPause * HConstants.RETRY_BACKOFF[attempt];
}
private void sleep(int attempt) {

View File

@ -21,13 +21,12 @@ package org.apache.hadoop.hbase.master;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import java.io.IOException;
import java.util.concurrent.Delayed;
import java.util.concurrent.TimeUnit;
abstract class RegionServerOperation implements Delayed, HConstants {
abstract class RegionServerOperation implements Delayed {
protected static final Log LOG =
LogFactory.getLog(RegionServerOperation.class.getName());

View File

@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HMsg;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.HServerAddress;
@ -94,7 +95,7 @@ public class RegionServerOperationQueue {
private final Sleeper sleeper;
RegionServerOperationQueue(final Configuration c, final AtomicBoolean closed) {
this.threadWakeFrequency = c.getInt(HMaster.THREAD_WAKE_FREQUENCY, 10 * 1000);
this.threadWakeFrequency = c.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
this.closed = closed;
this.sleeper = new Sleeper(this.threadWakeFrequency, this.closed);
}

View File

@ -58,7 +58,7 @@ import java.util.concurrent.atomic.AtomicInteger;
* The ServerManager class manages info about region servers - HServerInfo,
* load numbers, dying servers, etc.
*/
public class ServerManager implements HConstants {
public class ServerManager {
private static final Log LOG =
LogFactory.getLog(ServerManager.class.getName());

View File

@ -41,7 +41,7 @@ import java.util.TreeSet;
* objects in a table. (For a table, operate on each of its rows
* in .META.).
*/
abstract class TableOperation implements HConstants {
abstract class TableOperation {
private final Set<MetaRegion> metaRegions;
protected final byte [] tableName;
// Do regions in order.
@ -81,7 +81,8 @@ abstract class TableOperation implements HConstants {
// Open a scanner on the meta region
byte [] tableNameMetaStart =
Bytes.toBytes(Bytes.toString(tableName) + ",,");
Scan scan = new Scan(tableNameMetaStart).addFamily(CATALOG_FAMILY);
final Scan scan = new Scan(tableNameMetaStart)
.addFamily(HConstants.CATALOG_FAMILY);
long scannerId = this.server.openScanner(m.getRegionName(), scan);
int rows = this.master.getConfiguration().
getInt("hbase.meta.scanner.caching", 100);
@ -96,9 +97,10 @@ abstract class TableOperation implements HConstants {
HRegionInfo info = this.master.getHRegionInfo(values.getRow(), values);
if (info == null) {
emptyRows.add(values.getRow());
LOG.error(Bytes.toString(CATALOG_FAMILY) + ":" +
Bytes.toString(REGIONINFO_QUALIFIER) + " not found on " +
Bytes.toStringBinary(values.getRow()));
LOG.error(Bytes.toString(HConstants.CATALOG_FAMILY) + ":"
+ Bytes.toString(HConstants.REGIONINFO_QUALIFIER)
+ " not found on "
+ Bytes.toStringBinary(values.getRow()));
continue;
}
final String serverAddress = BaseScanner.getServerAddress(values);

View File

@ -40,7 +40,7 @@ import java.util.concurrent.locks.ReentrantLock;
/**
* Compact region on request and then run split if appropriate
*/
class CompactSplitThread extends Thread implements HConstants {
class CompactSplitThread extends Thread {
static final Log LOG = LogFactory.getLog(CompactSplitThread.class);
private HTable root = null;
@ -162,13 +162,13 @@ class CompactSplitThread extends Thread implements HConstants {
if (region.getRegionInfo().isMetaTable()) {
// We need to update the root region
if (this.root == null) {
this.root = new HTable(conf, ROOT_TABLE_NAME);
this.root = new HTable(conf, HConstants.ROOT_TABLE_NAME);
}
t = root;
} else {
// For normal regions we need to update the meta region
if (meta == null) {
meta = new HTable(conf, META_TABLE_NAME);
meta = new HTable(conf, HConstants.META_TABLE_NAME);
}
t = meta;
}
@ -181,13 +181,15 @@ class CompactSplitThread extends Thread implements HConstants {
this.server.removeFromOnlineRegions(oldRegionInfo);
Put put = new Put(oldRegionInfo.getRegionName());
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER,
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(oldRegionInfo));
put.add(CATALOG_FAMILY, SERVER_QUALIFIER, EMPTY_BYTE_ARRAY);
put.add(CATALOG_FAMILY, STARTCODE_QUALIFIER, EMPTY_BYTE_ARRAY);
put.add(CATALOG_FAMILY, SPLITA_QUALIFIER,
put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
HConstants.EMPTY_BYTE_ARRAY);
put.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
HConstants.EMPTY_BYTE_ARRAY);
put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER,
Writables.getBytes(newRegions[0].getRegionInfo()));
put.add(CATALOG_FAMILY, SPLITB_QUALIFIER,
put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER,
Writables.getBytes(newRegions[1].getRegionInfo()));
t.put(put);
@ -198,8 +200,8 @@ class CompactSplitThread extends Thread implements HConstants {
// Add new regions to META
for (int i = 0; i < newRegions.length; i++) {
put = new Put(newRegions[i].getRegionName());
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(
newRegions[i].getRegionInfo()));
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(newRegions[i].getRegionInfo()));
t.put(put);
}

View File

@ -115,7 +115,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
* regionName is a unique identifier for this HRegion. (startKey, endKey]
* defines the keyspace for this HRegion.
*/
public class HRegion implements HConstants, HeapSize { // , Writable{
public class HRegion implements HeapSize { // , Writable{
public static final Log LOG = LogFactory.getLog(HRegion.class);
static final String SPLITDIR = "splits";
static final String MERGEDIR = "merges";
@ -280,7 +280,8 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
this.conf = conf;
this.regionInfo = regionInfo;
this.flushListener = flushListener;
this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000);
this.threadWakeFrequency = conf.getLong(HConstants.THREAD_WAKE_FREQUENCY,
10 * 1000);
String encodedNameStr = this.regionInfo.getEncodedName();
this.regiondir = new Path(basedir, encodedNameStr);
if (LOG.isDebugEnabled()) {
@ -309,7 +310,7 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
*/
public void initialize(Path initialFiles, final Progressable reporter)
throws IOException {
Path oldLogFile = new Path(regiondir, HREGION_OLDLOGFILE_NAME);
Path oldLogFile = new Path(regiondir, HConstants.HREGION_OLDLOGFILE_NAME);
moveInitialFilesIntoPlace(this.fs, initialFiles, this.regiondir);
@ -743,7 +744,7 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
* @return compaction directory for the passed in <code>dir</code>
*/
static Path getCompactionDir(final Path dir) {
return new Path(dir, HREGION_COMPACTIONDIR_NAME);
return new Path(dir, HConstants.HREGION_COMPACTIONDIR_NAME);
}
/*
@ -1504,12 +1505,12 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
/**
* Checks if any stamps is Long.MAX_VALUE. If so, sets them to now.
* <p>
* This acts to replace LATEST_TIMESTAMP with now.
* This acts to replace {@link HConstants#LATEST_TIMESTAMP} with {@code now}.
* @param keys
* @param now
* @return <code>true</code> when updating the time stamp completed.
*/
private boolean updateKeys(List<KeyValue> keys, byte [] now) {
private boolean updateKeys(final List<KeyValue> keys, final byte[] now) {
if (keys == null || keys.isEmpty()) {
return false;
}
@ -2117,8 +2118,7 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
// Utility methods
/**
* A utility method to create new instances of HRegion based on the
* {@link org.apache.hadoop.hbase.HConstants#REGION_IMPL} configuration
* property.
* {@link HConstants#REGION_IMPL} configuration property.
* @param basedir qualified path of directory where region should be located,
* usually the table directory.
* @param log The HLog is the outbound log for any updates to the HRegion
@ -2177,8 +2177,8 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
FileSystem fs = FileSystem.get(conf);
fs.mkdirs(regionDir);
HRegion region = HRegion.newHRegion(tableDir,
new HLog(fs, new Path(regionDir, HREGION_LOGDIR_NAME),
new Path(regionDir, HREGION_OLDLOGDIR_NAME), conf, null),
new HLog(fs, new Path(regionDir, HConstants.HREGION_LOGDIR_NAME),
new Path(regionDir, HConstants.HREGION_OLDLOGDIR_NAME), conf, null),
fs, conf, info, null);
region.initialize(null, null);
return region;
@ -2230,12 +2230,14 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
throws IOException {
meta.checkResources();
// The row key is the region name
byte [] row = r.getRegionName();
byte[] row = r.getRegionName();
Integer lid = meta.obtainRowLock(row);
try {
List<KeyValue> edits = new ArrayList<KeyValue>();
edits.add(new KeyValue(row, CATALOG_FAMILY, REGIONINFO_QUALIFIER,
EnvironmentEdgeManager.currentTimeMillis(), Writables.getBytes(r.getRegionInfo())));
final List<KeyValue> edits = new ArrayList<KeyValue>(1);
edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER,
EnvironmentEdgeManager.currentTimeMillis(),
Writables.getBytes(r.getRegionInfo())));
meta.put(HConstants.CATALOG_FAMILY, edits);
} finally {
meta.releaseRowLock(lid);
@ -2275,7 +2277,8 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
byte [] row = info.getRegionName();
Put put = new Put(row);
info.setOffline(true);
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(info));
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(info));
srvr.put(metaRegionName, put);
cleanRegionInMETA(srvr, metaRegionName, info);
}
@ -2292,8 +2295,9 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
final byte [] metaRegionName, final HRegionInfo info)
throws IOException {
Delete del = new Delete(info.getRegionName());
del.deleteColumns(CATALOG_FAMILY, SERVER_QUALIFIER);
del.deleteColumns(CATALOG_FAMILY, STARTCODE_QUALIFIER);
del.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
del.deleteColumns(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER);
srvr.delete(metaRegionName, del);
}
@ -2449,23 +2453,28 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
Path basedir = a.getBaseDir();
// Presume both are of same region type -- i.e. both user or catalog
// table regions. This way can use comparator.
final byte [] startKey = a.comparator.matchingRows(a.getStartKey(), 0,
a.getStartKey().length,
EMPTY_BYTE_ARRAY, 0, EMPTY_BYTE_ARRAY.length) ||
b.comparator.matchingRows(b.getStartKey(), 0, b.getStartKey().length,
EMPTY_BYTE_ARRAY, 0, EMPTY_BYTE_ARRAY.length)?
EMPTY_BYTE_ARRAY:
a.comparator.compareRows(a.getStartKey(), 0, a.getStartKey().length,
b.getStartKey(), 0, b.getStartKey().length) <= 0?
a.getStartKey(): b.getStartKey();
final byte [] endKey = a.comparator.matchingRows(a.getEndKey(), 0,
a.getEndKey().length, EMPTY_BYTE_ARRAY, 0, EMPTY_BYTE_ARRAY.length) ||
a.comparator.matchingRows(b.getEndKey(), 0, b.getEndKey().length,
EMPTY_BYTE_ARRAY, 0, EMPTY_BYTE_ARRAY.length)?
EMPTY_BYTE_ARRAY:
a.comparator.compareRows(a.getEndKey(), 0, a.getEndKey().length,
b.getEndKey(), 0, b.getEndKey().length) <= 0?
b.getEndKey(): a.getEndKey();
final byte[] startKey =
(a.comparator.matchingRows(a.getStartKey(), 0, a.getStartKey().length,
HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
|| b.comparator.matchingRows(b.getStartKey(), 0,
b.getStartKey().length, HConstants.EMPTY_BYTE_ARRAY, 0,
HConstants.EMPTY_BYTE_ARRAY.length))
? HConstants.EMPTY_BYTE_ARRAY
: (a.comparator.compareRows(a.getStartKey(), 0, a.getStartKey().length,
b.getStartKey(), 0, b.getStartKey().length) <= 0
? a.getStartKey()
: b.getStartKey());
final byte[] endKey =
(a.comparator.matchingRows(a.getEndKey(), 0, a.getEndKey().length,
HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
|| a.comparator.matchingRows(b.getEndKey(), 0, b.getEndKey().length,
HConstants.EMPTY_BYTE_ARRAY, 0,
HConstants.EMPTY_BYTE_ARRAY.length))
? HConstants.EMPTY_BYTE_ARRAY
: (a.comparator.compareRows(a.getEndKey(), 0, a.getEndKey().length,
b.getEndKey(), 0, b.getEndKey().length) <= 0
? b.getEndKey()
: a.getEndKey());
HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey);
LOG.info("Creating new region " + newRegionInfo.toString());
@ -2883,13 +2892,15 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
}
majorCompact = true;
}
Path tableDir = new Path(args[0]);
Configuration c = HBaseConfiguration.create();
FileSystem fs = FileSystem.get(c);
Path logdir = new Path(c.get("hbase.tmp.dir"),
"hlog" + tableDir.getName() + EnvironmentEdgeManager.currentTimeMillis());
Path oldLogDir = new Path(c.get("hbase.tmp.dir"), HREGION_OLDLOGDIR_NAME);
HLog log = new HLog(fs, logdir, oldLogDir, c, null);
final Path tableDir = new Path(args[0]);
final Configuration c = HBaseConfiguration.create();
final FileSystem fs = FileSystem.get(c);
final Path logdir = new Path(c.get("hbase.tmp.dir"),
"hlog" + tableDir.getName()
+ EnvironmentEdgeManager.currentTimeMillis());
final Path oldLogDir = new Path(c.get("hbase.tmp.dir"),
HConstants.HREGION_OLDLOGDIR_NAME);
final HLog log = new HLog(fs, logdir, oldLogDir, c, null);
try {
processTable(fs, tableDir, log, c, majorCompact);
} finally {

View File

@ -115,7 +115,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
* HRegionServer makes a set of HRegions available to clients. It checks in with
* the HMaster. There are many HRegionServers in a single HBase deployment.
*/
public class HRegionServer implements HConstants, HRegionInterface,
public class HRegionServer implements HRegionInterface,
HBaseRPCErrorHandler, Runnable, Watcher {
public static final Log LOG = LogFactory.getLog(HRegionServer.class);
private static final HMsg REPORT_EXITING = new HMsg(Type.MSG_REPORT_EXITING);
@ -245,7 +245,8 @@ public class HRegionServer implements HConstants, HRegionInterface,
conf.get("hbase.regionserver.dns.interface","default"),
conf.get("hbase.regionserver.dns.nameserver","default"));
String addressStr = machineName + ":" +
conf.get(REGIONSERVER_PORT, Integer.toString(DEFAULT_REGIONSERVER_PORT));
conf.get(HConstants.REGIONSERVER_PORT,
Integer.toString(HConstants.DEFAULT_REGIONSERVER_PORT));
// This is not necessarily the address we will run with. The address we
// use will be in #serverInfo data member. For example, we may have been
// passed a port of 0 which means we should pick some ephemeral port to bind
@ -262,7 +263,8 @@ public class HRegionServer implements HConstants, HRegionInterface,
// Config'ed params
this.numRetries = conf.getInt("hbase.client.retries.number", 2);
this.threadWakeFrequency = conf.getInt(THREAD_WAKE_FREQUENCY, 10 * 1000);
this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY,
10 * 1000);
this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 1 * 1000);
sleeper = new Sleeper(this.msgInterval, this.stopRequested);
@ -277,7 +279,9 @@ public class HRegionServer implements HConstants, HRegionInterface,
this.numRegionsToReport =
conf.getInt("hbase.regionserver.numregionstoreport", 10);
this.rpcTimeout = conf.getLong(HBASE_REGIONSERVER_LEASE_PERIOD_KEY, DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD);
this.rpcTimeout =
conf.getLong(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD);
reinitialize();
}
@ -311,7 +315,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
reinitializeZooKeeper();
int nbBlocks = conf.getInt("hbase.regionserver.nbreservationblocks", 4);
for(int i = 0; i < nbBlocks; i++) {
reservedSpace.add(new byte[DEFAULT_SIZE_RESERVATION_BLOCK]);
reservedSpace.add(new byte[HConstants.DEFAULT_SIZE_RESERVATION_BLOCK]);
}
}
@ -334,13 +338,14 @@ public class HRegionServer implements HConstants, HRegionInterface,
// Background thread to check for major compactions; needed if region
// has not gotten updates in a while. Make it run at a lesser frequency.
int multiplier = this.conf.getInt(THREAD_WAKE_FREQUENCY +
int multiplier = this.conf.getInt(HConstants.THREAD_WAKE_FREQUENCY +
".multiplier", 1000);
this.majorCompactionChecker = new MajorCompactionChecker(this,
this.threadWakeFrequency * multiplier, this.stopRequested);
this.leases = new Leases(
(int) conf.getLong(HBASE_REGIONSERVER_LEASE_PERIOD_KEY, DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD),
(int) conf.getLong(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD),
this.threadWakeFrequency);
}
@ -899,7 +904,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
}
private HLog setupHLog() throws IOException {
Path oldLogDir = new Path(rootDir, HREGION_OLDLOGDIR_NAME);
final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
Path logdir = new Path(rootDir, HLog.getHLogDirectoryName(this.serverInfo));
if (LOG.isDebugEnabled()) {
LOG.debug("Log dir " + logdir);

View File

@ -93,7 +93,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
* <p>Locking and transactions are handled at a higher level. This API should
* not be called directly but by an HRegion manager.
*/
public class Store implements HConstants, HeapSize {
public class Store implements HeapSize {
static final Log LOG = LogFactory.getLog(Store.class);
/**
* Comparator that looks at columns and compares their family portions.
@ -207,7 +207,7 @@ public class Store implements HConstants, HeapSize {
// Check if this is in-memory store
this.inMemory = family.isInMemory();
// By default we split region if a file > DEFAULT_MAX_FILE_SIZE.
// By default we split region if a file > HConstants.DEFAULT_MAX_FILE_SIZE.
long maxFileSize = info.getTableDesc().getMaxFileSize();
if (maxFileSize == HConstants.DEFAULT_MAX_FILE_SIZE) {
maxFileSize = conf.getLong("hbase.hregion.max.filesize",
@ -1109,7 +1109,7 @@ public class Store implements HConstants, HeapSize {
/*
* @param wantedVersions How many versions were asked for.
* @return wantedVersions or this families' VERSIONS.
* @return wantedVersions or this families' {@link HConstants#VERSIONS}.
*/
int versionsToReturn(final int wantedVersions) {
if (wantedVersions <= 0) {
@ -1470,7 +1470,7 @@ public class Store implements HConstants, HeapSize {
}
} else if (result > 0) {
// Less than what was asked for but maybe < because we're asking for
// r/c/LATEST_TIMESTAMP -- what was returned was r/c-1/SOME_TS...
// r/c/HConstants.LATEST_TIMESTAMP -- what was returned was r/c-1/SOME_TS...
// A next will get us a r/c/SOME_TS.
if (!s.next()) {
return false;

View File

@ -72,7 +72,7 @@ import java.util.regex.Pattern;
* passing filesystem and path. To read, call {@link #createReader()}.
* <p>StoreFiles may also reference store files in another Store.
*/
public class StoreFile implements HConstants {
public class StoreFile {
static final Log LOG = LogFactory.getLog(StoreFile.class.getName());
private static final String HFILE_CACHE_SIZE_KEY = "hfile.block.cache.size";

View File

@ -116,7 +116,7 @@ import com.google.common.util.concurrent.NamingThreadFactory;
* org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration)}.
*
*/
public class HLog implements HConstants, Syncable {
public class HLog implements Syncable {
static final Log LOG = LogFactory.getLog(HLog.class);
public static final byte [] METAFAMILY = Bytes.toBytes("METAFAMILY");
static final byte [] METAROW = Bytes.toBytes("METAROW");
@ -223,7 +223,8 @@ public class HLog implements HConstants, Syncable {
static byte [] COMPLETE_CACHE_FLUSH;
static {
try {
COMPLETE_CACHE_FLUSH = "HBASE::CACHEFLUSH".getBytes(UTF8_ENCODING);
COMPLETE_CACHE_FLUSH =
"HBASE::CACHEFLUSH".getBytes(HConstants.UTF8_ENCODING);
} catch (UnsupportedEncodingException e) {
assert(false);
}
@ -1599,7 +1600,7 @@ public class HLog implements HConstants, Syncable {
final List<Path> processedLogs, final Path oldLogDir,
final FileSystem fs, final Configuration conf)
throws IOException{
final Path corruptDir = new Path(conf.get(HBASE_DIR),
final Path corruptDir = new Path(conf.get(HConstants.HBASE_DIR),
conf.get("hbase.regionserver.hlog.splitlog.corrupt.dir", ".corrupt"));
fs.mkdirs(corruptDir);
@ -1623,7 +1624,7 @@ public class HLog implements HConstants, Syncable {
HTableDescriptor.getTableDir(rootDir, logEntry.getKey().getTablename());
Path regionDir =
HRegion.getRegionDir(tableDir, HRegionInfo.encodeRegionName(logEntry.getKey().getRegionName()));
return new Path(regionDir, HREGION_OLDLOGFILE_NAME);
return new Path(regionDir, HConstants.HREGION_OLDLOGFILE_NAME);
}
@ -1671,8 +1672,8 @@ public class HLog implements HConstants, Syncable {
}
Configuration conf = HBaseConfiguration.create();
FileSystem fs = FileSystem.get(conf);
Path baseDir = new Path(conf.get(HBASE_DIR));
Path oldLogDir = new Path(baseDir, HREGION_OLDLOGDIR_NAME);
final Path baseDir = new Path(conf.get(HConstants.HBASE_DIR));
final Path oldLogDir = new Path(baseDir, HConstants.HREGION_OLDLOGDIR_NAME);
for (int i = 1; i < args.length; i++) {
Path logPath = new Path(args[i]);
if (!fs.exists(logPath)) {

View File

@ -51,7 +51,7 @@ import java.util.Properties;
* control over the process. Currently, this class allows us to parse the
* zoo.cfg and inject variables from HBase's site.xml configuration in.
*/
public class HQuorumPeer implements HConstants {
public class HQuorumPeer {
private static final Log LOG = LogFactory.getLog(HQuorumPeer.class);
private static final String VARIABLE_START = "${";
@ -171,12 +171,13 @@ public class HQuorumPeer implements HConstants {
// First check if there is a zoo.cfg in the CLASSPATH. If so, simply read
// it and grab its configuration properties.
ClassLoader cl = HQuorumPeer.class.getClassLoader();
InputStream inputStream = cl.getResourceAsStream(ZOOKEEPER_CONFIG_NAME);
final InputStream inputStream =
cl.getResourceAsStream(HConstants.ZOOKEEPER_CONFIG_NAME);
if (inputStream != null) {
try {
return parseZooCfg(conf, inputStream);
} catch (IOException e) {
LOG.warn("Cannot read " + ZOOKEEPER_CONFIG_NAME +
LOG.warn("Cannot read " + HConstants.ZOOKEEPER_CONFIG_NAME +
", loading from XML files", e);
}
}
@ -200,14 +201,16 @@ public class HQuorumPeer implements HConstants {
// If clientPort is not set, assign the default
if (zkProperties.getProperty(ZK_CLIENT_PORT_KEY) == null) {
zkProperties.put(ZK_CLIENT_PORT_KEY, DEFAULT_ZOOKEPER_CLIENT_PORT);
zkProperties.put(ZK_CLIENT_PORT_KEY,
HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT);
}
// Create the server.X properties.
int peerPort = conf.getInt("hbase.zookeeper.peerport", 2888);
int leaderPort = conf.getInt("hbase.zookeeper.leaderport", 3888);
String[] serverHosts = conf.getStrings(ZOOKEEPER_QUORUM, "localhost");
final String[] serverHosts = conf.getStrings(HConstants.ZOOKEEPER_QUORUM,
"localhost");
for (int i = 0; i < serverHosts.length; ++i) {
String serverHost = serverHosts[i];
String address = serverHost + ":" + peerPort + ":" + leaderPort;
@ -232,7 +235,8 @@ public class HQuorumPeer implements HConstants {
try {
properties.load(inputStream);
} catch (IOException e) {
String msg = "fail to read properties from " + ZOOKEEPER_CONFIG_NAME;
final String msg = "fail to read properties from "
+ HConstants.ZOOKEEPER_CONFIG_NAME;
LOG.fatal(msg);
throw new IOException(msg, e);
}
@ -269,8 +273,8 @@ public class HQuorumPeer implements HConstants {
}
// Special case for 'hbase.cluster.distributed' property being 'true'
if (key.startsWith("server.")) {
if(conf.get(CLUSTER_DISTRIBUTED).equals(CLUSTER_IS_DISTRIBUTED) &&
value.startsWith("localhost")) {
if (conf.get(HConstants.CLUSTER_DISTRIBUTED).equals(HConstants.CLUSTER_IS_DISTRIBUTED)
&& value.startsWith("localhost")) {
String msg = "The server in zoo.cfg cannot be set to localhost " +
"in a fully-distributed setup because it won't be reachable. " +
"See \"Getting Started\" for more information.";

View File

@ -31,15 +31,16 @@ import java.util.Properties;
* Tool for reading ZooKeeper servers from HBase XML configuation and producing
* a line-by-line list for use by bash scripts.
*/
public class ZKServerTool implements HConstants {
public class ZKServerTool {
/**
* Run the tool.
* @param args Command line arguments. First arg is path to zookeepers file.
*/
public static void main(String args[]) {
Configuration conf = HBaseConfiguration.create();
// Note that we do not simply grab the property ZOOKEEPER_QUORUM from
// the HBaseConfiguration because the user may be using a zoo.cfg file.
// Note that we do not simply grab the property
// HConstants.ZOOKEEPER_QUORUM from the HBaseConfiguration because the
// user may be using a zoo.cfg file.
Properties zkProps = HQuorumPeer.makeZKProps(conf);
for (Entry<Object, Object> entry : zkProps.entrySet()) {
String key = entry.getKey().toString().trim();

View File

@ -54,7 +54,7 @@ import java.util.Properties;
* - read/write/delete the root region location in ZooKeeper.
* - set/check out of safe mode flag.
*/
public class ZooKeeperWrapper implements HConstants {
public class ZooKeeperWrapper {
protected static final Log LOG = LogFactory.getLog(ZooKeeperWrapper.class);
// TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved.
@ -82,7 +82,7 @@ public class ZooKeeperWrapper implements HConstants {
setQuorumServers(properties);
if (quorumServers == null) {
throw new IOException("Could not read quorum servers from " +
ZOOKEEPER_CONFIG_NAME);
HConstants.ZOOKEEPER_CONFIG_NAME);
}
int sessionTimeout = conf.getInt("zookeeper.session.timeout", 60 * 1000);
@ -93,8 +93,8 @@ public class ZooKeeperWrapper implements HConstants {
throw new IOException(e);
}
parentZNode = conf.get(ZOOKEEPER_ZNODE_PARENT,
DEFAULT_ZOOKEEPER_ZNODE_PARENT);
parentZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
String rootServerZNodeName = conf.get("zookeeper.znode.rootserver",
"root-region-server");
@ -137,12 +137,13 @@ public class ZooKeeperWrapper implements HConstants {
}
if (!anyValid) {
LOG.error("no valid quorum servers found in " + ZOOKEEPER_CONFIG_NAME);
LOG.error("no valid quorum servers found in "
+ HConstants.ZOOKEEPER_CONFIG_NAME);
return;
}
if (clientPort == null) {
LOG.error("no clientPort found in " + ZOOKEEPER_CONFIG_NAME);
LOG.error("no clientPort found in " + HConstants.ZOOKEEPER_CONFIG_NAME);
return;
}
@ -815,8 +816,8 @@ public class ZooKeeperWrapper implements HConstants {
}
public static String getZookeeperClusterKey(Configuration conf) {
return conf.get(ZOOKEEPER_QUORUM)+":"+
conf.get(ZOOKEEPER_ZNODE_PARENT);
return (conf.get(HConstants.ZOOKEEPER_QUORUM)
+ ":" + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT));
}
/**

View File

@ -47,7 +47,7 @@ import org.apache.hadoop.security.UserGroupInformation;
* if we are running on DistributedFilesystem, create a FileSystem instance
* each and will close down their instance on the way out.
*/
public class MiniHBaseCluster implements HConstants {
public class MiniHBaseCluster {
static final Log LOG = LogFactory.getLog(MiniHBaseCluster.class.getName());
private Configuration conf;
public LocalHBaseCluster hbaseCluster;
@ -67,7 +67,7 @@ public class MiniHBaseCluster implements HConstants {
public MiniHBaseCluster(Configuration conf, int numRegionServers)
throws IOException {
this.conf = conf;
conf.set(MASTER_PORT, "0");
conf.set(HConstants.MASTER_PORT, "0");
init(numRegionServers);
}

View File

@ -94,16 +94,16 @@ import org.apache.hadoop.util.LineReader;
* <p>If number of clients > 1, we start up a MapReduce job. Each map task
* runs an individual client. Each client does about 1GB of data.
*/
public class PerformanceEvaluation implements HConstants {
public class PerformanceEvaluation {
protected static final Log LOG = LogFactory.getLog(PerformanceEvaluation.class.getName());
private static final int ROW_LENGTH = 1000;
private static final int ONE_GB = 1024 * 1024 * 1000;
private static final int ROWS_PER_GB = ONE_GB / ROW_LENGTH;
public static final byte [] TABLE_NAME = Bytes.toBytes("TestTable");
public static final byte [] FAMILY_NAME = Bytes.toBytes("info");
public static final byte [] QUALIFIER_NAME = Bytes.toBytes("data");
public static final byte[] TABLE_NAME = Bytes.toBytes("TestTable");
public static final byte[] FAMILY_NAME = Bytes.toBytes("info");
public static final byte[] QUALIFIER_NAME = Bytes.toBytes("data");
protected static final HTableDescriptor TABLE_DESCRIPTOR;
static {

View File

@ -31,15 +31,14 @@ import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
import org.apache.hadoop.hbase.util.Bytes;
public class TestExplicitColumnTracker extends HBaseTestCase
implements HConstants {
public class TestExplicitColumnTracker extends HBaseTestCase {
private boolean PRINT = false;
private final byte [] col1 = Bytes.toBytes("col1");
private final byte [] col2 = Bytes.toBytes("col2");
private final byte [] col3 = Bytes.toBytes("col3");
private final byte [] col4 = Bytes.toBytes("col4");
private final byte [] col5 = Bytes.toBytes("col5");
private final byte[] col1 = Bytes.toBytes("col1");
private final byte[] col2 = Bytes.toBytes("col2");
private final byte[] col3 = Bytes.toBytes("col3");
private final byte[] col4 = Bytes.toBytes("col4");
private final byte[] col5 = Bytes.toBytes("col5");
private void runTest(int maxVersions,
TreeSet<byte[]> trackColumns,

View File

@ -44,19 +44,19 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
* {@link TestGet} is a medley of tests of get all done up as a single test.
* This class
*/
public class TestGetClosestAtOrBefore extends HBaseTestCase implements HConstants {
static final Log LOG = LogFactory.getLog(TestGetClosestAtOrBefore.class);
public class TestGetClosestAtOrBefore extends HBaseTestCase {
private static final Log LOG = LogFactory.getLog(TestGetClosestAtOrBefore.class);
private MiniDFSCluster miniHdfs;
private static final byte [] T00 = Bytes.toBytes("000");
private static final byte [] T10 = Bytes.toBytes("010");
private static final byte [] T11 = Bytes.toBytes("011");
private static final byte [] T12 = Bytes.toBytes("012");
private static final byte [] T20 = Bytes.toBytes("020");
private static final byte [] T30 = Bytes.toBytes("030");
private static final byte [] T31 = Bytes.toBytes("031");
private static final byte [] T35 = Bytes.toBytes("035");
private static final byte [] T40 = Bytes.toBytes("040");
private static final byte[] T00 = Bytes.toBytes("000");
private static final byte[] T10 = Bytes.toBytes("010");
private static final byte[] T11 = Bytes.toBytes("011");
private static final byte[] T12 = Bytes.toBytes("012");
private static final byte[] T20 = Bytes.toBytes("020");
private static final byte[] T30 = Bytes.toBytes("030");
private static final byte[] T31 = Bytes.toBytes("031");
private static final byte[] T35 = Bytes.toBytes("035");
private static final byte[] T40 = Bytes.toBytes("040");
@Override
protected void setUp() throws Exception {
@ -86,7 +86,8 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase implements HConstant
i == 0? HConstants.EMPTY_BYTE_ARRAY: Bytes.toBytes((byte)i),
i == last? HConstants.EMPTY_BYTE_ARRAY: Bytes.toBytes((byte)i + interval));
Put put = new Put(hri.getRegionName());
put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(hri));
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
mr.put(put, false);
}
}

View File

@ -30,17 +30,17 @@ import org.apache.hadoop.hbase.regionserver.GetDeleteTracker.Delete;
import org.apache.hadoop.hbase.util.Bytes;
public class TestGetDeleteTracker extends HBaseTestCase implements HConstants {
public class TestGetDeleteTracker extends HBaseTestCase {
private static final boolean PRINT = true;
private byte [] col1 = null;
private byte [] col2 = null;
private byte[] col1 = null;
private byte[] col2 = null;
private int col1Len = 0;
private int col2Len = 0;
private byte [] empty = null;
private byte[] empty = null;
private long ts1 = 0L;
private long ts2 = 0L;

View File

@ -32,24 +32,23 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
public class TestKeyValueHeap extends HBaseTestCase
implements HConstants {
public class TestKeyValueHeap extends HBaseTestCase {
private static final boolean PRINT = false;
List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>();
private byte [] row1;
private byte [] fam1;
private byte [] col1;
private byte [] data;
private byte[] row1;
private byte[] fam1;
private byte[] col1;
private byte[] data;
private byte [] row2;
private byte [] fam2;
private byte [] col2;
private byte[] row2;
private byte[] fam2;
private byte[] col2;
private byte [] col3;
private byte [] col4;
private byte [] col5;
private byte[] col3;
private byte[] col4;
private byte[] col5;
public void setUp() throws Exception {
super.setUp();

View File

@ -34,21 +34,20 @@ import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
import org.apache.hadoop.hbase.util.Bytes;
public class TestQueryMatcher extends HBaseTestCase
implements HConstants {
public class TestQueryMatcher extends HBaseTestCase {
private static final boolean PRINT = false;
private byte [] row1;
private byte [] row2;
private byte [] fam1;
private byte [] fam2;
private byte [] col1;
private byte [] col2;
private byte [] col3;
private byte [] col4;
private byte [] col5;
private byte[] row1;
private byte[] row2;
private byte[] fam1;
private byte[] fam2;
private byte[] col1;
private byte[] col2;
private byte[] col3;
private byte[] col4;
private byte[] col5;
private byte [] data;
private byte[] data;
private Get get;

View File

@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
public class TestScanDeleteTracker extends HBaseTestCase implements HConstants {
public class TestScanDeleteTracker extends HBaseTestCase {
private ScanDeleteTracker sdt;
private long timestamp = 10L;

View File

@ -28,19 +28,18 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
import org.apache.hadoop.hbase.util.Bytes;
public class TestWildcardColumnTracker extends HBaseTestCase
implements HConstants {
public class TestWildcardColumnTracker extends HBaseTestCase {
private boolean PRINT = false;
public void testGet_SingleVersion() {
if(PRINT) {
if (PRINT) {
System.out.println("SingleVersion");
}
byte [] col1 = Bytes.toBytes("col1");
byte [] col2 = Bytes.toBytes("col2");
byte [] col3 = Bytes.toBytes("col3");
byte [] col4 = Bytes.toBytes("col4");
byte [] col5 = Bytes.toBytes("col5");
byte[] col1 = Bytes.toBytes("col1");
byte[] col2 = Bytes.toBytes("col2");
byte[] col3 = Bytes.toBytes("col3");
byte[] col4 = Bytes.toBytes("col4");
byte[] col5 = Bytes.toBytes("col5");
//Create tracker
List<MatchCode> expected = new ArrayList<MatchCode>();

View File

@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hdfs.MiniDFSCluster;
/** JUnit test case for HLog */
public class TestHLog extends HBaseTestCase implements HConstants {
static final Log LOG = LogFactory.getLog(TestHLog.class);
public class TestHLog extends HBaseTestCase {
private static final Log LOG = LogFactory.getLog(TestHLog.class);
private Path dir;
private Path oldLogDir;
private MiniDFSCluster cluster;