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:
parent
296b055f97
commit
4193875580
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -749,18 +752,19 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
|
|||
// table would sit should it exist. Open scanner on it. If a region
|
||||
// for the table we want to create already exists, then table already
|
||||
// created. Throw already-exists exception.
|
||||
MetaRegion m = regionManager.getFirstMetaRegionForRegion(newRegions[0]);
|
||||
MetaRegion m = regionManager.getFirstMetaRegionForRegion(newRegions[0]);
|
||||
byte [] metaRegionName = m.getRegionName();
|
||||
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));
|
||||
|
@ -1003,8 +1016,9 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
|
|||
HRegionInfo hri = getHRegionInfo(rr.getRow(), rr);
|
||||
if (hostnameAndPort == null) {
|
||||
// Get server from the .META. if it wasn't passed as argument
|
||||
hostnameAndPort =
|
||||
Bytes.toString(rr.getValue(CATALOG_FAMILY, SERVER_QUALIFIER));
|
||||
hostnameAndPort =
|
||||
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);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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() +
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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());
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -236,4 +238,4 @@ class CompactSplitThread extends Thread implements HConstants {
|
|||
public int getCompactionQueueSize() {
|
||||
return compactionQueue.size();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
@ -1696,7 +1701,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
|
|||
if (!region.getRegionInfo().isMetaTable()) {
|
||||
this.cacheFlusher.reclaimMemStoreMemory();
|
||||
}
|
||||
return region.checkAndMutate(row, family, qualifier, value, w, lock,
|
||||
return region.checkAndMutate(row, family, qualifier, value, w, lock,
|
||||
true);
|
||||
} catch (Throwable t) {
|
||||
throw convertThrowableToIOE(cleanup(t));
|
||||
|
@ -2017,7 +2022,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
|
|||
HRegion region = getRegion(regionName);
|
||||
region.bulkLoadHFile(hfilePath, familyName);
|
||||
}
|
||||
|
||||
|
||||
Map<String, Integer> rowlocks =
|
||||
new ConcurrentHashMap<String, Integer>();
|
||||
|
||||
|
|
|
@ -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",
|
||||
|
@ -477,7 +477,7 @@ public class Store implements HConstants, HeapSize {
|
|||
|
||||
public void bulkLoadHFile(String srcPathStr) throws IOException {
|
||||
Path srcPath = new Path(srcPathStr);
|
||||
|
||||
|
||||
HFile.Reader reader = null;
|
||||
try {
|
||||
LOG.info("Validating hfile at " + srcPath + " for inclusion in "
|
||||
|
@ -485,20 +485,20 @@ public class Store implements HConstants, HeapSize {
|
|||
reader = new HFile.Reader(srcPath.getFileSystem(conf),
|
||||
srcPath, null, false);
|
||||
reader.loadFileInfo();
|
||||
|
||||
|
||||
byte[] firstKey = reader.getFirstRowKey();
|
||||
byte[] lastKey = reader.getLastRowKey();
|
||||
|
||||
|
||||
LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) +
|
||||
" last=" + Bytes.toStringBinary(lastKey));
|
||||
LOG.debug("Region bounds: first=" +
|
||||
Bytes.toStringBinary(region.getStartKey()) +
|
||||
" last=" + Bytes.toStringBinary(region.getEndKey()));
|
||||
|
||||
|
||||
HRegionInfo hri = region.getRegionInfo();
|
||||
if (!hri.containsRange(firstKey, lastKey)) {
|
||||
throw new WrongRegionException(
|
||||
"Bulk load file " + srcPathStr + " does not fit inside region "
|
||||
"Bulk load file " + srcPathStr + " does not fit inside region "
|
||||
+ this.region);
|
||||
}
|
||||
} finally {
|
||||
|
@ -516,15 +516,15 @@ public class Store implements HConstants, HeapSize {
|
|||
LOG.info("Copied to temporary path on dst filesystem: " + tmpPath);
|
||||
srcPath = tmpPath;
|
||||
}
|
||||
|
||||
|
||||
Path dstPath = StoreFile.getRandomFilename(fs, homedir);
|
||||
LOG.info("Renaming bulk load file " + srcPath + " to " + dstPath);
|
||||
StoreFile.rename(fs, srcPath, dstPath);
|
||||
|
||||
|
||||
StoreFile sf = new StoreFile(fs, dstPath, blockcache,
|
||||
this.conf, this.family.getBloomFilterType(), this.inMemory);
|
||||
sf.createReader();
|
||||
|
||||
|
||||
LOG.info("Moved hfile " + srcPath + " into store directory " +
|
||||
homedir + " - updating store file list.");
|
||||
|
||||
|
@ -539,7 +539,7 @@ public class Store implements HConstants, HeapSize {
|
|||
this.lock.writeLock().unlock();
|
||||
}
|
||||
LOG.info("Successfully loaded store file " + srcPath
|
||||
+ " into store " + this + " (new location: " + dstPath + ")");
|
||||
+ " into store " + this + " (new location: " + dstPath + ")");
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -554,10 +554,10 @@ public class Store implements HConstants, HeapSize {
|
|||
this.lock.writeLock().lock();
|
||||
try {
|
||||
ImmutableList<StoreFile> result = storefiles;
|
||||
|
||||
|
||||
// Clear so metrics doesn't find them.
|
||||
storefiles = ImmutableList.of();
|
||||
|
||||
|
||||
for (StoreFile f: result) {
|
||||
f.closeReader();
|
||||
}
|
||||
|
@ -1059,7 +1059,7 @@ public class Store implements HConstants, HeapSize {
|
|||
newStoreFiles.add(sf);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// If a StoreFile result, move it into place. May be null.
|
||||
if (result != null) {
|
||||
newStoreFiles.add(result);
|
||||
|
@ -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) {
|
||||
|
@ -1363,7 +1363,7 @@ public class Store implements HConstants, HeapSize {
|
|||
|
||||
/**
|
||||
* Return a scanner for both the memstore and the HStore files
|
||||
* @throws IOException
|
||||
* @throws IOException
|
||||
*/
|
||||
protected KeyValueScanner getScanner(Scan scan,
|
||||
final NavigableSet<byte []> targetCols) throws IOException {
|
||||
|
@ -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;
|
||||
|
|
|
@ -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";
|
||||
|
@ -104,7 +104,7 @@ public class StoreFile implements HConstants {
|
|||
/** Constant for major compaction meta */
|
||||
public static final byte [] MAJOR_COMPACTION_KEY =
|
||||
Bytes.toBytes("MAJOR_COMPACTION_KEY");
|
||||
|
||||
|
||||
// If true, this file was product of a major compaction. Its then set
|
||||
// whenever you get a Reader.
|
||||
private AtomicBoolean majorCompaction = null;
|
||||
|
@ -115,10 +115,10 @@ public class StoreFile implements HConstants {
|
|||
public static final byte[] BULKLOAD_TIME_KEY =
|
||||
Bytes.toBytes("BULKLOAD_TIMESTAMP");
|
||||
|
||||
|
||||
|
||||
static final String BLOOM_FILTER_META_KEY = "BLOOM_FILTER_META";
|
||||
static final String BLOOM_FILTER_DATA_KEY = "BLOOM_FILTER_DATA";
|
||||
static final byte[] BLOOM_FILTER_TYPE_KEY =
|
||||
static final byte[] BLOOM_FILTER_TYPE_KEY =
|
||||
Bytes.toBytes("BLOOM_FILTER_TYPE");
|
||||
|
||||
/**
|
||||
|
@ -154,7 +154,7 @@ public class StoreFile implements HConstants {
|
|||
* @throws IOException When opening the reader fails.
|
||||
*/
|
||||
StoreFile(final FileSystem fs, final Path p, final boolean blockcache,
|
||||
final Configuration conf, final BloomType bt, final boolean inMemory)
|
||||
final Configuration conf, final BloomType bt, final boolean inMemory)
|
||||
throws IOException {
|
||||
this.conf = conf;
|
||||
this.fs = fs;
|
||||
|
@ -165,7 +165,7 @@ public class StoreFile implements HConstants {
|
|||
this.reference = Reference.read(fs, p);
|
||||
this.referencePath = getReferredToFile(this.path);
|
||||
}
|
||||
// ignore if the column family config says "no bloom filter"
|
||||
// ignore if the column family config says "no bloom filter"
|
||||
// even if there is one in the hfile.
|
||||
if (conf.getBoolean("io.hfile.bloom.enabled", true)) {
|
||||
this.bloomType = bt;
|
||||
|
@ -264,7 +264,7 @@ public class StoreFile implements HConstants {
|
|||
}
|
||||
return this.sequenceid;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Return the highest sequence ID found across all storefiles in
|
||||
* the given list. Store files that were created by a mapreduce
|
||||
|
@ -375,14 +375,14 @@ public class StoreFile implements HConstants {
|
|||
this.majorCompaction.set(mc);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
if (this.bloomType != BloomType.NONE) {
|
||||
this.reader.loadBloomfilter();
|
||||
}
|
||||
|
||||
return this.reader;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @return Reader for StoreFile. creates if necessary
|
||||
* @throws IOException
|
||||
|
@ -396,7 +396,7 @@ public class StoreFile implements HConstants {
|
|||
|
||||
/**
|
||||
* @return Current reader. Must call createReader first else returns null.
|
||||
* @throws IOException
|
||||
* @throws IOException
|
||||
* @see {@link #createReader()}
|
||||
*/
|
||||
public StoreFile.Reader getReader() {
|
||||
|
@ -423,11 +423,11 @@ public class StoreFile implements HConstants {
|
|||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
public String toString() {
|
||||
return this.path.toString() +
|
||||
(isReference()? "-" + this.referencePath + "-" + reference.toString(): "");
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @return a length description of this StoreFile, suitable for debug output
|
||||
*/
|
||||
|
@ -477,8 +477,8 @@ public class StoreFile implements HConstants {
|
|||
* @return HFile.Writer
|
||||
* @throws IOException
|
||||
*/
|
||||
public static StoreFile.Writer createWriter(final FileSystem fs, final Path dir,
|
||||
final int blocksize)
|
||||
public static StoreFile.Writer createWriter(final FileSystem fs, final Path dir,
|
||||
final int blocksize)
|
||||
throws IOException {
|
||||
return createWriter(fs,dir,blocksize,null,null,null,BloomType.NONE,0);
|
||||
}
|
||||
|
@ -498,10 +498,10 @@ public class StoreFile implements HConstants {
|
|||
* @return HFile.Writer
|
||||
* @throws IOException
|
||||
*/
|
||||
public static StoreFile.Writer createWriter(final FileSystem fs, final Path dir,
|
||||
final int blocksize, final Compression.Algorithm algorithm,
|
||||
final KeyValue.KVComparator c, final Configuration conf,
|
||||
BloomType bloomType, int maxKeySize)
|
||||
public static StoreFile.Writer createWriter(final FileSystem fs, final Path dir,
|
||||
final int blocksize, final Compression.Algorithm algorithm,
|
||||
final KeyValue.KVComparator c, final Configuration conf,
|
||||
BloomType bloomType, int maxKeySize)
|
||||
throws IOException {
|
||||
if (!fs.exists(dir)) {
|
||||
fs.mkdirs(dir);
|
||||
|
@ -614,7 +614,7 @@ public class StoreFile implements HConstants {
|
|||
/** Type of bloom filter (e.g. ROW vs ROWCOL) */
|
||||
protected BloomType bloomFilterType;
|
||||
|
||||
public Reader(FileSystem fs, Path path, BlockCache cache,
|
||||
public Reader(FileSystem fs, Path path, BlockCache cache,
|
||||
boolean inMemory)
|
||||
throws IOException {
|
||||
super(fs, path, cache, inMemory);
|
||||
|
@ -627,7 +627,7 @@ public class StoreFile implements HConstants {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<byte [], byte []> loadFileInfo()
|
||||
public Map<byte [], byte []> loadFileInfo()
|
||||
throws IOException {
|
||||
Map<byte [], byte []> fi = super.loadFileInfo();
|
||||
|
||||
|
@ -635,19 +635,19 @@ public class StoreFile implements HConstants {
|
|||
if (b != null) {
|
||||
bloomFilterType = BloomType.valueOf(Bytes.toString(b));
|
||||
}
|
||||
|
||||
|
||||
return fi;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Load the bloom filter for this HFile into memory.
|
||||
* Load the bloom filter for this HFile into memory.
|
||||
* Assumes the HFile has already been loaded
|
||||
*/
|
||||
public void loadBloomfilter() {
|
||||
if (this.bloomFilter != null) {
|
||||
return; // already loaded
|
||||
}
|
||||
|
||||
|
||||
// see if bloom filter information is in the metadata
|
||||
try {
|
||||
ByteBuffer b = getMetaBlock(BLOOM_FILTER_META_KEY, false);
|
||||
|
@ -656,7 +656,7 @@ public class StoreFile implements HConstants {
|
|||
throw new IOException("valid bloom filter type not found in FileInfo");
|
||||
}
|
||||
this.bloomFilter = new ByteBloomFilter(b);
|
||||
LOG.info("Loaded " + (bloomFilterType==BloomType.ROW? "row":"col")
|
||||
LOG.info("Loaded " + (bloomFilterType==BloomType.ROW? "row":"col")
|
||||
+ " bloom filter metadata for " + name);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
|
@ -667,11 +667,11 @@ public class StoreFile implements HConstants {
|
|||
this.bloomFilter = null;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
BloomFilter getBloomFilter() {
|
||||
return this.bloomFilter;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @return bloom type information associated with this store file
|
||||
*/
|
||||
|
@ -681,7 +681,7 @@ public class StoreFile implements HConstants {
|
|||
|
||||
@Override
|
||||
public int getFilterEntries() {
|
||||
return (this.bloomFilter != null) ? this.bloomFilter.getKeyCount()
|
||||
return (this.bloomFilter != null) ? this.bloomFilter.getKeyCount()
|
||||
: super.getFilterEntries();
|
||||
}
|
||||
|
||||
|
@ -696,12 +696,12 @@ public class StoreFile implements HConstants {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean shouldSeek(final byte[] row,
|
||||
public boolean shouldSeek(final byte[] row,
|
||||
final SortedSet<byte[]> columns) {
|
||||
if (bloomFilter == null) {
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
byte[] key;
|
||||
switch(bloomFilterType) {
|
||||
case ROW:
|
||||
|
@ -717,27 +717,27 @@ public class StoreFile implements HConstants {
|
|||
default:
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
try {
|
||||
ByteBuffer bloom = getMetaBlock(BLOOM_FILTER_DATA_KEY, true);
|
||||
if (bloom != null) {
|
||||
return bloomFilter.contains(key, bloom);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Error reading bloom filter data -- proceeding without",
|
||||
LOG.error("Error reading bloom filter data -- proceeding without",
|
||||
e);
|
||||
bloomFilter = null;
|
||||
} catch (IllegalArgumentException e) {
|
||||
LOG.error("Bad bloom filter data -- proceeding without", e);
|
||||
bloomFilter = null;
|
||||
}
|
||||
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
|
@ -769,10 +769,10 @@ public class StoreFile implements HConstants {
|
|||
this.kvComparator = comparator;
|
||||
|
||||
if (bloomType != BloomType.NONE && conf != null) {
|
||||
float err = conf.getFloat("io.hfile.bloom.error.rate", (float)0.01);
|
||||
float err = conf.getFloat("io.hfile.bloom.error.rate", (float)0.01);
|
||||
int maxFold = conf.getInt("io.hfile.bloom.max.fold", 7);
|
||||
|
||||
this.bloomFilter = new ByteBloomFilter(maxKeys, err,
|
||||
|
||||
this.bloomFilter = new ByteBloomFilter(maxKeys, err,
|
||||
Hash.getHashType(conf), maxFold);
|
||||
this.bloomFilter.allocBloom();
|
||||
this.bloomType = bloomType;
|
||||
|
@ -804,11 +804,11 @@ public class StoreFile implements HConstants {
|
|||
boolean newKey = true;
|
||||
if (this.lastKv != null) {
|
||||
switch(bloomType) {
|
||||
case ROW:
|
||||
newKey = ! kvComparator.matchingRows(kv, lastKv);
|
||||
case ROW:
|
||||
newKey = ! kvComparator.matchingRows(kv, lastKv);
|
||||
break;
|
||||
case ROWCOL:
|
||||
newKey = ! kvComparator.matchingRowColumn(kv, lastKv);
|
||||
case ROWCOL:
|
||||
newKey = ! kvComparator.matchingRowColumn(kv, lastKv);
|
||||
break;
|
||||
case NONE:
|
||||
newKey = false;
|
||||
|
@ -820,12 +820,12 @@ public class StoreFile implements HConstants {
|
|||
* Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp
|
||||
*
|
||||
* 2 Types of Filtering:
|
||||
* 1. Row = Row
|
||||
* 1. Row = Row
|
||||
* 2. RowCol = Row + Qualifier
|
||||
*/
|
||||
switch (bloomType) {
|
||||
case ROW:
|
||||
this.bloomFilter.add(kv.getBuffer(), kv.getRowOffset(),
|
||||
this.bloomFilter.add(kv.getBuffer(), kv.getRowOffset(),
|
||||
kv.getRowLength());
|
||||
break;
|
||||
case ROWCOL:
|
||||
|
@ -860,9 +860,9 @@ public class StoreFile implements HConstants {
|
|||
}
|
||||
super.append(key, value);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
public void close()
|
||||
throws IOException {
|
||||
// make sure we wrote something to the bloom before adding it
|
||||
if (this.bloomFilter != null && this.bloomFilter.getKeyCount() > 0) {
|
||||
|
@ -871,7 +871,7 @@ public class StoreFile implements HConstants {
|
|||
int b = this.bloomFilter.getByteSize();
|
||||
int k = this.bloomFilter.getKeyCount();
|
||||
int m = this.bloomFilter.getMaxKeys();
|
||||
StoreFile.LOG.info("Bloom added to HFile. " + b + "B, " +
|
||||
StoreFile.LOG.info("Bloom added to HFile. " + b + "B, " +
|
||||
k + "/" + m + " (" + NumberFormat.getPercentInstance().format(
|
||||
((double)k) / ((double)m)) + ")");
|
||||
}
|
||||
|
@ -881,9 +881,9 @@ public class StoreFile implements HConstants {
|
|||
}
|
||||
super.close();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Useful comparators for comparing StoreFiles.
|
||||
*/
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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.";
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -143,7 +143,7 @@ public class MiniHBaseCluster implements HConstants {
|
|||
/**
|
||||
* Subclass so can get at protected methods (none at moment). Also, creates
|
||||
* a FileSystem instance per instantiation. Adds a shutdown own FileSystem
|
||||
* on the way out. Shuts down own Filesystem only, not All filesystems as
|
||||
* on the way out. Shuts down own Filesystem only, not All filesystems as
|
||||
* the FileSystem system exit hook does.
|
||||
*/
|
||||
public static class MiniHBaseClusterRegionServer extends HRegionServer {
|
||||
|
@ -179,7 +179,7 @@ public class MiniHBaseCluster implements HConstants {
|
|||
new UnixUserGroupInformation(username, new String[]{"supergroup"}));
|
||||
return c2;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected void init(MapWritable c) throws IOException {
|
||||
super.init(c);
|
||||
|
@ -199,7 +199,7 @@ public class MiniHBaseCluster implements HConstants {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public void kill() {
|
||||
super.kill();
|
||||
}
|
||||
|
@ -370,7 +370,7 @@ public class MiniHBaseCluster implements HConstants {
|
|||
public HRegionServer getRegionServer(int serverNumber) {
|
||||
return hbaseCluster.getRegionServer(serverNumber);
|
||||
}
|
||||
|
||||
|
||||
public List<HRegion> getRegions(byte[] tableName) {
|
||||
List<HRegion> ret = new ArrayList<HRegion>();
|
||||
for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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,
|
||||
|
@ -50,13 +49,13 @@ implements HConstants {
|
|||
|
||||
|
||||
//Initialize result
|
||||
List<MatchCode> result = new ArrayList<MatchCode>();
|
||||
|
||||
List<MatchCode> result = new ArrayList<MatchCode>();
|
||||
|
||||
//"Match"
|
||||
for(byte [] col : scannerColumns){
|
||||
result.add(exp.checkColumn(col, 0, col.length));
|
||||
}
|
||||
|
||||
|
||||
assertEquals(expected.size(), result.size());
|
||||
for(int i=0; i< expected.size(); i++){
|
||||
assertEquals(expected.get(i), result.get(i));
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -346,4 +347,4 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase implements HConstant
|
|||
}
|
||||
super.tearDown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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>();
|
||||
|
|
|
@ -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;
|
||||
|
@ -395,4 +395,4 @@ public class TestHLog extends HBaseTestCase implements HConstants {
|
|||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue