HADOOP-1325. First complete, functioning version of HBase. Contributed by Jim Kellerman.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@535970 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
ec9a568764
commit
add0c0c5a0
|
@ -0,0 +1,207 @@
|
|||
#! /bin/sh
|
||||
#
|
||||
# The hbase command script. Based on the hadoop command script putting
|
||||
# in hbase classes, libs and configurations ahead of hadoop's.
|
||||
#
|
||||
# TODO: Narrow the amount of duplicated code.
|
||||
#
|
||||
# Environment Variables:
|
||||
#
|
||||
# JAVA_HOME The java implementation to use. Overrides JAVA_HOME.
|
||||
#
|
||||
# HBASE_HEAPSIZE The maximum amount of heap to use, in MB.
|
||||
# Default is 1000.
|
||||
#
|
||||
# HBASE_OPTS Extra Java runtime options.
|
||||
#
|
||||
# HBASE_CONF_DIR Alternate conf dir. Default is ${HBASE_HOME}/conf.
|
||||
#
|
||||
# HADOOP_CONF_DIR Alternate conf dir. Default is ${HADOOP_HOME}/conf.
|
||||
#
|
||||
# HADOOP_HOME Hadoop home directory.
|
||||
#
|
||||
# HADOOP_ROOT_LOGGER The root appender. Default is INFO,console
|
||||
#
|
||||
|
||||
bin=`dirname "$0"`
|
||||
bin=`cd "$bin"; pwd`
|
||||
|
||||
# This will set HBASE_HOME, HADOOP_HOME, etc.
|
||||
. "$bin"/hbase-config.sh
|
||||
|
||||
cygwin=false
|
||||
case "`uname`" in
|
||||
CYGWIN*) cygwin=true;;
|
||||
esac
|
||||
|
||||
# if no args specified, show usage
|
||||
if [ $# = 0 ]; then
|
||||
echo "Usage: hbase [--hadoop=hadoopdir] <command>"
|
||||
echo "where <command> is one of:"
|
||||
echo " client run a hbase client"
|
||||
echo " reader run a hbase region directory reader"
|
||||
echo " logreader output content of a logfile"
|
||||
echo " master run a hbase HMaster node"
|
||||
echo " regionserver run a hbase HRegionServer node"
|
||||
echo " or"
|
||||
echo " CLASSNAME run the class named CLASSNAME"
|
||||
echo "Most commands print help when invoked w/o parameters."
|
||||
exit 1
|
||||
fi
|
||||
|
||||
# get arguments
|
||||
COMMAND=$1
|
||||
shift
|
||||
|
||||
# Source the hadoop-env.sh. Will have JAVA_HOME defined. There is no
|
||||
# hbase-env.sh as yet.
|
||||
if [ -f "${HADOOP_CONF_DIR}/hadoop-env.sh" ]; then
|
||||
. "${HADOOP_CONF_DIR}/hadoop-env.sh"
|
||||
fi
|
||||
|
||||
# some Java parameters
|
||||
if [ "$JAVA_HOME" != "" ]; then
|
||||
#echo "run java in $JAVA_HOME"
|
||||
JAVA_HOME=$JAVA_HOME
|
||||
fi
|
||||
|
||||
if [ "$JAVA_HOME" = "" ]; then
|
||||
echo "Error: JAVA_HOME is not set."
|
||||
exit 1
|
||||
fi
|
||||
|
||||
JAVA=$JAVA_HOME/bin/java
|
||||
JAVA_HEAP_MAX=-Xmx1000m
|
||||
|
||||
# check envvars which might override default args
|
||||
if [ "$HBASE_HEAPSIZE" != "" ]; then
|
||||
#echo "run with heapsize $HBASE_HEAPSIZE"
|
||||
JAVA_HEAP_MAX="-Xmx""$HBASE_HEAPSIZE""m"
|
||||
#echo $JAVA_HEAP_MAX
|
||||
fi
|
||||
|
||||
# CLASSPATH initially contains $HBASE_CONF_DIR
|
||||
# Add HADOOP_CONF_DIR if its been defined.
|
||||
CLASSPATH="${HBASE_CONF_DIR}"
|
||||
if [ ! "$HADOOP_CONF_DIR" = "" ]; then
|
||||
CLASSPATH="${CLASSPATH}:${HADOOP_CONF_DIR}"
|
||||
fi
|
||||
CLASSPATH=${CLASSPATH}:$JAVA_HOME/lib/tools.jar
|
||||
|
||||
# for developers, add hbase and hadoop classes to CLASSPATH
|
||||
if [ -d "$HADOOP_HOME/build/contrib/hbase/classes" ]; then
|
||||
CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/contrib/hbase/classes
|
||||
fi
|
||||
if [ -d "$HADOOP_HOME/build/classes" ]; then
|
||||
CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/classes
|
||||
fi
|
||||
if [ -d "$HADOOP_HOME/build/webapps" ]; then
|
||||
CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build
|
||||
fi
|
||||
if [ -d "$HADOOP_HOME/build/test/classes" ]; then
|
||||
CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/test/classes
|
||||
fi
|
||||
|
||||
# so that filenames w/ spaces are handled correctly in loops below
|
||||
IFS=
|
||||
|
||||
# for releases, add core hbase, hadoop jar & webapps to CLASSPATH
|
||||
for f in "$HBASE_HOME/hadoop-hbase-*.jar"; do
|
||||
CLASSPATH=${CLASSPATH}:$f;
|
||||
done
|
||||
for f in "$HADOOP_HOME/build/contrib/hbase/hadoop-hbase-*.jar"; do
|
||||
CLASSPATH=${CLASSPATH}:$f;
|
||||
done
|
||||
if [ -d "$HADOOP_HOME/webapps" ]; then
|
||||
CLASSPATH=${CLASSPATH}:$HADOOP_HOME
|
||||
fi
|
||||
for f in $HADOOP_HOME/hadoop-*-core.jar; do
|
||||
CLASSPATH=${CLASSPATH}:$f;
|
||||
done
|
||||
|
||||
# add hbase and hadoop libs to CLASSPATH
|
||||
for f in $HBASE_HOME/lib/*.jar; do
|
||||
CLASSPATH=${CLASSPATH}:$f;
|
||||
done
|
||||
for f in $HADOOP_HOME/lib/*.jar; do
|
||||
CLASSPATH=${CLASSPATH}:$f;
|
||||
done
|
||||
|
||||
for f in $HADOOP_HOME/lib/jetty-ext/*.jar; do
|
||||
CLASSPATH=${CLASSPATH}:$f;
|
||||
done
|
||||
|
||||
|
||||
# default log directory & file
|
||||
# TODO: Should we log to hadoop or under hbase?
|
||||
if [ "$HADOOP_LOG_DIR" = "" ]; then
|
||||
HADOOP_LOG_DIR="$HADOOP_HOME/logs"
|
||||
fi
|
||||
if [ "$HADOOP_LOGFILE" = "" ]; then
|
||||
HADOOP_LOGFILE='hbase.log'
|
||||
fi
|
||||
|
||||
# cygwin path translation
|
||||
if $cygwin; then
|
||||
CLASSPATH=`cygpath -p -w "$CLASSPATH"`
|
||||
HADOOP_HOME=`cygpath -d "$HADOOP_HOME"`
|
||||
HBASE_HOME=`cygpath -d "$HBASE_HOME"`
|
||||
HADOOP_LOG_DIR=`cygpath -d "$HADOOP_LOG_DIR"`
|
||||
fi
|
||||
|
||||
# TODO: Can this be put into separate script so don't have to duplicate
|
||||
# hadoop command script code?
|
||||
# setup 'java.library.path' for native-hadoop code if necessary
|
||||
JAVA_LIBRARY_PATH=''
|
||||
if [ -d "${HADOOP_HOME}/build/native" -o -d "${HADOOP_HOME}/lib/native" ]; then
|
||||
JAVA_PLATFORM=`CLASSPATH=${CLASSPATH} ${JAVA} org.apache.hadoop.util.PlatformName | sed -e "s/ /_/g"`
|
||||
|
||||
if [ -d "$HADOOP_HOME/build/native" ]; then
|
||||
JAVA_LIBRARY_PATH=${HADOOP_HOME}/build/native/${JAVA_PLATFORM}/lib
|
||||
fi
|
||||
|
||||
if [ -d "${HADOOP_HOME}/lib/native" ]; then
|
||||
if [ "x$JAVA_LIBRARY_PATH" != "x" ]; then
|
||||
JAVA_LIBRARY_PATH=${JAVA_LIBRARY_PATH}:${HADOOP_HOME}/lib/native/${JAVA_PLATFORM}
|
||||
else
|
||||
JAVA_LIBRARY_PATH=${HADOOP_HOME}/lib/native/${JAVA_PLATFORM}
|
||||
fi
|
||||
fi
|
||||
fi
|
||||
|
||||
# cygwin path translation
|
||||
if $cygwin; then
|
||||
JAVA_LIBRARY_PATH=`cygpath -p "$JAVA_LIBRARY_PATH"`
|
||||
fi
|
||||
|
||||
# restore ordinary behaviour
|
||||
unset IFS
|
||||
|
||||
# figure out which class to run
|
||||
if [ "$COMMAND" = "client" ] ; then
|
||||
CLASS='org.apache.hadoop.hbase.HClient'
|
||||
elif [ "$COMMAND" = "reader" ] ; then
|
||||
CLASS='org.apache.hadoop.hbase.HRegiondirReader'
|
||||
elif [ "$COMMAND" = "logreader" ] ; then
|
||||
CLASS='org.apache.hadoop.hbase.HLog'
|
||||
elif [ "$COMMAND" = "master" ] ; then
|
||||
CLASS='org.apache.hadoop.hbase.HMaster'
|
||||
elif [ "$COMMAND" = "regionserver" ] ; then
|
||||
CLASS='org.apache.hadoop.hbase.HRegionServer'
|
||||
else
|
||||
CLASS=$COMMAND
|
||||
fi
|
||||
|
||||
|
||||
HBASE_OPTS="$HBASE_OPTS -Dhadoop.log.dir=$HADOOP_LOG_DIR"
|
||||
HBASE_OPTS="$HBASE_OPTS -Dhadoop.log.file=$HADOOP_LOGFILE"
|
||||
HBASE_OPTS="$HBASE_OPTS -Dhadoop.home.dir=$HADOOP_HOME"
|
||||
HBASE_OPTS="$HBASE_OPTS -Dhadoop.id.str=$HADOOP_IDENT_STRING"
|
||||
HBASE_OPTS="$HBASE_OPTS -Dhadoop.root.logger=${HADOOP_ROOT_LOGGER:-INFO,console}"
|
||||
HBASE_OPTS="$HBASE_OPTS -Dhbase.home.dir=$HBASE_HOME"
|
||||
if [ "x$JAVA_LIBRARY_PATH" != "x" ]; then
|
||||
HBASE_OPTS="$HBASE_OPTS -Djava.library.path=$JAVA_LIBRARY_PATH"
|
||||
fi
|
||||
|
||||
# run it
|
||||
exec "$JAVA" $JAVA_HEAP_MAX $HBASE_OPTS -classpath "$CLASSPATH" $CLASS "$@"
|
|
@ -0,0 +1,62 @@
|
|||
# included in all the hbase scripts with source command
|
||||
# should not be executable directly
|
||||
# also should not be passed any arguments, since we need original $*
|
||||
|
||||
# resolve links - $0 may be a softlink
|
||||
|
||||
this="$0"
|
||||
while [ -h "$this" ]; do
|
||||
ls=`ls -ld "$this"`
|
||||
link=`expr "$ls" : '.*-> \(.*\)$'`
|
||||
if expr "$link" : '.*/.*' > /dev/null; then
|
||||
this="$link"
|
||||
else
|
||||
this=`dirname "$this"`/"$link"
|
||||
fi
|
||||
done
|
||||
|
||||
# convert relative path to absolute path
|
||||
bin=`dirname "$this"`
|
||||
script=`basename "$this"`
|
||||
bin=`cd "$bin"; pwd`
|
||||
this="$bin/$script"
|
||||
|
||||
# the root of the hbase installation
|
||||
export HBASE_HOME=`dirname "$this"`/..
|
||||
|
||||
#check to see if the conf dir or hadoop home are given as an optional arguments
|
||||
while [ $# -gt 1 ]
|
||||
do
|
||||
case $1 in
|
||||
--config=*)
|
||||
HADOOP_CONF_DIR=`echo $1|sed 's/[^=]*=\(.*\)/\1/'`
|
||||
shift
|
||||
;;
|
||||
--hbaseconfig=*)
|
||||
HBASE_CONF_DIR=`echo $1|sed 's/[^=]*=\(.*\)/\1/'`
|
||||
shift
|
||||
;;
|
||||
|
||||
--hadoop=*)
|
||||
HADOOP_HOME=`echo $1|sed 's/[^=]*=\(.*\)/\1/'`
|
||||
shift
|
||||
;;
|
||||
--hosts=*)
|
||||
regionservers=`echo $1|sed 's/[^=]*=\(.*\)/\1/'`
|
||||
shift
|
||||
;;
|
||||
|
||||
*)
|
||||
break
|
||||
;;
|
||||
esac
|
||||
done
|
||||
|
||||
# If no hadoop home specified, then we assume its above this directory.
|
||||
HADOOP_HOME="${HADOOP_HOME:-$HBASE_HOME/../../../}"
|
||||
# Allow alternate hadoop conf dir location.
|
||||
HADOOP_CONF_DIR="${HADOOP_CONF_DIR:-$HADOOP_HOME/conf}"
|
||||
# Allow alternate hbase conf dir location.
|
||||
HBASE_CONF_DIR="${HBASE_CONF_DIR:-$HBASE_HOME/conf}"
|
||||
# List of hbase regions servers.
|
||||
HBASE_REGIONSERVERS="${HBASE_REGIONSERVERS:-$HBASE_HOME/conf/regionservers}"
|
|
@ -42,7 +42,43 @@
|
|||
<name>hbase.master.meta.thread.rescanfrequency</name>
|
||||
<value>60000</value>
|
||||
<description>How long the HMaster sleeps (in milliseconds) between scans of
|
||||
the META table.
|
||||
the root and meta tables.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.master.lease.period</name>
|
||||
<value>30000</value>
|
||||
<description>HMaster server lease period in milliseconds. Default is
|
||||
30 seconds.</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.server.thread.wakefrequency</name>
|
||||
<value>10000</value>
|
||||
<description>Time to sleep in between searches for work (in milliseconds).
|
||||
Used as sleep interval by service threads such as META scanner and log roller.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.regionserver.lease.period</name>
|
||||
<value>30000</value>
|
||||
<description>HRegion server lease period in milliseconds. Default is
|
||||
30 seconds.</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.regionserver.handler.count</name>
|
||||
<value>10</value>
|
||||
<description>Count of RPC Server instances spun up on RegionServers
|
||||
Same property is used by the HMaster for count of master handlers.
|
||||
Default is 10.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hbase.regionserver.msginterval</name>
|
||||
<value>15000</value>
|
||||
<description>Interval between messages from the RegionServer to HMaster
|
||||
in milliseconds. Default is 15. Set this value low if you want unit
|
||||
tests to be responsive.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
</configuration>
|
||||
|
|
|
@ -0,0 +1 @@
|
|||
localhost
|
|
@ -30,7 +30,7 @@ import org.apache.hadoop.io.Text;
|
|||
* Abstract base class that implements the HScannerInterface.
|
||||
* Used by the concrete HMemcacheScanner and HStoreScanners
|
||||
******************************************************************************/
|
||||
public abstract class HAbstractScanner implements HScannerInterface {
|
||||
public abstract class HAbstractScanner implements HInternalScannerInterface {
|
||||
|
||||
// Pattern to determine if a column key is a regex
|
||||
|
||||
|
@ -51,6 +51,7 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
// 3. Simple match: compare column family + column key literally
|
||||
|
||||
private class ColumnMatcher {
|
||||
private boolean wildCardmatch;
|
||||
private MATCH_TYPE matchType;
|
||||
private String family;
|
||||
private Pattern columnMatcher;
|
||||
|
@ -69,14 +70,17 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
if(columnkey == null || columnkey.length() == 0) {
|
||||
this.matchType = MATCH_TYPE.FAMILY_ONLY;
|
||||
this.family = column.substring(0, colpos);
|
||||
this.wildCardmatch = true;
|
||||
|
||||
} else if(isRegexPattern.matcher(columnkey).matches()) {
|
||||
this.matchType = MATCH_TYPE.REGEX;
|
||||
this.columnMatcher = Pattern.compile(column);
|
||||
this.wildCardmatch = true;
|
||||
|
||||
} else {
|
||||
this.matchType = MATCH_TYPE.SIMPLE;
|
||||
this.col = col;
|
||||
this.wildCardmatch = false;
|
||||
}
|
||||
} catch(Exception e) {
|
||||
throw new IOException("Column: " + column + ": " + e.getMessage());
|
||||
|
@ -99,8 +103,12 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
throw new IOException("Invalid match type: " + this.matchType);
|
||||
}
|
||||
}
|
||||
|
||||
boolean isWildCardMatch() {
|
||||
return this.wildCardmatch;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
protected TreeMap<Text, Vector<ColumnMatcher>> okCols; // Holds matchers for each column family
|
||||
|
||||
protected boolean scannerClosed = false; // True when scanning is done
|
||||
|
@ -109,14 +117,17 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
protected BytesWritable vals[]; // Values that correspond to those keys
|
||||
|
||||
protected long timestamp; // The timestamp to match entries against
|
||||
private boolean wildcardMatch;
|
||||
private boolean multipleMatchers;
|
||||
|
||||
protected DataOutputBuffer outbuf = new DataOutputBuffer();
|
||||
protected DataInputBuffer inbuf = new DataInputBuffer();
|
||||
|
||||
/** Constructor for abstract base class */
|
||||
HAbstractScanner(long timestamp, Text[] targetCols) throws IOException {
|
||||
|
||||
this.timestamp = timestamp;
|
||||
this.wildcardMatch = false;
|
||||
this.multipleMatchers = false;
|
||||
this.okCols = new TreeMap<Text, Vector<ColumnMatcher>>();
|
||||
for(int i = 0; i < targetCols.length; i++) {
|
||||
Text family = HStoreKey.extractFamily(targetCols[i]);
|
||||
|
@ -124,7 +135,14 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
if(matchers == null) {
|
||||
matchers = new Vector<ColumnMatcher>();
|
||||
}
|
||||
matchers.add(new ColumnMatcher(targetCols[i]));
|
||||
ColumnMatcher matcher = new ColumnMatcher(targetCols[i]);
|
||||
if(matcher.isWildCardMatch()) {
|
||||
this.wildcardMatch = true;
|
||||
}
|
||||
matchers.add(matcher);
|
||||
if(matchers.size() > 1) {
|
||||
this.multipleMatchers = true;
|
||||
}
|
||||
okCols.put(family, matchers);
|
||||
}
|
||||
}
|
||||
|
@ -170,6 +188,19 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
/** Mechanism used to shut down the whole scan */
|
||||
public abstract void close() throws IOException;
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.apache.hadoop.hbase.HInternalScannerInterface#isWildcardScanner()
|
||||
*/
|
||||
public boolean isWildcardScanner() {
|
||||
return this.wildcardMatch;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.apache.hadoop.hbase.HInternalScannerInterface#isMultipleMatchScanner()
|
||||
*/
|
||||
public boolean isMultipleMatchScanner() {
|
||||
return this.multipleMatchers;
|
||||
}
|
||||
/**
|
||||
* Get the next set of values for this scanner.
|
||||
*
|
||||
|
@ -179,7 +210,7 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
*
|
||||
* @see org.apache.hadoop.hbase.HScannerInterface#next(org.apache.hadoop.hbase.HStoreKey, java.util.TreeMap)
|
||||
*/
|
||||
public boolean next(HStoreKey key, TreeMap<Text, byte[]> results)
|
||||
public boolean next(HStoreKey key, TreeMap<Text, BytesWritable> results)
|
||||
throws IOException {
|
||||
|
||||
// Find the next row label (and timestamp)
|
||||
|
@ -187,7 +218,7 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
Text chosenRow = null;
|
||||
long chosenTimestamp = -1;
|
||||
for(int i = 0; i < keys.length; i++) {
|
||||
while((keys[i] != null)
|
||||
if((keys[i] != null)
|
||||
&& (columnMatch(i))
|
||||
&& (keys[i].getTimestamp() <= this.timestamp)
|
||||
&& ((chosenRow == null)
|
||||
|
@ -210,23 +241,31 @@ public abstract class HAbstractScanner implements HScannerInterface {
|
|||
|
||||
for(int i = 0; i < keys.length; i++) {
|
||||
// Fetch the data
|
||||
|
||||
while((keys[i] != null)
|
||||
&& (keys[i].getRow().compareTo(chosenRow) == 0)
|
||||
&& (keys[i].getTimestamp() == chosenTimestamp)) {
|
||||
|
||||
if(columnMatch(i)) {
|
||||
outbuf.reset();
|
||||
vals[i].write(outbuf);
|
||||
byte byteresults[] = outbuf.getData();
|
||||
inbuf.reset(byteresults, outbuf.getLength());
|
||||
BytesWritable tmpval = new BytesWritable();
|
||||
tmpval.readFields(inbuf);
|
||||
results.put(new Text(keys[i].getColumn()), tmpval.get());
|
||||
insertedItem = true;
|
||||
while((keys[i] != null)
|
||||
&& (keys[i].getRow().compareTo(chosenRow) == 0)) {
|
||||
|
||||
// If we are doing a wild card match or there are multiple matchers
|
||||
// per column, we need to scan all the older versions of this row
|
||||
// to pick up the rest of the family members
|
||||
|
||||
if(!wildcardMatch
|
||||
&& !multipleMatchers
|
||||
&& (keys[i].getTimestamp() != chosenTimestamp)) {
|
||||
break;
|
||||
}
|
||||
|
||||
if (! getNext(i)) {
|
||||
if(columnMatch(i)) {
|
||||
|
||||
// We only want the first result for any specific family member
|
||||
|
||||
if(!results.containsKey(keys[i].getColumn())) {
|
||||
results.put(new Text(keys[i].getColumn()), vals[i]);
|
||||
insertedItem = true;
|
||||
}
|
||||
}
|
||||
|
||||
if(!getNext(i)) {
|
||||
closeSubScanner(i);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,5 +21,6 @@ public class HBaseConfiguration extends Configuration {
|
|||
public HBaseConfiguration() {
|
||||
super();
|
||||
addDefaultResource("hbase-default.xml");
|
||||
addDefaultResource("hbase-site.xml");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,28 +19,30 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.Random;
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/*******************************************************************************
|
||||
* HClient manages a connection to a single HRegionServer.
|
||||
******************************************************************************/
|
||||
public class HClient implements HConstants {
|
||||
private final Logger LOG =
|
||||
Logger.getLogger(this.getClass().getName());
|
||||
private final Log LOG = LogFactory.getLog(this.getClass().getName());
|
||||
|
||||
private static final Text[] metaColumns = {
|
||||
META_COLUMN_FAMILY
|
||||
private static final Text[] META_COLUMNS = {
|
||||
COLUMN_FAMILY
|
||||
};
|
||||
private static final Text startRow = new Text();
|
||||
|
||||
private static final Text EMPTY_START_ROW = new Text();
|
||||
|
||||
private boolean closed;
|
||||
private long clientTimeout;
|
||||
|
@ -83,7 +85,7 @@ public class HClient implements HConstants {
|
|||
this.closed = false;
|
||||
this.conf = conf;
|
||||
|
||||
this.clientTimeout = conf.getLong("hbase.client.timeout.length", 10 * 1000);
|
||||
this.clientTimeout = conf.getLong("hbase.client.timeout.length", 30 * 1000);
|
||||
this.numTimeouts = conf.getInt("hbase.client.timeout.number", 5);
|
||||
this.numRetries = conf.getInt("hbase.client.retries.number", 2);
|
||||
|
||||
|
@ -98,40 +100,61 @@ public class HClient implements HConstants {
|
|||
this.currentServer = null;
|
||||
this.rand = new Random();
|
||||
}
|
||||
|
||||
public synchronized void createTable(HTableDescriptor desc) throws IOException {
|
||||
if(closed) {
|
||||
|
||||
/**
|
||||
* Check client is open.
|
||||
*/
|
||||
private synchronized void checkOpen() {
|
||||
if (this.closed) {
|
||||
throw new IllegalStateException("client is not open");
|
||||
}
|
||||
if(master == null) {
|
||||
locateRootRegion();
|
||||
}
|
||||
|
||||
private synchronized void checkMaster() throws IOException {
|
||||
if (this.master != null) {
|
||||
return;
|
||||
}
|
||||
master.createTable(desc);
|
||||
HServerAddress masterLocation =
|
||||
new HServerAddress(this.conf.get(MASTER_ADDRESS));
|
||||
this.master = (HMasterInterface)RPC.getProxy(HMasterInterface.class,
|
||||
HMasterInterface.versionID, masterLocation.getInetSocketAddress(), this.conf);
|
||||
}
|
||||
|
||||
public synchronized void createTable(HTableDescriptor desc)
|
||||
throws IOException {
|
||||
checkOpen();
|
||||
checkMaster();
|
||||
locateRootRegion();
|
||||
this.master.createTable(desc);
|
||||
}
|
||||
|
||||
public synchronized void deleteTable(Text tableName) throws IOException {
|
||||
if(closed) {
|
||||
throw new IllegalStateException("client is not open");
|
||||
}
|
||||
if(master == null) {
|
||||
locateRootRegion();
|
||||
}
|
||||
master.deleteTable(tableName);
|
||||
checkOpen();
|
||||
checkMaster();
|
||||
locateRootRegion();
|
||||
this.master.deleteTable(tableName);
|
||||
}
|
||||
|
||||
public synchronized void shutdown() throws IOException {
|
||||
checkOpen();
|
||||
checkMaster();
|
||||
this.master.shutdown();
|
||||
}
|
||||
|
||||
public synchronized void openTable(Text tableName) throws IOException {
|
||||
if(closed) {
|
||||
throw new IllegalStateException("client is not open");
|
||||
if(tableName == null || tableName.getLength() == 0) {
|
||||
throw new IllegalArgumentException("table name cannot be null or zero length");
|
||||
}
|
||||
|
||||
tableServers = tablesToServers.get(tableName);
|
||||
if(tableServers == null ) { // We don't know where the table is
|
||||
checkOpen();
|
||||
this.tableServers = tablesToServers.get(tableName);
|
||||
if(this.tableServers == null ) { // We don't know where the table is
|
||||
findTableInMeta(tableName); // Load the information from meta
|
||||
}
|
||||
}
|
||||
|
||||
private void findTableInMeta(Text tableName) throws IOException {
|
||||
TreeMap<Text, TableInfo> metaServers = tablesToServers.get(META_TABLE_NAME);
|
||||
TreeMap<Text, TableInfo> metaServers =
|
||||
this.tablesToServers.get(META_TABLE_NAME);
|
||||
|
||||
if(metaServers == null) { // Don't know where the meta is
|
||||
loadMetaFromRoot(tableName);
|
||||
|
@ -139,18 +162,51 @@ public class HClient implements HConstants {
|
|||
// All we really wanted was the meta or root table
|
||||
return;
|
||||
}
|
||||
metaServers = tablesToServers.get(META_TABLE_NAME);
|
||||
metaServers = this.tablesToServers.get(META_TABLE_NAME);
|
||||
}
|
||||
|
||||
tableServers = new TreeMap<Text, TableInfo>();
|
||||
for(Iterator<TableInfo> i = metaServers.tailMap(tableName).values().iterator();
|
||||
i.hasNext(); ) {
|
||||
this.tableServers = new TreeMap<Text, TableInfo>();
|
||||
for(int tries = 0;
|
||||
this.tableServers.size() == 0 && tries < this.numRetries;
|
||||
tries++) {
|
||||
|
||||
TableInfo t = i.next();
|
||||
Text firstMetaRegion = null;
|
||||
if(metaServers.containsKey(tableName)) {
|
||||
firstMetaRegion = tableName;
|
||||
|
||||
} else {
|
||||
firstMetaRegion = metaServers.headMap(tableName).lastKey();
|
||||
}
|
||||
for(Iterator<TableInfo> i
|
||||
= metaServers.tailMap(firstMetaRegion).values().iterator();
|
||||
i.hasNext(); ) {
|
||||
|
||||
scanOneMetaRegion(t, tableName);
|
||||
TableInfo t = i.next();
|
||||
|
||||
scanOneMetaRegion(t, tableName);
|
||||
}
|
||||
if(this.tableServers.size() == 0) {
|
||||
// Table not assigned. Sleep and try again
|
||||
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("Sleeping. Table " + tableName
|
||||
+ " not currently being served.");
|
||||
}
|
||||
try {
|
||||
Thread.sleep(this.clientTimeout);
|
||||
|
||||
} catch(InterruptedException e) {
|
||||
}
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("Wake. Retry finding table " + tableName);
|
||||
}
|
||||
}
|
||||
}
|
||||
tablesToServers.put(tableName, tableServers);
|
||||
if(this.tableServers.size() == 0) {
|
||||
throw new IOException("failed to scan " + META_TABLE_NAME + " after "
|
||||
+ this.numRetries + " retries");
|
||||
}
|
||||
this.tablesToServers.put(tableName, this.tableServers);
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -169,24 +225,23 @@ public class HClient implements HConstants {
|
|||
* could be.
|
||||
*/
|
||||
private void locateRootRegion() throws IOException {
|
||||
if(master == null) {
|
||||
HServerAddress masterLocation =
|
||||
new HServerAddress(this.conf.get(MASTER_ADDRESS));
|
||||
master = (HMasterInterface)RPC.getProxy(HMasterInterface.class,
|
||||
HMasterInterface.versionID,
|
||||
masterLocation.getInetSocketAddress(), conf);
|
||||
}
|
||||
checkMaster();
|
||||
|
||||
int tries = 0;
|
||||
HServerAddress rootRegionLocation = null;
|
||||
do {
|
||||
for(int tries = 0; rootRegionLocation == null && tries < numRetries; tries++){
|
||||
int localTimeouts = 0;
|
||||
while(rootRegionLocation == null && localTimeouts < numTimeouts) {
|
||||
rootRegionLocation = master.findRootRegion();
|
||||
|
||||
if(rootRegionLocation == null) {
|
||||
try {
|
||||
Thread.sleep(clientTimeout);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Sleeping. Waiting for root region.");
|
||||
}
|
||||
Thread.sleep(this.clientTimeout);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Wake. Retry finding root region.");
|
||||
}
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
localTimeouts++;
|
||||
|
@ -201,17 +256,18 @@ public class HClient implements HConstants {
|
|||
HRegionInterface rootRegion = getHRegionConnection(rootRegionLocation);
|
||||
|
||||
if(rootRegion.getRegionInfo(HGlobals.rootRegionInfo.regionName) != null) {
|
||||
tableServers = new TreeMap<Text, TableInfo>();
|
||||
tableServers.put(startRow, new TableInfo(HGlobals.rootRegionInfo, rootRegionLocation));
|
||||
tablesToServers.put(ROOT_TABLE_NAME, tableServers);
|
||||
this.tableServers = new TreeMap<Text, TableInfo>();
|
||||
this.tableServers.put(EMPTY_START_ROW,
|
||||
new TableInfo(HGlobals.rootRegionInfo, rootRegionLocation));
|
||||
|
||||
this.tablesToServers.put(ROOT_TABLE_NAME, this.tableServers);
|
||||
break;
|
||||
}
|
||||
rootRegionLocation = null;
|
||||
|
||||
} while(rootRegionLocation == null && tries++ < numRetries);
|
||||
}
|
||||
|
||||
if(rootRegionLocation == null) {
|
||||
closed = true;
|
||||
if (rootRegionLocation == null) {
|
||||
this.closed = true;
|
||||
throw new IOException("unable to locate root region server");
|
||||
}
|
||||
}
|
||||
|
@ -220,38 +276,78 @@ public class HClient implements HConstants {
|
|||
* Scans the root region to find all the meta regions
|
||||
*/
|
||||
private void scanRoot() throws IOException {
|
||||
tableServers = new TreeMap<Text, TableInfo>();
|
||||
TableInfo t = tablesToServers.get(ROOT_TABLE_NAME).get(startRow);
|
||||
scanOneMetaRegion(t, META_TABLE_NAME);
|
||||
tablesToServers.put(META_TABLE_NAME, tableServers);
|
||||
this.tableServers = new TreeMap<Text, TableInfo>();
|
||||
TableInfo t = this.tablesToServers.get(ROOT_TABLE_NAME).get(EMPTY_START_ROW);
|
||||
for(int tries = 0;
|
||||
scanOneMetaRegion(t, META_TABLE_NAME) == 0 && tries < this.numRetries;
|
||||
tries++) {
|
||||
|
||||
// The table is not yet being served. Sleep and retry.
|
||||
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("Sleeping. Table " + META_TABLE_NAME
|
||||
+ " not currently being served.");
|
||||
}
|
||||
try {
|
||||
Thread.sleep(this.clientTimeout);
|
||||
|
||||
} catch(InterruptedException e) {
|
||||
}
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("Wake. Retry finding table " + META_TABLE_NAME);
|
||||
}
|
||||
}
|
||||
if(this.tableServers.size() == 0) {
|
||||
throw new IOException("failed to scan " + ROOT_TABLE_NAME + " after "
|
||||
+ this.numRetries + " retries");
|
||||
}
|
||||
this.tablesToServers.put(META_TABLE_NAME, this.tableServers);
|
||||
}
|
||||
|
||||
/*
|
||||
* Scans a single meta region
|
||||
* @param t the table we're going to scan
|
||||
* @param tableName the name of the table we're looking for
|
||||
* @return returns the number of servers that are serving the table
|
||||
*/
|
||||
private void scanOneMetaRegion(TableInfo t, Text tableName) throws IOException {
|
||||
private int scanOneMetaRegion(TableInfo t, Text tableName)
|
||||
throws IOException {
|
||||
|
||||
HRegionInterface server = getHRegionConnection(t.serverAddress);
|
||||
int servers = 0;
|
||||
long scannerId = -1L;
|
||||
try {
|
||||
scannerId = server.openScanner(t.regionInfo.regionName, metaColumns, tableName);
|
||||
|
||||
scannerId =
|
||||
server.openScanner(t.regionInfo.regionName, META_COLUMNS, tableName);
|
||||
|
||||
DataInputBuffer inbuf = new DataInputBuffer();
|
||||
while(true) {
|
||||
HRegionInfo regionInfo = null;
|
||||
String serverAddress = null;
|
||||
HStoreKey key = new HStoreKey();
|
||||
|
||||
LabelledData[] values = server.next(scannerId, key);
|
||||
if(values.length == 0) {
|
||||
if(servers == 0) {
|
||||
// If we didn't find any servers then the table does not exist
|
||||
|
||||
throw new NoSuchElementException("table '" + tableName
|
||||
+ "' does not exist");
|
||||
}
|
||||
|
||||
// We found at least one server for the table and now we're done.
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
byte[] bytes = null;
|
||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||
for(int i = 0; i < values.length; i++) {
|
||||
results.put(values[i].getLabel(), values[i].getData().get());
|
||||
bytes = new byte[values[i].getData().getSize()];
|
||||
System.arraycopy(values[i].getData().get(), 0, bytes, 0, bytes.length);
|
||||
results.put(values[i].getLabel(), bytes);
|
||||
}
|
||||
HRegionInfo regionInfo = new HRegionInfo();
|
||||
byte[] bytes = results.get(META_COL_REGIONINFO);
|
||||
regionInfo = new HRegionInfo();
|
||||
bytes = results.get(COL_REGIONINFO);
|
||||
inbuf.reset(bytes, bytes.length);
|
||||
regionInfo.readFields(inbuf);
|
||||
|
||||
|
@ -259,15 +355,26 @@ public class HClient implements HConstants {
|
|||
// We're done
|
||||
break;
|
||||
}
|
||||
|
||||
bytes = results.get(META_COL_SERVER);
|
||||
String serverName = new String(bytes, UTF8_ENCODING);
|
||||
|
||||
bytes = results.get(COL_SERVER);
|
||||
if(bytes == null || bytes.length == 0) {
|
||||
// We need to rescan because the table we want is unassigned.
|
||||
|
||||
tableServers.put(regionInfo.startKey,
|
||||
new TableInfo(regionInfo, new HServerAddress(serverName)));
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("no server address for " + regionInfo.toString());
|
||||
}
|
||||
servers = 0;
|
||||
this.tableServers.clear();
|
||||
break;
|
||||
}
|
||||
servers += 1;
|
||||
serverAddress = new String(bytes, UTF8_ENCODING);
|
||||
|
||||
this.tableServers.put(regionInfo.startKey,
|
||||
new TableInfo(regionInfo, new HServerAddress(serverAddress)));
|
||||
}
|
||||
|
||||
return servers;
|
||||
|
||||
} finally {
|
||||
if(scannerId != -1L) {
|
||||
server.close(scannerId);
|
||||
|
@ -280,23 +387,24 @@ public class HClient implements HConstants {
|
|||
|
||||
// See if we already have a connection
|
||||
|
||||
HRegionInterface server = servers.get(regionServer.toString());
|
||||
HRegionInterface server = this.servers.get(regionServer.toString());
|
||||
|
||||
if(server == null) { // Get a connection
|
||||
|
||||
server = (HRegionInterface)RPC.waitForProxy(HRegionInterface.class,
|
||||
HRegionInterface.versionID, regionServer.getInetSocketAddress(), conf);
|
||||
HRegionInterface.versionID, regionServer.getInetSocketAddress(),
|
||||
this.conf);
|
||||
|
||||
servers.put(regionServer.toString(), server);
|
||||
this.servers.put(regionServer.toString(), server);
|
||||
}
|
||||
return server;
|
||||
}
|
||||
|
||||
/** Close the connection to the HRegionServer */
|
||||
/** Close the connection */
|
||||
public synchronized void close() throws IOException {
|
||||
if(! closed) {
|
||||
if(! this.closed) {
|
||||
RPC.stopClient();
|
||||
closed = true;
|
||||
this.closed = true;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -307,65 +415,75 @@ public class HClient implements HConstants {
|
|||
* catalog table that just contains table names and their descriptors.
|
||||
* Right now, it only exists as part of the META table's region info.
|
||||
*/
|
||||
public synchronized HTableDescriptor[] listTables() throws IOException {
|
||||
public synchronized HTableDescriptor[] listTables()
|
||||
throws IOException {
|
||||
TreeSet<HTableDescriptor> uniqueTables = new TreeSet<HTableDescriptor>();
|
||||
|
||||
TreeMap<Text, TableInfo> metaTables = tablesToServers.get(META_TABLE_NAME);
|
||||
TreeMap<Text, TableInfo> metaTables =
|
||||
this.tablesToServers.get(META_TABLE_NAME);
|
||||
|
||||
if(metaTables == null) {
|
||||
// Meta is not loaded yet so go do that
|
||||
loadMetaFromRoot(META_TABLE_NAME);
|
||||
metaTables = tablesToServers.get(META_TABLE_NAME);
|
||||
}
|
||||
|
||||
for(Iterator<TableInfo>it = metaTables.values().iterator(); it.hasNext(); ) {
|
||||
TableInfo t = it.next();
|
||||
for (TableInfo t: metaTables.values()) {
|
||||
HRegionInterface server = getHRegionConnection(t.serverAddress);
|
||||
long scannerId = -1L;
|
||||
try {
|
||||
scannerId = server.openScanner(t.regionInfo.regionName, metaColumns, startRow);
|
||||
HStoreKey key = new HStoreKey();
|
||||
scannerId = server.openScanner(t.regionInfo.regionName,
|
||||
META_COLUMNS, EMPTY_START_ROW);
|
||||
|
||||
HStoreKey key = new HStoreKey();
|
||||
DataInputBuffer inbuf = new DataInputBuffer();
|
||||
while(true) {
|
||||
LabelledData[] values = server.next(scannerId, key);
|
||||
if(values.length == 0) {
|
||||
break;
|
||||
}
|
||||
|
||||
for(int i = 0; i < values.length; i++) {
|
||||
if(values[i].getLabel().equals(META_COL_REGIONINFO)) {
|
||||
if(values[i].getLabel().equals(COL_REGIONINFO)) {
|
||||
byte[] bytes = values[i].getData().get();
|
||||
inbuf.reset(bytes, bytes.length);
|
||||
HRegionInfo info = new HRegionInfo();
|
||||
info.readFields(inbuf);
|
||||
|
||||
// Only examine the rows where the startKey is zero length
|
||||
|
||||
// Only examine the rows where the startKey is zero length
|
||||
if(info.startKey.getLength() == 0) {
|
||||
uniqueTables.add(info.tableDesc);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
} finally {
|
||||
if(scannerId != -1L) {
|
||||
server.close(scannerId);
|
||||
}
|
||||
}
|
||||
}
|
||||
return (HTableDescriptor[]) uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]);
|
||||
return (HTableDescriptor[])uniqueTables.
|
||||
toArray(new HTableDescriptor[uniqueTables.size()]);
|
||||
}
|
||||
|
||||
private synchronized TableInfo getTableInfo(Text row) {
|
||||
if(tableServers == null) {
|
||||
if(row == null || row.getLength() == 0) {
|
||||
throw new IllegalArgumentException("row key cannot be null or zero length");
|
||||
}
|
||||
if(this.tableServers == null) {
|
||||
throw new IllegalStateException("Must open table first");
|
||||
}
|
||||
|
||||
// Only one server will have the row we are looking for
|
||||
|
||||
Text serverKey = tableServers.tailMap(row).firstKey();
|
||||
return tableServers.get(serverKey);
|
||||
Text serverKey = null;
|
||||
if(this.tableServers.containsKey(row)) {
|
||||
serverKey = row;
|
||||
|
||||
} else {
|
||||
serverKey = this.tableServers.headMap(row).lastKey();
|
||||
}
|
||||
return this.tableServers.get(serverKey);
|
||||
}
|
||||
|
||||
/** Get a single value for the specified row and column */
|
||||
|
@ -416,7 +534,7 @@ public class HClient implements HConstants {
|
|||
* Return the specified columns.
|
||||
*/
|
||||
public synchronized HScannerInterface obtainScanner(Text[] columns, Text startRow) throws IOException {
|
||||
if(tableServers == null) {
|
||||
if(this.tableServers == null) {
|
||||
throw new IllegalStateException("Must open table first");
|
||||
}
|
||||
return new ClientScanner(columns, startRow);
|
||||
|
@ -427,14 +545,14 @@ public class HClient implements HConstants {
|
|||
TableInfo info = getTableInfo(row);
|
||||
long lockid;
|
||||
try {
|
||||
currentServer = getHRegionConnection(info.serverAddress);
|
||||
currentRegion = info.regionInfo.regionName;
|
||||
clientid = rand.nextLong();
|
||||
lockid = currentServer.startUpdate(currentRegion, clientid, row);
|
||||
this.currentServer = getHRegionConnection(info.serverAddress);
|
||||
this.currentRegion = info.regionInfo.regionName;
|
||||
this.clientid = rand.nextLong();
|
||||
lockid = currentServer.startUpdate(this.currentRegion, this.clientid, row);
|
||||
|
||||
} catch(IOException e) {
|
||||
currentServer = null;
|
||||
currentRegion = null;
|
||||
this.currentServer = null;
|
||||
this.currentRegion = null;
|
||||
throw e;
|
||||
}
|
||||
return lockid;
|
||||
|
@ -443,16 +561,17 @@ public class HClient implements HConstants {
|
|||
/** Change a value for the specified column */
|
||||
public void put(long lockid, Text column, byte val[]) throws IOException {
|
||||
try {
|
||||
currentServer.put(currentRegion, clientid, lockid, column, new BytesWritable(val));
|
||||
this.currentServer.put(this.currentRegion, this.clientid, lockid, column,
|
||||
new BytesWritable(val));
|
||||
|
||||
} catch(IOException e) {
|
||||
try {
|
||||
currentServer.abort(currentRegion, clientid, lockid);
|
||||
this.currentServer.abort(this.currentRegion, this.clientid, lockid);
|
||||
|
||||
} catch(IOException e2) {
|
||||
}
|
||||
currentServer = null;
|
||||
currentRegion = null;
|
||||
this.currentServer = null;
|
||||
this.currentRegion = null;
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
@ -460,16 +579,16 @@ public class HClient implements HConstants {
|
|||
/** Delete the value for a column */
|
||||
public void delete(long lockid, Text column) throws IOException {
|
||||
try {
|
||||
currentServer.delete(currentRegion, clientid, lockid, column);
|
||||
this.currentServer.delete(this.currentRegion, this.clientid, lockid, column);
|
||||
|
||||
} catch(IOException e) {
|
||||
try {
|
||||
currentServer.abort(currentRegion, clientid, lockid);
|
||||
this.currentServer.abort(this.currentRegion, this.clientid, lockid);
|
||||
|
||||
} catch(IOException e2) {
|
||||
}
|
||||
currentServer = null;
|
||||
currentRegion = null;
|
||||
this.currentServer = null;
|
||||
this.currentRegion = null;
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
@ -477,11 +596,10 @@ public class HClient implements HConstants {
|
|||
/** Abort a row mutation */
|
||||
public void abort(long lockid) throws IOException {
|
||||
try {
|
||||
currentServer.abort(currentRegion, clientid, lockid);
|
||||
|
||||
this.currentServer.abort(this.currentRegion, this.clientid, lockid);
|
||||
} catch(IOException e) {
|
||||
currentServer = null;
|
||||
currentRegion = null;
|
||||
this.currentServer = null;
|
||||
this.currentRegion = null;
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
@ -489,11 +607,11 @@ public class HClient implements HConstants {
|
|||
/** Finalize a row mutation */
|
||||
public void commit(long lockid) throws IOException {
|
||||
try {
|
||||
currentServer.commit(currentRegion, clientid, lockid);
|
||||
this.currentServer.commit(this.currentRegion, this.clientid, lockid);
|
||||
|
||||
} finally {
|
||||
currentServer = null;
|
||||
currentRegion = null;
|
||||
this.currentServer = null;
|
||||
this.currentRegion = null;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -515,7 +633,19 @@ public class HClient implements HConstants {
|
|||
this.columns = columns;
|
||||
this.startRow = startRow;
|
||||
this.closed = false;
|
||||
Collection<TableInfo> info = tableServers.tailMap(startRow).values();
|
||||
|
||||
Text firstServer = null;
|
||||
if(this.startRow == null || this.startRow.getLength() == 0) {
|
||||
firstServer = tableServers.firstKey();
|
||||
|
||||
} else if(tableServers.containsKey(startRow)) {
|
||||
firstServer = startRow;
|
||||
|
||||
} else {
|
||||
firstServer = tableServers.headMap(startRow).lastKey();
|
||||
}
|
||||
Collection<TableInfo> info = tableServers.tailMap(firstServer).values();
|
||||
|
||||
this.regions = info.toArray(new TableInfo[info.size()]);
|
||||
this.currentRegion = -1;
|
||||
this.server = null;
|
||||
|
@ -528,19 +658,20 @@ public class HClient implements HConstants {
|
|||
* Returns false if there are no more scanners.
|
||||
*/
|
||||
private boolean nextScanner() throws IOException {
|
||||
if(scannerId != -1L) {
|
||||
server.close(scannerId);
|
||||
scannerId = -1L;
|
||||
if(this.scannerId != -1L) {
|
||||
this.server.close(this.scannerId);
|
||||
this.scannerId = -1L;
|
||||
}
|
||||
currentRegion += 1;
|
||||
if(currentRegion == regions.length) {
|
||||
this.currentRegion += 1;
|
||||
if(this.currentRegion == this.regions.length) {
|
||||
close();
|
||||
return false;
|
||||
}
|
||||
try {
|
||||
server = getHRegionConnection(regions[currentRegion].serverAddress);
|
||||
scannerId = server.openScanner(regions[currentRegion].regionInfo.regionName,
|
||||
columns, startRow);
|
||||
this.server = getHRegionConnection(this.regions[currentRegion].serverAddress);
|
||||
this.scannerId = this.server.openScanner(
|
||||
this.regions[currentRegion].regionInfo.regionName, this.columns,
|
||||
this.startRow);
|
||||
|
||||
} catch(IOException e) {
|
||||
close();
|
||||
|
@ -553,16 +684,18 @@ public class HClient implements HConstants {
|
|||
* @see org.apache.hadoop.hbase.HScannerInterface#next(org.apache.hadoop.hbase.HStoreKey, java.util.TreeMap)
|
||||
*/
|
||||
public boolean next(HStoreKey key, TreeMap<Text, byte[]> results) throws IOException {
|
||||
if(closed) {
|
||||
if(this.closed) {
|
||||
return false;
|
||||
}
|
||||
LabelledData[] values = null;
|
||||
do {
|
||||
values = server.next(scannerId, key);
|
||||
values = this.server.next(this.scannerId, key);
|
||||
} while(values.length == 0 && nextScanner());
|
||||
|
||||
for(int i = 0; i < values.length; i++) {
|
||||
results.put(values[i].getLabel(), values[i].getData().get());
|
||||
byte[] bytes = new byte[values[i].getData().getSize()];
|
||||
System.arraycopy(values[i].getData().get(), 0, bytes, 0, bytes.length);
|
||||
results.put(values[i].getLabel(), bytes);
|
||||
}
|
||||
return values.length != 0;
|
||||
}
|
||||
|
@ -571,38 +704,112 @@ public class HClient implements HConstants {
|
|||
* @see org.apache.hadoop.hbase.HScannerInterface#close()
|
||||
*/
|
||||
public void close() throws IOException {
|
||||
if(scannerId != -1L) {
|
||||
server.close(scannerId);
|
||||
if(this.scannerId != -1L) {
|
||||
this.server.close(this.scannerId);
|
||||
}
|
||||
server = null;
|
||||
closed = true;
|
||||
this.server = null;
|
||||
this.closed = true;
|
||||
}
|
||||
}
|
||||
|
||||
private void printUsage() {
|
||||
System.err.println("Usage: java " + this.getClass().getName() +
|
||||
" [--master=hostname:port]");
|
||||
printUsage(null);
|
||||
}
|
||||
|
||||
private int doCommandLine(final String args[]) {
|
||||
// Process command-line args. TODO: Better cmd-line processing
|
||||
// (but hopefully something not as painful as cli options).
|
||||
for (String cmd: args) {
|
||||
if (cmd.equals("-h") || cmd.startsWith("--h")) {
|
||||
printUsage();
|
||||
return 0;
|
||||
}
|
||||
|
||||
final String masterArgKey = "--master=";
|
||||
if (cmd.startsWith(masterArgKey)) {
|
||||
this.conf.set(MASTER_ADDRESS,
|
||||
cmd.substring(masterArgKey.length()));
|
||||
}
|
||||
private void printUsage(final String message) {
|
||||
if (message != null && message.length() > 0) {
|
||||
System.err.println(message);
|
||||
}
|
||||
|
||||
System.err.println("Usage: java " + this.getClass().getName() +
|
||||
" [--master=host:port] <command> <args>");
|
||||
System.err.println("Options:");
|
||||
System.err.println(" master Specify host and port of HBase " +
|
||||
"cluster master. If not present,");
|
||||
System.err.println(" address is read from configuration.");
|
||||
System.err.println("Commands:");
|
||||
System.err.println(" shutdown Shutdown the HBase cluster.");
|
||||
System.err.println(" createTable Takes table name, column families, " +
|
||||
"and maximum versions.");
|
||||
System.err.println(" deleteTable Takes a table name.");
|
||||
System.err.println(" iistTables List all tables.");
|
||||
System.err.println("Example Usage:");
|
||||
System.err.println(" % java " + this.getClass().getName() + " shutdown");
|
||||
System.err.println(" % java " + this.getClass().getName() +
|
||||
" createTable webcrawl contents: anchors: 10");
|
||||
}
|
||||
|
||||
int doCommandLine(final String args[]) {
|
||||
// Process command-line args. TODO: Better cmd-line processing
|
||||
// (but hopefully something not as painful as cli options).
|
||||
int errCode = -1;
|
||||
if (args.length < 1) {
|
||||
printUsage();
|
||||
return errCode;
|
||||
}
|
||||
try {
|
||||
locateRootRegion();
|
||||
for (int i = 0; i < args.length; i++) {
|
||||
String cmd = args[i];
|
||||
if (cmd.equals("-h") || cmd.startsWith("--h")) {
|
||||
printUsage();
|
||||
errCode = 0;
|
||||
break;
|
||||
}
|
||||
|
||||
final String masterArgKey = "--master=";
|
||||
if (cmd.startsWith(masterArgKey)) {
|
||||
this.conf.set(MASTER_ADDRESS, cmd.substring(masterArgKey.length()));
|
||||
continue;
|
||||
}
|
||||
|
||||
if (cmd.equals("shutdown")) {
|
||||
shutdown();
|
||||
errCode = 0;
|
||||
break;
|
||||
}
|
||||
|
||||
if (cmd.equals("listTables")) {
|
||||
HTableDescriptor [] tables = listTables();
|
||||
for (int ii = 0; ii < tables.length; ii++) {
|
||||
System.out.println(tables[ii].getName());
|
||||
}
|
||||
errCode = 0;
|
||||
break;
|
||||
}
|
||||
|
||||
if (cmd.equals("createTable")) {
|
||||
if (i + 3 > args.length) {
|
||||
throw new IllegalArgumentException("Must supply a table name " +
|
||||
", at least one column family and maximum number of versions");
|
||||
}
|
||||
int maxVersions = (Integer.parseInt(args[args.length - 1]));
|
||||
HTableDescriptor desc =
|
||||
new HTableDescriptor(args[i + 1], maxVersions);
|
||||
boolean addedFamily = false;
|
||||
for (int ii = i + 2; ii < (args.length - 1); ii++) {
|
||||
desc.addFamily(new Text(args[ii]));
|
||||
addedFamily = true;
|
||||
}
|
||||
if (!addedFamily) {
|
||||
throw new IllegalArgumentException("Must supply at least one " +
|
||||
"column family");
|
||||
}
|
||||
createTable(desc);
|
||||
errCode = 0;
|
||||
break;
|
||||
}
|
||||
|
||||
if (cmd.equals("deleteTable")) {
|
||||
if (i + 1 > args.length) {
|
||||
throw new IllegalArgumentException("Must supply a table name");
|
||||
}
|
||||
deleteTable(new Text(args[i + 1]));
|
||||
errCode = 0;
|
||||
break;
|
||||
}
|
||||
|
||||
printUsage();
|
||||
break;
|
||||
}
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
|
|
|
@ -15,30 +15,40 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
||||
/*******************************************************************************
|
||||
/**
|
||||
* HConstants holds a bunch of HBase-related constants
|
||||
******************************************************************************/
|
||||
*/
|
||||
public interface HConstants {
|
||||
|
||||
// Configuration parameters
|
||||
|
||||
// TODO: URL for hbase master, like hdfs URLs with host and port.
|
||||
// Or, like jdbc URLs:
|
||||
// TODO: URL for hbase master like hdfs URLs with host and port.
|
||||
// Like jdbc URLs? URLs could be used to refer to table cells?
|
||||
// jdbc:mysql://[host][,failoverhost...][:port]/[database]
|
||||
// jdbc:mysql://[host][,failoverhost...][:port]/[database][?propertyName1][=propertyValue1][&propertyName2][=propertyValue2]...
|
||||
|
||||
static final String MASTER_ADDRESS = "hbase.master";
|
||||
// Key into HBaseConfiguration for the hbase.master address.
|
||||
// TODO: Support 'local': i.e. default of all running in single
|
||||
// process. Same for regionserver.
|
||||
// process. Same for regionserver. TODO: Is having HBase homed
|
||||
// on port 60k OK?
|
||||
static final String MASTER_ADDRESS = "hbase.master";
|
||||
static final String DEFAULT_MASTER_ADDRESS = "localhost:60000";
|
||||
|
||||
// Key for hbase.regionserver address.
|
||||
static final String REGIONSERVER_ADDRESS = "hbase.regionserver";
|
||||
static final String DEFAULT_REGIONSERVER_ADDRESS =
|
||||
"localhost:60010";
|
||||
static final String DEFAULT_REGIONSERVER_ADDRESS = "localhost:60010";
|
||||
|
||||
static final String THREAD_WAKE_FREQUENCY = "hbase.server.thread.wakefrequency";
|
||||
static final String HREGION_DIR = "hbase.regiondir";
|
||||
static final String DEFAULT_HREGION_DIR = "/hbase";
|
||||
static final String HREGIONDIR_PREFIX = "hregion_";
|
||||
|
||||
// TODO: Someone may try to name a column family 'log'. If they
|
||||
// do, it will clash with the HREGION log dir subdirectory. FIX.
|
||||
static final String HREGION_LOGDIR_NAME = "log";
|
||||
|
||||
// Always store the location of the root table's HRegion.
|
||||
// This HRegion is never split.
|
||||
|
@ -46,22 +56,29 @@ public interface HConstants {
|
|||
// region name = table + startkey + regionid. This is the row key.
|
||||
// each row in the root and meta tables describes exactly 1 region
|
||||
// Do we ever need to know all the information that we are storing?
|
||||
|
||||
static final Text ROOT_TABLE_NAME = new Text("--ROOT--");
|
||||
static final Text ROOT_COLUMN_FAMILY = new Text("info:");
|
||||
static final Text ROOT_COL_REGIONINFO = new Text(ROOT_COLUMN_FAMILY + "regioninfo");
|
||||
static final Text ROOT_COL_SERVER = new Text(ROOT_COLUMN_FAMILY + "server");
|
||||
static final Text ROOT_COL_STARTCODE = new Text(ROOT_COLUMN_FAMILY + "serverstartcode");
|
||||
|
||||
// The root tables' name.
|
||||
static final Text ROOT_TABLE_NAME = new Text("--ROOT--");
|
||||
|
||||
// The META tables' name.
|
||||
static final Text META_TABLE_NAME = new Text("--META--");
|
||||
static final Text META_COLUMN_FAMILY = new Text(ROOT_COLUMN_FAMILY);
|
||||
static final Text META_COL_REGIONINFO = new Text(ROOT_COL_REGIONINFO);
|
||||
static final Text META_COL_SERVER = new Text(ROOT_COL_SERVER);
|
||||
static final Text META_COL_STARTCODE = new Text(ROOT_COL_STARTCODE);
|
||||
|
||||
// Defines for the column names used in both ROOT and META HBase 'meta'
|
||||
// tables.
|
||||
static final Text COLUMN_FAMILY = new Text("info:");
|
||||
static final Text COL_REGIONINFO = new Text(COLUMN_FAMILY + "regioninfo");
|
||||
static final Text COL_SERVER = new Text(COLUMN_FAMILY + "server");
|
||||
static final Text COL_STARTCODE = new Text(COLUMN_FAMILY + "serverstartcode");
|
||||
|
||||
// Other constants
|
||||
|
||||
static final long DESIRED_MAX_FILE_SIZE = 128 * 1024 * 1024; // 128MB
|
||||
static final String UTF8_ENCODING = "UTF-8";
|
||||
|
||||
static final BytesWritable DELETE_BYTES =
|
||||
new BytesWritable("HBASE::DELETEVAL".getBytes());
|
||||
|
||||
static final BytesWritable COMPLETE_CACHEFLUSH =
|
||||
new BytesWritable("HBASE::CACHEFLUSH".getBytes());
|
||||
|
||||
}
|
||||
|
|
|
@ -15,9 +15,9 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
/*******************************************************************************
|
||||
/**
|
||||
* Global values used for finding and scanning the root and meta tables.
|
||||
******************************************************************************/
|
||||
*/
|
||||
public class HGlobals implements HConstants {
|
||||
|
||||
static HTableDescriptor rootTableDesc = null;
|
||||
|
@ -26,13 +26,11 @@ public class HGlobals implements HConstants {
|
|||
|
||||
static {
|
||||
rootTableDesc = new HTableDescriptor(ROOT_TABLE_NAME.toString(), 1);
|
||||
rootTableDesc.addFamily(ROOT_COLUMN_FAMILY);
|
||||
rootTableDesc.addFamily(COLUMN_FAMILY);
|
||||
|
||||
rootRegionInfo = new HRegionInfo(0L, rootTableDesc, null, null);
|
||||
|
||||
metaTableDesc = new HTableDescriptor(META_TABLE_NAME.toString(), 1);
|
||||
metaTableDesc.addFamily(META_COLUMN_FAMILY);
|
||||
metaTableDesc.addFamily(COLUMN_FAMILY);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,44 @@
|
|||
/**
|
||||
* Copyright 2007 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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.IOException;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
||||
/**
|
||||
* Internally, we need to be able to determine if the scanner is doing wildcard
|
||||
* column matches (when only a column family is specified or if a column regex
|
||||
* is specified) or if multiple members of the same column family were specified.
|
||||
*
|
||||
* If so, we need to ignore the timestamp to ensure that we get all the family
|
||||
* members, as they may have been last updated at different times.
|
||||
*
|
||||
* This interface exposes two APIs for querying the scanner.
|
||||
*/
|
||||
public interface HInternalScannerInterface {
|
||||
|
||||
public boolean next(HStoreKey key, TreeMap<Text, BytesWritable> results) throws IOException;
|
||||
public void close() throws IOException;
|
||||
/** Returns true if the scanner is matching a column family or regex */
|
||||
public boolean isWildcardScanner();
|
||||
|
||||
/** Returns true if the scanner is matching multiple column family members */
|
||||
public boolean isMultipleMatchScanner();
|
||||
|
||||
}
|
|
@ -18,40 +18,49 @@ package org.apache.hadoop.hbase;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.io.*;
|
||||
import org.apache.hadoop.io.SequenceFile.Reader;
|
||||
import org.apache.hadoop.fs.*;
|
||||
import org.apache.hadoop.conf.*;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.*;
|
||||
|
||||
/*******************************************************************************
|
||||
/**
|
||||
* HLog stores all the edits to the HStore.
|
||||
*
|
||||
* It performs logfile-rolling, so external callers are not aware that the
|
||||
* underlying file is being rolled.
|
||||
*
|
||||
* A single HLog is used by several HRegions simultaneously.
|
||||
* <p>A single HLog is used by several HRegions simultaneously.
|
||||
*
|
||||
* Each one is identified by a unique long int. HRegions do not need to declare
|
||||
* themselves before using the HLog; they simply include their HRegion-id in the
|
||||
* append() or completeCacheFlush() calls.
|
||||
* <p>Each HRegion is identified by a unique long int. HRegions do not need to
|
||||
* declare themselves before using the HLog; they simply include their
|
||||
* HRegion-id in the {@link #append(Text, Text, Text, TreeMap, long)} or
|
||||
* {@link #completeCacheFlush(Text, Text, long)} calls.
|
||||
*
|
||||
* An HLog consists of multiple on-disk files, which have a chronological order.
|
||||
* <p>An HLog consists of multiple on-disk files, which have a chronological
|
||||
* order. As data is flushed to other (better) on-disk structures, the log
|
||||
* becomes obsolete. We can destroy all the log messages for a given
|
||||
* HRegion-id up to the most-recent CACHEFLUSH message from that HRegion.
|
||||
*
|
||||
* As data is flushed to other (better) on-disk structures, the log becomes
|
||||
* obsolete. We can destroy all the log messages for a given HRegion-id up to
|
||||
* the most-recent CACHEFLUSH message from that HRegion.
|
||||
*
|
||||
* It's only practical to delete entire files. Thus, we delete an entire
|
||||
* <p>It's only practical to delete entire files. Thus, we delete an entire
|
||||
* on-disk file F when all of the messages in F have a log-sequence-id that's
|
||||
* older (smaller) than the most-recent CACHEFLUSH message for every HRegion
|
||||
* that has a message in F.
|
||||
******************************************************************************/
|
||||
public class HLog {
|
||||
*
|
||||
* <p>TODO: Vuk Ercegovac also pointed out that keeping HBase HRegion edit logs
|
||||
* in HDFS is currently flawed. HBase writes edits to logs and to a memcache.
|
||||
* The 'atomic' write to the log is meant to serve as insurance against
|
||||
* abnormal RegionServer exit: on startup, the log is rerun to reconstruct an
|
||||
* HRegion's last wholesome state. But files in HDFS do not 'exist' until they
|
||||
* are cleanly closed -- something that will not happen if RegionServer exits
|
||||
* without running its 'close'.
|
||||
*/
|
||||
public class HLog implements HConstants {
|
||||
private static final Log LOG = LogFactory.getLog(HLog.class);
|
||||
|
||||
static final String HLOG_DATFILE = "hlog.dat.";
|
||||
static final Text METACOLUMN = new Text("METACOLUMN");
|
||||
static final Text METACOLUMN = new Text("METACOLUMN:");
|
||||
static final Text METAROW = new Text("METAROW");
|
||||
|
||||
FileSystem fs;
|
||||
|
@ -66,28 +75,40 @@ public class HLog {
|
|||
long oldestOutstandingSeqNum = -1;
|
||||
|
||||
boolean closed = false;
|
||||
long logSeqNum = 0;
|
||||
transient long logSeqNum = 0;
|
||||
long filenum = 0;
|
||||
int numEntries = 0;
|
||||
transient int numEntries = 0;
|
||||
|
||||
Integer rollLock = new Integer(0);
|
||||
|
||||
/**
|
||||
* Bundle up a bunch of log files (which are no longer being written to),
|
||||
* into a new file. Delete the old log files when ready.
|
||||
* @param srcDir Directory of log files to bundle:
|
||||
* e.g. <code>${REGIONDIR}/log_HOST_PORT</code>
|
||||
* @param dstFile Destination file:
|
||||
* e.g. <code>${REGIONDIR}/oldlogfile_HOST_PORT</code>
|
||||
* @param fs FileSystem
|
||||
* @param conf HBaseConfiguration
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void consolidateOldLog(Path srcDir, Path dstFile, FileSystem fs, Configuration conf) throws IOException {
|
||||
LOG.debug("consolidating log files");
|
||||
public static void consolidateOldLog(Path srcDir, Path dstFile,
|
||||
FileSystem fs, Configuration conf)
|
||||
throws IOException {
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("consolidating log files");
|
||||
}
|
||||
|
||||
Path logfiles[] = fs.listPaths(srcDir);
|
||||
SequenceFile.Writer newlog = SequenceFile.createWriter(fs, conf, dstFile, HLogKey.class, HLogEdit.class);
|
||||
SequenceFile.Writer newlog = SequenceFile.createWriter(fs, conf, dstFile,
|
||||
HLogKey.class, HLogEdit.class);
|
||||
try {
|
||||
for(int i = 0; i < logfiles.length; i++) {
|
||||
SequenceFile.Reader in = new SequenceFile.Reader(fs, logfiles[i], conf);
|
||||
SequenceFile.Reader in =
|
||||
new SequenceFile.Reader(fs, logfiles[i], conf);
|
||||
try {
|
||||
HLogKey key = new HLogKey();
|
||||
HLogEdit val = new HLogEdit();
|
||||
|
||||
while(in.next(key, val)) {
|
||||
newlog.append(key, val);
|
||||
}
|
||||
|
@ -111,11 +132,13 @@ public class HLog {
|
|||
}
|
||||
}
|
||||
}
|
||||
LOG.debug("log file consolidation completed");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("log file consolidation completed");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create an edit log at the given location.
|
||||
* Create an edit log at the given <code>dir</code> location.
|
||||
*
|
||||
* You should never have to load an existing log. If there is a log
|
||||
* at startup, it should have already been processed and deleted by
|
||||
|
@ -125,19 +148,16 @@ public class HLog {
|
|||
this.fs = fs;
|
||||
this.dir = dir;
|
||||
this.conf = conf;
|
||||
this.logSeqNum = 0;
|
||||
|
||||
if(fs.exists(dir)) {
|
||||
if (fs.exists(dir)) {
|
||||
throw new IOException("Target HLog directory already exists: " + dir);
|
||||
}
|
||||
fs.mkdirs(dir);
|
||||
|
||||
rollWriter();
|
||||
}
|
||||
|
||||
/**
|
||||
* Roll the log writer. That is, start writing log messages to
|
||||
* a new file.
|
||||
* Roll the log writer. That is, start writing log messages to a new file.
|
||||
*
|
||||
* The 'rollLock' prevents us from entering rollWriter() more than
|
||||
* once at a time.
|
||||
|
@ -170,7 +190,9 @@ public class HLog {
|
|||
}
|
||||
}
|
||||
|
||||
LOG.debug("closing current log writer and getting a new one");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("closing current log writer and getting a new one");
|
||||
}
|
||||
|
||||
// Close the current writer (if any), and grab a new one.
|
||||
|
||||
|
@ -178,14 +200,16 @@ public class HLog {
|
|||
writer.close();
|
||||
|
||||
if(filenum > 0) {
|
||||
outputfiles.put(logSeqNum-1, computeFilename(filenum-1));
|
||||
outputfiles.put(logSeqNum - 1, computeFilename(filenum - 1));
|
||||
}
|
||||
}
|
||||
|
||||
Path newPath = computeFilename(filenum++);
|
||||
this.writer = SequenceFile.createWriter(fs, conf, newPath, HLogKey.class, HLogEdit.class);
|
||||
|
||||
LOG.debug("new log writer created");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("new log writer created");
|
||||
}
|
||||
|
||||
// Can we delete any of the old log files?
|
||||
// First, compute the oldest relevant log operation
|
||||
|
@ -203,7 +227,9 @@ public class HLog {
|
|||
// Next, remove all files with a final ID that's older
|
||||
// than the oldest pending region-operation.
|
||||
|
||||
LOG.debug("removing old log files");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("removing old log files");
|
||||
}
|
||||
|
||||
for(Iterator<Long> it = outputfiles.keySet().iterator(); it.hasNext(); ) {
|
||||
long maxSeqNum = it.next().longValue();
|
||||
|
@ -226,7 +252,9 @@ public class HLog {
|
|||
fs.delete(p);
|
||||
}
|
||||
|
||||
LOG.debug("old log files deleted");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("old log files deleted");
|
||||
}
|
||||
|
||||
this.numEntries = 0;
|
||||
}
|
||||
|
@ -247,21 +275,29 @@ public class HLog {
|
|||
}
|
||||
|
||||
/**
|
||||
* Append a set of edits to the log.
|
||||
* Log edits are keyed by regionName, rowname, and log-sequence-id.
|
||||
* Append a set of edits to the log. Log edits are keyed by regionName,
|
||||
* rowname, and log-sequence-id.
|
||||
*
|
||||
* Later, if we sort by these keys, we obtain all the relevant edits for
|
||||
* a given key-range of the HRegion. Any edits that do not have a matching
|
||||
* COMPLETE_CACHEFLUSH message can be discarded.
|
||||
* a given key-range of the HRegion (TODO). Any edits that do not have a
|
||||
* matching {@link HConstants#COMPLETE_CACHEFLUSH} message can be discarded.
|
||||
*
|
||||
* Logs cannot be restarted once closed, or once the HLog process dies.
|
||||
* <p>Logs cannot be restarted once closed, or once the HLog process dies.
|
||||
* Each time the HLog starts, it must create a new log. This means that
|
||||
* other systems should process the log appropriately upon each startup
|
||||
* (and prior to initializing HLog).
|
||||
*
|
||||
* We need to seize a lock on the writer so that writes are atomic.
|
||||
* @param regionName
|
||||
* @param tableName
|
||||
* @param row
|
||||
* @param columns
|
||||
* @param timestamp
|
||||
* @throws IOException
|
||||
*/
|
||||
public synchronized void append(Text regionName, Text tableName, Text row, TreeMap<Text, byte[]> columns, long timestamp) throws IOException {
|
||||
public synchronized void append(Text regionName, Text tableName, Text row,
|
||||
TreeMap<Text, BytesWritable> columns, long timestamp)
|
||||
throws IOException {
|
||||
if(closed) {
|
||||
throw new IOException("Cannot append; log is closed");
|
||||
}
|
||||
|
@ -272,19 +308,16 @@ public class HLog {
|
|||
// most recent flush for every regionName. However, for regions
|
||||
// that don't have any flush yet, the relevant operation is the
|
||||
// first one that's been added.
|
||||
|
||||
if(regionToLastFlush.get(regionName) == null) {
|
||||
if (regionToLastFlush.get(regionName) == null) {
|
||||
regionToLastFlush.put(regionName, seqNum[0]);
|
||||
}
|
||||
|
||||
int counter = 0;
|
||||
for(Iterator<Text> it = columns.keySet().iterator(); it.hasNext(); ) {
|
||||
Text column = it.next();
|
||||
byte[] val = columns.get(column);
|
||||
HLogKey logKey = new HLogKey(regionName, tableName, row, seqNum[counter++]);
|
||||
HLogEdit logEdit = new HLogEdit(column, val, timestamp);
|
||||
for (Text column: columns.keySet()) {
|
||||
HLogKey logKey =
|
||||
new HLogKey(regionName, tableName, row, seqNum[counter++]);
|
||||
HLogEdit logEdit = new HLogEdit(column, columns.get(column), timestamp);
|
||||
writer.append(logKey, logEdit);
|
||||
|
||||
numEntries++;
|
||||
}
|
||||
}
|
||||
|
@ -317,40 +350,76 @@ public class HLog {
|
|||
* Set a flag so that we do not roll the log between the start
|
||||
* and complete of a cache-flush. Otherwise the log-seq-id for
|
||||
* the flush will not appear in the correct logfile.
|
||||
* @return sequence ID to pass {@link #completeCacheFlush(Text, Text, long)}
|
||||
* @see {@link #completeCacheFlush(Text, Text, long)}
|
||||
*/
|
||||
public synchronized long startCacheFlush() {
|
||||
while(insideCacheFlush) {
|
||||
while (insideCacheFlush) {
|
||||
try {
|
||||
wait();
|
||||
} catch (InterruptedException ie) {
|
||||
}
|
||||
}
|
||||
|
||||
insideCacheFlush = true;
|
||||
notifyAll();
|
||||
return obtainSeqNum();
|
||||
}
|
||||
|
||||
/** Complete the cache flush */
|
||||
public synchronized void completeCacheFlush(Text regionName, Text tableName, long logSeqId) throws IOException {
|
||||
public synchronized void completeCacheFlush(final Text regionName,
|
||||
final Text tableName, final long logSeqId)
|
||||
throws IOException {
|
||||
if(closed) {
|
||||
return;
|
||||
}
|
||||
|
||||
if(! insideCacheFlush) {
|
||||
throw new IOException("Impossible situation: inside completeCacheFlush(), but 'insideCacheFlush' flag is false");
|
||||
throw new IOException("Impossible situation: inside " +
|
||||
"completeCacheFlush(), but 'insideCacheFlush' flag is false");
|
||||
}
|
||||
|
||||
writer.append(new HLogKey(regionName, tableName, HLog.METAROW, logSeqId),
|
||||
new HLogEdit(HLog.METACOLUMN, HStoreKey.COMPLETE_CACHEFLUSH, System.currentTimeMillis()));
|
||||
new HLogEdit(HLog.METACOLUMN, COMPLETE_CACHEFLUSH,
|
||||
System.currentTimeMillis()));
|
||||
numEntries++;
|
||||
|
||||
// Remember the most-recent flush for each region.
|
||||
// This is used to delete obsolete log files.
|
||||
|
||||
regionToLastFlush.put(regionName, logSeqId);
|
||||
|
||||
insideCacheFlush = false;
|
||||
notifyAll();
|
||||
}
|
||||
|
||||
/**
|
||||
* Pass a log file and it will dump out a text version on
|
||||
* <code>stdout</code>.
|
||||
* @param args
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void main(String[] args) throws IOException {
|
||||
if (args.length < 1) {
|
||||
System.err.println("Usage: java org.apache.hbase.HLog <logfile>");
|
||||
System.exit(-1);
|
||||
}
|
||||
Configuration conf = new HBaseConfiguration();
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
Path logfile = new Path(args[0]);
|
||||
if (!fs.exists(logfile)) {
|
||||
throw new FileNotFoundException(args[0] + " does not exist");
|
||||
}
|
||||
if (!fs.isFile(logfile)) {
|
||||
throw new IOException(args[0] + " is not a file");
|
||||
}
|
||||
Reader log = new SequenceFile.Reader(fs, logfile, conf);
|
||||
try {
|
||||
HLogKey key = new HLogKey();
|
||||
HLogEdit val = new HLogEdit();
|
||||
while(log.next(key, val)) {
|
||||
System.out.println(key.toString() + " " + val.toString());
|
||||
}
|
||||
} finally {
|
||||
log.close();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -34,9 +34,9 @@ public class HLogEdit implements Writable {
|
|||
public HLogEdit() {
|
||||
}
|
||||
|
||||
public HLogEdit(Text column, byte[] bval, long timestamp) {
|
||||
public HLogEdit(Text column, BytesWritable bval, long timestamp) {
|
||||
this.column.set(column);
|
||||
this.val = new BytesWritable(bval);
|
||||
this.val = bval;
|
||||
this.timestamp = timestamp;
|
||||
}
|
||||
|
||||
|
@ -52,6 +52,12 @@ public class HLogEdit implements Writable {
|
|||
return this.timestamp;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getColumn().toString() + " " + this.getTimestamp() + " " +
|
||||
new String(getVal().get()).trim();
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// Writable
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
|
|
@ -66,6 +66,12 @@ public class HLogKey implements WritableComparable {
|
|||
public long getLogSeqNum() {
|
||||
return logSeqNum;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getTablename().toString() + " " + getRegionName().toString() + " " +
|
||||
getRow().toString() + " " + getLogSeqNum();
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// Comparable
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -20,10 +20,11 @@ import org.apache.hadoop.ipc.VersionedProtocol;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
/*******************************************************************************
|
||||
* Clients interact with the HMasterInterface to gain access to meta-level HBase
|
||||
* functionality, like finding an HRegionServer and creating/destroying tables.
|
||||
******************************************************************************/
|
||||
/**
|
||||
* Clients interact with the HMasterInterface to gain access to meta-level
|
||||
* HBase functionality, like finding an HRegionServer and creating/destroying
|
||||
* tables.
|
||||
*/
|
||||
public interface HMasterInterface extends VersionedProtocol {
|
||||
public static final long versionID = 1L; // initial version
|
||||
|
||||
|
@ -33,6 +34,11 @@ public interface HMasterInterface extends VersionedProtocol {
|
|||
|
||||
public void createTable(HTableDescriptor desc) throws IOException;
|
||||
public void deleteTable(Text tableName) throws IOException;
|
||||
|
||||
/**
|
||||
* Shutdown an HBase cluster.
|
||||
*/
|
||||
public void shutdown() throws IOException;
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// These are the method calls of last resort when trying to find an HRegion
|
||||
|
|
|
@ -75,11 +75,15 @@ public class HMemcache {
|
|||
throw new IOException("Snapshot in progress!");
|
||||
}
|
||||
if(memcache.size() == 0) {
|
||||
LOG.debug("memcache empty. Skipping snapshot");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("memcache empty. Skipping snapshot");
|
||||
}
|
||||
return retval;
|
||||
}
|
||||
|
||||
LOG.debug("starting memcache snapshot");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("starting memcache snapshot");
|
||||
}
|
||||
|
||||
retval.memcacheSnapshot = memcache;
|
||||
this.snapshot = memcache;
|
||||
|
@ -87,7 +91,9 @@ public class HMemcache {
|
|||
memcache = new TreeMap<HStoreKey, BytesWritable>();
|
||||
retval.sequenceId = log.startCacheFlush();
|
||||
|
||||
LOG.debug("memcache snapshot complete");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("memcache snapshot complete");
|
||||
}
|
||||
|
||||
return retval;
|
||||
|
||||
|
@ -108,7 +114,9 @@ public class HMemcache {
|
|||
if(snapshot == null) {
|
||||
throw new IOException("Snapshot not present!");
|
||||
}
|
||||
LOG.debug("deleting snapshot");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("deleting snapshot");
|
||||
}
|
||||
|
||||
for(Iterator<TreeMap<HStoreKey, BytesWritable>> it = history.iterator();
|
||||
it.hasNext(); ) {
|
||||
|
@ -121,7 +129,9 @@ public class HMemcache {
|
|||
}
|
||||
this.snapshot = null;
|
||||
|
||||
LOG.debug("snapshot deleted");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("snapshot deleted");
|
||||
}
|
||||
|
||||
} finally {
|
||||
this.locker.writeLock().unlock();
|
||||
|
@ -133,15 +143,15 @@ public class HMemcache {
|
|||
*
|
||||
* Operation uses a write lock.
|
||||
*/
|
||||
public void add(Text row, TreeMap<Text, byte[]> columns, long timestamp) {
|
||||
public void add(Text row, TreeMap<Text, BytesWritable> columns, long timestamp) {
|
||||
this.locker.writeLock().lock();
|
||||
try {
|
||||
for(Iterator<Text> it = columns.keySet().iterator(); it.hasNext(); ) {
|
||||
Text column = it.next();
|
||||
byte[] val = columns.get(column);
|
||||
BytesWritable val = columns.get(column);
|
||||
|
||||
HStoreKey key = new HStoreKey(row, column, timestamp);
|
||||
memcache.put(key, new BytesWritable(val));
|
||||
memcache.put(key, val);
|
||||
}
|
||||
|
||||
} finally {
|
||||
|
@ -154,11 +164,11 @@ public class HMemcache {
|
|||
*
|
||||
* We only need a readlock here.
|
||||
*/
|
||||
public byte[][] get(HStoreKey key, int numVersions) {
|
||||
Vector<byte[]> results = new Vector<byte[]>();
|
||||
public BytesWritable[] get(HStoreKey key, int numVersions) {
|
||||
Vector<BytesWritable> results = new Vector<BytesWritable>();
|
||||
this.locker.readLock().lock();
|
||||
try {
|
||||
Vector<byte[]> result = get(memcache, key, numVersions-results.size());
|
||||
Vector<BytesWritable> result = get(memcache, key, numVersions-results.size());
|
||||
results.addAll(0, result);
|
||||
|
||||
for(int i = history.size()-1; i >= 0; i--) {
|
||||
|
@ -174,22 +184,22 @@ public class HMemcache {
|
|||
return null;
|
||||
|
||||
} else {
|
||||
return (byte[][]) results.toArray(new byte[results.size()][]);
|
||||
return results.toArray(new BytesWritable[results.size()]);
|
||||
}
|
||||
|
||||
} finally {
|
||||
this.locker.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Return all the available columns for the given key. The key indicates a
|
||||
* row and timestamp, but not a column name.
|
||||
*
|
||||
* The returned object should map column names to byte arrays (byte[]).
|
||||
*/
|
||||
public TreeMap<Text, byte[]> getFull(HStoreKey key) throws IOException {
|
||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||
public TreeMap<Text, BytesWritable> getFull(HStoreKey key) throws IOException {
|
||||
TreeMap<Text, BytesWritable> results = new TreeMap<Text, BytesWritable>();
|
||||
this.locker.readLock().lock();
|
||||
try {
|
||||
internalGetFull(memcache, key, results);
|
||||
|
@ -205,7 +215,7 @@ public class HMemcache {
|
|||
}
|
||||
|
||||
void internalGetFull(TreeMap<HStoreKey, BytesWritable> map, HStoreKey key,
|
||||
TreeMap<Text, byte[]> results) {
|
||||
TreeMap<Text, BytesWritable> results) {
|
||||
|
||||
SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(key);
|
||||
|
||||
|
@ -216,7 +226,7 @@ public class HMemcache {
|
|||
if(results.get(itCol) == null
|
||||
&& key.matchesWithoutColumn(itKey)) {
|
||||
BytesWritable val = tailMap.get(itKey);
|
||||
results.put(itCol, val.get());
|
||||
results.put(itCol, val);
|
||||
|
||||
} else if(key.getRow().compareTo(itKey.getRow()) > 0) {
|
||||
break;
|
||||
|
@ -233,8 +243,8 @@ public class HMemcache {
|
|||
* TODO - This is kinda slow. We need a data structure that allows for
|
||||
* proximity-searches, not just precise-matches.
|
||||
*/
|
||||
Vector<byte[]> get(TreeMap<HStoreKey, BytesWritable> map, HStoreKey key, int numVersions) {
|
||||
Vector<byte[]> result = new Vector<byte[]>();
|
||||
Vector<BytesWritable> get(TreeMap<HStoreKey, BytesWritable> map, HStoreKey key, int numVersions) {
|
||||
Vector<BytesWritable> result = new Vector<BytesWritable>();
|
||||
HStoreKey curKey = new HStoreKey(key.getRow(), key.getColumn(), key.getTimestamp());
|
||||
SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(curKey);
|
||||
|
||||
|
@ -242,7 +252,7 @@ public class HMemcache {
|
|||
HStoreKey itKey = it.next();
|
||||
|
||||
if(itKey.matchesRowCol(curKey)) {
|
||||
result.add(tailMap.get(itKey).get());
|
||||
result.add(tailMap.get(itKey));
|
||||
curKey.setVersion(itKey.getTimestamp() - 1);
|
||||
}
|
||||
|
||||
|
@ -256,7 +266,7 @@ public class HMemcache {
|
|||
/**
|
||||
* Return a scanner over the keys in the HMemcache
|
||||
*/
|
||||
public HScannerInterface getScanner(long timestamp, Text targetCols[], Text firstRow)
|
||||
public HInternalScannerInterface getScanner(long timestamp, Text targetCols[], Text firstRow)
|
||||
throws IOException {
|
||||
|
||||
return new HMemcacheScanner(timestamp, targetCols, firstRow);
|
||||
|
@ -280,13 +290,14 @@ public class HMemcache {
|
|||
locker.readLock().lock();
|
||||
try {
|
||||
this.backingMaps = new TreeMap[history.size() + 1];
|
||||
int i = 0;
|
||||
for(Iterator<TreeMap<HStoreKey, BytesWritable>> it = history.iterator();
|
||||
it.hasNext(); ) {
|
||||
|
||||
backingMaps[i++] = it.next();
|
||||
|
||||
//NOTE: Since we iterate through the backing maps from 0 to n, we need
|
||||
// to put the memcache first, the newest history second, ..., etc.
|
||||
|
||||
backingMaps[0] = memcache;
|
||||
for(int i = history.size() - 1; i > 0; i--) {
|
||||
backingMaps[i] = history.elementAt(i);
|
||||
}
|
||||
backingMaps[backingMaps.length - 1] = memcache;
|
||||
|
||||
this.keyIterators = new Iterator[backingMaps.length];
|
||||
this.keys = new HStoreKey[backingMaps.length];
|
||||
|
@ -295,7 +306,7 @@ public class HMemcache {
|
|||
// Generate list of iterators
|
||||
|
||||
HStoreKey firstKey = new HStoreKey(firstRow);
|
||||
for(i = 0; i < backingMaps.length; i++) {
|
||||
for(int i = 0; i < backingMaps.length; i++) {
|
||||
if(firstRow.getLength() != 0) {
|
||||
keyIterators[i] = backingMaps[i].tailMap(firstKey).keySet().iterator();
|
||||
|
||||
|
@ -327,7 +338,8 @@ public class HMemcache {
|
|||
* @return - true if this is the first row
|
||||
*/
|
||||
boolean findFirstRow(int i, Text firstRow) {
|
||||
return ((firstRow.getLength() == 0) || (keys[i].getRow().equals(firstRow)));
|
||||
return ((firstRow.getLength() == 0)
|
||||
|| (keys[i].getRow().toString().startsWith(firstRow.toString())));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -28,7 +28,7 @@ public class HMsg implements Writable {
|
|||
public static final byte MSG_REGION_CLOSE = 2;
|
||||
public static final byte MSG_REGION_MERGE = 3;
|
||||
public static final byte MSG_CALL_SERVER_STARTUP = 4;
|
||||
public static final byte MSG_REGIONSERVER_ALREADY_RUNNING = 5;
|
||||
public static final byte MSG_REGIONSERVER_STOP = 5;
|
||||
public static final byte MSG_REGION_CLOSE_WITHOUT_REPORT = 6;
|
||||
public static final byte MSG_REGION_CLOSE_AND_DELETE = 7;
|
||||
|
||||
|
|
|
@ -24,25 +24,37 @@ import org.apache.hadoop.conf.*;
|
|||
import java.io.*;
|
||||
import java.util.*;
|
||||
|
||||
/*******************************************************************************
|
||||
* HRegion stores data for a certain region of a table. It stores all columns
|
||||
/**
|
||||
* HRegion stores data for a certain region of a table. It stores all columns
|
||||
* for each row. A given table consists of one or more HRegions.
|
||||
*
|
||||
* We maintain multiple HStores for a single HRegion.
|
||||
* <p>We maintain multiple HStores for a single HRegion.
|
||||
*
|
||||
* An HStore is a set of rows with some column data; together, they make up all
|
||||
* the data for the rows.
|
||||
* <p>An HStore is a set of rows with some column data; together,
|
||||
* they make up all the data for the rows.
|
||||
*
|
||||
* Each HRegion has a 'startKey' and 'endKey'.
|
||||
* <p>Each HRegion has a 'startKey' and 'endKey'.
|
||||
*
|
||||
* The first is inclusive, the second is exclusive (except for the final region)
|
||||
* The endKey of region 0 is the same as startKey for region 1 (if it exists).
|
||||
* The startKey for the first region is null.
|
||||
* The endKey for the final region is null.
|
||||
* <p>The first is inclusive, the second is exclusive (except for
|
||||
* the final region) The endKey of region 0 is the same as
|
||||
* startKey for region 1 (if it exists). The startKey for the
|
||||
* first region is null. The endKey for the final region is null.
|
||||
*
|
||||
* The HStores have no locking built-in. All row-level locking and row-level
|
||||
* atomicity is provided by the HRegion.
|
||||
******************************************************************************/
|
||||
* <p>The HStores have no locking built-in. All row-level locking
|
||||
* and row-level atomicity is provided by the HRegion.
|
||||
*
|
||||
* <p>An HRegion is defined by its table and its key extent.
|
||||
*
|
||||
* <p>It consists of at least one HStore. The number of HStores should be
|
||||
* configurable, so that data which is accessed together is stored in the same
|
||||
* HStore. Right now, we approximate that by building a single HStore for
|
||||
* each column family. (This config info will be communicated via the
|
||||
* tabledesc.)
|
||||
*
|
||||
* The HTableDescriptor contains metainfo about the HRegion's table.
|
||||
* regionName is a unique identifier for this HRegion. (startKey, endKey]
|
||||
* defines the keyspace for this HRegion.
|
||||
*/
|
||||
public class HRegion implements HConstants {
|
||||
static String SPLITDIR = "splits";
|
||||
static String MERGEDIR = "merges";
|
||||
|
@ -53,8 +65,9 @@ public class HRegion implements HConstants {
|
|||
private static final Log LOG = LogFactory.getLog(HRegion.class);
|
||||
|
||||
/**
|
||||
* Merge two HRegions. They must be available on the current HRegionServer.
|
||||
* Returns a brand-new active HRegion, also running on the current HRegionServer.
|
||||
* Merge two HRegions. They must be available on the current
|
||||
* HRegionServer. Returns a brand-new active HRegion, also
|
||||
* running on the current HRegionServer.
|
||||
*/
|
||||
public static HRegion closeAndMerge(HRegion srcA, HRegion srcB) throws IOException {
|
||||
|
||||
|
@ -110,7 +123,9 @@ public class HRegion implements HConstants {
|
|||
// Flush each of the sources, and merge their files into a single
|
||||
// target for each column family.
|
||||
|
||||
LOG.debug("flushing and getting file names for region " + srcA.getRegionName());
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("flushing and getting file names for region " + srcA.getRegionName());
|
||||
}
|
||||
|
||||
TreeSet<HStoreFile> alreadyMerged = new TreeSet<HStoreFile>();
|
||||
TreeMap<Text, Vector<HStoreFile>> filesToMerge = new TreeMap<Text, Vector<HStoreFile>>();
|
||||
|
@ -124,7 +139,9 @@ public class HRegion implements HConstants {
|
|||
v.add(src);
|
||||
}
|
||||
|
||||
LOG.debug("flushing and getting file names for region " + srcB.getRegionName());
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("flushing and getting file names for region " + srcB.getRegionName());
|
||||
}
|
||||
|
||||
for(Iterator<HStoreFile> it = srcB.flushcache(true).iterator(); it.hasNext(); ) {
|
||||
HStoreFile src = it.next();
|
||||
|
@ -136,7 +153,9 @@ public class HRegion implements HConstants {
|
|||
v.add(src);
|
||||
}
|
||||
|
||||
LOG.debug("merging stores");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("merging stores");
|
||||
}
|
||||
|
||||
for(Iterator<Text> it = filesToMerge.keySet().iterator(); it.hasNext(); ) {
|
||||
Text colFamily = it.next();
|
||||
|
@ -152,8 +171,10 @@ public class HRegion implements HConstants {
|
|||
// Now close the source HRegions for good, and repeat the above to take care
|
||||
// of any last-minute inserts
|
||||
|
||||
LOG.debug("flushing changes since start of merge for region "
|
||||
+ srcA.getRegionName());
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("flushing changes since start of merge for region "
|
||||
+ srcA.getRegionName());
|
||||
}
|
||||
|
||||
filesToMerge.clear();
|
||||
for(Iterator<HStoreFile> it = srcA.close().iterator(); it.hasNext(); ) {
|
||||
|
@ -169,8 +190,10 @@ public class HRegion implements HConstants {
|
|||
}
|
||||
}
|
||||
|
||||
LOG.debug("flushing changes since start of merge for region "
|
||||
+ srcB.getRegionName());
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("flushing changes since start of merge for region "
|
||||
+ srcB.getRegionName());
|
||||
}
|
||||
|
||||
for(Iterator<HStoreFile> it = srcB.close().iterator(); it.hasNext(); ) {
|
||||
HStoreFile src = it.next();
|
||||
|
@ -185,7 +208,9 @@ public class HRegion implements HConstants {
|
|||
}
|
||||
}
|
||||
|
||||
LOG.debug("merging changes since start of merge");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("merging changes since start of merge");
|
||||
}
|
||||
|
||||
for(Iterator<Text> it = filesToMerge.keySet().iterator(); it.hasNext(); ) {
|
||||
Text colFamily = it.next();
|
||||
|
@ -217,8 +242,8 @@ public class HRegion implements HConstants {
|
|||
TreeMap<Text, Long> rowsToLocks = new TreeMap<Text, Long>();
|
||||
TreeMap<Long, Text> locksToRows = new TreeMap<Long, Text>();
|
||||
TreeMap<Text, HStore> stores = new TreeMap<Text, HStore>();
|
||||
TreeMap<Long, TreeMap<Text, byte[]>> targetColumns
|
||||
= new TreeMap<Long, TreeMap<Text, byte[]>>();
|
||||
TreeMap<Long, TreeMap<Text, BytesWritable>> targetColumns
|
||||
= new TreeMap<Long, TreeMap<Text, BytesWritable>>();
|
||||
|
||||
HMemcache memcache = new HMemcache();
|
||||
|
||||
|
@ -251,40 +276,25 @@ public class HRegion implements HConstants {
|
|||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
/**
|
||||
* An HRegion is defined by its table and its key extent.
|
||||
*
|
||||
* It consists of at least one HStore. The number of HStores should be
|
||||
* configurable, so that data which is accessed together is stored in the same
|
||||
* HStore. Right now, we approximate that by building a single HStore for
|
||||
* each column family. (This config info will be communicated via the
|
||||
* tabledesc.)
|
||||
* HRegion constructor.
|
||||
*
|
||||
* The HLog is the outbound log for any updates to the HRegion. (There's a
|
||||
* single HLog for all the HRegions on a single HRegionServer.)
|
||||
*
|
||||
* The HTableDescriptor contains metainfo about the HRegion's table.
|
||||
*
|
||||
* regionName is a unique identifier for this HRegion.
|
||||
*
|
||||
* (startKey, endKey] defines the keyspace for this HRegion. NULL values
|
||||
* indicate we're at the start or end of the table.
|
||||
*
|
||||
* fs is the filesystem. regiondir is where the HRegion is stored.
|
||||
*
|
||||
* logfile is a logfile from the previous execution that's custom-computed for
|
||||
* this HRegion. The HRegionServer computes and sorts the appropriate log
|
||||
* info for this HRegion.
|
||||
*
|
||||
* conf is global configuration settings.
|
||||
*
|
||||
* If there are initial files (implying that the HRegion is new), then read
|
||||
* them from the supplied path.
|
||||
*
|
||||
* If there is a previous log file (implying that the HRegion has been
|
||||
* written-to before), then read it from the supplied path.
|
||||
* @param log The HLog is the outbound log for any updates to the HRegion
|
||||
* (There's a single HLog for all the HRegions on a single HRegionServer.)
|
||||
* The log file is a logfile from the previous execution that's
|
||||
* custom-computed for this HRegion. The HRegionServer computes and sorts the
|
||||
* appropriate log info for this HRegion. If there is a previous log file
|
||||
* (implying that the HRegion has been written-to before), then read it from
|
||||
* the supplied path.
|
||||
* @param fs is the filesystem.
|
||||
* @param dir dir is where the HRegion is stored.
|
||||
* @param conf is global configuration settings.
|
||||
* @param initialFiles If there are initial files (implying that the HRegion
|
||||
* is new), then read them from the supplied path.
|
||||
* @throws IOException
|
||||
*/
|
||||
public HRegion(Path dir, HLog log, FileSystem fs, Configuration conf,
|
||||
HRegionInfo regionInfo, Path initialFiles, Path oldLogFile) throws IOException {
|
||||
HRegionInfo regionInfo, Path initialFiles, Path oldLogFile)
|
||||
throws IOException {
|
||||
|
||||
this.dir = dir;
|
||||
this.log = log;
|
||||
|
@ -308,33 +318,28 @@ public class HRegion implements HConstants {
|
|||
}
|
||||
|
||||
// Load in all the HStores.
|
||||
|
||||
for(Iterator<Text> it = this.regionInfo.tableDesc.families().iterator();
|
||||
it.hasNext(); ) {
|
||||
|
||||
it.hasNext(); ) {
|
||||
Text colFamily = HStoreKey.extractFamily(it.next());
|
||||
stores.put(colFamily, new HStore(dir, this.regionInfo.regionName, colFamily,
|
||||
this.regionInfo.tableDesc.getMaxVersions(), fs, oldLogFile, conf));
|
||||
stores.put(colFamily, new HStore(dir, this.regionInfo.regionName,
|
||||
colFamily, this.regionInfo.tableDesc.getMaxVersions(), fs,
|
||||
oldLogFile, conf));
|
||||
}
|
||||
|
||||
// Get rid of any splits or merges that were lost in-progress
|
||||
|
||||
Path splits = new Path(regiondir, SPLITDIR);
|
||||
if(fs.exists(splits)) {
|
||||
if (fs.exists(splits)) {
|
||||
fs.delete(splits);
|
||||
}
|
||||
|
||||
Path merges = new Path(regiondir, MERGEDIR);
|
||||
if(fs.exists(merges)) {
|
||||
if (fs.exists(merges)) {
|
||||
fs.delete(merges);
|
||||
}
|
||||
|
||||
this.maxUnflushedEntries = conf.getInt("hbase.hregion.maxunflushed", 10000);
|
||||
|
||||
// HRegion is ready to go!
|
||||
|
||||
this.writestate.writesOngoing = false;
|
||||
|
||||
LOG.info("region " + this.regionInfo.regionName + " available");
|
||||
}
|
||||
|
||||
|
@ -445,8 +450,10 @@ public class HRegion implements HConstants {
|
|||
for(Iterator<HStoreFile> it = hstoreFilesToSplit.iterator(); it.hasNext(); ) {
|
||||
HStoreFile hsf = it.next();
|
||||
|
||||
LOG.debug("splitting HStore " + hsf.getRegionName() + "/" + hsf.getColFamily()
|
||||
+ "/" + hsf.fileId());
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("splitting HStore " + hsf.getRegionName() + "/"
|
||||
+ hsf.getColFamily() + "/" + hsf.fileId());
|
||||
}
|
||||
|
||||
HStoreFile dstA = new HStoreFile(conf, splits, regionAInfo.regionName,
|
||||
hsf.getColFamily(), Math.abs(rand.nextLong()));
|
||||
|
@ -466,8 +473,10 @@ public class HRegion implements HConstants {
|
|||
HStoreFile hsf = it.next();
|
||||
|
||||
if(! alreadySplit.contains(hsf)) {
|
||||
LOG.debug("splitting HStore " + hsf.getRegionName() + "/" + hsf.getColFamily()
|
||||
+ "/" + hsf.fileId());
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("splitting HStore " + hsf.getRegionName() + "/"
|
||||
+ hsf.getColFamily() + "/" + hsf.fileId());
|
||||
}
|
||||
|
||||
HStoreFile dstA = new HStoreFile(conf, splits, regionAInfo.regionName,
|
||||
hsf.getColFamily(), Math.abs(rand.nextLong()));
|
||||
|
@ -672,7 +681,9 @@ public class HRegion implements HConstants {
|
|||
}
|
||||
|
||||
if(! shouldFlush) {
|
||||
LOG.debug("not flushing cache for region " + this.regionInfo.regionName);
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("not flushing cache for region " + this.regionInfo.regionName);
|
||||
}
|
||||
return null;
|
||||
|
||||
} else {
|
||||
|
@ -715,7 +726,9 @@ public class HRegion implements HConstants {
|
|||
Vector<HStoreFile> internalFlushcache() throws IOException {
|
||||
Vector<HStoreFile> allHStoreFiles = new Vector<HStoreFile>();
|
||||
|
||||
LOG.debug("flushing cache for region " + this.regionInfo.regionName);
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("flushing cache for region " + this.regionInfo.regionName);
|
||||
}
|
||||
|
||||
// We pass the log to the HMemcache, so we can lock down
|
||||
// both simultaneously. We only have to do this for a moment:
|
||||
|
@ -728,7 +741,9 @@ public class HRegion implements HConstants {
|
|||
// object stored that must be explicitly cleaned up using
|
||||
// a call to deleteSnapshot().
|
||||
|
||||
LOG.debug("starting memcache snapshot");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("starting memcache snapshot");
|
||||
}
|
||||
|
||||
HMemcache.Snapshot retval = memcache.snapshotMemcacheForLog(log);
|
||||
TreeMap<HStoreKey, BytesWritable> memcacheSnapshot = retval.memcacheSnapshot;
|
||||
|
@ -745,7 +760,9 @@ public class HRegion implements HConstants {
|
|||
|
||||
// A. Flush memcache to all the HStores.
|
||||
|
||||
LOG.debug("flushing memcache to HStores");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("flushing memcache to HStores");
|
||||
}
|
||||
|
||||
for(Iterator<HStore> it = stores.values().iterator(); it.hasNext(); ) {
|
||||
HStore hstore = it.next();
|
||||
|
@ -760,7 +777,9 @@ public class HRegion implements HConstants {
|
|||
// and that all updates to the log for this regionName that have lower
|
||||
// log-sequence-ids can be safely ignored.
|
||||
|
||||
LOG.debug("writing flush cache complete to log");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("writing flush cache complete to log");
|
||||
}
|
||||
|
||||
log.completeCacheFlush(this.regionInfo.regionName,
|
||||
regionInfo.tableDesc.getName(), logCacheFlushId);
|
||||
|
@ -768,11 +787,15 @@ public class HRegion implements HConstants {
|
|||
// C. Delete the now-irrelevant memcache snapshot; its contents have been
|
||||
// dumped to disk-based HStores.
|
||||
|
||||
LOG.debug("deleting memcache snapshot");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("deleting memcache snapshot");
|
||||
}
|
||||
|
||||
memcache.deleteSnapshot();
|
||||
|
||||
LOG.debug("cache flush complete for region " + this.regionInfo.regionName);
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("cache flush complete for region " + this.regionInfo.regionName);
|
||||
}
|
||||
|
||||
this.commitsSinceFlush = 0;
|
||||
return allHStoreFiles;
|
||||
|
@ -783,8 +806,8 @@ public class HRegion implements HConstants {
|
|||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
/** Fetch a single data item. */
|
||||
public byte[] get(Text row, Text column) throws IOException {
|
||||
byte results[][] = get(row, column, Long.MAX_VALUE, 1);
|
||||
public BytesWritable get(Text row, Text column) throws IOException {
|
||||
BytesWritable[] results = get(row, column, Long.MAX_VALUE, 1);
|
||||
if(results == null) {
|
||||
return null;
|
||||
|
||||
|
@ -794,12 +817,12 @@ public class HRegion implements HConstants {
|
|||
}
|
||||
|
||||
/** Fetch multiple versions of a single data item */
|
||||
public byte[][] get(Text row, Text column, int numVersions) throws IOException {
|
||||
public BytesWritable[] get(Text row, Text column, int numVersions) throws IOException {
|
||||
return get(row, column, Long.MAX_VALUE, numVersions);
|
||||
}
|
||||
|
||||
/** Fetch multiple versions of a single data item, with timestamp. */
|
||||
public byte[][] get(Text row, Text column, long timestamp, int numVersions)
|
||||
public BytesWritable[] get(Text row, Text column, long timestamp, int numVersions)
|
||||
throws IOException {
|
||||
|
||||
if(writestate.closed) {
|
||||
|
@ -825,11 +848,11 @@ public class HRegion implements HConstants {
|
|||
|
||||
// Private implementation: get the value for the indicated HStoreKey
|
||||
|
||||
private byte[][] get(HStoreKey key, int numVersions) throws IOException {
|
||||
private BytesWritable[] get(HStoreKey key, int numVersions) throws IOException {
|
||||
|
||||
// Check the memcache
|
||||
|
||||
byte[][] result = memcache.get(key, numVersions);
|
||||
BytesWritable[] result = memcache.get(key, numVersions);
|
||||
if(result != null) {
|
||||
return result;
|
||||
}
|
||||
|
@ -855,10 +878,10 @@ public class HRegion implements HConstants {
|
|||
* determine which column groups are useful for that row. That would let us
|
||||
* avoid a bunch of disk activity.
|
||||
*/
|
||||
public TreeMap<Text, byte[]> getFull(Text row) throws IOException {
|
||||
public TreeMap<Text, BytesWritable> getFull(Text row) throws IOException {
|
||||
HStoreKey key = new HStoreKey(row, System.currentTimeMillis());
|
||||
|
||||
TreeMap<Text, byte[]> memResult = memcache.getFull(key);
|
||||
TreeMap<Text, BytesWritable> memResult = memcache.getFull(key);
|
||||
for(Iterator<Text> it = stores.keySet().iterator(); it.hasNext(); ) {
|
||||
Text colFamily = it.next();
|
||||
HStore targetStore = stores.get(colFamily);
|
||||
|
@ -871,13 +894,13 @@ public class HRegion implements HConstants {
|
|||
* Return an iterator that scans over the HRegion, returning the indicated
|
||||
* columns. This Iterator must be closed by the caller.
|
||||
*/
|
||||
public HScannerInterface getScanner(Text cols[], Text firstRow) throws IOException {
|
||||
public HInternalScannerInterface getScanner(Text[] cols, Text firstRow) throws IOException {
|
||||
TreeSet<Text> families = new TreeSet<Text>();
|
||||
for(int i = 0; i < cols.length; i++) {
|
||||
families.add(HStoreKey.extractFamily(cols[i]));
|
||||
}
|
||||
|
||||
HStore storelist[] = new HStore[families.size()];
|
||||
HStore[] storelist = new HStore[families.size()];
|
||||
int i = 0;
|
||||
for(Iterator<Text> it = families.iterator(); it.hasNext(); ) {
|
||||
Text family = it.next();
|
||||
|
@ -917,21 +940,11 @@ public class HRegion implements HConstants {
|
|||
* This method really just tests the input, then calls an internal localput()
|
||||
* method.
|
||||
*/
|
||||
public void put(long lockid, Text targetCol, byte[] val) throws IOException {
|
||||
if(val.length == HStoreKey.DELETE_BYTES.length) {
|
||||
boolean matches = true;
|
||||
for(int i = 0; i < val.length; i++) {
|
||||
if(val[i] != HStoreKey.DELETE_BYTES[i]) {
|
||||
matches = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if(matches) {
|
||||
public void put(long lockid, Text targetCol, BytesWritable val) throws IOException {
|
||||
if(val.getSize() == DELETE_BYTES.getSize()
|
||||
&& val.compareTo(DELETE_BYTES) == 0) {
|
||||
throw new IOException("Cannot insert value: " + val);
|
||||
}
|
||||
}
|
||||
|
||||
localput(lockid, targetCol, val);
|
||||
}
|
||||
|
||||
|
@ -939,7 +952,7 @@ public class HRegion implements HConstants {
|
|||
* Delete a value or write a value. This is a just a convenience method for put().
|
||||
*/
|
||||
public void delete(long lockid, Text targetCol) throws IOException {
|
||||
localput(lockid, targetCol, HStoreKey.DELETE_BYTES);
|
||||
localput(lockid, targetCol, DELETE_BYTES);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -949,7 +962,7 @@ public class HRegion implements HConstants {
|
|||
* a per-row pending area, until a commit() or abort() call is received.
|
||||
* (Or until the user's write-lock expires.)
|
||||
*/
|
||||
void localput(long lockid, Text targetCol, byte[] val) throws IOException {
|
||||
void localput(long lockid, Text targetCol, BytesWritable val) throws IOException {
|
||||
checkColumn(targetCol);
|
||||
|
||||
Text row = getRowFromLock(lockid);
|
||||
|
@ -971,9 +984,9 @@ public class HRegion implements HConstants {
|
|||
+ " unexpected aborted by another thread");
|
||||
}
|
||||
|
||||
TreeMap<Text, byte[]> targets = targetColumns.get(lockid);
|
||||
TreeMap<Text, BytesWritable> targets = targetColumns.get(lockid);
|
||||
if(targets == null) {
|
||||
targets = new TreeMap<Text, byte[]>();
|
||||
targets = new TreeMap<Text, BytesWritable>();
|
||||
targetColumns.put(lockid, targets);
|
||||
}
|
||||
targets.put(targetCol, val);
|
||||
|
@ -1132,42 +1145,79 @@ public class HRegion implements HConstants {
|
|||
/*******************************************************************************
|
||||
* HScanner is an iterator through a bunch of rows in an HRegion.
|
||||
******************************************************************************/
|
||||
private class HScanner implements HScannerInterface {
|
||||
HScannerInterface scanners[] = null;
|
||||
TreeMap<Text, byte[]> resultSets[] = null;
|
||||
HStoreKey keys[] = null;
|
||||
private class HScanner implements HInternalScannerInterface {
|
||||
private HInternalScannerInterface[] scanners;
|
||||
private TreeMap<Text, BytesWritable>[] resultSets;
|
||||
private HStoreKey[] keys;
|
||||
private boolean wildcardMatch;
|
||||
private boolean multipleMatchers;
|
||||
|
||||
/** Create an HScanner with a handle on many HStores. */
|
||||
@SuppressWarnings("unchecked")
|
||||
public HScanner(Text cols[], Text firstRow, HMemcache memcache, HStore stores[]) throws IOException {
|
||||
public HScanner(Text[] cols, Text firstRow, HMemcache memcache, HStore[] stores) throws IOException {
|
||||
long scanTime = System.currentTimeMillis();
|
||||
this.scanners = new HScannerInterface[stores.length + 1];
|
||||
this.keys = new HStoreKey[scanners.length];
|
||||
this.scanners = new HInternalScannerInterface[stores.length + 1];
|
||||
this.resultSets = new TreeMap[scanners.length];
|
||||
this.keys = new HStoreKey[scanners.length];
|
||||
this.wildcardMatch = false;
|
||||
this.multipleMatchers = false;
|
||||
|
||||
// Advance to the first key in each store.
|
||||
// All results will match the required column-set and scanTime.
|
||||
|
||||
// NOTE: the memcache scanner should be the first scanner
|
||||
|
||||
for(int i = 0; i < stores.length; i++) {
|
||||
scanners[i] = stores[i].getScanner(scanTime, cols, firstRow);
|
||||
HInternalScannerInterface scanner =
|
||||
memcache.getScanner(scanTime, cols, firstRow);
|
||||
|
||||
if(scanner.isWildcardScanner()) {
|
||||
this.wildcardMatch = true;
|
||||
}
|
||||
if(scanner.isMultipleMatchScanner()) {
|
||||
this.multipleMatchers = true;
|
||||
}
|
||||
scanners[0] = scanner;
|
||||
|
||||
for(int i = 0; i < stores.length; i++) {
|
||||
scanner = stores[i].getScanner(scanTime, cols, firstRow);
|
||||
if(scanner.isWildcardScanner()) {
|
||||
this.wildcardMatch = true;
|
||||
}
|
||||
if(scanner.isMultipleMatchScanner()) {
|
||||
this.multipleMatchers = true;
|
||||
}
|
||||
scanners[i + 1] = scanner;
|
||||
}
|
||||
scanners[scanners.length-1] = memcache.getScanner(scanTime, cols, firstRow);
|
||||
|
||||
for(int i = 0; i < scanners.length; i++) {
|
||||
keys[i] = new HStoreKey();
|
||||
resultSets[i] = new TreeMap<Text, byte[]>();
|
||||
resultSets[i] = new TreeMap<Text, BytesWritable>();
|
||||
|
||||
if(! scanners[i].next(keys[i], resultSets[i])) {
|
||||
if(scanners[i] != null && ! scanners[i].next(keys[i], resultSets[i])) {
|
||||
closeScanner(i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.apache.hadoop.hbase.HInternalScannerInterface#isWildcardScanner()
|
||||
*/
|
||||
public boolean isWildcardScanner() {
|
||||
return wildcardMatch;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.apache.hadoop.hbase.HInternalScannerInterface#isMultipleMatchScanner()
|
||||
*/
|
||||
public boolean isMultipleMatchScanner() {
|
||||
return multipleMatchers;
|
||||
}
|
||||
|
||||
/**
|
||||
* Grab the next row's worth of values. The HScanner will return the most
|
||||
* recent data value for each row that is not newer than the target time.
|
||||
*/
|
||||
public boolean next(HStoreKey key, TreeMap<Text, byte[]> results) throws IOException {
|
||||
public boolean next(HStoreKey key, TreeMap<Text, BytesWritable> results) throws IOException {
|
||||
|
||||
// Find the lowest-possible key.
|
||||
|
||||
|
@ -1195,11 +1245,34 @@ public class HRegion implements HConstants {
|
|||
|
||||
for(int i = 0; i < scanners.length; i++) {
|
||||
while((scanners[i] != null)
|
||||
&& (keys[i].getRow().compareTo(chosenRow) == 0)
|
||||
&& (keys[i].getTimestamp() == chosenTimestamp)) {
|
||||
&& (keys[i].getRow().compareTo(chosenRow) == 0)) {
|
||||
|
||||
results.putAll(resultSets[i]);
|
||||
insertedItem = true;
|
||||
|
||||
// If we are doing a wild card match or there are multiple matchers
|
||||
// per column, we need to scan all the older versions of this row
|
||||
// to pick up the rest of the family members
|
||||
|
||||
if(!wildcardMatch
|
||||
&& !multipleMatchers
|
||||
&& (keys[i].getTimestamp() != chosenTimestamp)) {
|
||||
break;
|
||||
}
|
||||
|
||||
// NOTE: We used to do results.putAll(resultSets[i]);
|
||||
// but this had the effect of overwriting newer
|
||||
// values with older ones. So now we only insert
|
||||
// a result if the map does not contain the key.
|
||||
|
||||
for(Iterator<Map.Entry<Text, BytesWritable>> it
|
||||
= resultSets[i].entrySet().iterator();
|
||||
it.hasNext(); ) {
|
||||
|
||||
Map.Entry<Text, BytesWritable> e = it.next();
|
||||
if(!results.containsKey(e.getKey())) {
|
||||
results.put(e.getKey(), e.getValue());
|
||||
insertedItem = true;
|
||||
}
|
||||
}
|
||||
|
||||
resultSets[i].clear();
|
||||
if(! scanners[i].next(keys[i], resultSets[i])) {
|
||||
|
|
|
@ -15,13 +15,20 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
|
||||
/**
|
||||
* HRegion information.
|
||||
* Contains HRegion id, start and end keys, a reference to this
|
||||
* HRegions' table descriptor, etc.
|
||||
*/
|
||||
public class HRegionInfo implements Writable {
|
||||
public long regionId;
|
||||
public HTableDescriptor tableDesc;
|
||||
|
@ -36,9 +43,20 @@ public class HRegionInfo implements Writable {
|
|||
this.endKey = new Text();
|
||||
this.regionName = new Text();
|
||||
}
|
||||
|
||||
public HRegionInfo(final byte [] serializedBytes) {
|
||||
this();
|
||||
try {
|
||||
readFields(new DataInputStream(
|
||||
new ByteArrayInputStream(serializedBytes)));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public HRegionInfo(long regionId, HTableDescriptor tableDesc, Text startKey,
|
||||
Text endKey) throws IllegalArgumentException {
|
||||
public HRegionInfo(long regionId, HTableDescriptor tableDesc,
|
||||
Text startKey, Text endKey)
|
||||
throws IllegalArgumentException {
|
||||
|
||||
this.regionId = regionId;
|
||||
|
||||
|
@ -58,8 +76,16 @@ public class HRegionInfo implements Writable {
|
|||
this.endKey.set(endKey);
|
||||
}
|
||||
|
||||
this.regionName = new Text(tableDesc.getName() + "_"
|
||||
+ (startKey == null ? "" : startKey.toString()) + "_" + regionId);
|
||||
this.regionName = new Text(tableDesc.getName() + "_" +
|
||||
(startKey == null ? "" : startKey.toString()) + "_" +
|
||||
regionId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "regionname: " + this.regionName.toString() + ", startKey: <" +
|
||||
this.startKey.toString() + ">, tableDesc: {" +
|
||||
this.tableDesc.toString() + "}";
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
@ -81,4 +107,4 @@ public class HRegionInfo implements Writable {
|
|||
this.endKey.readFields(in);
|
||||
this.regionName.readFields(in);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -69,7 +69,7 @@ public interface HRegionInterface extends VersionedProtocol {
|
|||
* @param columns - columns to scan
|
||||
* @param startRow - starting row to scan
|
||||
*
|
||||
* @param scannerId - scanner identifier used in other calls
|
||||
* @return scannerId - scanner identifier used in other calls
|
||||
* @throws IOException
|
||||
*/
|
||||
public long openScanner(Text regionName, Text[] columns, Text startRow) throws IOException;
|
||||
|
|
|
@ -20,6 +20,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.io.*;
|
||||
import org.apache.hadoop.fs.*;
|
||||
import org.apache.hadoop.ipc.*;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.conf.*;
|
||||
|
||||
import java.io.*;
|
||||
|
@ -45,7 +46,7 @@ public class HRegionServer
|
|||
|
||||
private static final Log LOG = LogFactory.getLog(HRegionServer.class);
|
||||
|
||||
private boolean stopRequested;
|
||||
private volatile boolean stopRequested;
|
||||
private Path regionDir;
|
||||
private HServerAddress address;
|
||||
private Configuration conf;
|
||||
|
@ -82,12 +83,10 @@ public class HRegionServer
|
|||
long startTime = System.currentTimeMillis();
|
||||
|
||||
// Grab a list of regions to check
|
||||
|
||||
Vector<HRegion> checkSplit = new Vector<HRegion>();
|
||||
locker.readLock().lock();
|
||||
try {
|
||||
checkSplit.addAll(regions.values());
|
||||
|
||||
} finally {
|
||||
locker.readLock().unlock();
|
||||
}
|
||||
|
@ -132,13 +131,15 @@ public class HRegionServer
|
|||
= (oldRegion.find(META_TABLE_NAME.toString()) == 0)
|
||||
? ROOT_TABLE_NAME : META_TABLE_NAME;
|
||||
|
||||
LOG.debug("region split complete. updating meta");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("region split complete. updating meta");
|
||||
}
|
||||
|
||||
client.openTable(tableToUpdate);
|
||||
long lockid = client.startUpdate(oldRegion);
|
||||
client.delete(lockid, META_COL_REGIONINFO);
|
||||
client.delete(lockid, META_COL_SERVER);
|
||||
client.delete(lockid, META_COL_STARTCODE);
|
||||
client.delete(lockid, COL_REGIONINFO);
|
||||
client.delete(lockid, COL_SERVER);
|
||||
client.delete(lockid, COL_STARTCODE);
|
||||
client.commit(lockid);
|
||||
|
||||
for(int i = 0; i < newRegions.length; i++) {
|
||||
|
@ -147,13 +148,15 @@ public class HRegionServer
|
|||
newRegions[i].getRegionInfo().write(out);
|
||||
|
||||
lockid = client.startUpdate(newRegions[i].getRegionName());
|
||||
client.put(lockid, META_COL_REGIONINFO, bytes.toByteArray());
|
||||
client.put(lockid, COL_REGIONINFO, bytes.toByteArray());
|
||||
client.commit(lockid);
|
||||
}
|
||||
|
||||
// Now tell the master about the new regions
|
||||
|
||||
LOG.debug("reporting region split to master");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("reporting region split to master");
|
||||
}
|
||||
|
||||
reportSplit(newRegions[0].getRegionInfo(), newRegions[1].getRegionInfo());
|
||||
|
||||
|
@ -172,18 +175,24 @@ public class HRegionServer
|
|||
}
|
||||
|
||||
// Sleep
|
||||
|
||||
long waitTime =
|
||||
splitCheckFrequency - (System.currentTimeMillis() - startTime);
|
||||
|
||||
if(waitTime > 0) {
|
||||
long waitTime = stopRequested ? 0
|
||||
: splitCheckFrequency - (System.currentTimeMillis() - startTime);
|
||||
if (waitTime > 0) {
|
||||
try {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Sleep splitChecker");
|
||||
}
|
||||
Thread.sleep(waitTime);
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Wake splitChecker");
|
||||
}
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
}
|
||||
}
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("splitChecker exiting");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -221,18 +230,25 @@ public class HRegionServer
|
|||
}
|
||||
|
||||
// Sleep
|
||||
|
||||
long waitTime =
|
||||
threadWakeFrequency - (System.currentTimeMillis() - startTime);
|
||||
long waitTime = stopRequested ? 0
|
||||
: threadWakeFrequency - (System.currentTimeMillis() - startTime);
|
||||
|
||||
if(waitTime > 0) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Sleep cacheFlusher");
|
||||
}
|
||||
try {
|
||||
Thread.sleep(waitTime);
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Wake cacheFlusher");
|
||||
}
|
||||
}
|
||||
}
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("cacheFlusher exiting");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -249,24 +265,31 @@ public class HRegionServer
|
|||
private class LogRoller implements Runnable {
|
||||
public void run() {
|
||||
while(! stopRequested) {
|
||||
|
||||
// If the number of log entries is high enough, roll the log. This is a
|
||||
// very fast operation, but should not be done too frequently.
|
||||
|
||||
if(log.getNumEntries() > maxLogEntries) {
|
||||
try {
|
||||
log.rollWriter();
|
||||
|
||||
} catch(IOException iex) {
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
Thread.sleep(threadWakeFrequency);
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
|
||||
if(!stopRequested) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Sleep logRoller");
|
||||
}
|
||||
try {
|
||||
Thread.sleep(threadWakeFrequency);
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Wake logRoller");
|
||||
}
|
||||
}
|
||||
}
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("logRoller exiting");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -290,11 +313,11 @@ public class HRegionServer
|
|||
}
|
||||
|
||||
/** Start a HRegionServer at an indicated location */
|
||||
public HRegionServer(Path regionDir, HServerAddress address, Configuration conf)
|
||||
throws IOException {
|
||||
public HRegionServer(Path regionDir, HServerAddress address,
|
||||
Configuration conf)
|
||||
throws IOException {
|
||||
|
||||
// Basic setup
|
||||
|
||||
this.stopRequested = false;
|
||||
this.regionDir = regionDir;
|
||||
this.conf = conf;
|
||||
|
@ -302,69 +325,70 @@ public class HRegionServer
|
|||
this.regions = new TreeMap<Text, HRegion>();
|
||||
this.locker = new ReentrantReadWriteLock();
|
||||
this.outboundMsgs = new Vector<HMsg>();
|
||||
this.scanners = Collections.synchronizedMap(new TreeMap<Text, HScannerInterface>());
|
||||
this.scanners =
|
||||
Collections.synchronizedMap(new TreeMap<Text, HInternalScannerInterface>());
|
||||
|
||||
// Config'ed params
|
||||
|
||||
this.threadWakeFrequency = conf.getLong("hbase.hregionserver.thread.wakefrequency", 10 * 1000);
|
||||
this.maxLogEntries = conf.getInt("hbase.hregionserver.maxlogentries", 30 * 1000);
|
||||
this.msgInterval = conf.getLong("hbase.hregionserver.msginterval", 15 * 1000);
|
||||
this.splitCheckFrequency = conf.getLong("hbase.hregionserver.thread.splitcheckfrequency", 60 * 1000);
|
||||
this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000);
|
||||
this.maxLogEntries = conf.getInt("hbase.regionserver.maxlogentries", 30 * 1000);
|
||||
this.msgInterval = conf.getLong("hbase.regionserver.msginterval",
|
||||
15 * 1000);
|
||||
this.splitCheckFrequency =
|
||||
conf.getLong("hbase.regionserver.thread.splitcheckfrequency", 60 * 1000);
|
||||
|
||||
// Cache flushing
|
||||
|
||||
this.cacheFlusher = new Flusher();
|
||||
this.cacheFlusherThread = new Thread(cacheFlusher, "HRegionServer.cacheFlusher");
|
||||
this.cacheFlusherThread =
|
||||
new Thread(cacheFlusher, "HRegionServer.cacheFlusher");
|
||||
|
||||
// Check regions to see if they need to be split
|
||||
|
||||
this.splitChecker = new SplitChecker();
|
||||
this.splitCheckerThread = new Thread(splitChecker, "HRegionServer.splitChecker");
|
||||
this.splitCheckerThread =
|
||||
new Thread(splitChecker, "HRegionServer.splitChecker");
|
||||
|
||||
// Process requests from Master
|
||||
|
||||
this.toDo = new Vector<HMsg>();
|
||||
this.worker = new Worker();
|
||||
this.workerThread = new Thread(worker, "HRegionServer.worker");
|
||||
|
||||
try {
|
||||
|
||||
// Server to handle client requests
|
||||
|
||||
this.server = RPC.getServer(this, address.getBindAddress().toString(),
|
||||
address.getPort(), conf.getInt("hbase.hregionserver.handler.count", 10), false, conf);
|
||||
address.getPort(), conf.getInt("hbase.regionserver.handler.count", 10),
|
||||
false, conf);
|
||||
|
||||
this.address = new HServerAddress(server.getListenerAddress());
|
||||
|
||||
// Local file paths
|
||||
|
||||
String serverName = this.address.getBindAddress() + "_" + this.address.getPort();
|
||||
String serverName =
|
||||
this.address.getBindAddress() + "_" + this.address.getPort();
|
||||
Path newlogdir = new Path(regionDir, "log" + "_" + serverName);
|
||||
this.oldlogfile = new Path(regionDir, "oldlogfile" + "_" + serverName);
|
||||
|
||||
// Logging
|
||||
|
||||
this.fs = FileSystem.get(conf);
|
||||
HLog.consolidateOldLog(newlogdir, oldlogfile, fs, conf);
|
||||
// TODO: Now we have a consolidated log for all regions, sort and
|
||||
// then split result by region passing the splits as reconstruction
|
||||
// logs to HRegions on start. Or, rather than consolidate, split logs
|
||||
// into per region files.
|
||||
this.log = new HLog(fs, newlogdir, conf);
|
||||
this.logRoller = new LogRoller();
|
||||
this.logRollerThread = new Thread(logRoller, "HRegionServer.logRoller");
|
||||
|
||||
// Remote HMaster
|
||||
|
||||
this.hbaseMaster = (HMasterRegionInterface)
|
||||
RPC.waitForProxy(HMasterRegionInterface.class,
|
||||
HMasterRegionInterface.versionID,
|
||||
new HServerAddress(conf.get(MASTER_ADDRESS)).getInetSocketAddress(),
|
||||
conf);
|
||||
this.hbaseMaster = (HMasterRegionInterface)RPC.
|
||||
waitForProxy(HMasterRegionInterface.class,
|
||||
HMasterRegionInterface.versionID,
|
||||
new HServerAddress(conf.get(MASTER_ADDRESS)).getInetSocketAddress(),
|
||||
conf);
|
||||
|
||||
// Threads
|
||||
|
||||
this.workerThread.start();
|
||||
this.cacheFlusherThread.start();
|
||||
this.splitCheckerThread.start();
|
||||
this.logRollerThread.start();
|
||||
this.leases = new Leases(conf.getLong("hbase.hregionserver.lease.period",
|
||||
this.leases = new Leases(conf.getLong("hbase.regionserver.lease.period",
|
||||
3 * 60 * 1000), threadWakeFrequency);
|
||||
|
||||
// Server
|
||||
|
@ -380,56 +404,44 @@ public class HRegionServer
|
|||
}
|
||||
|
||||
/**
|
||||
* Stop all the HRegionServer threads and close everything down. All ongoing
|
||||
* transactions will be aborted all threads will be shut down. This method
|
||||
* will return immediately. The caller should call join to wait for all
|
||||
* processing to cease.
|
||||
* Set a flag that will cause all the HRegionServer threads to shut down
|
||||
* in an orderly fashion.
|
||||
*/
|
||||
public void stop() throws IOException {
|
||||
if(! stopRequested) {
|
||||
stopRequested = true;
|
||||
|
||||
closeAllRegions();
|
||||
log.close();
|
||||
fs.close();
|
||||
server.stop();
|
||||
}
|
||||
LOG.info("stopping server at: " + address.toString());
|
||||
public synchronized void stop() throws IOException {
|
||||
stopRequested = true;
|
||||
notifyAll(); // Wakes run() if it is sleeping
|
||||
}
|
||||
|
||||
/** Call join to wait for all the threads to finish */
|
||||
/** Wait on all threads to finish.
|
||||
* Presumption is that all closes and stops have already been called.
|
||||
*/
|
||||
public void join() {
|
||||
try {
|
||||
this.workerThread.join();
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
|
||||
try {
|
||||
this.logRollerThread.join();
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
|
||||
try {
|
||||
this.cacheFlusherThread.join();
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
|
||||
this.leases.close();
|
||||
|
||||
try {
|
||||
this.splitCheckerThread.join();
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
try {
|
||||
this.server.join();
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
LOG.info("server stopped at: " + address.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* The HRegionServer sticks in this loop until close. It repeatedly checks in
|
||||
* with the HMaster, sending heartbeats & reports, and receiving HRegion
|
||||
* The HRegionServer sticks in this loop until closed. It repeatedly checks
|
||||
* in with the HMaster, sending heartbeats & reports, and receiving HRegion
|
||||
* load/unload instructions.
|
||||
*/
|
||||
public void run() {
|
||||
|
@ -439,27 +451,40 @@ public class HRegionServer
|
|||
long waitTime;
|
||||
|
||||
// Let the master know we're here
|
||||
|
||||
try {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Telling master we are up");
|
||||
}
|
||||
|
||||
hbaseMaster.regionServerStartup(info);
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Done telling master we are up");
|
||||
}
|
||||
} catch(IOException e) {
|
||||
waitTime = msgInterval - (System.currentTimeMillis() - lastMsg);
|
||||
|
||||
waitTime = stopRequested ? 0
|
||||
: msgInterval - (System.currentTimeMillis() - lastMsg);
|
||||
|
||||
if(waitTime > 0) {
|
||||
try {
|
||||
Thread.sleep(waitTime);
|
||||
|
||||
} catch(InterruptedException iex) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Sleep");
|
||||
}
|
||||
synchronized(this) {
|
||||
try {
|
||||
Thread.sleep(waitTime);
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Wake");
|
||||
}
|
||||
}
|
||||
continue;
|
||||
}
|
||||
|
||||
// Now ask the master what it wants us to do and tell it what we have done.
|
||||
|
||||
while(! stopRequested) {
|
||||
if((System.currentTimeMillis() - lastMsg) >= msgInterval) {
|
||||
// Now ask master what it wants us to do and tell it what we have done.
|
||||
while (!stopRequested) {
|
||||
if ((System.currentTimeMillis() - lastMsg) >= msgInterval) {
|
||||
|
||||
HMsg outboundArray[] = null;
|
||||
synchronized(outboundMsgs) {
|
||||
|
@ -474,28 +499,43 @@ public class HRegionServer
|
|||
// Queue up the HMaster's instruction stream for processing
|
||||
|
||||
synchronized(toDo) {
|
||||
boolean restartOrStop = false;
|
||||
for(int i = 0; i < msgs.length; i++) {
|
||||
boolean restart = false;
|
||||
for(int i = 0; i < msgs.length && !stopRequested && !restart; i++) {
|
||||
switch(msgs[i].getMsg()) {
|
||||
|
||||
case HMsg.MSG_CALL_SERVER_STARTUP:
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Got call server startup message");
|
||||
}
|
||||
toDo.clear();
|
||||
closeAllRegions();
|
||||
restartOrStop = true;
|
||||
restart = true;
|
||||
break;
|
||||
|
||||
case HMsg.MSG_REGIONSERVER_ALREADY_RUNNING:
|
||||
stop();
|
||||
restartOrStop = true;
|
||||
case HMsg.MSG_REGIONSERVER_STOP:
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Got regionserver stop message");
|
||||
}
|
||||
toDo.clear();
|
||||
closeAllRegions();
|
||||
stopRequested = true;
|
||||
break;
|
||||
|
||||
default:
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Got default message");
|
||||
}
|
||||
toDo.add(msgs[i]);
|
||||
}
|
||||
}
|
||||
|
||||
if(toDo.size() > 0) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("notify on todo");
|
||||
}
|
||||
toDo.notifyAll();
|
||||
}
|
||||
if(restartOrStop) {
|
||||
if(restart || stopRequested) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -505,16 +545,47 @@ public class HRegionServer
|
|||
}
|
||||
}
|
||||
|
||||
waitTime = msgInterval - (System.currentTimeMillis() - lastMsg);
|
||||
|
||||
if(waitTime > 0) {
|
||||
try {
|
||||
Thread.sleep(waitTime);
|
||||
} catch(InterruptedException iex) {
|
||||
waitTime = stopRequested ? 0
|
||||
: msgInterval - (System.currentTimeMillis() - lastMsg);
|
||||
if (waitTime > 0) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Sleep");
|
||||
}
|
||||
synchronized(this) {
|
||||
try {
|
||||
Thread.sleep(waitTime);
|
||||
} catch(InterruptedException iex) {
|
||||
}
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Wake");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
try {
|
||||
LOG.info("stopping server at: " + address.toString());
|
||||
|
||||
// Send interrupts to wake up threads if sleeping so they notice shutdown.
|
||||
|
||||
this.logRollerThread.interrupt();
|
||||
this.cacheFlusherThread.interrupt();
|
||||
this.splitCheckerThread.interrupt();
|
||||
|
||||
this.worker.stop();
|
||||
this.server.stop();
|
||||
|
||||
closeAllRegions();
|
||||
log.close();
|
||||
leases.close();
|
||||
join();
|
||||
|
||||
} catch(IOException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("main thread exiting");
|
||||
}
|
||||
}
|
||||
|
||||
/** Add to the outbound message buffer */
|
||||
|
@ -556,48 +627,83 @@ public class HRegionServer
|
|||
private Worker worker;
|
||||
private Thread workerThread;
|
||||
private class Worker implements Runnable {
|
||||
public void stop() {
|
||||
synchronized(toDo) {
|
||||
toDo.notifyAll();
|
||||
}
|
||||
}
|
||||
|
||||
public void run() {
|
||||
while(!stopRequested) {
|
||||
HMsg msg = null;
|
||||
for(HMsg msg = null; !stopRequested; ) {
|
||||
synchronized(toDo) {
|
||||
while(toDo.size() == 0) {
|
||||
while(toDo.size() == 0 && !stopRequested) {
|
||||
try {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Wait on todo");
|
||||
}
|
||||
toDo.wait();
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Wake on todo");
|
||||
}
|
||||
} catch(InterruptedException e) {
|
||||
}
|
||||
}
|
||||
if(stopRequested) {
|
||||
continue;
|
||||
}
|
||||
msg = toDo.remove(0);
|
||||
}
|
||||
|
||||
try {
|
||||
switch(msg.getMsg()) {
|
||||
|
||||
case HMsg.MSG_REGION_OPEN: // Open a region
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("MSG_REGION_OPEN");
|
||||
}
|
||||
openRegion(msg.getRegionInfo());
|
||||
break;
|
||||
|
||||
case HMsg.MSG_REGION_CLOSE: // Close a region
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("MSG_REGION_CLOSE");
|
||||
}
|
||||
closeRegion(msg.getRegionInfo(), true);
|
||||
break;
|
||||
|
||||
case HMsg.MSG_REGION_MERGE: // Merge two regions
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("MSG_REGION_MERGE");
|
||||
}
|
||||
//TODO ???
|
||||
throw new IOException("TODO: need to figure out merge");
|
||||
//break;
|
||||
|
||||
case HMsg.MSG_CALL_SERVER_STARTUP: // Close regions, restart
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("MSG_CALL_SERVER_STARTUP");
|
||||
}
|
||||
closeAllRegions();
|
||||
continue;
|
||||
|
||||
case HMsg.MSG_REGIONSERVER_ALREADY_RUNNING: // Go away
|
||||
stop();
|
||||
case HMsg.MSG_REGIONSERVER_STOP: // Go away
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("MSG_REGIONSERVER_STOP");
|
||||
}
|
||||
stopRequested = true;
|
||||
continue;
|
||||
|
||||
case HMsg.MSG_REGION_CLOSE_WITHOUT_REPORT: // Close a region, don't reply
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("MSG_REGION_CLOSE_WITHOUT_REPORT");
|
||||
}
|
||||
closeRegion(msg.getRegionInfo(), false);
|
||||
break;
|
||||
|
||||
case HMsg.MSG_REGION_CLOSE_AND_DELETE:
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("MSG_REGION_CLOSE_AND_DELETE");
|
||||
}
|
||||
closeAndDeleteRegion(msg.getRegionInfo());
|
||||
break;
|
||||
|
||||
|
@ -608,15 +714,17 @@ public class HRegionServer
|
|||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("worker thread exiting");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void openRegion(HRegionInfo regionInfo) throws IOException {
|
||||
|
||||
this.locker.writeLock().lock();
|
||||
try {
|
||||
HRegion region = new HRegion(regionDir, log, fs, conf, regionInfo, null, oldlogfile);
|
||||
|
||||
HRegion region =
|
||||
new HRegion(regionDir, log, fs, conf, regionInfo, null, oldlogfile);
|
||||
regions.put(region.getRegionName(), region);
|
||||
reportOpen(region);
|
||||
|
||||
|
@ -661,12 +769,20 @@ public class HRegionServer
|
|||
}
|
||||
|
||||
/** Called either when the master tells us to restart or from stop() */
|
||||
private void closeAllRegions() throws IOException {
|
||||
private void closeAllRegions() {
|
||||
this.locker.writeLock().lock();
|
||||
try {
|
||||
for(Iterator<HRegion> it = regions.values().iterator(); it.hasNext(); ) {
|
||||
HRegion region = it.next();
|
||||
region.close();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("closing region " + region.getRegionName());
|
||||
}
|
||||
try {
|
||||
region.close();
|
||||
|
||||
} catch(IOException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
regions.clear();
|
||||
|
||||
|
@ -722,9 +838,12 @@ public class HRegionServer
|
|||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
|
||||
byte results[] = region.get(row, column);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("get " + row.toString() + ", " + column.toString());
|
||||
}
|
||||
BytesWritable results = region.get(row, column);
|
||||
if(results != null) {
|
||||
return new BytesWritable(results);
|
||||
return results;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -738,15 +857,9 @@ public class HRegionServer
|
|||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
|
||||
byte results[][] = region.get(row, column, numVersions);
|
||||
BytesWritable[] results = region.get(row, column, numVersions);
|
||||
if(results != null) {
|
||||
BytesWritable realResults[] = new BytesWritable[results.length];
|
||||
for(int i = 0; i < realResults.length; i++) {
|
||||
if(results[i] != null) {
|
||||
realResults[i] = new BytesWritable(results[i]);
|
||||
}
|
||||
}
|
||||
return realResults;
|
||||
return results;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -760,15 +873,9 @@ public class HRegionServer
|
|||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
|
||||
byte results[][] = region.get(row, column, timestamp, numVersions);
|
||||
BytesWritable[] results = region.get(row, column, timestamp, numVersions);
|
||||
if(results != null) {
|
||||
BytesWritable realResults[] = new BytesWritable[results.length];
|
||||
for(int i = 0; i < realResults.length; i++) {
|
||||
if(results[i] != null) {
|
||||
realResults[i] = new BytesWritable(results[i]);
|
||||
}
|
||||
}
|
||||
return realResults;
|
||||
return results;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -780,12 +887,12 @@ public class HRegionServer
|
|||
throw new IOException("Not serving region " + regionName);
|
||||
}
|
||||
|
||||
TreeMap<Text, byte[]> map = region.getFull(row);
|
||||
TreeMap<Text, BytesWritable> map = region.getFull(row);
|
||||
LabelledData result[] = new LabelledData[map.size()];
|
||||
int counter = 0;
|
||||
for(Iterator<Text> it = map.keySet().iterator(); it.hasNext(); ) {
|
||||
Text colname = it.next();
|
||||
byte val[] = map.get(colname);
|
||||
BytesWritable val = map.get(colname);
|
||||
result[counter++] = new LabelledData(colname, val);
|
||||
}
|
||||
return result;
|
||||
|
@ -842,7 +949,7 @@ public class HRegionServer
|
|||
leases.renewLease(new Text(String.valueOf(clientid)),
|
||||
new Text(String.valueOf(lockid)));
|
||||
|
||||
region.put(lockid, column, val.get());
|
||||
region.put(lockid, column, val);
|
||||
}
|
||||
|
||||
/** Remove a cell from the HBase. */
|
||||
|
@ -911,7 +1018,7 @@ public class HRegionServer
|
|||
// remote scanner interface
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
private Map<Text, HScannerInterface> scanners;
|
||||
private Map<Text, HInternalScannerInterface> scanners;
|
||||
private class ScannerListener extends LeaseListener {
|
||||
private Text scannerName;
|
||||
|
||||
|
@ -920,7 +1027,7 @@ public class HRegionServer
|
|||
}
|
||||
|
||||
public void leaseExpired() {
|
||||
HScannerInterface s = scanners.remove(scannerName);
|
||||
HInternalScannerInterface s = scanners.remove(scannerName);
|
||||
if(s != null) {
|
||||
try {
|
||||
s.close();
|
||||
|
@ -943,7 +1050,7 @@ public class HRegionServer
|
|||
|
||||
long scannerId = -1L;
|
||||
try {
|
||||
HScannerInterface s = r.getScanner(cols, firstRow);
|
||||
HInternalScannerInterface s = r.getScanner(cols, firstRow);
|
||||
scannerId = rand.nextLong();
|
||||
Text scannerName = new Text(String.valueOf(scannerId));
|
||||
scanners.put(scannerName, s);
|
||||
|
@ -959,18 +1066,30 @@ public class HRegionServer
|
|||
public LabelledData[] next(long scannerId, HStoreKey key) throws IOException {
|
||||
|
||||
Text scannerName = new Text(String.valueOf(scannerId));
|
||||
HScannerInterface s = scanners.get(scannerName);
|
||||
HInternalScannerInterface s = scanners.get(scannerName);
|
||||
if(s == null) {
|
||||
throw new IOException("unknown scanner");
|
||||
}
|
||||
leases.renewLease(scannerName, scannerName);
|
||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||
TreeMap<Text, BytesWritable> results = new TreeMap<Text, BytesWritable>();
|
||||
ArrayList<LabelledData> values = new ArrayList<LabelledData>();
|
||||
if(s.next(key, results)) {
|
||||
for(Iterator<Map.Entry<Text, byte[]>> it = results.entrySet().iterator();
|
||||
for(Iterator<Map.Entry<Text, BytesWritable>> it
|
||||
= results.entrySet().iterator();
|
||||
it.hasNext(); ) {
|
||||
Map.Entry<Text, byte[]> e = it.next();
|
||||
values.add(new LabelledData(e.getKey(), e.getValue()));
|
||||
|
||||
Map.Entry<Text, BytesWritable> e = it.next();
|
||||
BytesWritable val = e.getValue();
|
||||
if(val.getSize() == DELETE_BYTES.getSize()
|
||||
&& val.compareTo(DELETE_BYTES) == 0) {
|
||||
|
||||
// Value is deleted. Don't return a value
|
||||
|
||||
continue;
|
||||
|
||||
} else {
|
||||
values.add(new LabelledData(e.getKey(), val));
|
||||
}
|
||||
}
|
||||
}
|
||||
return values.toArray(new LabelledData[values.size()]);
|
||||
|
@ -978,7 +1097,7 @@ public class HRegionServer
|
|||
|
||||
public void close(long scannerId) throws IOException {
|
||||
Text scannerName = new Text(String.valueOf(scannerId));
|
||||
HScannerInterface s = scanners.remove(scannerName);
|
||||
HInternalScannerInterface s = scanners.remove(scannerName);
|
||||
if(s == null) {
|
||||
throw new IOException("unknown scanner");
|
||||
}
|
||||
|
@ -991,33 +1110,53 @@ public class HRegionServer
|
|||
leases.cancelLease(scannerName, scannerName);
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// Main program
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
private static void printUsage() {
|
||||
private static void printUsageAndExit() {
|
||||
printUsageAndExit(null);
|
||||
}
|
||||
|
||||
private static void printUsageAndExit(final String message) {
|
||||
if (message != null) {
|
||||
System.err.println(message);
|
||||
}
|
||||
System.err.println("Usage: java " +
|
||||
"org.apache.hbase.HRegionServer [--bind=hostname:port]");
|
||||
"org.apache.hbase.HRegionServer [--bind=hostname:port] start");
|
||||
System.exit(0);
|
||||
}
|
||||
|
||||
public static void main(String [] args) throws IOException {
|
||||
if (args.length < 1) {
|
||||
printUsageAndExit();
|
||||
}
|
||||
|
||||
Configuration conf = new HBaseConfiguration();
|
||||
|
||||
// Process command-line args. TODO: Better cmd-line processing
|
||||
// (but hopefully something not as painful as cli options).
|
||||
final String addressArgKey = "--bind=";
|
||||
for (String cmd: args) {
|
||||
if (cmd.equals("-h") || cmd.startsWith("--h")) {
|
||||
printUsage();
|
||||
return;
|
||||
if (cmd.startsWith(addressArgKey)) {
|
||||
conf.set(REGIONSERVER_ADDRESS, cmd.substring(addressArgKey.length()));
|
||||
continue;
|
||||
}
|
||||
|
||||
final String addressArgKey = "--bind=";
|
||||
if (cmd.startsWith(addressArgKey)) {
|
||||
conf.set(REGIONSERVER_ADDRESS,
|
||||
cmd.substring(addressArgKey.length()));
|
||||
if (cmd.equals("start")) {
|
||||
try {
|
||||
(new Thread(new HRegionServer(conf))).start();
|
||||
} catch (Throwable t) {
|
||||
LOG.error( "Can not start master because "+
|
||||
StringUtils.stringifyException(t) );
|
||||
System.exit(-1);
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
if (cmd.equals("stop")) {
|
||||
printUsageAndExit("There is no regionserver stop mechanism. To stop " +
|
||||
"regionservers, shutdown the hbase master");
|
||||
}
|
||||
|
||||
// Print out usage if we get to here.
|
||||
printUsageAndExit();
|
||||
}
|
||||
|
||||
new HRegionServer(conf);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,219 @@
|
|||
/**
|
||||
* Copyright 2007 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.TreeMap;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
||||
/**
|
||||
* A standalone HRegion directory reader. Currently reads content on
|
||||
* file system only.
|
||||
* TODO: Add dumping of HStoreFile content and HLog.
|
||||
*/
|
||||
public class HRegiondirReader {
|
||||
private final Configuration conf;
|
||||
private final Path parentdir;
|
||||
|
||||
private static final Pattern REGION_NAME_PARSER =
|
||||
Pattern.compile(HGlobals.HREGIONDIR_PREFIX +
|
||||
"([^_]+)_([^_]*)_([^_]*)");
|
||||
|
||||
private static final String USAGE = "Usage: " +
|
||||
"java org.apache.hadoop.hbase.HRegionDirReader <regiondir> " +
|
||||
"[<tablename>]";
|
||||
|
||||
private final List<HRegionInfo> infos;
|
||||
|
||||
public HRegiondirReader(final HBaseConfiguration conf,
|
||||
final String parentdirName)
|
||||
throws IOException {
|
||||
this.conf = conf;
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
this.parentdir = new Path(parentdirName);
|
||||
if (!fs.exists(parentdir)) {
|
||||
throw new FileNotFoundException(parentdirName);
|
||||
}
|
||||
if (!fs.isDirectory(parentdir)) {
|
||||
throw new IOException(parentdirName + " not a directory");
|
||||
}
|
||||
// Look for regions in parentdir.
|
||||
Path [] regiondirs =
|
||||
fs.listPaths(parentdir, new PathFilter() {
|
||||
public boolean accept(Path path) {
|
||||
Matcher m = REGION_NAME_PARSER.matcher(path.getName());
|
||||
return m != null && m.matches();
|
||||
}
|
||||
});
|
||||
// Create list of HRegionInfos for all regions found in
|
||||
// parentdir.
|
||||
this.infos = new ArrayList<HRegionInfo>();
|
||||
for (Path d: regiondirs) {
|
||||
Matcher m = REGION_NAME_PARSER.matcher(d.getName());
|
||||
if (m == null || !m.matches()) {
|
||||
throw new IOException("Unparseable region dir name");
|
||||
}
|
||||
String tableName = m.group(1);
|
||||
String endKey = m.group(2);
|
||||
long regionid = Long.parseLong(m.group(3));
|
||||
HTableDescriptor desc = getTableDescriptor(fs, d, tableName);
|
||||
HRegionInfo info = new HRegionInfo(regionid, desc,
|
||||
new Text(), (endKey == null || endKey.length() == 0)?
|
||||
new Text(): new Text(endKey));
|
||||
infos.add(info);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a populated table descriptor.
|
||||
* @param fs Current filesystem.
|
||||
* @param d The regiondir for <code>tableName</code>
|
||||
* @param tableName Name of this table.
|
||||
* @return A HTableDescriptor populated with all known column
|
||||
* families.
|
||||
* @throws IOException
|
||||
*/
|
||||
private HTableDescriptor getTableDescriptor(final FileSystem fs,
|
||||
final Path d, final String tableName)
|
||||
throws IOException {
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName, 1);
|
||||
Text [] families = getFamilies(fs, d);
|
||||
for (Text f: families) {
|
||||
desc.addFamily(f);
|
||||
}
|
||||
return desc;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get column families for this region by looking at
|
||||
* directory names under this region.
|
||||
* This is a hack. HRegions only know what columns they have
|
||||
* because they are told by passed-in metadata.
|
||||
* @param regiondir
|
||||
* @return Array of family names.
|
||||
* @throws IOException
|
||||
*/
|
||||
private Text [] getFamilies(final FileSystem fs,
|
||||
final Path regiondir)
|
||||
throws IOException {
|
||||
Path [] subdirs = fs.listPaths(regiondir, new PathFilter() {
|
||||
public boolean accept(Path path) {
|
||||
return !path.getName().equals("log");
|
||||
}
|
||||
});
|
||||
List<Text> families = new ArrayList<Text>();
|
||||
for (Path d: subdirs) {
|
||||
// Convert names of subdirectories into column family names
|
||||
// by adding the colon.
|
||||
Text family = new Text(d.getName() + ":");
|
||||
families.add(family);
|
||||
}
|
||||
return families.toArray(new Text [] {});
|
||||
}
|
||||
|
||||
public List <HRegionInfo> getRegions() {
|
||||
return this.infos;
|
||||
}
|
||||
|
||||
public HRegionInfo getRegionInfo(final String tableName)
|
||||
throws IOException {
|
||||
HRegionInfo result = null;
|
||||
for(HRegionInfo i: getRegions()) {
|
||||
if(i.tableDesc.getName().equals(tableName)) {
|
||||
result = i;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (result == null) {
|
||||
throw new NullPointerException("No such table: " +
|
||||
tableName);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private void dump(final String tableName) throws IOException {
|
||||
dump(getRegionInfo(tableName));
|
||||
}
|
||||
|
||||
private void dump(final HRegionInfo info) throws IOException {
|
||||
HRegion r = new HRegion(this.parentdir, null,
|
||||
FileSystem.get(this.conf), conf, info, null, null);
|
||||
Text [] families = info.tableDesc.families().toArray(new Text [] {});
|
||||
HInternalScannerInterface scanner = r.getScanner(families, new Text());
|
||||
HStoreKey key = new HStoreKey();
|
||||
TreeMap<Text, BytesWritable> results = new TreeMap<Text, BytesWritable>();
|
||||
// Print out table header line.
|
||||
String s = info.startKey.toString();
|
||||
String startKey = (s == null || s.length() <= 0)? "<>": s;
|
||||
s = info.endKey.toString();
|
||||
String endKey = (s == null || s.length() <= 0)? "<>": s;
|
||||
String tableName = info.tableDesc.getName().toString();
|
||||
System.out.println("table: " + tableName +
|
||||
", regionid: " + info.regionId +
|
||||
", startkey: " + startKey +
|
||||
", endkey: " + endKey);
|
||||
// Now print rows. Offset by a space to distingush rows from
|
||||
// table headers. TODO: Add in better formatting of output.
|
||||
// Every line starts with row name followed by column name
|
||||
// followed by cell content.
|
||||
while(scanner.next(key, results)) {
|
||||
for (Text colname: results.keySet()) {
|
||||
Object value = null;
|
||||
byte[] bytes = new byte[results.get(colname).getSize()];
|
||||
if (colname.toString().equals("info:regioninfo")) {
|
||||
// Then bytes are instance of an HRegionInfo.
|
||||
System.arraycopy(results.get(colname).get(), 0, bytes, 0, bytes.length);
|
||||
value = new HRegionInfo(bytes);
|
||||
} else {
|
||||
value = new String(bytes, HConstants.UTF8_ENCODING);
|
||||
}
|
||||
|
||||
System.out.println(" " + key + ", " + colname.toString() + ": \"" +
|
||||
value.toString() + "\"");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
if (args.length < 1) {
|
||||
System.err.println(USAGE);
|
||||
System.exit(-1);
|
||||
}
|
||||
HBaseConfiguration c = new HBaseConfiguration();
|
||||
HRegiondirReader reader = new HRegiondirReader(c, args[0]);
|
||||
if (args.length == 1) {
|
||||
// Do all regions.
|
||||
for(HRegionInfo info: reader.getRegions()) {
|
||||
reader.dump(info);
|
||||
}
|
||||
} else {
|
||||
for (int i = 1; i < args.length; i++) {
|
||||
reader.dump(args[i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -92,7 +92,7 @@ public class HServerAddress implements Writable {
|
|||
|
||||
} else {
|
||||
address = new InetSocketAddress(bindAddress, port);
|
||||
stringValue = bindAddress + "_" + port;
|
||||
stringValue = bindAddress + ":" + port;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -49,6 +49,11 @@ public class HServerInfo implements Writable {
|
|||
public long getStartCode() {
|
||||
return startCode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "address: " + this.serverAddress + ", startcode: " + this.startCode;
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// Writable
|
||||
|
@ -63,4 +68,4 @@ public class HServerInfo implements Writable {
|
|||
this.serverAddress.write(out);
|
||||
out.writeLong(this.startCode);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -78,30 +78,31 @@ public class HStore {
|
|||
|
||||
/**
|
||||
* An HStore is a set of zero or more MapFiles, which stretch backwards over
|
||||
* time. A given HStore is responsible for a certain set of columns for a row
|
||||
* in the HRegion.
|
||||
* time. A given HStore is responsible for a certain set of columns for a
|
||||
* row in the HRegion.
|
||||
*
|
||||
* The HRegion starts writing to its set of HStores when the HRegion's
|
||||
* <p>The HRegion starts writing to its set of HStores when the HRegion's
|
||||
* memcache is flushed. This results in a round of new MapFiles, one for
|
||||
* each HStore.
|
||||
*
|
||||
* There's no reason to consider append-logging at this level; all logging and
|
||||
* locking is handled at the HRegion level. HStore just provides services to
|
||||
* manage sets of MapFiles. One of the most important of those services is
|
||||
* MapFile-compaction services.
|
||||
* <p>There's no reason to consider append-logging at this level; all logging
|
||||
* and locking is handled at the HRegion level. HStore just provides
|
||||
* services to manage sets of MapFiles. One of the most important of those
|
||||
* services is MapFile-compaction services.
|
||||
*
|
||||
* The only thing having to do with logs that HStore needs to deal with is
|
||||
* <p>The only thing having to do with logs that HStore needs to deal with is
|
||||
* the reconstructionLog. This is a segment of an HRegion's log that might
|
||||
* be present upon startup. If the param is NULL, there's nothing to do.
|
||||
* NOT be present upon startup. If the param is NULL, there's nothing to do.
|
||||
* If the param is non-NULL, we need to process the log to reconstruct
|
||||
* a TreeMap that might not have been written to disk before the process died.
|
||||
* a TreeMap that might not have been written to disk before the process
|
||||
* died.
|
||||
*
|
||||
* It's assumed that after this constructor returns, the reconstructionLog file
|
||||
* will be deleted (by whoever has instantiated the HStore).
|
||||
* <p>It's assumed that after this constructor returns, the reconstructionLog
|
||||
* file will be deleted (by whoever has instantiated the HStore).
|
||||
*/
|
||||
public HStore(Path dir, Text regionName, Text colFamily, int maxVersions,
|
||||
FileSystem fs, Path reconstructionLog, Configuration conf) throws IOException {
|
||||
|
||||
FileSystem fs, Path reconstructionLog, Configuration conf)
|
||||
throws IOException {
|
||||
this.dir = dir;
|
||||
this.regionName = regionName;
|
||||
this.colFamily = colFamily;
|
||||
|
@ -114,14 +115,17 @@ public class HStore {
|
|||
this.loginfodir = HStoreFile.getInfoDir(dir, regionName, colFamily);
|
||||
fs.mkdirs(loginfodir);
|
||||
|
||||
LOG.debug("starting HStore for " + regionName + "/"+ colFamily);
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("starting HStore for " + regionName + "/"+ colFamily);
|
||||
}
|
||||
|
||||
// Either restart or get rid of any leftover compaction work. Either way,
|
||||
// by the time processReadyCompaction() returns, we can get rid of the
|
||||
// existing compaction-dir.
|
||||
|
||||
this.compactdir = new Path(dir, COMPACTION_DIR);
|
||||
Path curCompactStore = HStoreFile.getHStoreDir(compactdir, regionName, colFamily);
|
||||
Path curCompactStore =
|
||||
HStoreFile.getHStoreDir(compactdir, regionName, colFamily);
|
||||
if(fs.exists(curCompactStore)) {
|
||||
processReadyCompaction();
|
||||
fs.delete(curCompactStore);
|
||||
|
@ -129,28 +133,25 @@ public class HStore {
|
|||
|
||||
// Go through the 'mapdir' and 'loginfodir' together, make sure that all
|
||||
// MapFiles are in a reliable state. Every entry in 'mapdir' must have a
|
||||
// corresponding one in 'loginfodir'. Without a corresponding log info file,
|
||||
// the entry in 'mapdir'must be deleted.
|
||||
|
||||
// corresponding one in 'loginfodir'. Without a corresponding log info
|
||||
// file, the entry in 'mapdir' must be deleted.
|
||||
Vector<HStoreFile> hstoreFiles
|
||||
= HStoreFile.loadHStoreFiles(conf, dir, regionName, colFamily, fs);
|
||||
|
||||
for(Iterator<HStoreFile> it = hstoreFiles.iterator(); it.hasNext(); ) {
|
||||
HStoreFile hsf = it.next();
|
||||
mapFiles.put(hsf.loadInfo(fs), hsf);
|
||||
}
|
||||
|
||||
// Now go through all the HSTORE_LOGINFOFILEs and figure out the most-recent
|
||||
// log-seq-ID that's present. The most-recent such ID means we can ignore
|
||||
// all log messages up to and including that ID (because they're already
|
||||
// reflected in the TreeMaps).
|
||||
// Now go through all the HSTORE_LOGINFOFILEs and figure out the
|
||||
// most-recent log-seq-ID that's present. The most-recent such ID means we
|
||||
// can ignore all log messages up to and including that ID (because they're
|
||||
// already reflected in the TreeMaps).
|
||||
//
|
||||
// If the HSTORE_LOGINFOFILE doesn't contain a number, just ignore it. That
|
||||
// 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(Iterator<HStoreFile> it = hstoreFiles.iterator(); it.hasNext(); ) {
|
||||
for (Iterator<HStoreFile> it = hstoreFiles.iterator(); it.hasNext(); ) {
|
||||
HStoreFile hsf = it.next();
|
||||
long seqid = hsf.loadInfo(fs);
|
||||
if(seqid > 0) {
|
||||
|
@ -166,63 +167,72 @@ public class HStore {
|
|||
// We can ignore any log message that has a sequence ID that's equal to or
|
||||
// lower than maxSeqID. (Because we know such log messages are already
|
||||
// reflected in the MapFiles.)
|
||||
|
||||
LOG.debug("reading reconstructionLog");
|
||||
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("reading reconstructionLog");
|
||||
}
|
||||
if(reconstructionLog != null && fs.exists(reconstructionLog)) {
|
||||
long maxSeqIdInLog = -1;
|
||||
TreeMap<HStoreKey, BytesWritable> reconstructedCache
|
||||
= new TreeMap<HStoreKey, BytesWritable>();
|
||||
|
||||
SequenceFile.Reader login
|
||||
= new SequenceFile.Reader(fs, reconstructionLog, conf);
|
||||
|
||||
try {
|
||||
HLogKey key = new HLogKey();
|
||||
HLogEdit val = new HLogEdit();
|
||||
while(login.next(key, val)) {
|
||||
maxSeqIdInLog = Math.max(maxSeqIdInLog, key.getLogSeqNum());
|
||||
if(key.getLogSeqNum() <= maxSeqID) {
|
||||
if (key.getLogSeqNum() <= maxSeqID) {
|
||||
continue;
|
||||
}
|
||||
// Check this edit is for me. Also, guard against writing
|
||||
// METACOLUMN info such as HBASE::CACHEFLUSH entries
|
||||
Text column = val.getColumn();
|
||||
if (!key.getRegionName().equals(this.regionName) ||
|
||||
column.equals(HLog.METACOLUMN) ||
|
||||
HStoreKey.extractFamily(column).equals(this.colFamily)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Passing on edit " + key.getRegionName() + ", "
|
||||
+ key.getRegionName() + ", " + column.toString() + ": "
|
||||
+ new String(val.getVal().get()));
|
||||
}
|
||||
continue;
|
||||
}
|
||||
reconstructedCache.put(new HStoreKey(key.getRow(), val.getColumn(),
|
||||
val.getTimestamp()), val.getVal());
|
||||
}
|
||||
|
||||
} finally {
|
||||
login.close();
|
||||
}
|
||||
|
||||
if(reconstructedCache.size() > 0) {
|
||||
|
||||
// We create a "virtual flush" at maxSeqIdInLog+1.
|
||||
|
||||
LOG.debug("flushing reconstructionCache");
|
||||
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("flushing reconstructionCache");
|
||||
}
|
||||
flushCacheHelper(reconstructedCache, maxSeqIdInLog+1, true);
|
||||
}
|
||||
}
|
||||
|
||||
// Compact all the MapFiles into a single file. The resulting MapFile
|
||||
// should be "timeless"; that is, it should not have an associated seq-ID,
|
||||
// because all log messages have been reflected in the TreeMaps at this point.
|
||||
|
||||
// because all log messages have been reflected in the TreeMaps at this
|
||||
// point.
|
||||
if(mapFiles.size() >= 1) {
|
||||
compactHelper(true);
|
||||
}
|
||||
|
||||
// Finally, start up all the map readers! (There should be just one at this
|
||||
// point, as we've compacted them all.)
|
||||
|
||||
LOG.debug("starting map readers");
|
||||
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("starting map readers");
|
||||
}
|
||||
for(Iterator<Long> it = mapFiles.keySet().iterator(); it.hasNext(); ) {
|
||||
Long key = it.next().longValue();
|
||||
HStoreFile hsf = mapFiles.get(key);
|
||||
|
||||
//TODO - is this really necessary? Don't I do this inside compact()?
|
||||
|
||||
maps.put(key, new MapFile.Reader(fs, hsf.getMapFilePath().toString(), conf));
|
||||
// TODO - is this really necessary? Don't I do this inside compact()?
|
||||
maps.put(key,
|
||||
new MapFile.Reader(fs, hsf.getMapFilePath().toString(), conf));
|
||||
}
|
||||
|
||||
LOG.info("HStore online for " + this.regionName + "/" + this.colFamily);
|
||||
|
@ -273,7 +283,9 @@ public class HStore {
|
|||
long logCacheFlushId, boolean addToAvailableMaps) throws IOException {
|
||||
|
||||
synchronized(flushLock) {
|
||||
LOG.debug("flushing HStore " + this.regionName + "/" + this.colFamily);
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("flushing HStore " + this.regionName + "/" + this.colFamily);
|
||||
}
|
||||
|
||||
// A. Write the TreeMap out to the disk
|
||||
|
||||
|
@ -281,18 +293,23 @@ public class HStore {
|
|||
= HStoreFile.obtainNewHStoreFile(conf, dir, regionName, colFamily, fs);
|
||||
|
||||
Path mapfile = flushedFile.getMapFilePath();
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("map file is: " + mapfile.toString());
|
||||
}
|
||||
|
||||
MapFile.Writer out = new MapFile.Writer(conf, fs, mapfile.toString(),
|
||||
HStoreKey.class, BytesWritable.class);
|
||||
|
||||
try {
|
||||
for(Iterator<HStoreKey> it = inputCache.keySet().iterator(); it.hasNext(); ) {
|
||||
HStoreKey curkey = it.next();
|
||||
for (HStoreKey curkey: inputCache.keySet()) {
|
||||
if(this.colFamily.equals(HStoreKey.extractFamily(curkey.getColumn()))) {
|
||||
BytesWritable val = inputCache.get(curkey);
|
||||
out.append(curkey, val);
|
||||
}
|
||||
}
|
||||
LOG.debug("HStore " + this.regionName + "/" + this.colFamily + " flushed");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("HStore " + this.regionName + "/" + this.colFamily + " flushed");
|
||||
}
|
||||
|
||||
} finally {
|
||||
out.close();
|
||||
|
@ -301,7 +318,9 @@ public class HStore {
|
|||
// B. Write out the log sequence number that corresponds to this output
|
||||
// MapFile. The MapFile is current up to and including the log seq num.
|
||||
|
||||
LOG.debug("writing log cache flush id");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("writing log cache flush id");
|
||||
}
|
||||
flushedFile.writeInfo(fs, logCacheFlushId);
|
||||
|
||||
// C. Finally, make the new MapFile available.
|
||||
|
@ -312,7 +331,10 @@ public class HStore {
|
|||
try {
|
||||
maps.put(logCacheFlushId, new MapFile.Reader(fs, mapfile.toString(), conf));
|
||||
mapFiles.put(logCacheFlushId, flushedFile);
|
||||
LOG.debug("HStore available for " + this.regionName + "/" + this.colFamily);
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("HStore available for " + this.regionName + "/"
|
||||
+ this.colFamily + " flush id=" + logCacheFlushId);
|
||||
}
|
||||
|
||||
} finally {
|
||||
this.locker.writeLock().unlock();
|
||||
|
@ -357,7 +379,9 @@ public class HStore {
|
|||
|
||||
void compactHelper(boolean deleteSequenceInfo) throws IOException {
|
||||
synchronized(compactLock) {
|
||||
LOG.debug("started compaction of " + this.regionName + "/" + this.colFamily);
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("started compaction of " + this.regionName + "/" + this.colFamily);
|
||||
}
|
||||
|
||||
Path curCompactStore = HStoreFile.getHStoreDir(compactdir, regionName, colFamily);
|
||||
fs.mkdirs(curCompactStore);
|
||||
|
@ -387,13 +411,17 @@ public class HStore {
|
|||
}
|
||||
}
|
||||
}
|
||||
LOG.debug("max sequence id =" + maxSeenSeqID);
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("max sequence id =" + maxSeenSeqID);
|
||||
}
|
||||
|
||||
HStoreFile compactedOutputFile
|
||||
= new HStoreFile(conf, compactdir, regionName, colFamily, -1);
|
||||
|
||||
if(toCompactFiles.size() == 1) {
|
||||
LOG.debug("nothing to compact for " + this.regionName + "/" + this.colFamily);
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("nothing to compact for " + this.regionName + "/" + this.colFamily);
|
||||
}
|
||||
|
||||
HStoreFile hsf = toCompactFiles.elementAt(0);
|
||||
if(hsf.loadInfo(fs) == -1) {
|
||||
|
@ -438,7 +466,9 @@ public class HStore {
|
|||
// Now, advance through the readers in order. This will have the
|
||||
// effect of a run-time sort of the entire dataset.
|
||||
|
||||
LOG.debug("processing HStoreFile readers");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("processing HStoreFile readers");
|
||||
}
|
||||
|
||||
int numDone = 0;
|
||||
for(int i = 0; i < readers.length; i++) {
|
||||
|
@ -518,13 +548,17 @@ public class HStore {
|
|||
}
|
||||
}
|
||||
|
||||
LOG.debug("all HStores processed");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("all HStores processed");
|
||||
}
|
||||
|
||||
} finally {
|
||||
compactedOut.close();
|
||||
}
|
||||
|
||||
LOG.debug("writing new compacted HStore");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("writing new compacted HStore");
|
||||
}
|
||||
|
||||
// Now, write out an HSTORE_LOGINFOFILE for the brand-new TreeMap.
|
||||
|
||||
|
@ -564,7 +598,9 @@ public class HStore {
|
|||
|
||||
processReadyCompaction();
|
||||
|
||||
LOG.debug("compaction complete for " + this.regionName + "/" + this.colFamily);
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("compaction complete for " + this.regionName + "/" + this.colFamily);
|
||||
}
|
||||
|
||||
} finally {
|
||||
fs.delete(compactdir);
|
||||
|
@ -609,7 +645,9 @@ public class HStore {
|
|||
|
||||
// OK, there's actually compaction work that needs to be put into place.
|
||||
|
||||
LOG.debug("compaction starting");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("compaction starting");
|
||||
}
|
||||
|
||||
// 2. Load in the files to be deleted.
|
||||
// (Figuring out what MapFiles are going to be replaced)
|
||||
|
@ -629,7 +667,9 @@ public class HStore {
|
|||
in.close();
|
||||
}
|
||||
|
||||
LOG.debug("loaded files to be deleted");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("loaded files to be deleted");
|
||||
}
|
||||
|
||||
// 3. Unload all the replaced MapFiles.
|
||||
|
||||
|
@ -650,7 +690,9 @@ public class HStore {
|
|||
}
|
||||
}
|
||||
|
||||
LOG.debug("unloaded existing MapFiles");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("unloaded existing MapFiles");
|
||||
}
|
||||
|
||||
// What if we crash at this point? No big deal; we will restart
|
||||
// processReadyCompaction(), and nothing has been lost.
|
||||
|
@ -663,7 +705,9 @@ public class HStore {
|
|||
fs.delete(hsf.getInfoFilePath());
|
||||
}
|
||||
|
||||
LOG.debug("old files deleted");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("old files deleted");
|
||||
}
|
||||
|
||||
// What if we fail now? The above deletes will fail silently. We'd better
|
||||
// make sure not to write out any new files with the same names as
|
||||
|
@ -671,7 +715,9 @@ public class HStore {
|
|||
|
||||
// 5. Moving the new MapFile into place
|
||||
|
||||
LOG.debug("moving new MapFile into place");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("moving new MapFile into place");
|
||||
}
|
||||
|
||||
HStoreFile compactedFile
|
||||
= new HStoreFile(conf, compactdir, regionName, colFamily, -1);
|
||||
|
@ -691,7 +737,9 @@ public class HStore {
|
|||
|
||||
// 6. Loading the new TreeMap.
|
||||
|
||||
LOG.debug("loading new TreeMap");
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("loading new TreeMap");
|
||||
}
|
||||
|
||||
mapFiles.put(orderVal, finalCompactedFile);
|
||||
maps.put(orderVal, new MapFile.Reader(fs,
|
||||
|
@ -716,7 +764,7 @@ public class HStore {
|
|||
*
|
||||
* The returned object should map column names to byte arrays (byte[]).
|
||||
*/
|
||||
public void getFull(HStoreKey key, TreeMap<Text, byte[]> results) throws IOException {
|
||||
public void getFull(HStoreKey key, TreeMap<Text, BytesWritable> results) throws IOException {
|
||||
this.locker.readLock().lock();
|
||||
try {
|
||||
MapFile.Reader[] maparray
|
||||
|
@ -734,7 +782,7 @@ public class HStore {
|
|||
Text readcol = readkey.getColumn();
|
||||
if(results.get(readcol) == null
|
||||
&& key.matchesWithoutColumn(readkey)) {
|
||||
results.put(new Text(readcol), readval.get());
|
||||
results.put(new Text(readcol), readval);
|
||||
readval = new BytesWritable();
|
||||
|
||||
} else if(key.getRow().compareTo(readkey.getRow()) > 0) {
|
||||
|
@ -756,12 +804,12 @@ public class HStore {
|
|||
*
|
||||
* If 'numVersions' is negative, the method returns all available versions.
|
||||
*/
|
||||
public byte[][] get(HStoreKey key, int numVersions) throws IOException {
|
||||
if(numVersions == 0) {
|
||||
throw new IllegalArgumentException("Must request at least one value.");
|
||||
public BytesWritable[] get(HStoreKey key, int numVersions) throws IOException {
|
||||
if(numVersions <= 0) {
|
||||
throw new IllegalArgumentException("Number of versions must be > 0");
|
||||
}
|
||||
|
||||
Vector<byte[]> results = new Vector<byte[]>();
|
||||
Vector<BytesWritable> results = new Vector<BytesWritable>();
|
||||
this.locker.readLock().lock();
|
||||
try {
|
||||
MapFile.Reader[] maparray
|
||||
|
@ -776,7 +824,7 @@ public class HStore {
|
|||
HStoreKey readkey = (HStoreKey)map.getClosest(key, readval);
|
||||
|
||||
if(readkey.matchesRowCol(key)) {
|
||||
results.add(readval.get());
|
||||
results.add(readval);
|
||||
readval = new BytesWritable();
|
||||
|
||||
while(map.next(readkey, readval) && readkey.matchesRowCol(key)) {
|
||||
|
@ -784,7 +832,7 @@ public class HStore {
|
|||
break;
|
||||
|
||||
} else {
|
||||
results.add(readval.get());
|
||||
results.add(readval);
|
||||
readval = new BytesWritable();
|
||||
}
|
||||
}
|
||||
|
@ -799,14 +847,14 @@ public class HStore {
|
|||
return null;
|
||||
|
||||
} else {
|
||||
return (byte[][]) results.toArray(new byte[results.size()][]);
|
||||
return results.toArray(new BytesWritable[results.size()]);
|
||||
}
|
||||
|
||||
} finally {
|
||||
this.locker.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Gets the size of the largest MapFile and its mid key.
|
||||
*
|
||||
|
@ -866,31 +914,54 @@ public class HStore {
|
|||
* Return a set of MapFile.Readers, one for each HStore file.
|
||||
* These should be closed after the user is done with them.
|
||||
*/
|
||||
public HScannerInterface getScanner(long timestamp, Text targetCols[],
|
||||
public HInternalScannerInterface getScanner(long timestamp, Text targetCols[],
|
||||
Text firstRow) throws IOException {
|
||||
|
||||
return new HStoreScanner(timestamp, targetCols, firstRow);
|
||||
}
|
||||
|
||||
/** For debuging purposes. Dumps the keys from all the MapFiles */
|
||||
void dumpMaps() throws IOException {
|
||||
this.locker.readLock().lock();
|
||||
try {
|
||||
for(Iterator<MapFile.Reader> i = maps.values().iterator(); i.hasNext(); ) {
|
||||
MapFile.Reader r = i.next();
|
||||
synchronized(r) {
|
||||
r.reset();
|
||||
HStoreKey key = new HStoreKey();
|
||||
BytesWritable val = new BytesWritable();
|
||||
while(r.next(key, val)) {
|
||||
System.out.println(key);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
} finally {
|
||||
this.locker.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// This class implements the HScannerInterface.
|
||||
// It lets the caller scan the contents of this HStore.
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
class HStoreScanner extends HAbstractScanner {
|
||||
MapFile.Reader readers[];
|
||||
Text lastRow = null;
|
||||
private MapFile.Reader[] readers;
|
||||
|
||||
public HStoreScanner(long timestamp, Text targetCols[], Text firstRow) throws IOException {
|
||||
public HStoreScanner(long timestamp, Text[] targetCols, Text firstRow) throws IOException {
|
||||
super(timestamp, targetCols);
|
||||
|
||||
locker.readLock().lock();
|
||||
try {
|
||||
this.readers = new MapFile.Reader[mapFiles.size()];
|
||||
int i = 0;
|
||||
|
||||
// Most recent map file should be first
|
||||
|
||||
int i = readers.length - 1;
|
||||
for(Iterator<HStoreFile> it = mapFiles.values().iterator(); it.hasNext(); ) {
|
||||
HStoreFile curHSF = it.next();
|
||||
readers[i++] = new MapFile.Reader(fs, curHSF.getMapFilePath().toString(), conf);
|
||||
readers[i--] = new MapFile.Reader(fs, curHSF.getMapFilePath().toString(), conf);
|
||||
}
|
||||
|
||||
this.keys = new HStoreKey[readers.length];
|
||||
|
@ -952,6 +1023,7 @@ public class HStore {
|
|||
* @return - true if there is more data available
|
||||
*/
|
||||
boolean getNext(int i) throws IOException {
|
||||
vals[i] = new BytesWritable();
|
||||
if(! readers[i].next(keys[i], vals[i])) {
|
||||
closeSubScanner(i);
|
||||
return false;
|
||||
|
|
|
@ -23,9 +23,6 @@ import java.io.*;
|
|||
* A Key for a stored row
|
||||
******************************************************************************/
|
||||
public class HStoreKey implements WritableComparable {
|
||||
public static final byte[] DELETE_BYTES = "HSTOREKEY::DELETEVAL".getBytes();
|
||||
public static final byte[] COMPLETE_CACHEFLUSH = "HSTOREKEY::CACHEFLUSH".getBytes();
|
||||
|
||||
public static Text extractFamily(Text col) throws IOException {
|
||||
String column = col.toString();
|
||||
int colpos = column.indexOf(":");
|
||||
|
@ -97,32 +94,42 @@ public class HStoreKey implements WritableComparable {
|
|||
* @param other Key to compare against. Compares row and column.
|
||||
* @return True if same row and column.
|
||||
* @see {@link #matchesWithoutColumn(HStoreKey)}
|
||||
* @see {@link #matchesRowFamily(HStoreKey)}
|
||||
*/
|
||||
public boolean matchesRowCol(HStoreKey other) {
|
||||
if(this.row.compareTo(other.row) == 0 &&
|
||||
this.column.compareTo(other.column) == 0) {
|
||||
return true;
|
||||
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
return this.row.compareTo(other.row) == 0
|
||||
&& this.column.compareTo(other.column) == 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param other Key to copmare against. Compares row and
|
||||
* timestamp.
|
||||
* @return True if same row and timestamp is greater than
|
||||
* <code>other</code>
|
||||
* @param other Key to copmare against. Compares row and timestamp.
|
||||
*
|
||||
* @return True if same row and timestamp is greater than <code>other</code>
|
||||
* @see {@link #matchesRowCol(HStoreKey)}
|
||||
* @see {@link #matchesRowFamily(HStoreKey)}
|
||||
*/
|
||||
public boolean matchesWithoutColumn(HStoreKey other) {
|
||||
if((this.row.compareTo(other.row) == 0) &&
|
||||
(this.timestamp >= other.getTimestamp())) {
|
||||
return true;
|
||||
return this.row.compareTo(other.row) == 0
|
||||
&& this.timestamp >= other.getTimestamp();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param other Key to compare against. Compares row and column family
|
||||
*
|
||||
* @return true if same row and column family
|
||||
* @see {@link #matchesRowCol(HStoreKey)}
|
||||
* @see {@link #matchesWithoutColumn(HStoreKey)}
|
||||
*/
|
||||
public boolean matchesRowFamily(HStoreKey other) {
|
||||
boolean status = false;
|
||||
try {
|
||||
status = this.row.compareTo(other.row) == 0
|
||||
&& extractFamily(this.column).compareTo(
|
||||
extractFamily(other.getColumn())) == 0;
|
||||
|
||||
} else {
|
||||
return false;
|
||||
} catch(IOException e) {
|
||||
}
|
||||
return status;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
|
|
|
@ -15,26 +15,68 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import org.apache.hadoop.io.*;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.TreeSet;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.*;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
|
||||
/*******************************************************************************
|
||||
* HTableDescriptor contains various facts about an HTable, like its columns,
|
||||
* column families, etc.
|
||||
******************************************************************************/
|
||||
/**
|
||||
* HTableDescriptor contains various facts about an HTable, like
|
||||
* column families, maximum number of column versions, etc.
|
||||
*/
|
||||
public class HTableDescriptor implements WritableComparable {
|
||||
Text name;
|
||||
int maxVersions;
|
||||
TreeSet<Text> families = new TreeSet<Text>();
|
||||
|
||||
/**
|
||||
* Legal table names can only contain 'word characters':
|
||||
* i.e. <code>[a-zA-Z_0-9]</code>.
|
||||
*
|
||||
* Lets be restrictive until a reason to be otherwise.
|
||||
*/
|
||||
private static final Pattern LEGAL_TABLE_NAME =
|
||||
Pattern.compile("[\\w-]+");
|
||||
|
||||
/**
|
||||
* Legal family names can only contain 'word characters' and
|
||||
* end in a colon.
|
||||
*/
|
||||
private static final Pattern LEGAL_FAMILY_NAME =
|
||||
Pattern.compile("\\w+:");
|
||||
|
||||
public HTableDescriptor() {
|
||||
this.name = new Text();
|
||||
this.families.clear();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
* @param name Table name.
|
||||
* @param maxVersions Number of versions of a column to keep.
|
||||
* @throws IllegalArgumentException if passed a table name
|
||||
* that is made of other than 'word' characters: i.e.
|
||||
* <code>[a-zA-Z_0-9]
|
||||
*/
|
||||
public HTableDescriptor(String name, int maxVersions) {
|
||||
Matcher m = LEGAL_TABLE_NAME.matcher(name);
|
||||
if (m == null || !m.matches()) {
|
||||
throw new IllegalArgumentException("Table names can only " +
|
||||
"contain 'word characters': i.e. [a-zA-Z_0-9");
|
||||
}
|
||||
if (maxVersions <= 0) {
|
||||
// TODO: Allow maxVersion of 0 to be the way you say
|
||||
// "Keep all versions". Until there is support, consider
|
||||
// 0 -- or < 0 -- a configuration error.
|
||||
throw new IllegalArgumentException("Maximum versions " +
|
||||
"must be positive");
|
||||
}
|
||||
this.name = new Text(name);
|
||||
this.maxVersions = maxVersions;
|
||||
}
|
||||
|
@ -47,19 +89,28 @@ public class HTableDescriptor implements WritableComparable {
|
|||
return maxVersions;
|
||||
}
|
||||
|
||||
/** Add a column */
|
||||
/**
|
||||
* Add a column family.
|
||||
* @param family Column family name to add. Column family names
|
||||
* must end in a <code>:</code>
|
||||
* @throws IllegalArgumentException if passed a table name
|
||||
* that is made of other than 'word' characters: i.e.
|
||||
* <code>[a-zA-Z_0-9]
|
||||
*/
|
||||
public void addFamily(Text family) {
|
||||
String familyStr = family.toString();
|
||||
Matcher m = LEGAL_FAMILY_NAME.matcher(familyStr);
|
||||
if (m == null || !m.matches()) {
|
||||
throw new IllegalArgumentException("Family names can " +
|
||||
"only contain 'word characters' and must end with a " +
|
||||
"':'");
|
||||
}
|
||||
families.add(family);
|
||||
}
|
||||
|
||||
/** Do we contain a given column? */
|
||||
public boolean hasFamily(Text family) {
|
||||
if(families.contains(family)) {
|
||||
return true;
|
||||
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
return families.contains(family);
|
||||
}
|
||||
|
||||
/** All the column families in this table. */
|
||||
|
@ -67,6 +118,12 @@ public class HTableDescriptor implements WritableComparable {
|
|||
return families;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "name: " + this.name.toString() +
|
||||
", maxVersions: " + this.maxVersions + ", families: " + this.families;
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
// Writable
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
@ -120,4 +177,4 @@ public class HTableDescriptor implements WritableComparable {
|
|||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -31,9 +31,9 @@ public class LabelledData implements Writable {
|
|||
this.data = new BytesWritable();
|
||||
}
|
||||
|
||||
public LabelledData(Text label, byte[] data) {
|
||||
public LabelledData(Text label, BytesWritable data) {
|
||||
this.label = new Text(label);
|
||||
this.data = new BytesWritable(data);
|
||||
this.data = data;
|
||||
}
|
||||
|
||||
public Text getLabel() {
|
||||
|
|
|
@ -50,6 +50,7 @@ public class Leases {
|
|||
|
||||
this.leaseMonitor = new LeaseMonitor();
|
||||
this.leaseMonitorThread = new Thread(leaseMonitor);
|
||||
this.leaseMonitorThread.setName("Lease.monitor");
|
||||
leaseMonitorThread.start();
|
||||
}
|
||||
|
||||
|
@ -60,6 +61,7 @@ public class Leases {
|
|||
public void close() {
|
||||
this.running = false;
|
||||
try {
|
||||
this.leaseMonitorThread.interrupt();
|
||||
this.leaseMonitorThread.join();
|
||||
} catch (InterruptedException iex) {
|
||||
}
|
||||
|
|
|
@ -0,0 +1,28 @@
|
|||
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 3.2 Final//EN">
|
||||
<html>
|
||||
<head />
|
||||
<body bgcolor="white">
|
||||
Provides HBase, the <a href="http://lucene.apache.org/hadoop">Hadoop</a> simple database.
|
||||
|
||||
<h2>Requirements</h2>
|
||||
<ul>
|
||||
<li><a href="http://lucene.apache.org/hadoop">Hadoop</a> It has its own set of <a href="http://lucene.apache.org/hadoop/api/overview-summary.html">requirements</a> (Scroll down the page).</li>
|
||||
<li>Java 1.5.x, preferably from <a href="http://lucene.apache.org/hadoop/api/index.html">Sun</a> Set JAVA_HOME to the root of your Java installation</li>
|
||||
|
||||
<h2>Getting Started</h2>
|
||||
<p>First, you need a working instance of Hadoop. Download releases at <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/">Hadoop downloads</a>.
|
||||
Unpack the release and connect to its top-level directory. Edit the file conf/hadoop-env.sh to define at least JAVA_HOME. Try the following command:
|
||||
<pre>bin/hadoop
|
||||
</pre>
|
||||
This will display the documentation for the Hadoop command script.
|
||||
</p>
|
||||
<p>TODO</p>
|
||||
|
||||
<h2>Related Documentation</h2>
|
||||
|
||||
<ul>
|
||||
<li><a href="http://wiki.apache.org/lucene-hadoop/Hbase/HbaseArchitecture">Hbase/HbaseArchitecture</a>
|
||||
</ul>
|
||||
|
||||
</body>
|
||||
</html>
|
|
@ -22,22 +22,38 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.dfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* This class creates a single process HBase cluster for junit testing.
|
||||
* One thread is created for each server.
|
||||
*/
|
||||
public class MiniHBaseCluster implements HConstants {
|
||||
private static final Logger LOG =
|
||||
Logger.getLogger(MiniHBaseCluster.class.getName());
|
||||
private Configuration conf;
|
||||
private MiniDFSCluster cluster;
|
||||
private FileSystem fs;
|
||||
private Path parentdir;
|
||||
private HMasterRunner master;
|
||||
private Thread masterThread;
|
||||
private HMasterRunner masterRunner;
|
||||
private Thread masterRunnerThread;
|
||||
private HRegionServerRunner[] regionServers;
|
||||
private Thread[] regionThreads;
|
||||
|
||||
public MiniHBaseCluster(Configuration conf, int nRegionNodes) {
|
||||
this(conf, nRegionNodes, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
* @param conf
|
||||
* @param nRegionNodes
|
||||
* @param miniHdfsFilesystem If true, set the hbase mini
|
||||
* cluster atop a mini hdfs cluster. Otherwise, use the
|
||||
* filesystem configured in <code>conf</code>.
|
||||
*/
|
||||
public MiniHBaseCluster(Configuration conf, int nRegionNodes,
|
||||
final boolean miniHdfsFilesystem) {
|
||||
this.conf = conf;
|
||||
|
||||
try {
|
||||
|
@ -47,21 +63,20 @@ public class MiniHBaseCluster implements HConstants {
|
|||
"build/contrib/hbase/test");
|
||||
|
||||
String dir = testDir.getAbsolutePath();
|
||||
System.out.println(dir);
|
||||
LOG.info("Setting test.build.data to " + dir);
|
||||
System.setProperty("test.build.data", dir);
|
||||
}
|
||||
|
||||
// To run using configured filesystem, comment out this
|
||||
// line below that starts up the MiniDFSCluster.
|
||||
this.cluster = new MiniDFSCluster(this.conf, 2, true, (String[])null);
|
||||
if (miniHdfsFilesystem) {
|
||||
this.cluster =
|
||||
new MiniDFSCluster(this.conf, 2, true, (String[])null);
|
||||
}
|
||||
this.fs = FileSystem.get(conf);
|
||||
this.parentdir =
|
||||
new Path(conf.get(HREGION_DIR, DEFAULT_HREGION_DIR));
|
||||
this.parentdir = new Path(conf.get(HREGION_DIR, DEFAULT_HREGION_DIR));
|
||||
fs.mkdirs(parentdir);
|
||||
|
||||
} catch(Throwable e) {
|
||||
System.err.println("Mini DFS cluster failed to start");
|
||||
e.printStackTrace();
|
||||
LOG.error("Failed setup of FileSystem", e);
|
||||
throw e;
|
||||
}
|
||||
|
||||
|
@ -70,28 +85,27 @@ public class MiniHBaseCluster implements HConstants {
|
|||
}
|
||||
|
||||
// Create the master
|
||||
|
||||
this.master = new HMasterRunner();
|
||||
this.masterThread = new Thread(master, "HMaster");
|
||||
this.masterRunner = new HMasterRunner();
|
||||
this.masterRunnerThread = new Thread(masterRunner, "masterRunner");
|
||||
|
||||
// Start up the master
|
||||
|
||||
masterThread.start();
|
||||
while(! master.isCrashed() && ! master.isInitialized()) {
|
||||
LOG.info("Starting HMaster");
|
||||
masterRunnerThread.start();
|
||||
while(! masterRunner.isCrashed() && ! masterRunner.isInitialized()) {
|
||||
try {
|
||||
System.err.println("Waiting for HMaster to initialize...");
|
||||
LOG.info("...waiting for HMaster to initialize...");
|
||||
Thread.sleep(1000);
|
||||
|
||||
} catch(InterruptedException e) {
|
||||
}
|
||||
if(master.isCrashed()) {
|
||||
if(masterRunner.isCrashed()) {
|
||||
throw new RuntimeException("HMaster crashed");
|
||||
}
|
||||
}
|
||||
|
||||
LOG.info("HMaster started.");
|
||||
|
||||
// Set the master's port for the HRegionServers
|
||||
|
||||
this.conf.set(MASTER_ADDRESS, master.getHMasterAddress().toString());
|
||||
String address = masterRunner.getHMasterAddress().toString();
|
||||
this.conf.set(MASTER_ADDRESS, address);
|
||||
|
||||
// Start the HRegionServers
|
||||
|
||||
|
@ -99,28 +113,26 @@ public class MiniHBaseCluster implements HConstants {
|
|||
this.conf.set(REGIONSERVER_ADDRESS, "localhost:0");
|
||||
}
|
||||
|
||||
LOG.info("Starting HRegionServers");
|
||||
startRegionServers(this.conf, nRegionNodes);
|
||||
LOG.info("HRegionServers running");
|
||||
|
||||
// Wait for things to get started
|
||||
|
||||
while(! master.isCrashed() && ! master.isUp()) {
|
||||
while(! masterRunner.isCrashed() && ! masterRunner.isUp()) {
|
||||
try {
|
||||
System.err.println("Waiting for Mini HBase cluster to start...");
|
||||
LOG.info("Waiting for Mini HBase cluster to start...");
|
||||
Thread.sleep(1000);
|
||||
|
||||
} catch(InterruptedException e) {
|
||||
}
|
||||
if(master.isCrashed()) {
|
||||
if(masterRunner.isCrashed()) {
|
||||
throw new RuntimeException("HMaster crashed");
|
||||
}
|
||||
}
|
||||
|
||||
} catch(Throwable e) {
|
||||
|
||||
// Delete all DFS files
|
||||
|
||||
deleteFile(new File(System.getProperty("test.build.data"), "dfs"));
|
||||
|
||||
throw new RuntimeException("Mini HBase cluster did not start");
|
||||
}
|
||||
}
|
||||
|
@ -141,39 +153,35 @@ public class MiniHBaseCluster implements HConstants {
|
|||
* supplied port is not necessarily the actual port used.
|
||||
*/
|
||||
public HServerAddress getHMasterAddress() {
|
||||
return master.getHMasterAddress();
|
||||
return masterRunner.getHMasterAddress();
|
||||
}
|
||||
|
||||
/** Shut down the HBase cluster */
|
||||
public void shutdown() {
|
||||
System.out.println("Shutting down the HBase Cluster");
|
||||
LOG.info("Shutting down the HBase Cluster");
|
||||
for(int i = 0; i < regionServers.length; i++) {
|
||||
regionServers[i].shutdown();
|
||||
}
|
||||
master.shutdown();
|
||||
|
||||
masterRunner.shutdown();
|
||||
for(int i = 0; i < regionServers.length; i++) {
|
||||
try {
|
||||
regionThreads[i].join();
|
||||
|
||||
} catch(InterruptedException e) {
|
||||
} catch (InterruptedException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
try {
|
||||
masterThread.join();
|
||||
|
||||
} catch(InterruptedException e) {
|
||||
masterRunnerThread.join();
|
||||
} catch (InterruptedException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
|
||||
System.out.println("Shutting down Mini DFS cluster");
|
||||
if (cluster != null) {
|
||||
LOG.info("Shutting down Mini DFS cluster");
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
||||
// Delete all DFS files
|
||||
|
||||
deleteFile(new File(System.getProperty("test.build.data"), "dfs"));
|
||||
|
||||
}
|
||||
|
||||
private void deleteFile(File f) {
|
||||
|
@ -188,12 +196,14 @@ public class MiniHBaseCluster implements HConstants {
|
|||
|
||||
private class HMasterRunner implements Runnable {
|
||||
private HMaster master = null;
|
||||
private Thread masterThread = null;
|
||||
private volatile boolean isInitialized = false;
|
||||
private boolean isCrashed = false;
|
||||
private boolean isRunning = true;
|
||||
private long threadSleepTime = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000);
|
||||
|
||||
public HServerAddress getHMasterAddress() {
|
||||
return master.getMasterAddress();
|
||||
return this.master.getMasterAddress();
|
||||
}
|
||||
|
||||
public synchronized boolean isInitialized() {
|
||||
|
@ -218,33 +228,46 @@ public class MiniHBaseCluster implements HConstants {
|
|||
try {
|
||||
synchronized(this) {
|
||||
if(isRunning) {
|
||||
master = new HMaster(conf);
|
||||
this.master = new HMaster(conf);
|
||||
masterThread = new Thread(this.master);
|
||||
masterThread.start();
|
||||
}
|
||||
isInitialized = true;
|
||||
}
|
||||
} catch(Throwable e) {
|
||||
shutdown();
|
||||
System.err.println("HMaster crashed:");
|
||||
e.printStackTrace();
|
||||
LOG.error("HMaster crashed:", e);
|
||||
synchronized(this) {
|
||||
isCrashed = true;
|
||||
}
|
||||
}
|
||||
|
||||
while(this.master != null && this.master.isMasterRunning()) {
|
||||
try {
|
||||
Thread.sleep(threadSleepTime);
|
||||
|
||||
} catch(InterruptedException e) {
|
||||
}
|
||||
}
|
||||
synchronized(this) {
|
||||
isCrashed = true;
|
||||
}
|
||||
shutdown();
|
||||
}
|
||||
|
||||
/** Shut down the HMaster and wait for it to finish */
|
||||
public synchronized void shutdown() {
|
||||
isRunning = false;
|
||||
if(master != null) {
|
||||
if (this.master != null) {
|
||||
try {
|
||||
master.stop();
|
||||
|
||||
this.master.shutdown();
|
||||
} catch(IOException e) {
|
||||
System.err.println("Master crashed during stop");
|
||||
e.printStackTrace();
|
||||
|
||||
LOG.error("Master crashed during stop", e);
|
||||
} finally {
|
||||
master.join();
|
||||
try {
|
||||
masterThread.join();
|
||||
} catch(InterruptedException e) {
|
||||
}
|
||||
master = null;
|
||||
}
|
||||
}
|
||||
|
@ -272,8 +295,7 @@ public class MiniHBaseCluster implements HConstants {
|
|||
|
||||
} catch(Throwable e) {
|
||||
shutdown();
|
||||
System.err.println("HRegionServer crashed:");
|
||||
e.printStackTrace();
|
||||
LOG.error("HRegionServer crashed:", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -285,9 +307,7 @@ public class MiniHBaseCluster implements HConstants {
|
|||
server.stop();
|
||||
|
||||
} catch(IOException e) {
|
||||
System.err.println("HRegionServer crashed during stop");
|
||||
e.printStackTrace();
|
||||
|
||||
LOG.error("HRegionServer crashed during stop", e);
|
||||
} finally {
|
||||
server.join();
|
||||
server = null;
|
||||
|
|
|
@ -0,0 +1,231 @@
|
|||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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.ByteArrayOutputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Enumeration;
|
||||
import java.util.Iterator;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.dfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
||||
import org.apache.log4j.Appender;
|
||||
import org.apache.log4j.ConsoleAppender;
|
||||
import org.apache.log4j.Layout;
|
||||
import org.apache.log4j.Level;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.log4j.PatternLayout;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
public class TestGet extends TestCase {
|
||||
private static final Text CONTENTS = new Text("contents:");
|
||||
private static final Text ROW_KEY = new Text(HGlobals.rootRegionInfo.regionName);
|
||||
|
||||
|
||||
private void dumpRegion(HRegion r) throws IOException {
|
||||
for(Iterator<HStore> i = r.stores.values().iterator(); i.hasNext(); ) {
|
||||
i.next().dumpMaps();
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyGet(HRegion r) throws IOException {
|
||||
// This should return a value because there is only one family member
|
||||
|
||||
BytesWritable value = r.get(ROW_KEY, CONTENTS);
|
||||
assertNotNull(value);
|
||||
|
||||
// This should not return a value because there are multiple family members
|
||||
|
||||
value = r.get(ROW_KEY, HConstants.COLUMN_FAMILY);
|
||||
assertNull(value);
|
||||
|
||||
// Find out what getFull returns
|
||||
|
||||
TreeMap<Text, BytesWritable> values = r.getFull(ROW_KEY);
|
||||
//assertEquals(4, values.keySet().size());
|
||||
for(Iterator<Text> i = values.keySet().iterator(); i.hasNext(); ) {
|
||||
Text column = i.next();
|
||||
System.out.println(column);
|
||||
if(column.equals(HConstants.COL_SERVER)) {
|
||||
BytesWritable val = values.get(column);
|
||||
byte[] bytes = new byte[val.getSize()];
|
||||
System.arraycopy(val.get(), 0, bytes, 0, bytes.length);
|
||||
System.out.println(" " + new String(bytes, HConstants.UTF8_ENCODING));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testGet() throws IOException {
|
||||
MiniDFSCluster cluster = null;
|
||||
|
||||
try {
|
||||
|
||||
// Initialization
|
||||
|
||||
if(System.getProperty("test.build.data") == null) {
|
||||
String dir = new File(new File("").getAbsolutePath(), "build/contrib/hbase/test").getAbsolutePath();
|
||||
System.out.println(dir);
|
||||
System.setProperty("test.build.data", dir);
|
||||
}
|
||||
Configuration conf = new HBaseConfiguration();
|
||||
|
||||
Environment.getenv();
|
||||
if(Environment.debugging) {
|
||||
Logger rootLogger = Logger.getRootLogger();
|
||||
rootLogger.setLevel(Level.WARN);
|
||||
|
||||
ConsoleAppender consoleAppender = null;
|
||||
for(Enumeration<Appender> e = (Enumeration<Appender>)rootLogger.getAllAppenders();
|
||||
e.hasMoreElements();) {
|
||||
|
||||
Appender a = e.nextElement();
|
||||
if(a instanceof ConsoleAppender) {
|
||||
consoleAppender = (ConsoleAppender)a;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if(consoleAppender != null) {
|
||||
Layout layout = consoleAppender.getLayout();
|
||||
if(layout instanceof PatternLayout) {
|
||||
PatternLayout consoleLayout = (PatternLayout)layout;
|
||||
consoleLayout.setConversionPattern("%d %-5p [%t] %l: %m%n");
|
||||
}
|
||||
}
|
||||
Logger.getLogger("org.apache.hadoop.hbase").setLevel(Environment.logLevel);
|
||||
}
|
||||
cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
Path dir = new Path("/hbase");
|
||||
fs.mkdirs(dir);
|
||||
|
||||
HTableDescriptor desc = new HTableDescriptor("test", 1);
|
||||
desc.addFamily(CONTENTS);
|
||||
desc.addFamily(HConstants.COLUMN_FAMILY);
|
||||
|
||||
HRegionInfo info = new HRegionInfo(0L, desc, null, null);
|
||||
Path regionDir = HStoreFile.getHRegionDir(dir, info.regionName);
|
||||
fs.mkdirs(regionDir);
|
||||
|
||||
HLog log = new HLog(fs, new Path(regionDir, "log"), conf);
|
||||
|
||||
HRegion r = new HRegion(dir, log, fs, conf, info, null, null);
|
||||
|
||||
// Write information to the table
|
||||
|
||||
long lockid = r.startUpdate(ROW_KEY);
|
||||
ByteArrayOutputStream bytes = new ByteArrayOutputStream();
|
||||
DataOutputStream s = new DataOutputStream(bytes);
|
||||
CONTENTS.write(s);
|
||||
r.put(lockid, CONTENTS, new BytesWritable(bytes.toByteArray()));
|
||||
|
||||
bytes.reset();
|
||||
HGlobals.rootRegionInfo.write(s);
|
||||
|
||||
r.put(lockid, HConstants.COL_REGIONINFO, new BytesWritable(bytes.toByteArray()));
|
||||
|
||||
r.commit(lockid);
|
||||
|
||||
lockid = r.startUpdate(ROW_KEY);
|
||||
|
||||
r.put(lockid, HConstants.COL_SERVER,
|
||||
new BytesWritable(
|
||||
new HServerAddress("foo.bar.com:1234").toString().getBytes(HConstants.UTF8_ENCODING)
|
||||
)
|
||||
);
|
||||
|
||||
r.put(lockid, HConstants.COL_STARTCODE,
|
||||
new BytesWritable(
|
||||
String.valueOf(lockid).getBytes(HConstants.UTF8_ENCODING)
|
||||
)
|
||||
);
|
||||
|
||||
r.put(lockid, new Text(HConstants.COLUMN_FAMILY + "region"),
|
||||
new BytesWritable("region".getBytes(HConstants.UTF8_ENCODING)));
|
||||
|
||||
r.commit(lockid);
|
||||
|
||||
// Verify that get works the same from memcache as when reading from disk
|
||||
// NOTE dumpRegion won't work here because it only reads from disk.
|
||||
|
||||
verifyGet(r);
|
||||
|
||||
// Close and re-open region, forcing updates to disk
|
||||
|
||||
r.close();
|
||||
log.rollWriter();
|
||||
r = new HRegion(dir, log, fs, conf, info, null, null);
|
||||
|
||||
// Read it back
|
||||
|
||||
dumpRegion(r);
|
||||
verifyGet(r);
|
||||
|
||||
// Update one family member and add a new one
|
||||
|
||||
lockid = r.startUpdate(ROW_KEY);
|
||||
|
||||
r.put(lockid, new Text(HConstants.COLUMN_FAMILY + "region"),
|
||||
new BytesWritable("region2".getBytes()));
|
||||
|
||||
r.put(lockid, HConstants.COL_SERVER,
|
||||
new BytesWritable(
|
||||
new HServerAddress("bar.foo.com:4321").toString().getBytes(HConstants.UTF8_ENCODING)
|
||||
)
|
||||
);
|
||||
|
||||
r.put(lockid, new Text(HConstants.COLUMN_FAMILY + "junk"),
|
||||
new BytesWritable("junk".getBytes()));
|
||||
|
||||
r.commit(lockid);
|
||||
|
||||
verifyGet(r);
|
||||
|
||||
// Close region and re-open it
|
||||
|
||||
r.close();
|
||||
log.rollWriter();
|
||||
r = new HRegion(dir, log, fs, conf, info, null, null);
|
||||
|
||||
// Read it back
|
||||
|
||||
dumpRegion(r);
|
||||
verifyGet(r);
|
||||
|
||||
// Close region once and for all
|
||||
|
||||
r.close();
|
||||
|
||||
} catch(IOException e) {
|
||||
e.printStackTrace();
|
||||
throw e;
|
||||
|
||||
} finally {
|
||||
if(cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,289 @@
|
|||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Enumeration;
|
||||
import java.util.Iterator;
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
import junit.framework.Test;
|
||||
import junit.framework.TestSuite;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
||||
import org.apache.log4j.Appender;
|
||||
import org.apache.log4j.ConsoleAppender;
|
||||
import org.apache.log4j.Layout;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.log4j.Level;
|
||||
import org.apache.log4j.PatternLayout;
|
||||
|
||||
/**
|
||||
* Test HBase Master and Region servers, client API
|
||||
*/
|
||||
public class TestHBaseCluster extends TestCase {
|
||||
|
||||
/** constructor */
|
||||
public TestHBaseCluster(String name) {
|
||||
super(name);
|
||||
}
|
||||
|
||||
/** Test suite so that all tests get run */
|
||||
public static Test suite() {
|
||||
TestSuite suite = new TestSuite();
|
||||
suite.addTest(new TestHBaseCluster("testSetup"));
|
||||
suite.addTest(new TestHBaseCluster("testBasic"));
|
||||
suite.addTest(new TestHBaseCluster("testScanner"));
|
||||
suite.addTest(new TestHBaseCluster("testCleanup"));
|
||||
return suite;
|
||||
}
|
||||
|
||||
private static final int FIRST_ROW = 1;
|
||||
private static final int NUM_VALS = 1000;
|
||||
private static final Text CONTENTS = new Text("contents:");
|
||||
private static final Text CONTENTS_BASIC = new Text("contents:basic");
|
||||
private static final String CONTENTSTR = "contentstr";
|
||||
private static final Text ANCHOR = new Text("anchor:");
|
||||
private static final String ANCHORNUM = "anchor:anchornum-";
|
||||
private static final String ANCHORSTR = "anchorstr";
|
||||
|
||||
private static Configuration conf = null;
|
||||
private static boolean failures = false;
|
||||
private static boolean initialized = false;
|
||||
private static MiniHBaseCluster cluster = null;
|
||||
private static HTableDescriptor desc = null;
|
||||
private static HClient client = null;
|
||||
|
||||
// Set up environment, start mini cluster, etc.
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testSetup() throws Exception {
|
||||
try {
|
||||
if(System.getProperty("test.build.data") == null) {
|
||||
String dir = new File(new File("").getAbsolutePath(), "build/contrib/hbase/test").getAbsolutePath();
|
||||
System.out.println(dir);
|
||||
System.setProperty("test.build.data", dir);
|
||||
}
|
||||
conf = new HBaseConfiguration();
|
||||
|
||||
Environment.getenv();
|
||||
if(Environment.debugging) {
|
||||
Logger rootLogger = Logger.getRootLogger();
|
||||
rootLogger.setLevel(Level.WARN);
|
||||
|
||||
ConsoleAppender consoleAppender = null;
|
||||
for(Enumeration<Appender> e = (Enumeration<Appender>)rootLogger.getAllAppenders();
|
||||
e.hasMoreElements();) {
|
||||
|
||||
Appender a = e.nextElement();
|
||||
if(a instanceof ConsoleAppender) {
|
||||
consoleAppender = (ConsoleAppender)a;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if(consoleAppender != null) {
|
||||
Layout layout = consoleAppender.getLayout();
|
||||
if(layout instanceof PatternLayout) {
|
||||
PatternLayout consoleLayout = (PatternLayout)layout;
|
||||
consoleLayout.setConversionPattern("%d %-5p [%t] %l: %m%n");
|
||||
}
|
||||
}
|
||||
Logger.getLogger("org.apache.hadoop.hbase").setLevel(Environment.logLevel);
|
||||
}
|
||||
cluster = new MiniHBaseCluster(conf, 1);
|
||||
client = new HClient(conf);
|
||||
|
||||
desc = new HTableDescriptor("test", 3);
|
||||
desc.addFamily(new Text(CONTENTS));
|
||||
desc.addFamily(new Text(ANCHOR));
|
||||
client.createTable(desc);
|
||||
|
||||
} catch(Exception e) {
|
||||
failures = true;
|
||||
throw e;
|
||||
}
|
||||
initialized = true;
|
||||
}
|
||||
|
||||
// Test basic functionality. Writes to contents:basic and anchor:anchornum-*
|
||||
|
||||
public void testBasic() throws IOException {
|
||||
if(!initialized) {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
try {
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
client.openTable(desc.getName());
|
||||
|
||||
// Write out a bunch of values
|
||||
|
||||
for (int k = FIRST_ROW; k <= NUM_VALS; k++) {
|
||||
long writeid = client.startUpdate(new Text("row_" + k));
|
||||
client.put(writeid, CONTENTS_BASIC, (CONTENTSTR + k).getBytes());
|
||||
client.put(writeid, new Text(ANCHORNUM + k), (ANCHORSTR + k).getBytes());
|
||||
client.commit(writeid);
|
||||
}
|
||||
System.out.println("Write " + NUM_VALS + " rows. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
// Read them back in
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
Text collabel = null;
|
||||
for (int k = FIRST_ROW; k <= NUM_VALS; k++) {
|
||||
Text rowlabel = new Text("row_" + k);
|
||||
|
||||
byte bodydata[] = client.get(rowlabel, CONTENTS_BASIC);
|
||||
assertNotNull(bodydata);
|
||||
String bodystr = new String(bodydata).toString().trim();
|
||||
String teststr = CONTENTSTR + k;
|
||||
assertEquals("Incorrect value for key: (" + rowlabel + "," + CONTENTS_BASIC
|
||||
+ "), expected: '" + teststr + "' got: '" + bodystr + "'",
|
||||
bodystr, teststr);
|
||||
collabel = new Text(ANCHORNUM + k);
|
||||
bodydata = client.get(rowlabel, collabel);
|
||||
bodystr = new String(bodydata).toString().trim();
|
||||
teststr = ANCHORSTR + k;
|
||||
assertEquals("Incorrect value for key: (" + rowlabel + "," + collabel
|
||||
+ "), expected: '" + teststr + "' got: '" + bodystr + "'",
|
||||
bodystr, teststr);
|
||||
}
|
||||
|
||||
System.out.println("Read " + NUM_VALS + " rows. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
} catch(IOException e) {
|
||||
failures = true;
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
public void testScanner() throws IOException {
|
||||
if(!initialized || failures) {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
Text[] cols = new Text[] {
|
||||
new Text(ANCHORNUM + "[0-9]+"),
|
||||
new Text(CONTENTS_BASIC)
|
||||
};
|
||||
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
HScannerInterface s = client.obtainScanner(cols, new Text());
|
||||
try {
|
||||
|
||||
int contentsFetched = 0;
|
||||
int anchorFetched = 0;
|
||||
HStoreKey curKey = new HStoreKey();
|
||||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
String curval = new String(val).trim();
|
||||
|
||||
if(col.compareTo(CONTENTS_BASIC) == 0) {
|
||||
assertTrue("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp()
|
||||
+ ", Value for " + col + " should start with: " + CONTENTSTR
|
||||
+ ", but was fetched as: " + curval,
|
||||
curval.startsWith(CONTENTSTR));
|
||||
contentsFetched++;
|
||||
|
||||
} else if(col.toString().startsWith(ANCHORNUM)) {
|
||||
assertTrue("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp()
|
||||
+ ", Value for " + col + " should start with: " + ANCHORSTR
|
||||
+ ", but was fetched as: " + curval,
|
||||
curval.startsWith(ANCHORSTR));
|
||||
anchorFetched++;
|
||||
|
||||
} else {
|
||||
System.out.println(col);
|
||||
}
|
||||
}
|
||||
curVals.clear();
|
||||
k++;
|
||||
}
|
||||
assertEquals("Expected " + NUM_VALS + " " + CONTENTS_BASIC + " values, but fetched " + contentsFetched, NUM_VALS, contentsFetched);
|
||||
assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM + " values, but fetched " + anchorFetched, NUM_VALS, anchorFetched);
|
||||
|
||||
System.out.println("Scanned " + NUM_VALS
|
||||
+ " rows. Elapsed time: "
|
||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||
|
||||
} catch(IOException e) {
|
||||
failures = true;
|
||||
throw e;
|
||||
|
||||
} finally {
|
||||
s.close();
|
||||
}
|
||||
}
|
||||
|
||||
public void testListTables() throws IOException {
|
||||
if(!initialized || failures) {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
try {
|
||||
HTableDescriptor[] tables = client.listTables();
|
||||
assertEquals(1, tables.length);
|
||||
assertEquals(desc.getName(), tables[0].getName());
|
||||
TreeSet<Text> families = tables[0].families();
|
||||
assertEquals(2, families.size());
|
||||
assertTrue(families.contains(new Text(CONTENTS)));
|
||||
assertTrue(families.contains(new Text(ANCHOR)));
|
||||
|
||||
} catch(IOException e) {
|
||||
failures = true;
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
public void testCleanup() throws IOException {
|
||||
if(!initialized) {
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
try {
|
||||
if(!failures) {
|
||||
// Delete the table we created
|
||||
|
||||
client.deleteTable(desc.getName());
|
||||
try {
|
||||
Thread.sleep(60000); // Wait for table to be deleted
|
||||
|
||||
} catch(InterruptedException e) {
|
||||
}
|
||||
}
|
||||
|
||||
} finally {
|
||||
// Shut down the cluster
|
||||
|
||||
cluster.shutdown();
|
||||
client.close();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -16,6 +16,8 @@
|
|||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
@ -26,11 +28,8 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.HMemcache.Snapshot;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
public class TestHMemcache extends TestCase {
|
||||
private final Logger LOG =
|
||||
Logger.getLogger(this.getClass().getName());
|
||||
|
||||
private HMemcache hmemcache;
|
||||
|
||||
|
@ -77,10 +76,10 @@ public class TestHMemcache extends TestCase {
|
|||
*/
|
||||
private void addRows(final HMemcache hmc) {
|
||||
for (int i = 0; i < ROW_COUNT; i++) {
|
||||
TreeMap<Text, byte[]> columns = new TreeMap<Text, byte[]>();
|
||||
TreeMap<Text, BytesWritable> columns = new TreeMap<Text, BytesWritable>();
|
||||
for (int ii = 0; ii < COLUMNS_COUNT; ii++) {
|
||||
Text k = getColumnName(i, ii);
|
||||
columns.put(k, k.toString().getBytes());
|
||||
columns.put(k, new BytesWritable(k.toString().getBytes()));
|
||||
}
|
||||
hmc.add(getRowName(i), columns, System.currentTimeMillis());
|
||||
}
|
||||
|
@ -139,7 +138,7 @@ public class TestHMemcache extends TestCase {
|
|||
}
|
||||
|
||||
private void isExpectedRow(final int rowIndex,
|
||||
TreeMap<Text, byte[]> row) {
|
||||
TreeMap<Text, BytesWritable> row) {
|
||||
int i = 0;
|
||||
for (Text colname: row.keySet()) {
|
||||
String expectedColname =
|
||||
|
@ -150,8 +149,10 @@ public class TestHMemcache extends TestCase {
|
|||
// 100 bytes in size at least. This is the default size
|
||||
// for BytesWriteable. For comparison, comvert bytes to
|
||||
// String and trim to remove trailing null bytes.
|
||||
String colvalueStr =
|
||||
new String(row.get(colname)).trim();
|
||||
BytesWritable value = row.get(colname);
|
||||
byte[] bytes = new byte[value.getSize()];
|
||||
System.arraycopy(value.get(), 0, bytes, 0, bytes.length);
|
||||
String colvalueStr = new String(bytes).trim();
|
||||
assertEquals("Content", colnameStr, colvalueStr);
|
||||
}
|
||||
}
|
||||
|
@ -160,7 +161,7 @@ public class TestHMemcache extends TestCase {
|
|||
addRows(this.hmemcache);
|
||||
for (int i = 0; i < ROW_COUNT; i++) {
|
||||
HStoreKey hsk = new HStoreKey(getRowName(i));
|
||||
TreeMap<Text, byte[]> all = this.hmemcache.getFull(hsk);
|
||||
TreeMap<Text, BytesWritable> all = this.hmemcache.getFull(hsk);
|
||||
isExpectedRow(i, all);
|
||||
}
|
||||
}
|
||||
|
@ -174,16 +175,22 @@ public class TestHMemcache extends TestCase {
|
|||
cols[(ii + (i * COLUMNS_COUNT))] = getColumnName(i, ii);
|
||||
}
|
||||
}
|
||||
HScannerInterface scanner =
|
||||
HInternalScannerInterface scanner =
|
||||
this.hmemcache.getScanner(timestamp, cols, new Text());
|
||||
HStoreKey key = new HStoreKey();
|
||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||
TreeMap<Text, BytesWritable> results = new TreeMap<Text, BytesWritable>();
|
||||
for (int i = 0; scanner.next(key, results); i++) {
|
||||
assertTrue("Row name",
|
||||
key.toString().startsWith(getRowName(i).toString()));
|
||||
assertEquals("Count of columns", COLUMNS_COUNT,
|
||||
results.size());
|
||||
isExpectedRow(i, results);
|
||||
TreeMap<Text, BytesWritable> row = new TreeMap<Text, BytesWritable>();
|
||||
for(Iterator<Map.Entry<Text, BytesWritable>> it = results.entrySet().iterator();
|
||||
it.hasNext(); ) {
|
||||
Map.Entry<Text, BytesWritable> e = it.next();
|
||||
row.put(e.getKey(), e.getValue());
|
||||
}
|
||||
isExpectedRow(i, row);
|
||||
// Clear out set. Otherwise row results accumulate.
|
||||
results.clear();
|
||||
}
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.dfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
||||
import org.apache.log4j.Appender;
|
||||
|
@ -167,8 +168,11 @@ public class TestHRegion extends TestCase {
|
|||
|
||||
for (int k = FIRST_ROW; k <= NUM_VALS; k++) {
|
||||
long writeid = region.startUpdate(new Text("row_" + k));
|
||||
region.put(writeid, CONTENTS_BASIC, (CONTENTSTR + k).getBytes());
|
||||
region.put(writeid, new Text(ANCHORNUM + k), (ANCHORSTR + k).getBytes());
|
||||
region.put(writeid, CONTENTS_BASIC,
|
||||
new BytesWritable((CONTENTSTR + k).getBytes()));
|
||||
|
||||
region.put(writeid, new Text(ANCHORNUM + k),
|
||||
new BytesWritable((ANCHORSTR + k).getBytes()));
|
||||
region.commit(writeid);
|
||||
}
|
||||
System.out.println("Write " + NUM_VALS + " rows. Elapsed time: "
|
||||
|
@ -191,16 +195,20 @@ public class TestHRegion extends TestCase {
|
|||
for (int k = FIRST_ROW; k <= NUM_VALS; k++) {
|
||||
Text rowlabel = new Text("row_" + k);
|
||||
|
||||
byte bodydata[] = region.get(rowlabel, CONTENTS_BASIC);
|
||||
BytesWritable bodydata = region.get(rowlabel, CONTENTS_BASIC);
|
||||
assertNotNull(bodydata);
|
||||
String bodystr = new String(bodydata).toString().trim();
|
||||
byte[] bytes = new byte[bodydata.getSize()];
|
||||
System.arraycopy(bodydata.get(), 0, bytes, 0, bytes.length);
|
||||
String bodystr = new String(bytes).toString().trim();
|
||||
String teststr = CONTENTSTR + k;
|
||||
assertEquals("Incorrect value for key: (" + rowlabel + "," + CONTENTS_BASIC
|
||||
+ "), expected: '" + teststr + "' got: '" + bodystr + "'",
|
||||
bodystr, teststr);
|
||||
collabel = new Text(ANCHORNUM + k);
|
||||
bodydata = region.get(rowlabel, collabel);
|
||||
bodystr = new String(bodydata).toString().trim();
|
||||
bytes = new byte[bodydata.getSize()];
|
||||
System.arraycopy(bodydata.get(), 0, bytes, 0, bytes.length);
|
||||
bodystr = new String(bytes).toString().trim();
|
||||
teststr = ANCHORSTR + k;
|
||||
assertEquals("Incorrect value for key: (" + rowlabel + "," + collabel
|
||||
+ "), expected: '" + teststr + "' got: '" + bodystr + "'",
|
||||
|
@ -224,7 +232,7 @@ public class TestHRegion extends TestCase {
|
|||
// Try put with bad lockid.
|
||||
boolean exceptionThrown = false;
|
||||
try {
|
||||
region.put(-1, CONTENTS_BASIC, "bad input".getBytes());
|
||||
region.put(-1, CONTENTS_BASIC, new BytesWritable("bad input".getBytes()));
|
||||
} catch (LockException e) {
|
||||
exceptionThrown = true;
|
||||
}
|
||||
|
@ -237,7 +245,7 @@ public class TestHRegion extends TestCase {
|
|||
lockid = region.startUpdate(new Text("Some old key"));
|
||||
String unregisteredColName = "FamilyGroup:FamilyLabel";
|
||||
region.put(lockid, new Text(unregisteredColName),
|
||||
unregisteredColName.getBytes());
|
||||
new BytesWritable(unregisteredColName.getBytes()));
|
||||
} catch (IOException e) {
|
||||
exceptionThrown = true;
|
||||
} finally {
|
||||
|
@ -333,8 +341,8 @@ public class TestHRegion extends TestCase {
|
|||
String kLabel = String.format("%1$03d", k);
|
||||
|
||||
long lockid = region.startUpdate(new Text("row_vals1_" + kLabel));
|
||||
region.put(lockid, cols[0], vals1[k].getBytes());
|
||||
region.put(lockid, cols[1], vals1[k].getBytes());
|
||||
region.put(lockid, cols[0], new BytesWritable(vals1[k].getBytes()));
|
||||
region.put(lockid, cols[1], new BytesWritable(vals1[k].getBytes()));
|
||||
region.commit(lockid);
|
||||
numInserted += 2;
|
||||
}
|
||||
|
@ -346,17 +354,19 @@ public class TestHRegion extends TestCase {
|
|||
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
HScannerInterface s = region.getScanner(cols, new Text());
|
||||
HInternalScannerInterface s = region.getScanner(cols, new Text());
|
||||
int numFetched = 0;
|
||||
try {
|
||||
HStoreKey curKey = new HStoreKey();
|
||||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
TreeMap<Text, BytesWritable> curVals = new TreeMap<Text, BytesWritable>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
int curval = Integer.parseInt(new String(val).trim());
|
||||
BytesWritable val = curVals.get(col);
|
||||
byte[] bytes = new byte[val.getSize()];
|
||||
System.arraycopy(val.get(), 0, bytes, 0, bytes.length);
|
||||
int curval = Integer.parseInt(new String(bytes).trim());
|
||||
|
||||
for(int j = 0; j < cols.length; j++) {
|
||||
if(col.compareTo(cols[j]) == 0) {
|
||||
|
@ -396,13 +406,15 @@ public class TestHRegion extends TestCase {
|
|||
numFetched = 0;
|
||||
try {
|
||||
HStoreKey curKey = new HStoreKey();
|
||||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
TreeMap<Text, BytesWritable> curVals = new TreeMap<Text, BytesWritable>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
int curval = Integer.parseInt(new String(val).trim());
|
||||
BytesWritable val = curVals.get(col);
|
||||
byte[] bytes = new byte[val.getSize()];
|
||||
System.arraycopy(val.get(), 0, bytes, 0, bytes.length);
|
||||
int curval = Integer.parseInt(new String(bytes).trim());
|
||||
|
||||
for(int j = 0; j < cols.length; j++) {
|
||||
if(col.compareTo(cols[j]) == 0) {
|
||||
|
@ -433,8 +445,8 @@ public class TestHRegion extends TestCase {
|
|||
String kLabel = String.format("%1$03d", k);
|
||||
|
||||
long lockid = region.startUpdate(new Text("row_vals1_" + kLabel));
|
||||
region.put(lockid, cols[0], vals1[k].getBytes());
|
||||
region.put(lockid, cols[1], vals1[k].getBytes());
|
||||
region.put(lockid, cols[0], new BytesWritable(vals1[k].getBytes()));
|
||||
region.put(lockid, cols[1], new BytesWritable(vals1[k].getBytes()));
|
||||
region.commit(lockid);
|
||||
numInserted += 2;
|
||||
}
|
||||
|
@ -450,13 +462,15 @@ public class TestHRegion extends TestCase {
|
|||
numFetched = 0;
|
||||
try {
|
||||
HStoreKey curKey = new HStoreKey();
|
||||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
TreeMap<Text, BytesWritable> curVals = new TreeMap<Text, BytesWritable>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
int curval = Integer.parseInt(new String(val).trim());
|
||||
BytesWritable val = curVals.get(col);
|
||||
byte[] bytes = new byte[val.getSize()];
|
||||
System.arraycopy(val.get(), 0, bytes, 0, bytes.length);
|
||||
int curval = Integer.parseInt(new String(bytes).trim());
|
||||
|
||||
for(int j = 0; j < cols.length; j++) {
|
||||
if(col.compareTo(cols[j]) == 0) {
|
||||
|
@ -496,13 +510,15 @@ public class TestHRegion extends TestCase {
|
|||
numFetched = 0;
|
||||
try {
|
||||
HStoreKey curKey = new HStoreKey();
|
||||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
TreeMap<Text, BytesWritable> curVals = new TreeMap<Text, BytesWritable>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
int curval = Integer.parseInt(new String(val).trim());
|
||||
BytesWritable val = curVals.get(col);
|
||||
byte[] bytes = new byte[val.getSize()];
|
||||
System.arraycopy(val.get(), 0, bytes, 0, bytes.length);
|
||||
int curval = Integer.parseInt(new String(bytes).trim());
|
||||
|
||||
for (int j = 0; j < cols.length; j++) {
|
||||
if (col.compareTo(cols[j]) == 0) {
|
||||
|
@ -532,13 +548,15 @@ public class TestHRegion extends TestCase {
|
|||
numFetched = 0;
|
||||
try {
|
||||
HStoreKey curKey = new HStoreKey();
|
||||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
TreeMap<Text, BytesWritable> curVals = new TreeMap<Text, BytesWritable>();
|
||||
int k = 500;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
int curval = Integer.parseInt(new String(val).trim());
|
||||
BytesWritable val = curVals.get(col);
|
||||
byte[] bytes = new byte[val.getSize()];
|
||||
System.arraycopy(val.get(), 0, bytes, 0, bytes.length);
|
||||
int curval = Integer.parseInt(new String(bytes).trim());
|
||||
|
||||
for (int j = 0; j < cols.length; j++) {
|
||||
if (col.compareTo(cols[j]) == 0) {
|
||||
|
@ -592,7 +610,7 @@ public class TestHRegion extends TestCase {
|
|||
|
||||
// Write to the HRegion
|
||||
long writeid = region.startUpdate(new Text("row_" + k));
|
||||
region.put(writeid, CONTENTS_BODY, buf1.toString().getBytes());
|
||||
region.put(writeid, CONTENTS_BODY, new BytesWritable(buf1.toString().getBytes()));
|
||||
region.commit(writeid);
|
||||
if (k > 0 && k % (N_ROWS / 100) == 0) {
|
||||
System.out.println("Flushing write #" + k);
|
||||
|
@ -707,20 +725,22 @@ public class TestHRegion extends TestCase {
|
|||
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
HScannerInterface s = region.getScanner(cols, new Text());
|
||||
HInternalScannerInterface s = region.getScanner(cols, new Text());
|
||||
|
||||
try {
|
||||
|
||||
int contentsFetched = 0;
|
||||
int anchorFetched = 0;
|
||||
HStoreKey curKey = new HStoreKey();
|
||||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
TreeMap<Text, BytesWritable> curVals = new TreeMap<Text, BytesWritable>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
String curval = new String(val).trim();
|
||||
BytesWritable val = curVals.get(col);
|
||||
byte[] bytes = new byte[val.getSize()];
|
||||
System.arraycopy(val.get(), 0, bytes, 0, bytes.length);
|
||||
String curval = new String(bytes).trim();
|
||||
|
||||
if(col.compareTo(CONTENTS_BASIC) == 0) {
|
||||
assertTrue("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp()
|
||||
|
@ -767,13 +787,15 @@ public class TestHRegion extends TestCase {
|
|||
try {
|
||||
int numFetched = 0;
|
||||
HStoreKey curKey = new HStoreKey();
|
||||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
TreeMap<Text, BytesWritable> curVals = new TreeMap<Text, BytesWritable>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
int curval = Integer.parseInt(new String(val).trim());
|
||||
BytesWritable val = curVals.get(col);
|
||||
byte[] bytes = new byte[val.getSize()];
|
||||
System.arraycopy(val.get(), 0, bytes, 0, bytes.length);
|
||||
int curval = Integer.parseInt(new String(bytes).trim());
|
||||
|
||||
for (int j = 0; j < cols.length; j++) {
|
||||
if (col.compareTo(cols[j]) == 0) {
|
||||
|
@ -805,12 +827,12 @@ public class TestHRegion extends TestCase {
|
|||
try {
|
||||
int numFetched = 0;
|
||||
HStoreKey curKey = new HStoreKey();
|
||||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
TreeMap<Text, BytesWritable> curVals = new TreeMap<Text, BytesWritable>();
|
||||
int k = 0;
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
Text col = it.next();
|
||||
byte val[] = curVals.get(col);
|
||||
BytesWritable val = curVals.get(col);
|
||||
|
||||
assertTrue(col.compareTo(CONTENTS_BODY) == 0);
|
||||
assertNotNull(val);
|
||||
|
@ -843,7 +865,7 @@ public class TestHRegion extends TestCase {
|
|||
try {
|
||||
int fetched = 0;
|
||||
HStoreKey curKey = new HStoreKey();
|
||||
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
|
||||
TreeMap<Text, BytesWritable> curVals = new TreeMap<Text, BytesWritable>();
|
||||
while(s.next(curKey, curVals)) {
|
||||
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
|
||||
it.next();
|
||||
|
|
|
@ -0,0 +1,319 @@
|
|||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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.ByteArrayOutputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Enumeration;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.dfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
||||
import org.apache.log4j.Appender;
|
||||
import org.apache.log4j.ConsoleAppender;
|
||||
import org.apache.log4j.Layout;
|
||||
import org.apache.log4j.Level;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.log4j.PatternLayout;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
public class TestScanner extends TestCase {
|
||||
private static final Text FIRST_ROW = new Text();
|
||||
private static final Text[] COLS = {
|
||||
HConstants.COLUMN_FAMILY
|
||||
};
|
||||
private static final Text[] EXPLICIT_COLS = {
|
||||
HConstants.COL_REGIONINFO,
|
||||
HConstants.COL_SERVER,
|
||||
HConstants.COL_STARTCODE
|
||||
};
|
||||
|
||||
private static final Text ROW_KEY = new Text(HGlobals.rootRegionInfo.regionName);
|
||||
private static final HRegionInfo REGION_INFO =
|
||||
new HRegionInfo(0L, HGlobals.rootTableDesc, null, null);
|
||||
|
||||
private static final long START_CODE = Long.MAX_VALUE;
|
||||
|
||||
private HRegion region;
|
||||
private DataInputBuffer in = new DataInputBuffer();
|
||||
|
||||
/** Compare the HRegionInfo we read from HBase to what we stored */
|
||||
private void validateRegionInfo(BytesWritable regionBytes) throws IOException {
|
||||
in.reset(regionBytes.get(), regionBytes.getSize());
|
||||
HRegionInfo info = new HRegionInfo();
|
||||
info.readFields(in);
|
||||
|
||||
assertEquals(REGION_INFO.regionId, info.regionId);
|
||||
assertEquals(0, info.startKey.getLength());
|
||||
assertEquals(0, info.endKey.getLength());
|
||||
assertEquals(0, info.regionName.compareTo(REGION_INFO.regionName));
|
||||
assertEquals(0, info.tableDesc.compareTo(REGION_INFO.tableDesc));
|
||||
}
|
||||
|
||||
/** Use a scanner to get the region info and then validate the results */
|
||||
private void scan(boolean validateStartcode, String serverName)
|
||||
throws IOException {
|
||||
|
||||
HInternalScannerInterface scanner = null;
|
||||
TreeMap<Text, BytesWritable> results = new TreeMap<Text, BytesWritable>();
|
||||
HStoreKey key = new HStoreKey();
|
||||
|
||||
Text[][] scanColumns = {
|
||||
COLS,
|
||||
EXPLICIT_COLS
|
||||
};
|
||||
|
||||
for(int i = 0; i < scanColumns.length; i++) {
|
||||
try {
|
||||
scanner = region.getScanner(scanColumns[i], FIRST_ROW);
|
||||
while(scanner.next(key, results)) {
|
||||
assertTrue(results.containsKey(HConstants.COL_REGIONINFO));
|
||||
BytesWritable val = results.get(HConstants.COL_REGIONINFO);
|
||||
byte[] bytes = new byte[val.getSize()];
|
||||
System.arraycopy(val.get(), 0, bytes, 0, bytes.length);
|
||||
|
||||
validateRegionInfo(new BytesWritable(bytes));
|
||||
|
||||
if(validateStartcode) {
|
||||
assertTrue(results.containsKey(HConstants.COL_STARTCODE));
|
||||
val = results.get(HConstants.COL_STARTCODE);
|
||||
assertNotNull(val);
|
||||
bytes = new byte[val.getSize()];
|
||||
System.arraycopy(val.get(), 0, bytes, 0, bytes.length);
|
||||
assertFalse(bytes.length == 0);
|
||||
long startCode =
|
||||
Long.valueOf(new String(bytes, HConstants.UTF8_ENCODING));
|
||||
assertEquals(START_CODE, startCode);
|
||||
}
|
||||
|
||||
if(serverName != null) {
|
||||
assertTrue(results.containsKey(HConstants.COL_SERVER));
|
||||
val = results.get(HConstants.COL_SERVER);
|
||||
assertNotNull(val);
|
||||
bytes = new byte[val.getSize()];
|
||||
System.arraycopy(val.get(), 0, bytes, 0, bytes.length);
|
||||
assertFalse(bytes.length == 0);
|
||||
String server = new String(bytes, HConstants.UTF8_ENCODING);
|
||||
assertEquals(0, server.compareTo(serverName));
|
||||
}
|
||||
results.clear();
|
||||
}
|
||||
|
||||
} catch(IOException e) {
|
||||
e.printStackTrace();
|
||||
throw e;
|
||||
|
||||
} finally {
|
||||
if(scanner != null) {
|
||||
try {
|
||||
scanner.close();
|
||||
|
||||
} catch(IOException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
scanner = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Use get to retrieve the HRegionInfo and validate it */
|
||||
private void getRegionInfo() throws IOException {
|
||||
BytesWritable bytes = region.get(ROW_KEY, HConstants.COL_REGIONINFO);
|
||||
validateRegionInfo(bytes);
|
||||
}
|
||||
|
||||
/** The test! */
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testScanner() throws IOException {
|
||||
MiniDFSCluster cluster = null;
|
||||
FileSystem fs = null;
|
||||
|
||||
try {
|
||||
|
||||
// Initialization
|
||||
|
||||
if(System.getProperty("test.build.data") == null) {
|
||||
String dir = new File(new File("").getAbsolutePath(), "build/contrib/hbase/test").getAbsolutePath();
|
||||
System.out.println(dir);
|
||||
System.setProperty("test.build.data", dir);
|
||||
}
|
||||
Configuration conf = new HBaseConfiguration();
|
||||
|
||||
Environment.getenv();
|
||||
if(Environment.debugging) {
|
||||
Logger rootLogger = Logger.getRootLogger();
|
||||
rootLogger.setLevel(Level.WARN);
|
||||
|
||||
ConsoleAppender consoleAppender = null;
|
||||
for(Enumeration<Appender> e = (Enumeration<Appender>)rootLogger.getAllAppenders();
|
||||
e.hasMoreElements();) {
|
||||
|
||||
Appender a = e.nextElement();
|
||||
if(a instanceof ConsoleAppender) {
|
||||
consoleAppender = (ConsoleAppender)a;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if(consoleAppender != null) {
|
||||
Layout layout = consoleAppender.getLayout();
|
||||
if(layout instanceof PatternLayout) {
|
||||
PatternLayout consoleLayout = (PatternLayout)layout;
|
||||
consoleLayout.setConversionPattern("%d %-5p [%t] %l: %m%n");
|
||||
}
|
||||
}
|
||||
Logger.getLogger("org.apache.hadoop.hbase").setLevel(Environment.logLevel);
|
||||
}
|
||||
cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
|
||||
fs = cluster.getFileSystem();
|
||||
Path dir = new Path("/hbase");
|
||||
fs.mkdirs(dir);
|
||||
|
||||
Path regionDir = HStoreFile.getHRegionDir(dir, REGION_INFO.regionName);
|
||||
fs.mkdirs(regionDir);
|
||||
|
||||
HLog log = new HLog(fs, new Path(regionDir, "log"), conf);
|
||||
|
||||
region = new HRegion(dir, log, fs, conf, REGION_INFO, null, null);
|
||||
|
||||
// Write information to the meta table
|
||||
|
||||
long lockid = region.startUpdate(ROW_KEY);
|
||||
|
||||
ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
|
||||
DataOutputStream s = new DataOutputStream(byteStream);
|
||||
HGlobals.rootRegionInfo.write(s);
|
||||
region.put(lockid, HConstants.COL_REGIONINFO,
|
||||
new BytesWritable(byteStream.toByteArray()));
|
||||
region.commit(lockid);
|
||||
|
||||
// What we just committed is in the memcache. Verify that we can get
|
||||
// it back both with scanning and get
|
||||
|
||||
scan(false, null);
|
||||
getRegionInfo();
|
||||
|
||||
// Close and re-open
|
||||
|
||||
region.close();
|
||||
log.rollWriter();
|
||||
region = new HRegion(dir, log, fs, conf, REGION_INFO, null, null);
|
||||
|
||||
// Verify we can get the data back now that it is on disk.
|
||||
|
||||
scan(false, null);
|
||||
getRegionInfo();
|
||||
|
||||
// Store some new information
|
||||
|
||||
HServerAddress address = new HServerAddress("foo.bar.com:1234");
|
||||
|
||||
lockid = region.startUpdate(ROW_KEY);
|
||||
|
||||
region.put(lockid, HConstants.COL_SERVER,
|
||||
new BytesWritable(address.toString().getBytes(HConstants.UTF8_ENCODING)));
|
||||
|
||||
region.put(lockid, HConstants.COL_STARTCODE,
|
||||
new BytesWritable(
|
||||
String.valueOf(START_CODE).getBytes(HConstants.UTF8_ENCODING)));
|
||||
|
||||
region.commit(lockid);
|
||||
|
||||
// Validate that we can still get the HRegionInfo, even though it is in
|
||||
// an older row on disk and there is a newer row in the memcache
|
||||
|
||||
scan(true, address.toString());
|
||||
getRegionInfo();
|
||||
|
||||
// flush cache
|
||||
|
||||
region.flushcache(false);
|
||||
|
||||
// Validate again
|
||||
|
||||
scan(true, address.toString());
|
||||
getRegionInfo();
|
||||
|
||||
// Close and reopen
|
||||
|
||||
region.close();
|
||||
log.rollWriter();
|
||||
region = new HRegion(dir, log, fs, conf, REGION_INFO, null, null);
|
||||
|
||||
// Validate again
|
||||
|
||||
scan(true, address.toString());
|
||||
getRegionInfo();
|
||||
|
||||
// Now update the information again
|
||||
|
||||
address = new HServerAddress("bar.foo.com:4321");
|
||||
|
||||
lockid = region.startUpdate(ROW_KEY);
|
||||
|
||||
region.put(lockid, HConstants.COL_SERVER,
|
||||
new BytesWritable(address.toString().getBytes(HConstants.UTF8_ENCODING)));
|
||||
|
||||
region.commit(lockid);
|
||||
|
||||
// Validate again
|
||||
|
||||
scan(true, address.toString());
|
||||
getRegionInfo();
|
||||
|
||||
// flush cache
|
||||
|
||||
region.flushcache(false);
|
||||
|
||||
// Validate again
|
||||
|
||||
scan(true, address.toString());
|
||||
getRegionInfo();
|
||||
|
||||
// Close and reopen
|
||||
|
||||
region.close();
|
||||
log.rollWriter();
|
||||
region = new HRegion(dir, log, fs, conf, REGION_INFO, null, null);
|
||||
|
||||
// Validate again
|
||||
|
||||
scan(true, address.toString());
|
||||
getRegionInfo();
|
||||
|
||||
} catch(IOException e) {
|
||||
e.printStackTrace();
|
||||
throw e;
|
||||
|
||||
} finally {
|
||||
if(fs != null) {
|
||||
fs.close();
|
||||
}
|
||||
if(cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,29 @@
|
|||
package org.apache.hadoop.hbase;
|
||||
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
public class TestToString extends TestCase {
|
||||
public void testServerInfo() throws Exception {
|
||||
final String hostport = "127.0.0.1:9999";
|
||||
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);
|
||||
}
|
||||
|
||||
public void testHRegionInfo() throws Exception {
|
||||
HTableDescriptor htd = new HTableDescriptor("hank", 10);
|
||||
htd.addFamily(new Text("hankfamily:"));
|
||||
htd.addFamily(new Text("hankotherfamily:"));
|
||||
assertEquals("Table descriptor", htd.toString(),
|
||||
"name: hank, maxVersions: 10, families: [hankfamily:, hankotherfamily:]");
|
||||
HRegionInfo hri = new HRegionInfo(-1, htd, new Text(), new Text("10"));
|
||||
assertEquals("HRegionInfo",
|
||||
"regionname: hank__-1, startKey: <>, tableDesc: {name: hank, " +
|
||||
"maxVersions: 10, families: [hankfamily:, hankotherfamily:]}",
|
||||
hri.toString());
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue