HADOOP-1678 On region split, master should designate which host should serve daughter splits.

Phase 1: Master balances load for new regions and when a region server fails.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@564780 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jim Kellerman 2007-08-10 22:11:05 +00:00
parent 790e3d767e
commit 931d452cb2
23 changed files with 1433 additions and 709 deletions

View File

@ -86,3 +86,6 @@ Trunk (unreleased changes)
53. HADOOP-1528 HClient for multiple tables - expose close table function
54. HADOOP-1466 Clean up warnings, visibility and javadoc issues in HBase.
55. HADOOP-1662 Make region splits faster
56. HADOOP-1678 On region split, master should designate which host should
serve daughter splits. Phase 1: Master balances load for new regions and
when a region server fails.

View File

@ -488,8 +488,9 @@ public class HBaseAdmin implements HConstants {
* @throws IllegalArgumentException - if the table name is reserved
*/
protected void checkReservedTableName(Text tableName) {
if(tableName.equals(ROOT_TABLE_NAME)
|| tableName.equals(META_TABLE_NAME)) {
if(tableName.charAt(0) == '-' ||
tableName.charAt(0) == '.' ||
tableName.find(",") != -1) {
throw new IllegalArgumentException(tableName + " is a reserved table name");
}

View File

@ -92,15 +92,17 @@ public interface HConstants {
// Do we ever need to know all the information that we are storing?
/** The root table's name. */
static final Text ROOT_TABLE_NAME = new Text("--ROOT--");
static final Text ROOT_TABLE_NAME = new Text("-ROOT-");
/** The META table's name. */
static final Text META_TABLE_NAME = new Text("--META--");
static final Text META_TABLE_NAME = new Text(".META.");
// Defines for the column names used in both ROOT and META HBase 'meta' tables.
/** The ROOT and META column family */
/** The ROOT and META column family (string) */
static final String COLUMN_FAMILY_STR = "info:";
/** The ROOT and META column family (Text) */
static final Text COLUMN_FAMILY = new Text(COLUMN_FAMILY_STR);
/** Array of meta column names */

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.conf.*;
import java.io.*;
import java.util.*;
import java.util.concurrent.atomic.AtomicInteger;
/**
* HLog stores all the edits to the HStore.
@ -73,14 +74,14 @@ public class HLog implements HConstants {
SequenceFile.Writer writer;
TreeMap<Long, Path> outputfiles = new TreeMap<Long, Path>();
boolean insideCacheFlush = false;
volatile boolean insideCacheFlush = false;
TreeMap<Text, Long> regionToLastFlush = new TreeMap<Text, Long>();
boolean closed = false;
transient long logSeqNum = 0;
volatile boolean closed = false;
volatile long logSeqNum = 0;
long filenum = 0;
transient int numEntries = 0;
AtomicInteger numEntries = new AtomicInteger(0);
Integer rollLock = new Integer(0);
@ -125,7 +126,7 @@ public class HLog implements HConstants {
logWriters.put(regionName, w);
}
if (LOG.isDebugEnabled()) {
LOG.debug("Edit " + key.toString());
LOG.debug("Edit " + key.toString() + "=" + val.toString());
}
w.append(key, val);
}
@ -173,6 +174,16 @@ public class HLog implements HConstants {
fs.mkdirs(dir);
rollWriter();
}
synchronized void setSequenceNumber(long newvalue) {
if (newvalue > logSeqNum) {
if (LOG.isDebugEnabled()) {
LOG.debug("changing sequence number from " + logSeqNum + " to " +
newvalue);
}
logSeqNum = newvalue;
}
}
/**
* Roll the log writer. That is, start writing log messages to a new file.
@ -266,7 +277,7 @@ public class HLog implements HConstants {
}
fs.delete(p);
}
this.numEntries = 0;
this.numEntries.set(0);
}
}
@ -343,13 +354,13 @@ public class HLog implements HConstants {
new HLogKey(regionName, tableName, row, seqNum[counter++]);
HLogEdit logEdit = new HLogEdit(es.getKey(), es.getValue(), timestamp);
writer.append(logKey, logEdit);
numEntries++;
numEntries.getAndIncrement();
}
}
/** @return How many items have been added to the log */
int getNumEntries() {
return numEntries;
return numEntries.get();
}
/**
@ -418,7 +429,7 @@ public class HLog implements HConstants {
writer.append(new HLogKey(regionName, tableName, HLog.METAROW, logSeqId),
new HLogEdit(HLog.METACOLUMN, COMPLETE_CACHEFLUSH.get(),
System.currentTimeMillis()));
numEntries++;
numEntries.getAndIncrement();
// Remember the most-recent flush for each region.
// This is used to delete obsolete log files.

View File

@ -72,8 +72,14 @@ public class HLogEdit implements Writable {
/** {@inheritDoc} */
@Override
public String toString() {
return getColumn().toString() + " " + this.getTimestamp() + " " +
new String(getVal()).trim();
String value = "";
try {
value = new String(getVal(), HConstants.UTF8_ENCODING);
} catch (UnsupportedEncodingException e) {
throw new RuntimeException("UTF8 encoding not present?", e);
}
return "(" + getColumn().toString() + "/" + getTimestamp() + "/" + value + ")";
}
// Writable

View File

@ -84,7 +84,7 @@ public class HLogKey implements WritableComparable {
*/
@Override
public String toString() {
return tablename + "," + regionName + "," + row + "," + logSeqNum;
return tablename + "/" + regionName + "/" + row + "/" + logSeqNum;
}
/**

File diff suppressed because it is too large Load Diff

View File

@ -275,6 +275,7 @@ public class HRegion implements HConstants {
int compactionThreshold = 0;
private final HLocking lock = new HLocking();
private long desiredMaxFileSize;
private final long maxSequenceId;
//////////////////////////////////////////////////////////////////////////////
// Constructor
@ -324,12 +325,26 @@ public class HRegion implements HConstants {
}
// Load in all the HStores.
long maxSeqId = -1;
for(Map.Entry<Text, HColumnDescriptor> e :
this.regionInfo.tableDesc.families().entrySet()) {
Text colFamily = HStoreKey.extractFamily(e.getKey());
stores.put(colFamily,
new HStore(rootDir, this.regionInfo.regionName, e.getValue(), fs,
oldLogFile, conf));
HStore store = new HStore(rootDir, this.regionInfo.regionName,
e.getValue(), fs, oldLogFile, conf);
stores.put(colFamily, store);
long storeSeqId = store.getMaxSequenceId();
if (storeSeqId > maxSeqId) {
maxSeqId = storeSeqId;
}
}
this.maxSequenceId = maxSeqId;
if (LOG.isDebugEnabled()) {
LOG.debug("maximum sequence id for region " + regionInfo.getRegionName() +
" is " + this.maxSequenceId);
}
// Get rid of any splits or merges that were lost in-progress
@ -361,6 +376,10 @@ public class HRegion implements HConstants {
this.writestate.writesOngoing = false;
LOG.info("region " + this.regionInfo.regionName + " available");
}
long getMaxSequenceId() {
return this.maxSequenceId;
}
/** Returns a HRegionInfo object for this region */
HRegionInfo getRegionInfo() {
@ -464,8 +483,8 @@ public class HRegion implements HConstants {
* @throws IOException
*/
HRegion[] closeAndSplit(final Text midKey,
final RegionUnavailableListener listener)
throws IOException {
final RegionUnavailableListener listener) throws IOException {
checkMidKey(midKey);
long startTime = System.currentTimeMillis();
Path splits = getSplitsDir();
@ -496,6 +515,7 @@ public class HRegion implements HConstants {
Vector<HStoreFile> hstoreFilesToSplit = close();
if (hstoreFilesToSplit == null) {
LOG.warn("Close came back null (Implement abort of close?)");
throw new RuntimeException("close returned empty vector of HStoreFiles");
}
// Tell listener that region is now closed and that they can therefore
@ -690,8 +710,11 @@ public class HRegion implements HConstants {
biggest = size;
}
}
biggest.setSplitable(splitable);
if (biggest != null) {
biggest.setSplitable(splitable);
}
return biggest;
} finally {
lock.releaseReadLock();
}
@ -1405,6 +1428,7 @@ public class HRegion implements HConstants {
}
}
/** {@inheritDoc} */
@Override
public String toString() {
return getRegionName().toString();
@ -1842,9 +1866,7 @@ public class HRegion implements HConstants {
if (bytes == null || bytes.length == 0) {
return null;
}
return (HRegionInfo)((bytes == null || bytes.length == 0)?
null:
Writables.getWritable(bytes, new HRegionInfo()));
return (HRegionInfo) Writables.getWritable(bytes, new HRegionInfo());
}
/**
@ -1905,6 +1927,13 @@ public class HRegion implements HConstants {
return startCode;
}
/**
* Computes the Path of the HRegion
*
* @param dir parent directory
* @param regionName name of the region
* @return Path of HRegion directory
*/
public static Path getRegionDir(final Path dir, final Text regionName) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName));
}

View File

@ -32,6 +32,7 @@ import java.util.Vector;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.commons.logging.Log;
@ -391,6 +392,9 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
// Leases
private Leases leases;
// Request counter
private AtomicInteger requestCount;
/**
* Starts a HRegionServer at the default location
@ -424,6 +428,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
Collections.synchronizedSortedMap(new TreeMap<Text, HRegion>());
this.outboundMsgs = new Vector<HMsg>();
this.requestCount = new AtomicInteger();
// Config'ed params
this.numRetries = conf.getInt("hbase.client.retries.number", 2);
@ -597,6 +602,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
if (LOG.isDebugEnabled()) {
LOG.debug("Telling master we are up");
}
requestCount.set(0);
serverInfo.setLoad(new HServerLoad(0, onlineRegions.size()));
hbaseMaster.regionServerStartup(serverInfo);
if (LOG.isDebugEnabled()) {
LOG.debug("Done telling master we are up");
@ -626,6 +633,10 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
}
try {
serverInfo.setLoad(new HServerLoad(requestCount.get(),
onlineRegions.size()));
requestCount.set(0);
HMsg msgs[] =
hbaseMaster.regionServerReport(serverInfo, outboundArray);
lastMsg = System.currentTimeMillis();
@ -897,6 +908,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
this.lock.writeLock().lock();
try {
this.log.setSequenceNumber(region.getMaxSequenceId());
this.onlineRegions.put(region.getRegionName(), region);
} finally {
this.lock.writeLock().unlock();
@ -963,6 +975,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
*/
public HRegionInfo getRegionInfo(final Text regionName)
throws NotServingRegionException {
requestCount.incrementAndGet();
return getRegion(regionName).getRegionInfo();
}
@ -971,6 +984,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
*/
public void batchUpdate(Text regionName, long timestamp, BatchUpdate b)
throws IOException {
requestCount.incrementAndGet();
long clientid = rand.nextLong();
long lockid = startUpdate(regionName, clientid, b.getRow());
for(BatchOperation op: b) {
@ -993,6 +1007,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
public byte [] get(final Text regionName, final Text row,
final Text column)
throws IOException {
requestCount.incrementAndGet();
return getRegion(regionName).get(row, column);
}
@ -1002,6 +1017,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
public byte [][] get(final Text regionName, final Text row,
final Text column, final int numVersions)
throws IOException {
requestCount.incrementAndGet();
return getRegion(regionName).get(row, column, numVersions);
}
@ -1010,6 +1026,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
*/
public byte [][] get(final Text regionName, final Text row, final Text column,
final long timestamp, final int numVersions) throws IOException {
requestCount.incrementAndGet();
return getRegion(regionName).get(row, column, timestamp, numVersions);
}
@ -1018,6 +1035,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
*/
public KeyedData[] getRow(final Text regionName, final Text row)
throws IOException {
requestCount.incrementAndGet();
HRegion region = getRegion(regionName);
TreeMap<Text, byte[]> map = region.getFull(row);
KeyedData result[] = new KeyedData[map.size()];
@ -1034,6 +1052,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
*/
public KeyedData[] next(final long scannerId)
throws IOException {
requestCount.incrementAndGet();
String scannerName = String.valueOf(scannerId);
HInternalScannerInterface s = scanners.get(scannerName);
if (s == null) {
@ -1077,6 +1096,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
*/
public long startUpdate(Text regionName, long clientid, Text row)
throws IOException {
requestCount.incrementAndGet();
HRegion region = getRegion(regionName);
long lockid = region.startUpdate(row);
this.leases.createLease(clientid, lockid,
@ -1120,6 +1140,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
public void put(final Text regionName, final long clientid,
final long lockid, final Text column, final byte [] val)
throws IOException {
requestCount.incrementAndGet();
HRegion region = getRegion(regionName, true);
leases.renewLease(clientid, lockid);
region.put(lockid, column, val);
@ -1130,6 +1151,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
*/
public void delete(Text regionName, long clientid, long lockid, Text column)
throws IOException {
requestCount.incrementAndGet();
HRegion region = getRegion(regionName);
leases.renewLease(clientid, lockid);
region.delete(lockid, column);
@ -1140,6 +1162,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
*/
public void abort(Text regionName, long clientid, long lockid)
throws IOException {
requestCount.incrementAndGet();
HRegion region = getRegion(regionName, true);
leases.cancelLease(clientid, lockid);
region.abort(lockid);
@ -1150,6 +1173,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
*/
public void commit(Text regionName, final long clientid, final long lockid,
final long timestamp) throws IOException {
requestCount.incrementAndGet();
HRegion region = getRegion(regionName, true);
leases.cancelLease(clientid, lockid);
region.commit(lockid, timestamp);
@ -1159,6 +1183,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
* {@inheritDoc}
*/
public void renewLease(long lockid, long clientid) throws IOException {
requestCount.incrementAndGet();
leases.renewLease(clientid, lockid);
}
@ -1247,6 +1272,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
public long openScanner(Text regionName, Text[] cols, Text firstRow,
final long timestamp, final RowFilterInterface filter)
throws IOException {
requestCount.incrementAndGet();
HRegion r = getRegion(regionName);
long scannerId = -1L;
try {
@ -1277,6 +1303,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
* {@inheritDoc}
*/
public void close(final long scannerId) throws IOException {
requestCount.incrementAndGet();
String scannerName = String.valueOf(scannerId);
HInternalScannerInterface s = null;
synchronized(scanners) {

View File

@ -33,21 +33,24 @@ import java.io.*;
public class HServerInfo implements Writable {
private HServerAddress serverAddress;
private long startCode;
private HServerLoad load;
/** default constructor - used by Writable */
public HServerInfo() {
this.serverAddress = new HServerAddress();
this.startCode = 0;
this.load = new HServerLoad();
}
/**
* Constructs a fully initialized object
* Constructor
* @param serverAddress
* @param startCode
*/
public HServerInfo(HServerAddress serverAddress, long startCode) {
this.serverAddress = new HServerAddress(serverAddress);
this.startCode = startCode;
this.load = new HServerLoad();
}
/**
@ -57,6 +60,21 @@ public class HServerInfo implements Writable {
public HServerInfo(HServerInfo other) {
this.serverAddress = new HServerAddress(other.getServerAddress());
this.startCode = other.getStartCode();
this.load = other.getLoad();
}
/**
* @return the load
*/
public HServerLoad getLoad() {
return load;
}
/**
* @param load the load to set
*/
public void setLoad(HServerLoad load) {
this.load = load;
}
/** @return the server address */
@ -72,7 +90,8 @@ public class HServerInfo implements Writable {
/** {@inheritDoc} */
@Override
public String toString() {
return "address: " + this.serverAddress + ", startcode: " + this.startCode;
return "address: " + this.serverAddress + ", startcode: " + this.startCode
+ ", load: (" + this.load.toString() + ")";
}
// Writable
@ -81,11 +100,13 @@ public class HServerInfo implements Writable {
public void readFields(DataInput in) throws IOException {
this.serverAddress.readFields(in);
this.startCode = in.readLong();
this.load.readFields(in);
}
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
this.serverAddress.write(out);
out.writeLong(this.startCode);
this.load.write(out);
}
}

View File

@ -0,0 +1,136 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.io.WritableComparable;
/**
* This class encapsulates metrics for determining the load on a HRegionServer
*/
public class HServerLoad implements WritableComparable {
private int numberOfRequests; // number of requests since last report
private int numberOfRegions; // number of regions being served
/*
* TODO: Other metrics that might be considered when the master is actually
* doing load balancing instead of merely trying to decide where to assign
* a region:
* <ul>
* <li># of CPUs, heap size (to determine the "class" of machine). For
* now, we consider them to be homogeneous.</li>
* <li>#requests per region (Map<{String|HRegionInfo}, Integer>)</li>
* <li>#compactions and/or #splits (churn)</li>
* <li>server death rate (maybe there is something wrong with this server)</li>
* </ul>
*/
/** default constructior (used by Writable) */
public HServerLoad() {}
/**
* Constructor
* @param numberOfRequests
* @param numberOfRegions
*/
public HServerLoad(int numberOfRequests, int numberOfRegions) {
this.numberOfRequests = numberOfRequests;
this.numberOfRegions = numberOfRegions;
}
/**
* @return load factor for this server
*/
public int getLoad() {
int load = numberOfRequests == 0 ? 1 : numberOfRequests;
load *= numberOfRegions == 0 ? 1 : numberOfRegions;
return load;
}
/** {@inheritDoc} */
@Override
public String toString() {
return "requests: " + numberOfRequests + " regions: " + numberOfRegions;
}
/** {@inheritDoc} */
@Override
public boolean equals(Object o) {
return compareTo(o) == 0;
}
/** {@inheritDoc} */
@Override
public int hashCode() {
int result = Integer.valueOf(numberOfRequests).hashCode();
result ^= Integer.valueOf(numberOfRegions).hashCode();
return result;
}
// Getters
/**
* @return the numberOfRegions
*/
public int getNumberOfRegions() {
return numberOfRegions;
}
/**
* @return the numberOfRequests
*/
public int getNumberOfRequests() {
return numberOfRequests;
}
// Setters
/**
* @param numberOfRegions the numberOfRegions to set
*/
public void setNumberOfRegions(int numberOfRegions) {
this.numberOfRegions = numberOfRegions;
}
// Writable
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
numberOfRequests = in.readInt();
numberOfRegions = in.readInt();
}
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
out.writeInt(numberOfRequests);
out.writeInt(numberOfRegions);
}
// Comparable
/** {@inheritDoc} */
public int compareTo(Object o) {
HServerLoad other = (HServerLoad) o;
return this.getLoad() - other.getLoad();
}
}

View File

@ -90,6 +90,8 @@ class HStore implements HConstants {
TreeMap<Long, MapFile.Reader> readers = new TreeMap<Long, MapFile.Reader>();
Random rand = new Random();
private long maxSeqId;
/**
* An HStore is a set of zero or more MapFiles, which stretch backwards over
@ -196,6 +198,7 @@ class HStore implements HConstants {
// If the HSTORE_LOGINFOFILE doesn't contain a number, just ignore it. That
// means it was built prior to the previous run of HStore, and so it cannot
// contain any updates also contained in the log.
long maxSeqID = -1;
for (HStoreFile hsf: hstoreFiles) {
long seqid = hsf.loadInfo(fs);
@ -205,8 +208,14 @@ class HStore implements HConstants {
}
}
}
doReconstructionLog(reconstructionLog, maxSeqID);
this.maxSeqId = maxSeqID;
if (LOG.isDebugEnabled()) {
LOG.debug("maximum sequence id for hstore " + storeName + " is " +
this.maxSeqId);
}
doReconstructionLog(reconstructionLog, maxSeqId);
this.maxSeqId += 1;
// Compact all the MapFiles into a single file. The resulting MapFile
// should be "timeless"; that is, it should not have an associated seq-ID,
@ -228,6 +237,10 @@ class HStore implements HConstants {
}
}
long getMaxSequenceId() {
return this.maxSeqId;
}
/*
* Read the reconstructionLog to see whether we need to build a brand-new
* MapFile out of non-flushed log entries.
@ -258,6 +271,11 @@ class HStore implements HConstants {
while (login.next(key, val)) {
maxSeqIdInLog = Math.max(maxSeqIdInLog, key.getLogSeqNum());
if (key.getLogSeqNum() <= maxSeqID) {
if (LOG.isDebugEnabled()) {
LOG.debug("Skipping edit <" + key.toString() + "=" +
val.toString() + "> key sequence: " + key.getLogSeqNum() +
" max sequence: " + maxSeqID);
}
continue;
}
// Check this edit is for me. Also, guard against writing
@ -277,7 +295,8 @@ class HStore implements HConstants {
}
HStoreKey k = new HStoreKey(key.getRow(), column, val.getTimestamp());
if (LOG.isDebugEnabled()) {
LOG.debug("Applying edit " + k.toString());
LOG.debug("Applying edit <" + k.toString() + "=" + val.toString() +
">");
}
reconstructedCache.put(k, val.getVal());
}
@ -428,16 +447,12 @@ class HStore implements HConstants {
String name = flushedFile.toString();
MapFile.Writer out = flushedFile.getWriter(this.fs, this.compression,
this.bloomFilter);
int count = 0;
int total = 0;
try {
for (Map.Entry<HStoreKey, byte []> es: inputCache.entrySet()) {
HStoreKey curkey = es.getKey();
total++;
if (this.familyName.
equals(HStoreKey.extractFamily(curkey.getColumn()))) {
out.append(curkey, new ImmutableBytesWritable(es.getValue()));
count++;
}
}
} finally {
@ -1030,6 +1045,7 @@ class HStore implements HConstants {
//////////////////////////////////////////////////////////////////////////////
class HStoreScanner extends HAbstractScanner {
@SuppressWarnings("hiding")
private MapFile.Reader[] readers;
HStoreScanner(long timestamp, Text[] targetCols, Text firstRow)

View File

@ -94,7 +94,17 @@ public class HStoreFile implements HConstants, WritableComparable {
static final String HSTORE_DATFILE_DIR = "mapfiles";
static final String HSTORE_INFO_DIR = "info";
static final String HSTORE_FILTER_DIR = "filter";
public static enum Range {top, bottom}
/**
* For split HStoreFiles, specifies if the file covers the lower half or
* the upper half of the key range
*/
public static enum Range {
/** HStoreFile contains upper half of key range */
top,
/** HStoreFile contains lower half of key range */
bottom
}
/*
* Regex that will work for straight filenames and for reference names.
@ -156,7 +166,7 @@ public class HStoreFile implements HConstants, WritableComparable {
/*
* Data structure to hold reference to a store file over in another region.
*/
static class Reference {
static class Reference implements Writable {
Text regionName;
long fileid;
Range region;
@ -190,11 +200,15 @@ public class HStoreFile implements HConstants, WritableComparable {
return this.regionName;
}
/** {@inheritDoc} */
@Override
public String toString() {
return this.regionName + "/" + this.fileid + "/" + this.region;
}
// Make it serializable.
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
this.regionName.write(out);
out.writeLong(this.fileid);
@ -203,6 +217,7 @@ public class HStoreFile implements HConstants, WritableComparable {
this.midkey.write(out);
}
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
this.regionName = new Text();
this.regionName.readFields(in);
@ -417,6 +432,8 @@ public class HStoreFile implements HConstants, WritableComparable {
private static boolean isReference(final Path p, final Matcher m) {
if (m == null || !m.matches()) {
LOG.warn("Failed match of store file name " + p.toString());
throw new RuntimeException("Failed match of store file name " +
p.toString());
}
return m.groupCount() > 1 && m.group(2) != null;
}
@ -662,6 +679,7 @@ public class HStoreFile implements HConstants, WritableComparable {
}
}
/** {@inheritDoc} */
@SuppressWarnings({ "unused"})
@Override
public synchronized void finalKey(WritableComparable key)
@ -669,6 +687,7 @@ public class HStoreFile implements HConstants, WritableComparable {
throw new UnsupportedOperationException("Unsupported");
}
/** {@inheritDoc} */
@Override
public synchronized Writable get(WritableComparable key, Writable val)
throws IOException {
@ -676,6 +695,7 @@ public class HStoreFile implements HConstants, WritableComparable {
return super.get(key, val);
}
/** {@inheritDoc} */
@SuppressWarnings("unchecked")
@Override
public synchronized WritableComparable getClosest(WritableComparable key,
@ -692,6 +712,7 @@ public class HStoreFile implements HConstants, WritableComparable {
return super.getClosest(key, val);
}
/** {@inheritDoc} */
@SuppressWarnings("unused")
@Override
public synchronized WritableComparable midKey() throws IOException {
@ -699,6 +720,7 @@ public class HStoreFile implements HConstants, WritableComparable {
return null;
}
/** {@inheritDoc} */
@SuppressWarnings("unchecked")
@Override
public synchronized boolean next(WritableComparable key, Writable val)
@ -727,6 +749,7 @@ public class HStoreFile implements HConstants, WritableComparable {
return false;
}
/** {@inheritDoc} */
@Override
public synchronized void reset() throws IOException {
if (top) {
@ -737,6 +760,7 @@ public class HStoreFile implements HConstants, WritableComparable {
super.reset();
}
/** {@inheritDoc} */
@Override
public synchronized boolean seek(WritableComparable key)
throws IOException {
@ -758,6 +782,15 @@ public class HStoreFile implements HConstants, WritableComparable {
static class Reader extends MapFile.Reader {
private final Filter bloomFilter;
/**
* Constructor
*
* @param fs
* @param dirName
* @param conf
* @param filter
* @throws IOException
*/
public Reader(FileSystem fs, String dirName, Configuration conf,
final Filter filter)
throws IOException {
@ -810,6 +843,18 @@ public class HStoreFile implements HConstants, WritableComparable {
private final Filter bloomFilter;
/**
* Constructor
*
* @param conf
* @param fs
* @param dirName
* @param keyClass
* @param valClass
* @param compression
* @param filter
* @throws IOException
*/
@SuppressWarnings("unchecked")
public Writer(Configuration conf, FileSystem fs, String dirName,
Class keyClass, Class valClass,
@ -905,6 +950,7 @@ public class HStoreFile implements HConstants, WritableComparable {
return (isReference())? l / 2: l;
}
/** {@inheritDoc} */
@Override
public String toString() {
return this.regionName.toString() + "/" + this.colFamily.toString() +
@ -912,6 +958,7 @@ public class HStoreFile implements HConstants, WritableComparable {
(isReference()? "/" + this.reference.toString(): "");
}
/** {@inheritDoc} */
@Override
public boolean equals(Object o) {
return this.compareTo(o) == 0;

View File

@ -133,6 +133,7 @@ public class HTable implements HConstants {
}
/** @return the table name */
public Text getTableName() {
return this.tableName;
}

View File

@ -41,13 +41,13 @@ public class HTableDescriptor implements WritableComparable {
/*
* Legal table names can only contain 'word characters':
* i.e. <code>[a-zA-Z_0-9]</code>.
* i.e. <code>[a-zA-Z_0-9-.]</code>.
* Lets be restrictive until a reason to be otherwise. One reason to limit
* characters in table name is to ensure table regions as entries in META
* regions can be found (See HADOOP-1581 'HBASE: Un-openable tablename bug').
*/
private static final Pattern LEGAL_TABLE_NAME =
Pattern.compile("[\\w-]+");
Pattern.compile("^[\\w-.]+$");
/** Constructs an empty object */
public HTableDescriptor() {

View File

@ -143,33 +143,43 @@ public class MiniHBaseCluster implements HConstants {
}
}
/** runs the master server */
public static class MasterThread extends Thread {
private final HMaster master;
MasterThread(final HMaster m) {
super(m, "Master:" + m.getMasterAddress().toString());
this.master = m;
}
/** {@inheritDoc} */
@Override
public void run() {
LOG.info("Starting " + getName());
super.run();
}
/** @return master server */
public HMaster getMaster() {
return this.master;
}
}
/** runs region servers */
public static class RegionServerThread extends Thread {
private final HRegionServer regionServer;
RegionServerThread(final HRegionServer r, final int index) {
super(r, "RegionServer:" + index);
this.regionServer = r;
}
/** {@inheritDoc} */
@Override
public void run() {
LOG.info("Starting " + getName());
super.run();
}
/** @return the region server */
public HRegionServer getRegionServer() {
return this.regionServer;
}
@ -227,6 +237,11 @@ public class MiniHBaseCluster implements HConstants {
return threads;
}
/**
* Starts a region server thread running
*
* @throws IOException
*/
public void startRegionServer() throws IOException {
RegionServerThread t =
startRegionServer(this.conf, this.regionThreads.size());
@ -275,6 +290,7 @@ public class MiniHBaseCluster implements HConstants {
* Shut down the specified region server cleanly
*
* @param serverNumber
* @return the region server that was stopped
*/
public HRegionServer stopRegionServer(int serverNumber) {
HRegionServer server =

View File

@ -34,10 +34,10 @@ public class TestCleanRegionServerExit extends HBaseClusterTestCase {
/** constructor */
public TestCleanRegionServerExit() {
super();
super(2);
conf.setInt("ipc.client.timeout", 5000); // reduce ipc client timeout
conf.setInt("ipc.client.connect.max.retries", 5); // and number of retries
conf.setInt("hbase.client.retries.number", 2); // reduce HBase retries
conf.setInt("hbase.client.retries.number", 3); // reduce HBase retries
Logger.getRootLogger().setLevel(Level.WARN);
Logger.getLogger(this.getClass().getPackage().getName()).setLevel(Level.DEBUG);
}

View File

@ -40,7 +40,7 @@ import org.apache.log4j.Logger;
* HRegions or in the HBaseMaster, so only basic testing is possible.
*/
public class TestHRegion extends HBaseTestCase implements RegionUnavailableListener {
private static final Logger LOG =
static final Logger LOG =
Logger.getLogger(TestHRegion.class.getName());
/** Constructor */

View File

@ -34,6 +34,9 @@ import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.WritableComparable;
/**
* Test HStoreFile
*/
public class TestHStoreFile extends TestCase {
static final Log LOG = LogFactory.getLog(TestHStoreFile.class);
private static String DIR = System.getProperty("test.build.data", ".");
@ -226,7 +229,9 @@ public class TestHStoreFile extends TestCase {
}
assertTrue(key.compareTo(midkey) < 0);
}
LOG.info("Last in bottom: " + previous.toString());
if (previous != null) {
LOG.info("Last in bottom: " + previous.toString());
}
// Now test reading from the top.
top = new HStoreFile.HalfMapFileReader(this.fs, p.toString(), this.conf,
HStoreFile.Range.top, midkey);
@ -249,17 +254,17 @@ public class TestHStoreFile extends TestCase {
// Next test using a midkey that does not exist in the file.
// First, do a key that is < than first key. Ensure splits behave
// properly.
midkey = new HStoreKey(new Text(" "));
WritableComparable badkey = new HStoreKey(new Text(" "));
bottom = new HStoreFile.HalfMapFileReader(this.fs, p.toString(),
this.conf, HStoreFile.Range.bottom, midkey);
// When midkey is < than the bottom, should return no values.
this.conf, HStoreFile.Range.bottom, badkey);
// When badkey is < than the bottom, should return no values.
assertFalse(bottom.next(key, value));
// Now read from the top.
top = new HStoreFile.HalfMapFileReader(this.fs, p.toString(), this.conf,
HStoreFile.Range.top, midkey);
HStoreFile.Range.top, badkey);
first = true;
while (top.next(key, value)) {
assertTrue(key.compareTo(midkey) >= 0);
assertTrue(key.compareTo(badkey) >= 0);
if (first) {
first = false;
LOG.info("First top when key < bottom: " + key.toString());
@ -275,10 +280,10 @@ public class TestHStoreFile extends TestCase {
assertTrue(tmp.charAt(i) == 'z');
}
// Test when midkey is > than last key in file ('||' > 'zz').
midkey = new HStoreKey(new Text("|||"));
// Test when badkey is > than last key in file ('||' > 'zz').
badkey = new HStoreKey(new Text("|||"));
bottom = new HStoreFile.HalfMapFileReader(this.fs, p.toString(),
this.conf, HStoreFile.Range.bottom, midkey);
this.conf, HStoreFile.Range.bottom, badkey);
first = true;
while (bottom.next(key, value)) {
if (first) {
@ -297,7 +302,7 @@ public class TestHStoreFile extends TestCase {
}
// Now look at top. Should not return any values.
top = new HStoreFile.HalfMapFileReader(this.fs, p.toString(), this.conf,
HStoreFile.Range.top, midkey);
HStoreFile.Range.top, badkey);
assertFalse(top.next(key, value));
} finally {

View File

@ -34,10 +34,10 @@ public class TestRegionServerAbort extends HBaseClusterTestCase {
/** constructor */
public TestRegionServerAbort() {
super();
super(2);
conf.setInt("ipc.client.timeout", 5000); // reduce client timeout
conf.setInt("ipc.client.connect.max.retries", 5); // and number of retries
conf.setInt("hbase.client.retries.number", 2); // reduce HBase retries
conf.setInt("hbase.client.retries.number", 3); // reduce HBase retries
Logger.getRootLogger().setLevel(Level.WARN);
Logger.getLogger(this.getClass().getPackage().getName()).setLevel(Level.DEBUG);
}

View File

@ -206,7 +206,8 @@ public class TestScanner2 extends HBaseClusterTestCase {
HRegionLocation rl = t.getRegionLocation(table);
regionServer = t.getConnection().getHRegionConnection(rl.getServerAddress());
scannerId = regionServer.openScanner(rl.getRegionInfo().getRegionName(),
HMaster.METACOLUMNS, new Text(), System.currentTimeMillis(), null);
HConstants.COLUMN_FAMILY_ARRAY, new Text(),
System.currentTimeMillis(), null);
while (true) {
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
KeyedData[] values = regionServer.next(scannerId);

View File

@ -46,6 +46,7 @@ public class TestSplit extends HBaseTestCase {
private static final char FIRST_CHAR = 'a';
private static final char LAST_CHAR = 'z';
/** {@inheritDoc} */
@Override
public void setUp() throws Exception {
super.setUp();
@ -59,6 +60,7 @@ public class TestSplit extends HBaseTestCase {
conf.setLong("hbase.hregion.max.filesize", 1024 * 128);
}
/** {@inheritDoc} */
@Override
public void tearDown() throws Exception {
try {

View File

@ -35,8 +35,8 @@ public class TestToString extends TestCase {
HServerAddress address = new HServerAddress(hostport);
assertEquals("HServerAddress toString", address.toString(), hostport);
HServerInfo info = new HServerInfo(address, -1);
assertEquals("HServerInfo", info.toString(),
"address: " + hostport + ", startcode: " + -1);
assertEquals("HServerInfo", "address: " + hostport + ", startcode: -1" +
", load: (requests: 0 regions: 0)", info.toString());
}
/**