HADOOP-1045. Add contrib/hbase, a BigTable-like online database.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@525267 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Doug Cutting 2007-04-03 20:34:28 +00:00
commit 948d0cdec4
31 changed files with 8698 additions and 0 deletions

285
README.txt Normal file
View File

@ -0,0 +1,285 @@
HBASE
Michael Cafarella
This document gives a quick overview of HBase, the Hadoop simple
database. It is extremely similar to Google's BigTable, with a just a
few differences. If you understand BigTable, great. If not, you should
still be able to understand this document.
---------------------------------------------------------------
I.
HBase uses a data model very similar to that of BigTable. Users store
data rows in labelled tables. A data row has a sortable key and an
arbitrary number of columns. The table is stored sparsely, so that
rows in the same table can have crazily-varying columns, if the user
likes.
A column name has the form "<group>:<label>" where <group> and <label>
can be any string you like. A single table enforces its set of
<group>s (called "column groups"). You can only adjust this set of
groups by performing administrative operations on the table. However,
you can use new <label> strings at any write without preannouncing
it. HBase stores column groups physically close on disk. So the items
in a given column group should have roughly the same write/read
behavior.
Writes are row-locked only. You cannot lock multiple rows at once. All
row-writes are atomic by default.
All updates to the database have an associated timestamp. The HBase
will store a configurable number of versions of a given cell. Clients
can get data by asking for the "most recent value as of a certain
time". Or, clients can fetch all available versions at once.
---------------------------------------------------------------
II.
To the user, a table seems like a list of data tuples, sorted by row
key. Physically, tables are broken into HRegions. An HRegion is
identified by its tablename plus a start/end-key pair. A given HRegion
with keys <start> and <end> will store all the rows from (<start>,
<end>]. A set of HRegions, sorted appropriately, forms an entire
table.
All data is physically stored using Hadoop's DFS. Data is served to
clients by a set of HRegionServers, usually one per machine. A given
HRegion is served by only one HRegionServer at a time.
When a client wants to make updates, it contacts the relevant
HRegionServer and commits the update to an HRegion. Upon commit, the
data is added to the HRegion's HMemcache and to the HRegionServer's
HLog. The HMemcache is a memory buffer that stores and serves the
most-recent updates. The HLog is an on-disk log file that tracks all
updates. The commit() call will not return to the client until the
update has been written to the HLog.
When serving data, the HRegion will first check its HMemcache. If not
available, it will then check its on-disk HStores. There is an HStore
for each column family in an HRegion. An HStore might consist of
multiple on-disk HStoreFiles. Each HStoreFile is a B-Tree-like
structure that allow for relatively fast access.
Periodically, we invoke HRegion.flushcache() to write the contents of
the HMemcache to an on-disk HStore's files. This adds a new HStoreFile
to each HStore. The HMemcache is then emptied, and we write a special
token to the HLog, indicating the HMemcache has been flushed.
On startup, each HRegion checks to see if there have been any writes
to the HLog since the most-recent invocation of flushcache(). If not,
then all relevant HRegion data is reflected in the on-disk HStores. If
yes, the HRegion reconstructs the updates from the HLog, writes them
to the HMemcache, and then calls flushcache(). Finally, it deletes the
HLog and is now available for serving data.
Thus, calling flushcache() infrequently will be less work, but
HMemcache will consume more memory and the HLog will take a longer
time to reconstruct upon restart. If flushcache() is called
frequently, the HMemcache will take less memory, and the HLog will be
faster to reconstruct, but each flushcache() call imposes some
overhead.
The HLog is periodically rolled, so it consists of multiple
time-sorted files. Whenever we roll the HLog, the HLog will delete all
old log files that contain only flushed data. Rolling the HLog takes
very little time and is generally a good idea to do from time to time.
Each call to flushcache() will add an additional HStoreFile to each
HStore. Fetching a file from an HStore can potentially access all of
its HStoreFiles. This is time-consuming, so we want to periodically
compact these HStoreFiles into a single larger one. This is done by
calling HStore.compact().
Compaction is a very expensive operation. It's done automatically at
startup, and should probably be done periodically during operation.
The Google BigTable paper has a slightly-confusing hierarchy of major
and minor compactions. We have just two things to keep in mind:
1) A "flushcache()" drives all updates out of the memory buffer into
on-disk structures. Upon flushcache, the log-reconstruction time
goes to zero. Each flushcache() will add a new HStoreFile to each
HStore.
2) a "compact()" consolidates all the HStoreFiles into a single
one. It's expensive, and is always done at startup.
Unlike BigTable, Hadoop's HBase allows no period where updates have
been "committed" but have not been written to the log. This is not
hard to add, if it's really wanted.
We can merge two HRegions into a single new HRegion by calling
HRegion.closeAndMerge(). We can split an HRegion into two smaller
HRegions by calling HRegion.closeAndSplit().
OK, to sum up so far:
1) Clients access data in tables.
2) tables are broken into HRegions.
3) HRegions are served by HRegionServers. Clients contact an
HRegionServer to access the data within its row-range.
4) HRegions store data in:
a) HMemcache, a memory buffer for recent writes
b) HLog, a write-log for recent writes
c) HStores, an efficient on-disk set of files. One per col-group.
(HStores use HStoreFiles.)
---------------------------------------------------------------
III.
Each HRegionServer stays in contact with the single HBaseMaster. The
HBaseMaster is responsible for telling each HRegionServer what
HRegions it should load and make available.
The HBaseMaster keeps a constant tally of which HRegionServers are
alive at any time. If the connection between an HRegionServer and the
HBaseMaster times out, then:
a) The HRegionServer kills itself and restarts in an empty state.
b) The HBaseMaster assumes the HRegionServer has died and reallocates
its HRegions to other HRegionServers
Note that this is unlike Google's BigTable, where a TabletServer can
still serve Tablets after its connection to the Master has died. We
tie them together, because we do not use an external lock-management
system like BigTable. With BigTable, there's a Master that allocates
tablets and a lock manager (Chubby) that guarantees atomic access by
TabletServers to tablets. HBase uses just a single central point for
all HRegionServers to access: the HBaseMaster.
(This is no more dangerous than what BigTable does. Each system is
reliant on a network structure (whether HBaseMaster or Chubby) that
must survive for the data system to survive. There may be some
Chubby-specific advantages, but that's outside HBase's goals right
now.)
As HRegionServers check in with a new HBaseMaster, the HBaseMaster
asks each HRegionServer to load in zero or more HRegions. When the
HRegionServer dies, the HBaseMaster marks those HRegions as
unallocated, and attempts to give them to different HRegionServers.
Recall that each HRegion is identified by its table name and its
key-range. Since key ranges are contiguous, and they always start and
end with NULL, it's enough to simply indicate the end-key.
Unfortunately, this is not quite enough. Because of merge() and
split(), we may (for just a moment) have two quite different HRegions
with the same name. If the system dies at an inopportune moment, both
HRegions may exist on disk simultaneously. The arbiter of which
HRegion is "correct" is the HBase meta-information (to be discussed
shortly). In order to distinguish between different versions of the
same HRegion, we also add a unique 'regionId' to the HRegion name.
Thus, we finally get to this identifier for an HRegion:
tablename + endkey + regionId.
You can see this identifier being constructed in
HRegion.buildRegionName().
We can also use this identifier as a row-label in a different
HRegion. Thus, the HRegion meta-info is itself stored in an
HRegion. We call this table, which maps from HRegion identifiers to
physical HRegionServer locations, the META table.
The META table itself can grow large, and may be broken into separate
HRegions. To locate all components of the META table, we list all META
HRegions in a ROOT table. The ROOT table is always contained in a
single HRegion.
Upon startup, the HRegionServer immediately attempts to scan the ROOT
table (because there is only one HRegion for the ROOT table, that
HRegion's name is hard-coded). It may have to wait for the ROOT table
to be allocated to an HRegionServer.
Once the ROOT table is available, the HBaseMaster can scan it and
learn of all the META HRegions. It then scans the META table. Again,
the HBaseMaster may have to wait for all the META HRegions to be
allocated to different HRegionServers.
Finally, when the HBaseMaster has scanned the META table, it knows the
entire set of HRegions. It can then allocate these HRegions to the set
of HRegionServers.
The HBaseMaster keeps the set of currently-available HRegionServers in
memory. Since the death of the HBaseMaster means the death of the
entire system, there's no reason to store this information on
disk. All information about the HRegion->HRegionServer mapping is
stored physically on different tables. Thus, a client does not need to
contact the HBaseMaster after it learns the location of the ROOT
HRegion. The load on HBaseMaster should be relatively small: it deals
with timing out HRegionServers, scanning the ROOT and META upon
startup, and serving the location of the ROOT HRegion.
The HClient is fairly complicated, and often needs to navigate the
ROOT and META HRegions when serving a user's request to scan a
specific user table. If an HRegionServer is unavailable or it does not
have an HRegion it should have, the HClient will wait and retry. At
startup or in case of a recent HRegionServer failure, the correct
mapping info from HRegion to HRegionServer may not always be
available.
In summary:
1) HRegionServers offer access to HRegions (an HRegion lives at one
HRegionServer)
2) HRegionServers check in with the HBaseMaster
3) If the HBaseMaster dies, the whole system dies
4) The set of current HRegionServers is known only to the HBaseMaster
5) The mapping between HRegions and HRegionServers is stored in two
special HRegions, which are allocated to HRegionServers like any
other.
6) The ROOT HRegion is a special one, the location of which the
HBaseMaster always knows.
7) It's the HClient's responsibility to navigate all this.
---------------------------------------------------------------
IV.
What's the current status of all this code?
As of this writing, there is just shy of 7000 lines of code in the
"hbase" directory.
All of the single-machine operations (safe-committing, merging,
splitting, versioning, flushing, compacting, log-recovery) are
complete, have been tested, and seem to work great.
The multi-machine stuff (the HBaseMaster, the HRegionServer, and the
HClient) have not been fully tested. The reason is that the HClient is
still incomplete, so the rest of the distributed code cannot be
fully-tested. I think it's good, but can't be sure until the HClient
is done. However, the code is now very clean and in a state where
other people can understand it and contribute.
Other related features and TODOs:
1) Single-machine log reconstruction works great, but distributed log
recovery is not yet implemented. This is relatively easy, involving
just a sort of the log entries, placing the shards into the right
DFS directories
2) Data compression is not yet implemented, but there is an obvious
place to do so in the HStore.
3) We need easy interfaces to MapReduce jobs, so they can scan tables
4) The HMemcache lookup structure is relatively inefficient
5) File compaction is relatively slow; we should have a more
conservative algorithm for deciding when to apply compaction.
6) For the getFull() operation, use of Bloom filters would speed
things up
7) We need stress-test and performance-number tools for the whole
system
8) There's some HRegion-specific testing code that worked fine during
development, but it has to be rewritten so it works against an
HRegion while it's hosted by an HRegionServer, and connected to an
HBaseMaster. This code is at the bottom of the HRegion.java file.

23
build.xml Normal file
View File

@ -0,0 +1,23 @@
<?xml version="1.0"?>
<!--
Before you can run these subtargets directly, you need
to call at top-level: ant deploy-contrib compile-core-test
-->
<project name="hbase" default="jar">
<import file="../build-contrib.xml"/>
<!-- Override jar target to specify main class -->
<target name="jar" depends="compile">
<jar
jarfile="${build.dir}/hadoop-${name}.jar"
basedir="${build.classes}"
/>
</target>
<target name="test">
<antcall target="hadoopbuildcontrib.test"/>
</target>
</project>

View File

@ -0,0 +1,248 @@
/**
* 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.IOException;
import java.util.TreeMap;
import java.util.Vector;
import java.util.regex.Pattern;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.BytesWritable;
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 {
// Pattern to determine if a column key is a regex
private static Pattern isRegexPattern = Pattern.compile("^.*[\\\\+|^&*$\\[\\]\\}{)(]+.*$");
// The kind of match we are doing on a column:
private static enum MATCH_TYPE {
FAMILY_ONLY, // Just check the column family name
REGEX, // Column family + matches regex
SIMPLE // Literal matching
};
// This class provides column matching functions that are more sophisticated
// than a simple string compare. There are three types of matching:
// 1. Match on the column family name only
// 2. Match on the column family + column key regex
// 3. Simple match: compare column family + column key literally
private class ColumnMatcher {
private MATCH_TYPE matchType;
private String family;
private Pattern columnMatcher;
private Text col;
ColumnMatcher(Text col) throws IOException {
String column = col.toString();
try {
int colpos = column.indexOf(":") + 1;
if(colpos == 0) {
throw new IllegalArgumentException("Column name has no family indicator.");
}
String columnkey = column.substring(colpos);
if(columnkey == null || columnkey.length() == 0) {
this.matchType = MATCH_TYPE.FAMILY_ONLY;
this.family = column.substring(0, colpos);
} else if(isRegexPattern.matcher(columnkey).matches()) {
this.matchType = MATCH_TYPE.REGEX;
this.columnMatcher = Pattern.compile(column);
} else {
this.matchType = MATCH_TYPE.SIMPLE;
this.col = col;
}
} catch(Exception e) {
throw new IOException("Column: " + column + ": " + e.getMessage());
}
}
// Matching method
boolean matches(Text col) throws IOException {
if(this.matchType == MATCH_TYPE.SIMPLE) {
return col.equals(this.col);
} else if(this.matchType == MATCH_TYPE.FAMILY_ONLY) {
return col.toString().startsWith(this.family);
} else if(this.matchType == MATCH_TYPE.REGEX) {
return this.columnMatcher.matcher(col.toString()).matches();
} else {
throw new IOException("Invalid match type: " + this.matchType);
}
}
}
protected TreeMap<Text, Vector<ColumnMatcher>> okCols; // Holds matchers for each column family
protected boolean scannerClosed = false; // True when scanning is done
protected HStoreKey keys[]; // Keys retrieved from the sources
protected BytesWritable vals[]; // Values that correspond to those keys
protected long timestamp; // The timestamp to match entries against
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.okCols = new TreeMap<Text, Vector<ColumnMatcher>>();
for(int i = 0; i < targetCols.length; i++) {
Text family = HStoreKey.extractFamily(targetCols[i]);
Vector<ColumnMatcher> matchers = okCols.get(family);
if(matchers == null) {
matchers = new Vector<ColumnMatcher>();
}
matchers.add(new ColumnMatcher(targetCols[i]));
okCols.put(family, matchers);
}
}
/**
* For a particular column i, find all the matchers defined for the column.
* Compare the column family and column key using the matchers. The first one
* that matches returns true. If no matchers are successful, return false.
*
* @param i index into the keys array
* @return true - if any of the matchers for the column match the column family
* and the column key.
*
* @throws IOException
*/
boolean columnMatch(int i) throws IOException {
Text column = keys[i].getColumn();
Text family = HStoreKey.extractFamily(column);
Vector<ColumnMatcher> matchers = okCols.get(family);
if(matchers == null) {
return false;
}
for(int m = 0; m < matchers.size(); m++) {
if(matchers.get(m).matches(column)) {
return true;
}
}
return false;
}
/**
* If the user didn't want to start scanning at the first row, this method
* seeks to the requested row.
*/
abstract boolean findFirstRow(int i, Text firstRow) throws IOException;
/** The concrete implementations provide a mechanism to find the next set of values */
abstract boolean getNext(int i) throws IOException;
/** Mechanism used by concrete implementation to shut down a particular scanner */
abstract void closeSubScanner(int i) throws IOException;
/** Mechanism used to shut down the whole scan */
public abstract void close() throws IOException;
/**
* Get the next set of values for this scanner.
*
* @param key - The key that matched
* @param results - all the results for that key.
* @return - true if a match was found
*
* @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 {
// Find the next row label (and timestamp)
Text chosenRow = null;
long chosenTimestamp = -1;
for(int i = 0; i < keys.length; i++) {
while((keys[i] != null)
&& (columnMatch(i))
&& (keys[i].getTimestamp() <= this.timestamp)
&& ((chosenRow == null)
|| (keys[i].getRow().compareTo(chosenRow) < 0)
|| ((keys[i].getRow().compareTo(chosenRow) == 0)
&& (keys[i].getTimestamp() > chosenTimestamp)))) {
chosenRow = new Text(keys[i].getRow());
chosenTimestamp = keys[i].getTimestamp();
}
}
// Grab all the values that match this row/timestamp
boolean insertedItem = false;
if(chosenRow != null) {
key.setRow(chosenRow);
key.setVersion(chosenTimestamp);
key.setColumn(new Text(""));
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;
}
if (! getNext(i)) {
closeSubScanner(i);
}
}
// Advance the current scanner beyond the chosen row, to
// a valid timestamp, so we're ready next time.
while((keys[i] != null)
&& ((keys[i].getRow().compareTo(chosenRow) <= 0)
|| (keys[i].getTimestamp() > this.timestamp)
|| (! columnMatch(i)))) {
getNext(i);
}
}
}
return insertedItem;
}
}

View File

@ -0,0 +1,533 @@
/**
* 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 org.apache.hadoop.io.*;
import org.apache.hadoop.ipc.*;
import org.apache.hadoop.conf.*;
import java.io.*;
import java.util.*;
/*******************************************************************************
* HClient manages a connection to a single HRegionServer.
******************************************************************************/
public class HClient extends HGlobals implements HConstants {
private static final Text[] metaColumns = {
META_COLUMN_FAMILY
};
private static final Text startRow = new Text();
private boolean closed;
private Configuration conf;
private HServerAddress masterLocation;
private long clientTimeout;
private int numTimeouts;
private int numRetries;
private HMasterInterface master;
private class TableInfo {
public HRegionInfo regionInfo;
public HServerAddress serverAddress;
TableInfo(HRegionInfo regionInfo, HServerAddress serverAddress) {
this.regionInfo = regionInfo;
this.serverAddress = serverAddress;
}
}
// Map tableName -> (Map startRow -> (HRegionInfo, HServerAddress)
private TreeMap<Text, TreeMap<Text, TableInfo>> tablesToServers;
// For the "current" table: Map startRow -> (HRegionInfo, HServerAddress)
private TreeMap<Text, TableInfo> tableServers;
// Known region HServerAddress.toString() -> HRegionInterface
private TreeMap<String, HRegionInterface> servers;
// For row mutation operations
private Text currentRegion;
private HRegionInterface currentServer;
private Random rand;
private long clientid;
/** Creates a new HClient */
public HClient(Configuration conf) {
this.closed = false;
this.conf = conf;
// Load config settings
this.masterLocation = new HServerAddress(this.conf.get(MASTER_DEFAULT_NAME));
this.clientTimeout = this.conf.getLong("hbase.client.timeout.length", 10 * 1000);
this.numTimeouts = this.conf.getInt("hbase.client.timeout.number", 5);
this.numRetries = this.conf.getInt("hbase.client.retries.number", 2);
// Finish initialization
this.master = null;
this.tablesToServers = new TreeMap<Text, TreeMap<Text, TableInfo>>();
this.tableServers = null;
this.servers = new TreeMap<String, HRegionInterface>();
// For row mutation operations
this.currentRegion = null;
this.currentServer = null;
this.rand = new Random();
}
public synchronized void openTable(Text tableName) throws IOException {
if(closed) {
throw new IllegalStateException("client is not open");
}
tableServers = tablesToServers.get(tableName);
if(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);
if(metaServers == null) { // Don't know where the meta is
loadMetaFromRoot(tableName);
if(tableName.equals(META_TABLE_NAME) || tableName.equals(ROOT_TABLE_NAME)) {
// All we really wanted was the meta or root table
return;
}
metaServers = tablesToServers.get(META_TABLE_NAME);
}
tableServers = new TreeMap<Text, TableInfo>();
for(Iterator<TableInfo> i = metaServers.tailMap(tableName).values().iterator();
i.hasNext(); ) {
TableInfo t = i.next();
scanOneMetaRegion(t, tableName);
}
tablesToServers.put(tableName, tableServers);
}
/*
* Load the meta table from the root table.
*/
private void loadMetaFromRoot(Text tableName) throws IOException {
locateRootRegion();
if(tableName.equals(ROOT_TABLE_NAME)) { // All we really wanted was the root
return;
}
scanRoot();
}
/*
* Repeatedly try to find the root region by asking the HMaster for where it
* could be.
*/
private void locateRootRegion() throws IOException {
if(master == null) {
master = (HMasterInterface)RPC.getProxy(HMasterInterface.class,
HMasterInterface.versionID,
masterLocation.getInetSocketAddress(), conf);
}
int tries = 0;
HServerAddress rootRegionLocation = null;
do {
int localTimeouts = 0;
while(rootRegionLocation == null && localTimeouts < numTimeouts) {
rootRegionLocation = master.findRootRegion();
if(rootRegionLocation == null) {
try {
Thread.sleep(clientTimeout);
} catch(InterruptedException iex) {
}
localTimeouts++;
}
}
if(rootRegionLocation == null) {
throw new IOException("Timed out trying to locate root region");
}
// Verify that this server still serves the root region
HRegionInterface rootRegion = getHRegionConnection(rootRegionLocation);
if(rootRegion.getRegionInfo(rootRegionInfo.regionName) != null) {
tableServers = new TreeMap<Text, TableInfo>();
tableServers.put(startRow, new TableInfo(rootRegionInfo, rootRegionLocation));
tablesToServers.put(ROOT_TABLE_NAME, tableServers);
break;
}
rootRegionLocation = null;
} while(rootRegionLocation == null && tries++ < numRetries);
if(rootRegionLocation == null) {
closed = true;
throw new IOException("unable to locate root region server");
}
}
/*
* 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);
}
/*
* 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
*/
private void scanOneMetaRegion(TableInfo t, Text tableName) throws IOException {
HRegionInterface server = getHRegionConnection(t.serverAddress);
HScannerInterface scanner = null;
try {
scanner = server.openScanner(t.regionInfo.regionName, metaColumns, tableName);
HStoreKey key = new HStoreKey();
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
DataInputBuffer inbuf = new DataInputBuffer();
while(scanner.next(key, results)) {
byte hRegionInfoBytes[] = results.get(META_COL_REGIONINFO);
inbuf.reset(hRegionInfoBytes, hRegionInfoBytes.length);
HRegionInfo regionInfo = new HRegionInfo();
regionInfo.readFields(inbuf);
if(! regionInfo.tableDesc.getName().equals(tableName)) {
// We're done
break;
}
byte serverBytes[] = results.get(META_COL_SERVER);
String serverName = new String(serverBytes, UTF8_ENCODING);
tableServers.put(regionInfo.startKey,
new TableInfo(regionInfo, new HServerAddress(serverName)));
results.clear();
}
} finally {
scanner.close();
}
}
public synchronized HRegionInterface getHRegionConnection(HServerAddress regionServer)
throws IOException {
// See if we already have a connection
HRegionInterface server = servers.get(regionServer.toString());
if(server == null) { // Get a connection
server = (HRegionInterface)RPC.waitForProxy(HRegionInterface.class,
HRegionInterface.versionID, regionServer.getInetSocketAddress(), conf);
servers.put(regionServer.toString(), server);
}
return server;
}
/** Close the connection to the HRegionServer */
public synchronized void close() throws IOException {
if(! closed) {
RPC.stopClient();
closed = true;
}
}
/**
* List all the userspace tables. In other words, scan the META table.
*
* If we wanted this to be really fast, we could implement a special
* catalog table that just contains table names and their descriptors.
* Right now, it only exists as part of the META table's region info.
*/
public HTableDescriptor[] listTables() throws IOException {
TreeSet<HTableDescriptor> uniqueTables = new TreeSet<HTableDescriptor>();
TreeMap<Text, TableInfo> metaTables = 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>i = metaTables.values().iterator(); i.hasNext(); ) {
TableInfo t = i.next();
HRegionInterface server = getHRegionConnection(t.serverAddress);
HScannerInterface scanner = null;
try {
scanner = server.openScanner(t.regionInfo.regionName, metaColumns, startRow);
HStoreKey key = new HStoreKey();
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
DataInputBuffer inbuf = new DataInputBuffer();
while(scanner.next(key, results)) {
byte infoBytes[] = (byte[]) results.get(ROOT_COL_REGIONINFO);
inbuf.reset(infoBytes, infoBytes.length);
HRegionInfo info = new HRegionInfo();
info.readFields(inbuf);
// Only examine the rows where the startKey is zero length
if(info.startKey.getLength() == 0) {
uniqueTables.add(info.tableDesc);
}
results.clear();
}
} finally {
scanner.close();
}
}
return (HTableDescriptor[]) uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]);
}
private TableInfo getTableInfo(Text row) {
if(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);
}
/** Get a single value for the specified row and column */
public byte[] get(Text row, Text column) throws IOException {
TableInfo info = getTableInfo(row);
return getHRegionConnection(info.serverAddress).get(
info.regionInfo.regionName, row, column).get();
}
/** Get the specified number of versions of the specified row and column */
public byte[][] get(Text row, Text column, int numVersions) throws IOException {
TableInfo info = getTableInfo(row);
BytesWritable[] values = getHRegionConnection(info.serverAddress).get(
info.regionInfo.regionName, row, column, numVersions);
ArrayList<byte[]> bytes = new ArrayList<byte[]>();
for(int i = 0 ; i < values.length; i++) {
bytes.add(values[i].get());
}
return bytes.toArray(new byte[values.length][]);
}
/**
* Get the specified number of versions of the specified row and column with
* the specified timestamp.
*/
public byte[][] get(Text row, Text column, long timestamp, int numVersions) throws IOException {
TableInfo info = getTableInfo(row);
BytesWritable[] values = getHRegionConnection(info.serverAddress).get(
info.regionInfo.regionName, row, column, timestamp, numVersions);
ArrayList<byte[]> bytes = new ArrayList<byte[]>();
for(int i = 0 ; i < values.length; i++) {
bytes.add(values[i].get());
}
return bytes.toArray(new byte[values.length][]);
}
/** Get all the data for the specified row */
public LabelledData[] getRow(Text row) throws IOException {
TableInfo info = getTableInfo(row);
return getHRegionConnection(info.serverAddress).getRow(
info.regionInfo.regionName, row);
}
/**
* Get a scanner on the current table starting at the specified row.
* Return the specified columns.
*/
public HScannerInterface obtainScanner(Text[] columns, Text startRow) throws IOException {
if(tableServers == null) {
throw new IllegalStateException("Must open table first");
}
return new ClientScanner(columns, startRow);
}
/** Start an atomic row insertion or update */
public long startUpdate(Text row) throws IOException {
TableInfo info = getTableInfo(row);
long lockid;
try {
currentServer = getHRegionConnection(info.serverAddress);
currentRegion = info.regionInfo.regionName;
clientid = rand.nextLong();
lockid = currentServer.startUpdate(currentRegion, clientid, row);
} catch(IOException e) {
currentServer = null;
currentRegion = null;
throw e;
}
return lockid;
}
/** 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));
} catch(IOException e) {
try {
currentServer.abort(currentRegion, clientid, lockid);
} catch(IOException e2) {
}
currentServer = null;
currentRegion = null;
throw e;
}
}
/** Delete the value for a column */
public void delete(long lockid, Text column) throws IOException {
try {
currentServer.delete(currentRegion, clientid, lockid, column);
} catch(IOException e) {
try {
currentServer.abort(currentRegion, clientid, lockid);
} catch(IOException e2) {
}
currentServer = null;
currentRegion = null;
throw e;
}
}
/** Abort a row mutation */
public void abort(long lockid) throws IOException {
try {
currentServer.abort(currentRegion, clientid, lockid);
} catch(IOException e) {
currentServer = null;
currentRegion = null;
throw e;
}
}
/** Finalize a row mutation */
public void commit(long lockid) throws IOException {
try {
currentServer.commit(currentRegion, clientid, lockid);
} finally {
currentServer = null;
currentRegion = null;
}
}
/**
* Implements the scanner interface for the HBase client.
* If there are multiple regions in a table, this scanner will iterate
* through them all.
*/
private class ClientScanner implements HScannerInterface {
private Text[] columns;
private Text startRow;
private boolean closed;
private TableInfo[] regions;
private int currentRegion;
private HRegionInterface server;
private HScannerInterface scanner;
public ClientScanner(Text[] columns, Text startRow) throws IOException {
this.columns = columns;
this.startRow = startRow;
this.closed = false;
Collection<TableInfo> info = tableServers.tailMap(startRow).values();
this.regions = info.toArray(new TableInfo[info.size()]);
this.currentRegion = -1;
this.server = null;
this.scanner = null;
nextScanner();
}
/*
* Gets a scanner for the next region.
* Returns false if there are no more scanners.
*/
private boolean nextScanner() throws IOException {
if(scanner != null) {
scanner.close();
}
currentRegion += 1;
if(currentRegion == regions.length) {
close();
return false;
}
try {
server = getHRegionConnection(regions[currentRegion].serverAddress);
scanner = server.openScanner(regions[currentRegion].regionInfo.regionName,
columns, startRow);
} catch(IOException e) {
close();
throw e;
}
return true;
}
/* (non-Javadoc)
* @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) {
return false;
}
boolean status = scanner.next(key, results);
if(! status) {
status = nextScanner();
if(status) {
status = scanner.next(key, results);
}
}
return status;
}
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.HScannerInterface#close()
*/
public void close() throws IOException {
if(scanner != null) {
scanner.close();
}
server = null;
closed = true;
}
}
}

View File

@ -0,0 +1,56 @@
/**
* 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 org.apache.hadoop.io.Text;
/*******************************************************************************
* HConstants holds a bunch of HBase-related constants
******************************************************************************/
public interface HConstants {
// Configuration parameters
static final String MASTER_DEFAULT_NAME = "hbase.master.default.name";
static final String HREGION_DIR = "hbase.regiondir";
static final String DEFAULT_HREGION_DIR = "/hbase";
static final String HREGIONDIR_PREFIX = "hregion_";
// Always store the location of the root table's HRegion.
// This HRegion is never split.
// 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");
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);
// Other constants
static final long DESIRED_MAX_FILE_SIZE = 128 * 1024 * 1024; // 128MB
static final String UTF8_ENCODING = "UTF-8";
}

View File

@ -0,0 +1,38 @@
/**
* 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;
/*******************************************************************************
* Global values used for finding and scanning the root and meta tables.
******************************************************************************/
public class HGlobals implements HConstants {
static HTableDescriptor rootTableDesc = null;
static HRegionInfo rootRegionInfo = null;
static HTableDescriptor metaTableDesc = null;
static {
rootTableDesc = new HTableDescriptor(ROOT_TABLE_NAME.toString(), 1);
rootTableDesc.addFamily(ROOT_COLUMN_FAMILY);
rootRegionInfo = new HRegionInfo(0L, rootTableDesc, null, null);
metaTableDesc = new HTableDescriptor(META_TABLE_NAME.toString(), 1);
metaTableDesc.addFamily(META_COLUMN_FAMILY);
}
}

View File

@ -0,0 +1,90 @@
/**
* 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;
/*******************************************************************************
* HLocking is a set of lock primitives that are pretty helpful in a few places
* around the HBase code. For each independent entity that needs locking, create
* a new HLocking instance.
******************************************************************************/
public class HLocking {
Integer readerLock = new Integer(0);
Integer writerLock = new Integer(0);
int numReaders = 0;
int numWriters = 0;
public HLocking() {
}
/** Caller needs the nonexclusive read-lock */
public void obtainReadLock() {
synchronized(readerLock) {
synchronized(writerLock) {
while(numWriters > 0) {
try {
writerLock.wait();
} catch (InterruptedException ie) {
}
}
numReaders++;
readerLock.notifyAll();
}
}
}
/** Caller is finished with the nonexclusive read-lock */
public void releaseReadLock() {
synchronized(readerLock) {
synchronized(writerLock) {
numReaders--;
readerLock.notifyAll();
}
}
}
/** Caller needs the exclusive write-lock */
public void obtainWriteLock() {
synchronized(readerLock) {
synchronized(writerLock) {
while(numReaders > 0) {
try {
readerLock.wait();
} catch (InterruptedException ie) {
}
}
while(numWriters > 0) {
try {
writerLock.wait();
} catch (InterruptedException ie) {
}
}
numWriters++;
writerLock.notifyAll();
}
}
}
/** Caller is finished with the write lock */
public void releaseWriteLock() {
synchronized(readerLock) {
synchronized(writerLock) {
numWriters--;
writerLock.notifyAll();
}
}
}
}

View File

@ -0,0 +1,356 @@
/**
* 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 org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.*;
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.
*
* 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.
*
* 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.
*
* 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 {
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 METAROW = new Text("METAROW");
FileSystem fs;
Path dir;
Configuration conf;
SequenceFile.Writer writer;
TreeMap<Long, Path> outputfiles = new TreeMap<Long, Path>();
boolean insideCacheFlush = false;
TreeMap<Text, Long> regionToLastFlush = new TreeMap<Text, Long>();
long oldestOutstandingSeqNum = -1;
boolean closed = false;
long logSeqNum = 0;
long filenum = 0;
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.
*/
public static void consolidateOldLog(Path srcDir, Path dstFile, FileSystem fs, Configuration conf) throws IOException {
LOG.debug("consolidating log files");
Path logfiles[] = fs.listPaths(srcDir);
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);
try {
HLogKey key = new HLogKey();
HLogEdit val = new HLogEdit();
while(in.next(key, val)) {
newlog.append(key, val);
}
} finally {
in.close();
}
}
} finally {
newlog.close();
}
if(fs.exists(srcDir)) {
if(! fs.delete(srcDir)) {
LOG.error("Cannot delete: " + srcDir);
if(! FileUtil.fullyDelete(new File(srcDir.toString()))) {
throw new IOException("Cannot delete: " + srcDir);
}
}
}
LOG.debug("log file consolidation completed");
}
/**
* Create an edit log at the given 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
* the time the HLog object is started up.
*/
public HLog(FileSystem fs, Path dir, Configuration conf) throws IOException {
this.fs = fs;
this.dir = dir;
this.conf = conf;
this.logSeqNum = 0;
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.
*
* The 'rollLock' prevents us from entering rollWriter() more than
* once at a time.
*
* The 'this' lock limits access to the current writer so
* we don't append multiple items simultaneously.
*/
public void rollWriter() throws IOException {
synchronized(rollLock) {
// Try to roll the writer to a new file. We may have to
// wait for a cache-flush to complete. In the process,
// compute a list of old log files that can be deleted.
Vector<Path> toDeleteList = new Vector<Path>();
synchronized(this) {
if(closed) {
throw new IOException("Cannot roll log; log is closed");
}
// Make sure we do not roll the log while inside a
// cache-flush. Otherwise, the log sequence number for
// the CACHEFLUSH operation will appear in a "newer" log file
// than it should.
while(insideCacheFlush) {
try {
wait();
} catch (InterruptedException ie) {
}
}
LOG.debug("closing current log writer and getting a new one");
// Close the current writer (if any), and grab a new one.
if(writer != null) {
writer.close();
if(filenum > 0) {
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");
// Can we delete any of the old log files?
// First, compute the oldest relevant log operation
// over all the regions.
long oldestOutstandingSeqNum = Long.MAX_VALUE;
for(Iterator<Long> it = regionToLastFlush.values().iterator(); it.hasNext(); ) {
long curSeqNum = it.next().longValue();
if(curSeqNum < oldestOutstandingSeqNum) {
oldestOutstandingSeqNum = curSeqNum;
}
}
// Next, remove all files with a final ID that's older
// than the oldest pending region-operation.
LOG.debug("removing old log files");
for(Iterator<Long> it = outputfiles.keySet().iterator(); it.hasNext(); ) {
long maxSeqNum = it.next().longValue();
if(maxSeqNum < oldestOutstandingSeqNum) {
Path p = outputfiles.get(maxSeqNum);
it.remove();
toDeleteList.add(p);
} else {
break;
}
}
}
// Actually delete them, if any!
for(Iterator<Path> it = toDeleteList.iterator(); it.hasNext(); ) {
Path p = it.next();
fs.delete(p);
}
LOG.debug("old log files deleted");
this.numEntries = 0;
}
}
/**
* This is a convenience method that computes a new filename with
* a given file-number.
*/
Path computeFilename(long filenum) {
return new Path(dir, HLOG_DATFILE + String.format("%1$03d", filenum));
}
/** Shut down the log. */
public synchronized void close() throws IOException {
this.writer.close();
this.closed = true;
}
/**
* 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.
*
* 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.
*/
public synchronized void append(Text regionName, Text tableName, Text row, TreeMap<Text, byte[]> columns, long timestamp) throws IOException {
if(closed) {
throw new IOException("Cannot append; log is closed");
}
long seqNum[] = obtainSeqNum(columns.size());
// The 'regionToLastFlush' map holds the sequence id of the
// 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) {
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);
writer.append(logKey, logEdit);
numEntries++;
}
}
/** How many items have been added to the log? */
public int getNumEntries() {
return numEntries;
}
/**
* Obtain a log sequence number. This seizes the whole HLog
* lock, but it shouldn't last too long.
*/
synchronized long obtainSeqNum() {
return logSeqNum++;
}
synchronized long[] obtainSeqNum(int num) {
long[] results = new long[num];
for (int i = 0; i < num; i++) {
results[i] = logSeqNum++;
}
return results;
}
/**
* By acquiring a log sequence ID, we can allow log messages
* to continue while we flush the cache.
*
* 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.
*/
public synchronized long startCacheFlush() {
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 {
if(closed) {
return;
}
if(! insideCacheFlush) {
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()));
numEntries++;
// Remember the most-recent flush for each region.
// This is used to delete obsolete log files.
regionToLastFlush.put(regionName, logSeqId);
insideCacheFlush = false;
notifyAll();
}
}

View File

@ -0,0 +1,71 @@
/**
* 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 org.apache.hadoop.io.*;
import java.io.*;
/*******************************************************************************
* A log value.
*
* These aren't sortable; you need to sort by the matching HLogKey.
* The table and row are already identified in HLogKey.
* This just indicates the column and value.
******************************************************************************/
public class HLogEdit implements Writable {
Text column = new Text();
BytesWritable val = new BytesWritable();
long timestamp;
public HLogEdit() {
}
public HLogEdit(Text column, byte[] bval, long timestamp) {
this.column.set(column);
this.val = new BytesWritable(bval);
this.timestamp = timestamp;
}
public Text getColumn() {
return this.column;
}
public BytesWritable getVal() {
return this.val;
}
public long getTimestamp() {
return this.timestamp;
}
//////////////////////////////////////////////////////////////////////////////
// Writable
//////////////////////////////////////////////////////////////////////////////
public void write(DataOutput out) throws IOException {
this.column.write(out);
this.val.write(out);
out.writeLong(timestamp);
}
public void readFields(DataInput in) throws IOException {
this.column.readFields(in);
this.val.readFields(in);
this.timestamp = in.readLong();
}
}

View File

@ -0,0 +1,117 @@
/**
* 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 org.apache.hadoop.io.*;
import java.io.*;
/*******************************************************************************
* A Key for an entry in the change log.
*
* The log intermingles edits to many tables and rows, so each log entry
* identifies the appropriate table and row. Within a table and row, they're
* also sorted.
******************************************************************************/
public class HLogKey implements WritableComparable {
Text regionName = new Text();
Text tablename = new Text();
Text row = new Text();
long logSeqNum = 0L;
/**
* Create the log key!
* We maintain the tablename mainly for debugging purposes.
* A regionName is always a sub-table object.
*/
public HLogKey() {
}
public HLogKey(Text regionName, Text tablename, Text row, long logSeqNum) {
this.regionName.set(regionName);
this.tablename.set(tablename);
this.row.set(row);
this.logSeqNum = logSeqNum;
}
//////////////////////////////////////////////////////////////////////////////
// A bunch of accessors
//////////////////////////////////////////////////////////////////////////////
public Text getRegionName() {
return regionName;
}
public Text getTablename() {
return tablename;
}
public Text getRow() {
return row;
}
public long getLogSeqNum() {
return logSeqNum;
}
//////////////////////////////////////////////////////////////////////////////
// Comparable
//////////////////////////////////////////////////////////////////////////////
/**
* When sorting through log entries, we want to group items
* first in the same table, then to the same row, then finally
* ordered by write-order.
*/
public int compareTo(Object o) {
HLogKey other = (HLogKey) o;
int result = this.regionName.compareTo(other.regionName);
if(result == 0) {
result = this.row.compareTo(other.row);
if(result == 0) {
if (this.logSeqNum < other.logSeqNum) {
result = -1;
} else if (this.logSeqNum > other.logSeqNum) {
result = 1;
}
}
}
return result;
}
//////////////////////////////////////////////////////////////////////////////
// Writable
//////////////////////////////////////////////////////////////////////////////
public void write(DataOutput out) throws IOException {
this.regionName.write(out);
this.tablename.write(out);
this.row.write(out);
out.writeLong(logSeqNum);
}
public void readFields(DataInput in) throws IOException {
this.regionName.readFields(in);
this.tablename.readFields(in);
this.row.readFields(in);
this.logSeqNum = in.readLong();
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,41 @@
/**
* 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 org.apache.hadoop.io.*;
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.
******************************************************************************/
public interface HMasterInterface {
public static final long versionID = 1L; // initial version
//////////////////////////////////////////////////////////////////////////////
// Admin tools would use these cmds
//////////////////////////////////////////////////////////////////////////////
public void createTable(HTableDescriptor desc) throws IOException;
public void deleteTable(Text tableName) throws IOException;
//////////////////////////////////////////////////////////////////////////////
// These are the method calls of last resort when trying to find an HRegion
//////////////////////////////////////////////////////////////////////////////
public HServerAddress findRootRegion();
}

View File

@ -0,0 +1,27 @@
/**
* 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.*;
/*******************************************************************************
* HRegionServers interact with the HMasterRegionInterface to report on local
* goings-on and to obtain data-handling instructions from the HMaster.
*********************************************/
public interface HMasterRegionInterface {
public static final long versionId = 1L;
public void regionServerStartup(HServerInfo info) throws IOException;
public HMsg[] regionServerReport(HServerInfo info, HMsg msgs[]) throws IOException;
}

View File

@ -0,0 +1,368 @@
/**
* 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 org.apache.hadoop.io.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.io.*;
import java.util.*;
/*******************************************************************************
* The HMemcache holds in-memory modifications to the HRegion. This is really a
* wrapper around a TreeMap that helps us when staging the Memcache out to disk.
******************************************************************************/
public class HMemcache {
private static final Log LOG = LogFactory.getLog(HMemcache.class);
TreeMap<HStoreKey, BytesWritable> memcache
= new TreeMap<HStoreKey, BytesWritable>();
Vector<TreeMap<HStoreKey, BytesWritable>> history
= new Vector<TreeMap<HStoreKey, BytesWritable>>();
TreeMap<HStoreKey, BytesWritable> snapshot = null;
HLocking locking = new HLocking();
public HMemcache() {
}
public static class Snapshot {
public TreeMap<HStoreKey, BytesWritable> memcacheSnapshot = null;
public long sequenceId = 0;
public Snapshot() {
}
}
/**
* We want to return a snapshot of the current HMemcache with a known HLog
* sequence number at the same time.
*
* Return both the frozen HMemcache TreeMap, as well as the HLog seq number.
*
* We need to prevent any writing to the cache during this time, so we obtain
* a write lock for the duration of the operation.
*/
public Snapshot snapshotMemcacheForLog(HLog log) throws IOException {
Snapshot retval = new Snapshot();
locking.obtainWriteLock();
try {
if(snapshot != null) {
throw new IOException("Snapshot in progress!");
}
if(memcache.size() == 0) {
LOG.debug("memcache empty. Skipping snapshot");
return retval;
}
LOG.debug("starting memcache snapshot");
retval.memcacheSnapshot = memcache;
this.snapshot = memcache;
history.add(memcache);
memcache = new TreeMap<HStoreKey, BytesWritable>();
retval.sequenceId = log.startCacheFlush();
LOG.debug("memcache snapshot complete");
return retval;
} finally {
locking.releaseWriteLock();
}
}
/**
* Delete the snapshot, remove from history.
*
* Modifying the structure means we need to obtain a writelock.
*/
public void deleteSnapshot() throws IOException {
locking.obtainWriteLock();
try {
if(snapshot == null) {
throw new IOException("Snapshot not present!");
}
LOG.debug("deleting snapshot");
for(Iterator<TreeMap<HStoreKey, BytesWritable>> it = history.iterator();
it.hasNext(); ) {
TreeMap<HStoreKey, BytesWritable> cur = it.next();
if(snapshot == cur) {
it.remove();
break;
}
}
this.snapshot = null;
LOG.debug("snapshot deleted");
} finally {
locking.releaseWriteLock();
}
}
/**
* Store a value.
*
* Operation uses a write lock.
*/
public void add(Text row, TreeMap<Text, byte[]> columns, long timestamp) {
locking.obtainWriteLock();
try {
for(Iterator<Text> it = columns.keySet().iterator(); it.hasNext(); ) {
Text column = it.next();
byte[] val = columns.get(column);
HStoreKey key = new HStoreKey(row, column, timestamp);
memcache.put(key, new BytesWritable(val));
}
} finally {
locking.releaseWriteLock();
}
}
/**
* Look back through all the backlog TreeMaps to find the target.
*
* We only need a readlock here.
*/
public byte[][] get(HStoreKey key, int numVersions) {
Vector<byte[]> results = new Vector<byte[]>();
locking.obtainReadLock();
try {
Vector<byte[]> result = get(memcache, key, numVersions-results.size());
results.addAll(0, result);
for(int i = history.size()-1; i >= 0; i--) {
if(numVersions > 0 && results.size() >= numVersions) {
break;
}
result = get(history.elementAt(i), key, numVersions-results.size());
results.addAll(results.size(), result);
}
if(results.size() == 0) {
return null;
} else {
return (byte[][]) results.toArray(new byte[results.size()][]);
}
} finally {
locking.releaseReadLock();
}
}
/**
* 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[]>();
locking.obtainReadLock();
try {
internalGetFull(memcache, key, results);
for(int i = history.size()-1; i >= 0; i--) {
TreeMap<HStoreKey, BytesWritable> cur = history.elementAt(i);
internalGetFull(cur, key, results);
}
return results;
} finally {
locking.releaseReadLock();
}
}
void internalGetFull(TreeMap<HStoreKey, BytesWritable> map, HStoreKey key,
TreeMap<Text, byte[]> results) {
SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(key);
for(Iterator<HStoreKey> it = tailMap.keySet().iterator(); it.hasNext(); ) {
HStoreKey itKey = it.next();
Text itCol = itKey.getColumn();
if(results.get(itCol) == null
&& key.matchesWithoutColumn(itKey)) {
BytesWritable val = tailMap.get(itKey);
results.put(itCol, val.get());
} else if(key.getRow().compareTo(itKey.getRow()) > 0) {
break;
}
}
}
/**
* Examine a single map for the desired key.
*
* We assume that all locking is done at a higher-level. No locking within
* this method.
*
* 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[]>();
HStoreKey curKey = new HStoreKey(key.getRow(), key.getColumn(), key.getTimestamp());
SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(curKey);
for(Iterator<HStoreKey> it = tailMap.keySet().iterator(); it.hasNext(); ) {
HStoreKey itKey = it.next();
if(itKey.matchesRowCol(curKey)) {
result.add(tailMap.get(itKey).get());
curKey.setVersion(itKey.getTimestamp() - 1);
}
if(numVersions > 0 && result.size() >= numVersions) {
break;
}
}
return result;
}
/**
* Return a scanner over the keys in the HMemcache
*/
public HScannerInterface getScanner(long timestamp, Text targetCols[], Text firstRow)
throws IOException {
return new HMemcacheScanner(timestamp, targetCols, firstRow);
}
//////////////////////////////////////////////////////////////////////////////
// HMemcacheScanner implements the HScannerInterface.
// It lets the caller scan the contents of the Memcache.
//////////////////////////////////////////////////////////////////////////////
class HMemcacheScanner extends HAbstractScanner {
TreeMap<HStoreKey, BytesWritable> backingMaps[];
Iterator<HStoreKey> keyIterators[];
@SuppressWarnings("unchecked")
public HMemcacheScanner(long timestamp, Text targetCols[], Text firstRow)
throws IOException {
super(timestamp, targetCols);
locking.obtainReadLock();
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();
}
backingMaps[backingMaps.length - 1] = memcache;
this.keyIterators = new Iterator[backingMaps.length];
this.keys = new HStoreKey[backingMaps.length];
this.vals = new BytesWritable[backingMaps.length];
// Generate list of iterators
HStoreKey firstKey = new HStoreKey(firstRow);
for(i = 0; i < backingMaps.length; i++) {
if(firstRow.getLength() != 0) {
keyIterators[i] = backingMaps[i].tailMap(firstKey).keySet().iterator();
} else {
keyIterators[i] = backingMaps[i].keySet().iterator();
}
while(getNext(i)) {
if(! findFirstRow(i, firstRow)) {
continue;
}
if(columnMatch(i)) {
break;
}
}
}
} catch(Exception ex) {
close();
}
}
/**
* The user didn't want to start scanning at the first row. This method
* seeks to the requested row.
*
* @param i - which iterator to advance
* @param firstRow - seek to this row
* @return - true if this is the first row
*/
boolean findFirstRow(int i, Text firstRow) {
return ((firstRow.getLength() == 0) || (keys[i].getRow().equals(firstRow)));
}
/**
* Get the next value from the specified iterater.
*
* @param i - which iterator to fetch next value from
* @return - true if there is more data available
*/
boolean getNext(int i) {
if(! keyIterators[i].hasNext()) {
closeSubScanner(i);
return false;
}
this.keys[i] = keyIterators[i].next();
this.vals[i] = backingMaps[i].get(keys[i]);
return true;
}
/** Shut down an individual map iterator. */
void closeSubScanner(int i) {
keyIterators[i] = null;
keys[i] = null;
vals[i] = null;
backingMaps[i] = null;
}
/** Shut down map iterators, and release the lock */
public void close() throws IOException {
if(! scannerClosed) {
try {
for(int i = 0; i < keys.length; i++) {
if(keyIterators[i] != null) {
closeSubScanner(i);
}
}
} finally {
locking.releaseReadLock();
scannerClosed = true;
}
}
}
}
}

View File

@ -0,0 +1,79 @@
/**
* Copyright 2006-7 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 org.apache.hadoop.io.*;
import java.io.*;
/*******************************************************************************
* HMsg is for communicating instructions between the HMaster and the
* HRegionServers.
******************************************************************************/
public class HMsg implements Writable {
public static final byte MSG_REGION_OPEN = 1;
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_REGION_CLOSE_WITHOUT_REPORT = 6;
public static final byte MSG_REGION_CLOSE_AND_DELETE = 7;
public static final byte MSG_REPORT_OPEN = 100;
public static final byte MSG_REPORT_CLOSE = 101;
public static final byte MSG_REGION_SPLIT = 102;
public static final byte MSG_NEW_REGION = 103;
byte msg;
HRegionInfo info;
public HMsg() {
this.info = new HRegionInfo();
}
public HMsg(byte msg) {
this.msg = msg;
this.info = new HRegionInfo();
}
public HMsg(byte msg, HRegionInfo info) {
this.msg = msg;
this.info = info;
}
public byte getMsg() {
return msg;
}
public HRegionInfo getRegionInfo() {
return info;
}
//////////////////////////////////////////////////////////////////////////////
// Writable
//////////////////////////////////////////////////////////////////////////////
public void write(DataOutput out) throws IOException {
out.writeByte(msg);
info.write(out);
}
public void readFields(DataInput in) throws IOException {
this.msg = in.readByte();
this.info.readFields(in);
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,84 @@
/**
* 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.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
public class HRegionInfo implements Writable {
public long regionId;
public HTableDescriptor tableDesc;
public Text startKey;
public Text endKey;
public Text regionName;
public HRegionInfo() {
this.regionId = 0;
this.tableDesc = new HTableDescriptor();
this.startKey = new Text();
this.endKey = new Text();
this.regionName = new Text();
}
public HRegionInfo(long regionId, HTableDescriptor tableDesc, Text startKey,
Text endKey) throws IllegalArgumentException {
this.regionId = regionId;
if(tableDesc == null) {
throw new IllegalArgumentException("tableDesc cannot be null");
}
this.tableDesc = tableDesc;
this.startKey = new Text();
if(startKey != null) {
this.startKey.set(startKey);
}
this.endKey = new Text();
if(endKey != null) {
this.endKey.set(endKey);
}
this.regionName = new Text(tableDesc.getName() + "_"
+ (startKey == null ? "" : startKey.toString()) + "_" + regionId);
}
//////////////////////////////////////////////////////////////////////////////
// Writable
//////////////////////////////////////////////////////////////////////////////
public void write(DataOutput out) throws IOException {
out.writeLong(regionId);
tableDesc.write(out);
startKey.write(out);
endKey.write(out);
regionName.write(out);
}
public void readFields(DataInput in) throws IOException {
this.regionId = in.readLong();
this.tableDesc.readFields(in);
this.startKey.readFields(in);
this.endKey.readFields(in);
this.regionName.readFields(in);
}
}

View File

@ -0,0 +1,61 @@
/**
* 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 org.apache.hadoop.io.*;
import java.io.*;
/*******************************************************************************
* Clients interact with HRegionServers using
* a handle to the HRegionInterface.
******************************************************************************/
public interface HRegionInterface {
public static final long versionID = 1L; // initial version
// Get metainfo about an HRegion
public HRegionInfo getRegionInfo(Text regionName);
// Start a scanner for a given HRegion.
public HScannerInterface openScanner(Text regionName, Text[] columns, Text startRow) throws IOException;
// GET methods for an HRegion.
public BytesWritable get(Text regionName, Text row, Text column) throws IOException;
public BytesWritable[] get(Text regionName, Text row, Text column, int numVersions) throws IOException;
public BytesWritable[] get(Text regionName, Text row, Text column, long timestamp, int numVersions) throws IOException;
public LabelledData[] getRow(Text regionName, Text row) throws IOException;
//////////////////////////////////////////////////////////////////////////////
// Start an atomic row insertion/update. No changes are committed until the
// call to commit() returns. A call to abort() will abandon any updates in progress.
//
// Callers to this method are given a lease for each unique lockid; before the
// lease expires, either abort() or commit() must be called. If it is not
// called, the system will automatically call abort() on the client's behalf.
//
// The client can gain extra time with a call to renewLease().
//////////////////////////////////////////////////////////////////////////////
public long startUpdate(Text regionName, long clientid, Text row) throws IOException;
public void put(Text regionName, long clientid, long lockid, Text column, BytesWritable val) throws IOException;
public void delete(Text regionName, long clientid, long lockid, Text column) throws IOException;
public void abort(Text regionName, long clientid, long lockid) throws IOException;
public void commit(Text regionName, long clientid, long lockid) throws IOException;
public void renewLease(long lockid, long clientid) throws IOException;
}

View File

@ -0,0 +1,818 @@
/**
* 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 org.apache.hadoop.io.*;
import org.apache.hadoop.fs.*;
import org.apache.hadoop.ipc.*;
import org.apache.hadoop.conf.*;
import java.io.*;
import java.util.*;
/*******************************************************************************
* HRegionServer makes a set of HRegions available to clients. It checks in with
* the HMaster. There are many HRegionServers in a single HBase deployment.
******************************************************************************/
public class HRegionServer implements HConstants, HRegionInterface, Runnable {
private boolean stopRequested;
private Path regionDir;
private HServerAddress address;
private Configuration conf;
private Random rand;
private TreeMap<Text, HRegion> regions; // region name -> HRegion
private HLocking locking;
private Vector<HMsg> outboundMsgs;
private long threadWakeFrequency;
private int maxLogEntries;
private long msgInterval;
// Check to see if regions should be split
private long splitCheckFrequency;
private SplitChecker splitChecker;
private Thread splitCheckerThread;
private class SplitChecker implements Runnable {
private HClient client = new HClient(conf);
private class SplitRegion {
public HRegion region;
public Text midKey;
SplitRegion(HRegion region, Text midKey) {
this.region = region;
this.midKey = midKey;
}
}
public void run() {
while(! stopRequested) {
long startTime = System.currentTimeMillis();
// Grab a list of regions to check
Vector<HRegion> checkSplit = new Vector<HRegion>();
locking.obtainReadLock();
try {
checkSplit.addAll(regions.values());
} finally {
locking.releaseReadLock();
}
// Check to see if they need splitting
Vector<SplitRegion> toSplit = new Vector<SplitRegion>();
for(Iterator<HRegion> it = checkSplit.iterator(); it.hasNext(); ) {
HRegion cur = it.next();
Text midKey = new Text();
try {
if(cur.needsSplit(midKey)) {
toSplit.add(new SplitRegion(cur, midKey));
}
} catch(IOException iex) {
iex.printStackTrace();
}
}
for(Iterator<SplitRegion> it = toSplit.iterator(); it.hasNext(); ) {
SplitRegion r = it.next();
locking.obtainWriteLock();
regions.remove(r.region.getRegionName());
locking.releaseWriteLock();
HRegion[] newRegions = null;
try {
Text oldRegion = r.region.getRegionName();
newRegions = r.region.closeAndSplit(r.midKey);
// When a region is split, the META table needs to updated if we're
// splitting a 'normal' region, and the ROOT table needs to be
// updated if we are splitting a META region.
Text tableToUpdate
= (oldRegion.find(META_TABLE_NAME.toString()) == 0)
? ROOT_TABLE_NAME : META_TABLE_NAME;
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.commit(lockid);
for(int i = 0; i < newRegions.length; i++) {
ByteArrayOutputStream bytes = new ByteArrayOutputStream();
DataOutputStream out = new DataOutputStream(bytes);
newRegions[i].getRegionInfo().write(out);
lockid = client.startUpdate(newRegions[i].getRegionName());
client.put(lockid, META_COL_REGIONINFO, bytes.toByteArray());
client.commit(lockid);
}
// Now tell the master about the new regions
reportSplit(newRegions[0].getRegionInfo(), newRegions[1].getRegionInfo());
newRegions[0].close();
newRegions[1].close();
} catch(IOException e) {
//TODO: What happens if this fails? Are we toast?
e.printStackTrace();
continue;
}
}
// Sleep
long endTime = System.currentTimeMillis();
try {
Thread.sleep(splitCheckFrequency - (endTime - startTime));
} catch(InterruptedException iex) {
}
}
}
}
// Cache flushing
private Flusher cacheFlusher;
private Thread cacheFlusherThread;
private class Flusher implements Runnable {
public void run() {
while(! stopRequested) {
long startTime = System.currentTimeMillis();
// Grab a list of items to flush
Vector<HRegion> toFlush = new Vector<HRegion>();
locking.obtainReadLock();
try {
toFlush.addAll(regions.values());
} finally {
locking.releaseReadLock();
}
// Flush them, if necessary
for(Iterator<HRegion> it = toFlush.iterator(); it.hasNext(); ) {
HRegion cur = it.next();
try {
cur.optionallyFlush();
} catch(IOException iex) {
iex.printStackTrace();
}
}
// Sleep
long endTime = System.currentTimeMillis();
try {
Thread.sleep(threadWakeFrequency - (endTime - startTime));
} catch(InterruptedException iex) {
}
}
}
}
// File paths
private FileSystem fs;
private Path oldlogfile;
// Logging
private HLog log;
private LogRoller logRoller;
private Thread logRollerThread;
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) {
}
}
}
}
// Remote HMaster
private HMasterRegionInterface hbaseMaster;
// Server
private Server server;
// Leases
private Leases leases;
/** Start a HRegionServer at the default location */
public HRegionServer(Configuration conf) throws IOException {
this(new Path(conf.get(HREGION_DIR, DEFAULT_HREGION_DIR)),
new HServerAddress(conf.get("hbase.regionserver.default.name")),
conf);
}
/** Start a HRegionServer at an indicated location */
public HRegionServer(Path regionDir, HServerAddress address, Configuration conf)
throws IOException {
// Basic setup
this.stopRequested = false;
this.regionDir = regionDir;
this.address = address;
this.conf = conf;
this.rand = new Random();
this.regions = new TreeMap<Text, HRegion>();
this.locking = new HLocking();
this.outboundMsgs = new Vector<HMsg>();
// 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);
// Cache flushing
this.cacheFlusher = new Flusher();
this.cacheFlusherThread = new Thread(cacheFlusher);
// Check regions to see if they need to be split
this.splitChecker = new SplitChecker();
this.splitCheckerThread = new Thread(splitChecker);
try {
// Local file paths
this.fs = FileSystem.get(conf);
Path newlogdir = new Path(regionDir, "log" + "_" + address.toString());
this.oldlogfile = new Path(regionDir, "oldlogfile" + "_" + address.toString());
// Logging
HLog.consolidateOldLog(newlogdir, oldlogfile, fs, conf);
this.log = new HLog(fs, newlogdir, conf);
this.logRoller = new LogRoller();
this.logRollerThread = new Thread(logRoller);
// Remote HMaster
this.hbaseMaster = (HMasterRegionInterface)
RPC.waitForProxy(HMasterRegionInterface.class,
HMasterRegionInterface.versionId,
new HServerAddress(conf.get(MASTER_DEFAULT_NAME)).getInetSocketAddress(),
conf);
// Threads
this.cacheFlusherThread.start();
this.splitCheckerThread.start();
this.logRollerThread.start();
this.leases = new Leases(conf.getLong("hbase.hregionserver.lease.period",
3 * 60 * 1000), threadWakeFrequency);
// Server
this.server = RPC.getServer(this, address.getBindAddress().toString(),
address.getPort(), conf.getInt("hbase.hregionserver.handler.count", 10), false, conf);
this.server.start();
} catch(IOException e) {
this.stopRequested = true;
throw e;
}
}
/**
* 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.
*/
public void stop() throws IOException {
if(! stopRequested) {
stopRequested = true;
closeAllRegions();
log.close();
fs.close();
server.stop();
}
}
/** Call join to wait for all the threads to finish */
public void join() {
try {
this.logRollerThread.join();
} catch(InterruptedException iex) {
}
try {
this.cacheFlusherThread.join();
} catch(InterruptedException iex) {
}
this.leases.close();
try {
this.server.join();
} catch(InterruptedException iex) {
}
}
/**
* The HRegionServer sticks in this loop until close. It repeatedly checks in
* with the HMaster, sending heartbeats & reports, and receiving HRegion
* load/unload instructions.
*/
public void run() {
while(! stopRequested) {
HServerInfo info = new HServerInfo(address, rand.nextLong());
long lastMsg = 0;
long waitTime;
// Let the master know we're here
try {
hbaseMaster.regionServerStartup(info);
} catch(IOException e) {
waitTime = msgInterval - (System.currentTimeMillis() - lastMsg);
try {
Thread.sleep(waitTime);
} catch(InterruptedException iex) {
}
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) {
HMsg outboundArray[] = null;
synchronized(outboundMsgs) {
outboundArray = outboundMsgs.toArray(new HMsg[outboundMsgs.size()]);
outboundMsgs.clear();
}
try {
HMsg msgs[] = hbaseMaster.regionServerReport(info, outboundArray);
lastMsg = System.currentTimeMillis();
// Process the HMaster's instruction stream
if(! processMessages(msgs)) {
break;
}
} catch(IOException e) {
e.printStackTrace();
}
}
waitTime = msgInterval - (System.currentTimeMillis() - lastMsg);
try {
Thread.sleep(waitTime);
} catch(InterruptedException iex) {
}
}
}
}
private boolean processMessages(HMsg[] msgs) throws IOException {
for(int i = 0; i < msgs.length; i++) {
switch(msgs[i].getMsg()) {
case HMsg.MSG_REGION_OPEN: // Open a region
openRegion(msgs[i].getRegionInfo());
break;
case HMsg.MSG_REGION_CLOSE: // Close a region
closeRegion(msgs[i].getRegionInfo(), true);
break;
case HMsg.MSG_REGION_MERGE: // Merge two regions
//TODO ???
throw new IOException("TODO: need to figure out merge");
//break;
case HMsg.MSG_CALL_SERVER_STARTUP: // Close regions, restart
closeAllRegions();
return false;
case HMsg.MSG_REGIONSERVER_ALREADY_RUNNING: // Go away
stop();
return false;
case HMsg.MSG_REGION_CLOSE_WITHOUT_REPORT: // Close a region, don't reply
closeRegion(msgs[i].getRegionInfo(), false);
break;
case HMsg.MSG_REGION_CLOSE_AND_DELETE:
closeAndDeleteRegion(msgs[i].getRegionInfo());
break;
default:
throw new IOException("Impossible state during msg processing. Instruction: " + msgs[i]);
}
}
return true;
}
/** Add to the outbound message buffer */
private void reportOpen(HRegion region) {
synchronized(outboundMsgs) {
outboundMsgs.add(new HMsg(HMsg.MSG_REPORT_OPEN, region.getRegionInfo()));
}
}
/** Add to the outbound message buffer */
private void reportClose(HRegion region) {
synchronized(outboundMsgs) {
outboundMsgs.add(new HMsg(HMsg.MSG_REPORT_CLOSE, region.getRegionInfo()));
}
}
/**
* Add to the outbound message buffer
*
* When a region splits, we need to tell the master that there are two new
* regions that need to be assigned.
*
* We do not need to inform the master about the old region, because we've
* updated the meta or root regions, and the master will pick that up on its
* next rescan of the root or meta tables.
*/
private void reportSplit(HRegionInfo newRegionA, HRegionInfo newRegionB) {
synchronized(outboundMsgs) {
outboundMsgs.add(new HMsg(HMsg.MSG_NEW_REGION, newRegionA));
outboundMsgs.add(new HMsg(HMsg.MSG_NEW_REGION, newRegionB));
}
}
//////////////////////////////////////////////////////////////////////////////
// HMaster-given operations
//////////////////////////////////////////////////////////////////////////////
private void openRegion(HRegionInfo regionInfo) throws IOException {
locking.obtainWriteLock();
try {
HRegion region = new HRegion(regionDir, log, fs, conf, regionInfo, null, oldlogfile);
regions.put(region.getRegionName(), region);
reportOpen(region);
} finally {
locking.releaseWriteLock();
}
}
private void closeRegion(HRegionInfo info, boolean reportWhenCompleted)
throws IOException {
locking.obtainWriteLock();
try {
HRegion region = regions.remove(info.regionName);
if(region != null) {
region.close();
if(reportWhenCompleted) {
reportClose(region);
}
}
} finally {
locking.releaseWriteLock();
}
}
private void closeAndDeleteRegion(HRegionInfo info) throws IOException {
locking.obtainWriteLock();
try {
HRegion region = regions.remove(info.regionName);
if(region != null) {
region.closeAndDelete();
}
} finally {
locking.releaseWriteLock();
}
}
/** Called either when the master tells us to restart or from stop() */
private void closeAllRegions() throws IOException {
locking.obtainWriteLock();
try {
for(Iterator<HRegion> it = regions.values().iterator(); it.hasNext(); ) {
HRegion region = it.next();
region.close();
}
regions.clear();
} finally {
locking.releaseWriteLock();
}
}
/*****************************************************************************
* TODO - Figure out how the master is to determine when regions should be
* merged. It once it makes this determination, it needs to ensure that
* the regions to be merged are first being served by the same
* HRegionServer and if not, move them so they are.
*
* For now, we do not do merging. Splits are driven by the HRegionServer.
****************************************************************************/
/*
private void mergeRegions(Text regionNameA, Text regionNameB) throws IOException {
locking.obtainWriteLock();
try {
HRegion srcA = regions.remove(regionNameA);
HRegion srcB = regions.remove(regionNameB);
HRegion newRegion = HRegion.closeAndMerge(srcA, srcB);
regions.put(newRegion.getRegionName(), newRegion);
reportClose(srcA);
reportClose(srcB);
reportOpen(newRegion);
} finally {
locking.releaseWriteLock();
}
}
*/
//////////////////////////////////////////////////////////////////////////////
// HRegionInterface
//////////////////////////////////////////////////////////////////////////////
/** Obtain a table descriptor for the given region */
public HRegionInfo getRegionInfo(Text regionName) {
HRegion region = getRegion(regionName);
if(region == null) {
return null;
}
return region.getRegionInfo();
}
/** Start a scanner for a given HRegion. */
public HScannerInterface openScanner(Text regionName, Text[] cols,
Text firstRow) throws IOException {
HRegion r = getRegion(regionName);
if(r == null) {
throw new IOException("Not serving region " + regionName);
}
return r.getScanner(cols, firstRow);
}
/** Get the indicated row/column */
public BytesWritable get(Text regionName, Text row, Text column) throws IOException {
HRegion region = getRegion(regionName);
if(region == null) {
throw new IOException("Not serving region " + regionName);
}
byte results[] = region.get(row, column);
if(results != null) {
return new BytesWritable(results);
}
return null;
}
/** Get multiple versions of the indicated row/col */
public BytesWritable[] get(Text regionName, Text row, Text column,
int numVersions) throws IOException {
HRegion region = getRegion(regionName);
if(region == null) {
throw new IOException("Not serving region " + regionName);
}
byte 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 null;
}
/** Get multiple timestamped versions of the indicated row/col */
public BytesWritable[] get(Text regionName, Text row, Text column,
long timestamp, int numVersions) throws IOException {
HRegion region = getRegion(regionName);
if(region == null) {
throw new IOException("Not serving region " + regionName);
}
byte 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 null;
}
/** Get all the columns (along with their names) for a given row. */
public LabelledData[] getRow(Text regionName, Text row) throws IOException {
HRegion region = getRegion(regionName);
if(region == null) {
throw new IOException("Not serving region " + regionName);
}
TreeMap<Text, byte[]> 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);
result[counter++] = new LabelledData(colname, val);
}
return result;
}
/**
* Start an update to the HBase. This also creates a lease associated with
* the caller.
*/
private class RegionListener extends LeaseListener {
private HRegion localRegion;
private long localLockId;
public RegionListener(HRegion region, long lockId) {
this.localRegion = region;
this.localLockId = lockId;
}
public void leaseExpired() {
try {
localRegion.abort(localLockId);
} catch(IOException iex) {
iex.printStackTrace();
}
}
}
public long startUpdate(Text regionName, long clientid, Text row)
throws IOException {
HRegion region = getRegion(regionName);
if(region == null) {
throw new IOException("Not serving region " + regionName);
}
long lockid = region.startUpdate(row);
leases.createLease(new Text(String.valueOf(clientid)),
new Text(String.valueOf(lockid)),
new RegionListener(region, lockid));
return lockid;
}
/** Add something to the HBase. */
public void put(Text regionName, long clientid, long lockid, Text column,
BytesWritable val) throws IOException {
HRegion region = getRegion(regionName);
if(region == null) {
throw new IOException("Not serving region " + regionName);
}
leases.renewLease(new Text(String.valueOf(clientid)),
new Text(String.valueOf(lockid)));
region.put(lockid, column, val.get());
}
/** Remove a cell from the HBase. */
public void delete(Text regionName, long clientid, long lockid, Text column)
throws IOException {
HRegion region = getRegion(regionName);
if(region == null) {
throw new IOException("Not serving region " + regionName);
}
leases.renewLease(new Text(String.valueOf(clientid)),
new Text(String.valueOf(lockid)));
region.delete(lockid, column);
}
/** Abandon the transaction */
public void abort(Text regionName, long clientid, long lockid)
throws IOException {
HRegion region = getRegion(regionName);
if(region == null) {
throw new IOException("Not serving region " + regionName);
}
leases.cancelLease(new Text(String.valueOf(clientid)),
new Text(String.valueOf(lockid)));
region.abort(lockid);
}
/** Confirm the transaction */
public void commit(Text regionName, long clientid, long lockid)
throws IOException {
HRegion region = getRegion(regionName);
if(region == null) {
throw new IOException("Not serving region " + regionName);
}
leases.cancelLease(new Text(String.valueOf(clientid)),
new Text(String.valueOf(lockid)));
region.commit(lockid);
}
/** Don't let the client's lease expire just yet... */
public void renewLease(long lockid, long clientid) throws IOException {
leases.renewLease(new Text(String.valueOf(clientid)),
new Text(String.valueOf(lockid)));
}
/** Private utility method for safely obtaining an HRegion handle. */
private HRegion getRegion(Text regionName) {
locking.obtainReadLock();
try {
return regions.get(regionName);
} finally {
locking.releaseReadLock();
}
}
}

View File

@ -0,0 +1,29 @@
/**
* 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 org.apache.hadoop.io.*;
import java.io.*;
import java.util.*;
/*******************************************************************************
* HScannerInterface iterates through a set of rows. It's implemented by several classes.
******************************************************************************/
public interface HScannerInterface {
public boolean next(HStoreKey key, TreeMap<Text, byte[]> results) throws IOException;
public void close() throws IOException;
}

View File

@ -0,0 +1,103 @@
/**
* Copyright 2006-7 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 org.apache.hadoop.io.*;
import java.io.*;
import java.net.InetSocketAddress;
/*******************************************************************************
* HServerAddress is a "label" for a HBase server that combines the host
* name and port number.
******************************************************************************/
public class HServerAddress implements Writable {
private InetSocketAddress address;
private String stringValue;
public HServerAddress() {
this.address = null;
this.stringValue = null;
}
public HServerAddress(String hostAndPort) {
int colonIndex = hostAndPort.indexOf(':');
if(colonIndex < 0) {
throw new IllegalArgumentException("Not a host:port pair: " + hostAndPort);
}
String host = hostAndPort.substring(0, colonIndex);
int port = Integer.valueOf(hostAndPort.substring(colonIndex + 1));
this.address = new InetSocketAddress(host, port);
this.stringValue = new String(hostAndPort);
}
public HServerAddress(String bindAddress, int port) {
this.address = new InetSocketAddress(bindAddress, port);
this.stringValue = new String(bindAddress + ":" + port);
}
public HServerAddress(HServerAddress other) {
String bindAddress = other.getBindAddress();
int port = other.getPort();
address = new InetSocketAddress(bindAddress, port);
stringValue = new String(bindAddress + ":" + port);
}
public String getBindAddress() {
return address.getAddress().getHostAddress();
}
public int getPort() {
return address.getPort();
}
public InetSocketAddress getInetSocketAddress() {
return address;
}
public String toString() {
return (stringValue == null ? "" : stringValue);
}
//////////////////////////////////////////////////////////////////////////////
// Writable
//////////////////////////////////////////////////////////////////////////////
public void readFields(DataInput in) throws IOException {
String bindAddress = in.readUTF();
int port = in.readInt();
if(bindAddress == null || bindAddress.length() == 0) {
address = null;
stringValue = null;
} else {
address = new InetSocketAddress(bindAddress, port);
stringValue = bindAddress + "_" + port;
}
}
public void write(DataOutput out) throws IOException {
if(address == null) {
out.writeUTF("");
out.writeInt(0);
} else {
out.writeUTF(address.getAddress().getHostAddress());
out.writeInt(address.getPort());
}
}
}

View File

@ -0,0 +1,66 @@
/**
* Copyright 2006-7 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 org.apache.hadoop.io.*;
import java.io.*;
/*******************************************************************************
* HRSInfo contains metainfo about an HRegionServer, including details about the
* source machine and load statistics.
******************************************************************************/
public class HServerInfo implements Writable {
private HServerAddress serverAddress;
private long startCode;
public HServerInfo() {
this.serverAddress = new HServerAddress();
this.startCode = 0;
}
public HServerInfo(HServerAddress serverAddress, long startCode) {
this.serverAddress = new HServerAddress(serverAddress);
this.startCode = startCode;
}
public HServerInfo(HServerInfo other) {
this.serverAddress = new HServerAddress(other.getServerAddress());
this.startCode = other.getStartCode();
}
public HServerAddress getServerAddress() {
return serverAddress;
}
public long getStartCode() {
return startCode;
}
//////////////////////////////////////////////////////////////////////////////
// Writable
//////////////////////////////////////////////////////////////////////////////
public void readFields(DataInput in) throws IOException {
this.serverAddress.readFields(in);
this.startCode = in.readLong();
}
public void write(DataOutput out) throws IOException {
this.serverAddress.write(out);
out.writeLong(this.startCode);
}
}

View File

@ -0,0 +1,976 @@
/**
* 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 org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.io.*;
import org.apache.hadoop.fs.*;
import java.io.*;
import java.util.*;
/*******************************************************************************
* HStore maintains a bunch of data files. It is responsible for maintaining
* the memory/file hierarchy and for periodic flushes to disk and compacting
* edits to the file.
*
* Locking and transactions are handled at a higher level. This API should not
* be called directly by any writer, but rather by an HRegion manager.
******************************************************************************/
public class HStore {
private static final Log LOG = LogFactory.getLog(HStore.class);
static final String COMPACTION_DIR = "compaction.tmp";
static final String WORKING_COMPACTION = "compaction.inprogress";
static final String COMPACTION_TO_REPLACE = "toreplace";
static final String COMPACTION_DONE = "done";
Path dir;
Text regionName;
Text colFamily;
int maxVersions;
FileSystem fs;
Configuration conf;
Path mapdir;
Path compactdir;
Path loginfodir;
Integer compactLock = new Integer(0);
Integer flushLock = new Integer(0);
HLocking locking = new HLocking();
TreeMap<Long, MapFile.Reader> maps = new TreeMap<Long, MapFile.Reader>();
TreeMap<Long, HStoreFile> mapFiles = new TreeMap<Long, HStoreFile>();
Random rand = new Random();
//////////////////////////////////////////////////////////////////////////////
// Constructors, destructors, etc
//////////////////////////////////////////////////////////////////////////////
/**
* 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.
*
* 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.
*
* 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.
* 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.
*
* 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 {
this.dir = dir;
this.regionName = regionName;
this.colFamily = colFamily;
this.maxVersions = maxVersions;
this.fs = fs;
this.conf = conf;
this.mapdir = HStoreFile.getMapDir(dir, regionName, colFamily);
fs.mkdirs(mapdir);
this.loginfodir = HStoreFile.getInfoDir(dir, regionName, colFamily);
fs.mkdirs(loginfodir);
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);
if(fs.exists(curCompactStore)) {
processReadyCompaction();
fs.delete(curCompactStore);
}
// 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.
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).
//
// 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(); ) {
HStoreFile hsf = it.next();
long seqid = hsf.loadInfo(fs);
if(seqid > 0) {
if(seqid > maxSeqID) {
maxSeqID = seqid;
}
}
}
// Read the reconstructionLog to see whether we need to build a brand-new
// MapFile out of non-flushed log entries.
//
// 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(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) {
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");
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.
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");
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));
}
LOG.info("HStore online for " + this.regionName + "/" + this.colFamily);
}
/** Turn off all the MapFile readers */
public void close() throws IOException {
locking.obtainWriteLock();
LOG.info("closing HStore for " + this.regionName + "/" + this.colFamily);
try {
for(Iterator<MapFile.Reader> it = maps.values().iterator(); it.hasNext(); ) {
MapFile.Reader map = it.next();
map.close();
}
maps.clear();
mapFiles.clear();
LOG.info("HStore closed for " + this.regionName + "/" + this.colFamily);
} finally {
locking.releaseWriteLock();
}
}
//////////////////////////////////////////////////////////////////////////////
// Flush changes to disk
//////////////////////////////////////////////////////////////////////////////
/**
* Write out a brand-new set of items to the disk.
*
* We should only store key/vals that are appropriate for the data-columns
* stored in this HStore.
*
* Also, we are not expecting any reads of this MapFile just yet.
*
* Return the entire list of HStoreFiles currently used by the HStore.
*/
public Vector<HStoreFile> flushCache(TreeMap<HStoreKey, BytesWritable> inputCache,
long logCacheFlushId) throws IOException {
return flushCacheHelper(inputCache, logCacheFlushId, true);
}
Vector<HStoreFile> flushCacheHelper(TreeMap<HStoreKey, BytesWritable> inputCache,
long logCacheFlushId, boolean addToAvailableMaps) throws IOException {
synchronized(flushLock) {
LOG.debug("flushing HStore " + this.regionName + "/" + this.colFamily);
// A. Write the TreeMap out to the disk
HStoreFile flushedFile
= HStoreFile.obtainNewHStoreFile(conf, dir, regionName, colFamily, fs);
Path mapfile = flushedFile.getMapFilePath();
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();
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");
} finally {
out.close();
}
// 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");
flushedFile.writeInfo(fs, logCacheFlushId);
// C. Finally, make the new MapFile available.
if(addToAvailableMaps) {
locking.obtainWriteLock();
try {
maps.put(logCacheFlushId, new MapFile.Reader(fs, mapfile.toString(), conf));
mapFiles.put(logCacheFlushId, flushedFile);
LOG.debug("HStore available for " + this.regionName + "/" + this.colFamily);
} finally {
locking.releaseWriteLock();
}
}
return getAllMapFiles();
}
}
public Vector<HStoreFile> getAllMapFiles() {
Vector<HStoreFile> flushedFiles = new Vector<HStoreFile>();
for(Iterator<HStoreFile> it = mapFiles.values().iterator(); it.hasNext(); ) {
HStoreFile hsf = it.next();
flushedFiles.add(hsf);
}
return flushedFiles;
}
//////////////////////////////////////////////////////////////////////////////
// Compaction
//////////////////////////////////////////////////////////////////////////////
/**
* Compact the back-HStores. This method may take some time, so the calling
* thread must be able to block for long periods.
*
* During this time, the HStore can work as usual, getting values from MapFiles
* and writing new MapFiles from given memcaches.
*
* Existing MapFiles are not destroyed until the new compacted TreeMap is
* completely written-out to disk.
*
* The compactLock block prevents multiple simultaneous compactions.
* The structureLock prevents us from interfering with other write operations.
*
* We don't want to hold the structureLock for the whole time, as a compact()
* can be lengthy and we want to allow cache-flushes during this period.
*/
public void compact() throws IOException {
compactHelper(false);
}
void compactHelper(boolean deleteSequenceInfo) throws IOException {
synchronized(compactLock) {
LOG.debug("started compaction of " + this.regionName + "/" + this.colFamily);
Path curCompactStore = HStoreFile.getHStoreDir(compactdir, regionName, colFamily);
fs.mkdirs(curCompactStore);
try {
// Grab a list of files to compact.
Vector<HStoreFile> toCompactFiles = null;
locking.obtainWriteLock();
try {
toCompactFiles = new Vector<HStoreFile>(mapFiles.values());
} finally {
locking.releaseWriteLock();
}
// Compute the max-sequenceID seen in any of the to-be-compacted TreeMaps
long maxSeenSeqID = -1;
for(Iterator<HStoreFile> it = toCompactFiles.iterator(); it.hasNext(); ) {
HStoreFile hsf = it.next();
long seqid = hsf.loadInfo(fs);
if(seqid > 0) {
if(seqid > maxSeenSeqID) {
maxSeenSeqID = seqid;
}
}
}
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);
HStoreFile hsf = toCompactFiles.elementAt(0);
if(hsf.loadInfo(fs) == -1) {
return;
}
}
// Step through them, writing to the brand-new TreeMap
MapFile.Writer compactedOut = new MapFile.Writer(conf, fs,
compactedOutputFile.getMapFilePath().toString(), HStoreKey.class,
BytesWritable.class);
try {
// We create a new set of MapFile.Reader objects so we don't screw up
// the caching associated with the currently-loaded ones.
//
// Our iteration-based access pattern is practically designed to ruin
// the cache.
//
// We work by opening a single MapFile.Reader for each file, and
// iterating through them in parallel. We always increment the
// lowest-ranked one. Updates to a single row/column will appear
// ranked by timestamp. This allows us to throw out deleted values or
// obsolete versions.
MapFile.Reader[] readers = new MapFile.Reader[toCompactFiles.size()];
HStoreKey[] keys = new HStoreKey[toCompactFiles.size()];
BytesWritable[] vals = new BytesWritable[toCompactFiles.size()];
boolean[] done = new boolean[toCompactFiles.size()];
int pos = 0;
for(Iterator<HStoreFile> it = toCompactFiles.iterator(); it.hasNext(); ) {
HStoreFile hsf = it.next();
readers[pos] = new MapFile.Reader(fs, hsf.getMapFilePath().toString(), conf);
keys[pos] = new HStoreKey();
vals[pos] = new BytesWritable();
done[pos] = false;
pos++;
}
// 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");
int numDone = 0;
for(int i = 0; i < readers.length; i++) {
readers[i].reset();
done[i] = ! readers[i].next(keys[i], vals[i]);
if(done[i]) {
numDone++;
}
}
int timesSeen = 0;
Text lastRow = new Text();
Text lastColumn = new Text();
while(numDone < done.length) {
// Find the reader with the smallest key
int smallestKey = -1;
for(int i = 0; i < readers.length; i++) {
if(done[i]) {
continue;
}
if(smallestKey < 0) {
smallestKey = i;
} else {
if(keys[i].compareTo(keys[smallestKey]) < 0) {
smallestKey = i;
}
}
}
// Reflect the current key/val in the output
HStoreKey sk = keys[smallestKey];
if(lastRow.equals(sk.getRow())
&& lastColumn.equals(sk.getColumn())) {
timesSeen++;
} else {
timesSeen = 1;
}
if(timesSeen <= maxVersions) {
// Keep old versions until we have maxVersions worth.
// Then just skip them.
if(sk.getRow().getLength() != 0
&& sk.getColumn().getLength() != 0) {
// Only write out objects which have a non-zero length key and value
compactedOut.append(sk, vals[smallestKey]);
}
}
//TODO: I don't know what to do about deleted values. I currently
// include the fact that the item was deleted as a legitimate
// "version" of the data. Maybe it should just drop the deleted val?
// Update last-seen items
lastRow.set(sk.getRow());
lastColumn.set(sk.getColumn());
// Advance the smallest key. If that reader's all finished, then
// mark it as done.
if(! readers[smallestKey].next(keys[smallestKey], vals[smallestKey])) {
done[smallestKey] = true;
readers[smallestKey].close();
numDone++;
}
}
LOG.debug("all HStores processed");
} finally {
compactedOut.close();
}
LOG.debug("writing new compacted HStore");
// Now, write out an HSTORE_LOGINFOFILE for the brand-new TreeMap.
if((! deleteSequenceInfo) && maxSeenSeqID >= 0) {
compactedOutputFile.writeInfo(fs, maxSeenSeqID);
} else {
compactedOutputFile.writeInfo(fs, -1);
}
// Write out a list of data files that we're replacing
Path filesToReplace = new Path(curCompactStore, COMPACTION_TO_REPLACE);
DataOutputStream out = new DataOutputStream(fs.create(filesToReplace));
try {
out.writeInt(toCompactFiles.size());
for(Iterator<HStoreFile> it = toCompactFiles.iterator(); it.hasNext(); ) {
HStoreFile hsf = it.next();
hsf.write(out);
}
} finally {
out.close();
}
// Indicate that we're done.
Path doneFile = new Path(curCompactStore, COMPACTION_DONE);
out = new DataOutputStream(fs.create(doneFile));
try {
} finally {
out.close();
}
// Move the compaction into place.
processReadyCompaction();
LOG.debug("compaction complete for " + this.regionName + "/" + this.colFamily);
} finally {
fs.delete(compactdir);
}
}
}
/**
* It's assumed that the compactLock will be acquired prior to calling this
* method! Otherwise, it is not thread-safe!
*
* It works by processing a compaction that's been written to disk.
*
* It is usually invoked at the end of a compaction, but might also be invoked
* at HStore startup, if the prior execution died midway through.
*/
void processReadyCompaction() throws IOException {
// Move the compacted TreeMap into place.
// That means:
// 1) Acquiring the write-lock
// 2) Figuring out what MapFiles are going to be replaced
// 3) Unloading all the replaced MapFiles.
// 4) Deleting all the old MapFile files.
// 5) Moving the new MapFile into place
// 6) Loading the new TreeMap.
// 7) Releasing the write-lock
// 1. Acquiring the write-lock
locking.obtainWriteLock();
Path curCompactStore = HStoreFile.getHStoreDir(compactdir, regionName, colFamily);
try {
Path doneFile = new Path(curCompactStore, COMPACTION_DONE);
if(! fs.exists(doneFile)) {
// The last execution didn't finish the compaction, so there's nothing
// we can do. We'll just have to redo it. Abandon it and return.
return;
}
// OK, there's actually compaction work that needs to be put into place.
LOG.debug("compaction starting");
// 2. Load in the files to be deleted.
// (Figuring out what MapFiles are going to be replaced)
Vector<HStoreFile> toCompactFiles = new Vector<HStoreFile>();
Path filesToReplace = new Path(curCompactStore, COMPACTION_TO_REPLACE);
DataInputStream in = new DataInputStream(fs.open(filesToReplace));
try {
int numfiles = in.readInt();
for(int i = 0; i < numfiles; i++) {
HStoreFile hsf = new HStoreFile(conf);
hsf.readFields(in);
toCompactFiles.add(hsf);
}
} finally {
in.close();
}
LOG.debug("loaded files to be deleted");
// 3. Unload all the replaced MapFiles.
Iterator<HStoreFile> it2 = mapFiles.values().iterator();
for(Iterator<MapFile.Reader> it = maps.values().iterator(); it.hasNext(); ) {
MapFile.Reader curReader = it.next();
HStoreFile curMapFile = it2.next();
if(toCompactFiles.contains(curMapFile)) {
curReader.close();
it.remove();
}
}
for(Iterator<HStoreFile> it = mapFiles.values().iterator(); it.hasNext(); ) {
HStoreFile curMapFile = it.next();
if(toCompactFiles.contains(curMapFile)) {
it.remove();
}
}
LOG.debug("unloaded existing MapFiles");
// What if we crash at this point? No big deal; we will restart
// processReadyCompaction(), and nothing has been lost.
// 4. Delete all the old files, no longer needed
for(Iterator<HStoreFile> it = toCompactFiles.iterator(); it.hasNext(); ) {
HStoreFile hsf = it.next();
fs.delete(hsf.getMapFilePath());
fs.delete(hsf.getInfoFilePath());
}
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
// something we delete, though.
// 5. Moving the new MapFile into place
LOG.debug("moving new MapFile into place");
HStoreFile compactedFile
= new HStoreFile(conf, compactdir, regionName, colFamily, -1);
HStoreFile finalCompactedFile
= HStoreFile.obtainNewHStoreFile(conf, dir, regionName, colFamily, fs);
fs.rename(compactedFile.getMapFilePath(), finalCompactedFile.getMapFilePath());
// Fail here? No problem.
fs.rename(compactedFile.getInfoFilePath(), finalCompactedFile.getInfoFilePath());
// Fail here? No worries.
long orderVal = finalCompactedFile.loadInfo(fs);
// 6. Loading the new TreeMap.
LOG.debug("loading new TreeMap");
mapFiles.put(orderVal, finalCompactedFile);
maps.put(orderVal, new MapFile.Reader(fs,
finalCompactedFile.getMapFilePath().toString(), conf));
} finally {
// 7. Releasing the write-lock
locking.releaseWriteLock();
}
}
//////////////////////////////////////////////////////////////////////////////
// Accessors.
// (This is the only section that is directly useful!)
//////////////////////////////////////////////////////////////////////////////
/**
* 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 void getFull(HStoreKey key, TreeMap<Text, byte[]> results) throws IOException {
locking.obtainReadLock();
try {
MapFile.Reader[] maparray
= maps.values().toArray(new MapFile.Reader[maps.size()]);
for(int i = maparray.length-1; i >= 0; i--) {
MapFile.Reader map = maparray[i];
synchronized(map) {
BytesWritable readval = new BytesWritable();
map.reset();
HStoreKey readkey = (HStoreKey)map.getClosest(key, readval);
do {
Text readcol = readkey.getColumn();
if(results.get(readcol) == null
&& key.matchesWithoutColumn(readkey)) {
results.put(new Text(readcol), readval.get());
readval = new BytesWritable();
} else if(key.getRow().compareTo(readkey.getRow()) > 0) {
break;
}
} while(map.next(readkey, readval));
}
}
} finally {
locking.releaseReadLock();
}
}
/**
* Get the value for the indicated HStoreKey. Grab the target value and the
* previous 'numVersions-1' values, as well.
*
* 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.");
}
Vector<byte[]> results = new Vector<byte[]>();
locking.obtainReadLock();
try {
MapFile.Reader[] maparray
= maps.values().toArray(new MapFile.Reader[maps.size()]);
for(int i = maparray.length-1; i >= 0; i--) {
MapFile.Reader map = maparray[i];
synchronized(map) {
BytesWritable readval = new BytesWritable();
map.reset();
HStoreKey readkey = (HStoreKey)map.getClosest(key, readval);
if(readkey.matchesRowCol(key)) {
results.add(readval.get());
readval = new BytesWritable();
while(map.next(readkey, readval) && readkey.matchesRowCol(key)) {
if(numVersions > 0 && (results.size() >= numVersions)) {
break;
} else {
results.add(readval.get());
readval = new BytesWritable();
}
}
}
}
if(results.size() >= numVersions) {
break;
}
}
if(results.size() == 0) {
return null;
} else {
return (byte[][]) results.toArray(new byte[results.size()][]);
}
} finally {
locking.releaseReadLock();
}
}
/**
* Gets the size of the largest MapFile and its mid key.
*
* @param midKey - the middle key for the largest MapFile
* @return - size of the largest MapFile
* @throws IOException
*/
public long getLargestFileSize(Text midKey) throws IOException {
long maxSize = 0L;
long mapIndex = 0L;
// Iterate through all the MapFiles
for(Iterator<Map.Entry<Long, HStoreFile>> it = mapFiles.entrySet().iterator();
it.hasNext(); ) {
Map.Entry<Long, HStoreFile> e = it.next();
HStoreFile curHSF = e.getValue();
long size = fs.getLength(new Path(curHSF.getMapFilePath(), MapFile.DATA_FILE_NAME));
if(size > maxSize) { // This is the largest one so far
maxSize = size;
mapIndex = e.getKey();
}
}
MapFile.Reader r = maps.get(mapIndex);
synchronized(r) {
midKey.set(((HStoreKey)r.midKey()).getRow());
}
return maxSize;
}
//////////////////////////////////////////////////////////////////////////////
// File administration
//////////////////////////////////////////////////////////////////////////////
/** Generate a random unique filename suffix */
String obtainFileLabel(Path prefix) throws IOException {
String testsuffix = String.valueOf(Math.abs(rand.nextInt()));
Path testpath = new Path(prefix.toString() + testsuffix);
while(fs.exists(testpath)) {
testsuffix = String.valueOf(Math.abs(rand.nextInt()));
testpath = new Path(prefix.toString() + testsuffix);
}
return testsuffix;
}
/**
* 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[],
Text firstRow) throws IOException {
return new HStoreScanner(timestamp, targetCols, firstRow);
}
//////////////////////////////////////////////////////////////////////////////
// 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;
public HStoreScanner(long timestamp, Text targetCols[], Text firstRow) throws IOException {
super(timestamp, targetCols);
locking.obtainReadLock();
try {
this.readers = new MapFile.Reader[mapFiles.size()];
int i = 0;
for(Iterator<HStoreFile> it = mapFiles.values().iterator(); it.hasNext(); ) {
HStoreFile curHSF = it.next();
readers[i++] = new MapFile.Reader(fs, curHSF.getMapFilePath().toString(), conf);
}
this.keys = new HStoreKey[readers.length];
this.vals = new BytesWritable[readers.length];
// Advance the readers to the first pos.
for(i = 0; i < readers.length; i++) {
keys[i] = new HStoreKey();
vals[i] = new BytesWritable();
if(firstRow.getLength() != 0) {
if(findFirstRow(i, firstRow)) {
continue;
}
}
while(getNext(i)) {
if(columnMatch(i)) {
break;
}
}
}
} catch (Exception ex) {
close();
}
}
/**
* The user didn't want to start scanning at the first row. This method
* seeks to the requested row.
*
* @param i - which iterator to advance
* @param firstRow - seek to this row
* @return - true if this is the first row or if the row was not found
*/
boolean findFirstRow(int i, Text firstRow) throws IOException {
HStoreKey firstKey
= (HStoreKey)readers[i].getClosest(new HStoreKey(firstRow), vals[i]);
if(firstKey == null) {
// Didn't find it. Close the scanner and return TRUE
closeSubScanner(i);
return true;
}
keys[i].setRow(firstKey.getRow());
keys[i].setColumn(firstKey.getColumn());
keys[i].setVersion(firstKey.getTimestamp());
return columnMatch(i);
}
/**
* Get the next value from the specified reader.
*
* @param i - which reader to fetch next value from
* @return - true if there is more data available
*/
boolean getNext(int i) throws IOException {
if(! readers[i].next(keys[i], vals[i])) {
closeSubScanner(i);
return false;
}
return true;
}
/** Close down the indicated reader. */
void closeSubScanner(int i) throws IOException {
try {
if(readers[i] != null) {
readers[i].close();
}
} finally {
readers[i] = null;
keys[i] = null;
vals[i] = null;
}
}
/** Shut it down! */
public void close() throws IOException {
if(! scannerClosed) {
try {
for(int i = 0; i < readers.length; i++) {
if(readers[i] != null) {
readers[i].close();
}
}
} finally {
locking.releaseReadLock();
scannerClosed = true;
}
}
}
}
}

View File

@ -0,0 +1,378 @@
/**
* 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 org.apache.hadoop.io.*;
import org.apache.hadoop.fs.*;
import org.apache.hadoop.conf.*;
import java.io.*;
import java.util.*;
/*******************************************************************************
* Each HStore maintains a bunch of different data files.
*
* The filename is a mix of the parent dir, the region name, the column name,
* and the file identifier.
*
* This class handles all that path-building stuff for you.
******************************************************************************/
public class HStoreFile implements HConstants, WritableComparable {
public static final byte INFO_SEQ_NUM = 0;
public static final String HSTORE_DATFILE_PREFIX = "mapfile.dat.";
public static final String HSTORE_INFOFILE_PREFIX = "mapfile.info.";
public static final String HSTORE_DATFILE_DIR = "mapfiles";
public static final String HSTORE_INFO_DIR = "info";
static Random rand = new Random();
Path dir;
Text regionName;
Text colFamily;
long fileId;
Configuration conf;
/**
* An HStoreFile tracks 4 things: its parent dir, the region identifier, the
* column family, and the file identifier. If you know those four things, you
* know how to obtain the right HStoreFile.
*
* When merging or splitting HRegions, we might want to modify one of the
* params for an HStoreFile (effectively moving it elsewhere).
*/
public HStoreFile(Configuration conf) {
this.conf = conf;
this.dir = new Path("");
this.regionName = new Text();
this.colFamily = new Text();
this.fileId = 0;
}
public HStoreFile(Configuration conf, Path dir, Text regionName,
Text colFamily, long fileId) {
this.conf = conf;
this.dir = dir;
this.regionName = new Text(regionName);
this.colFamily = new Text(colFamily);
this.fileId = fileId;
}
// Get the individual components
public Path getDir() {
return dir;
}
public Text getRegionName() {
return regionName;
}
public Text getColFamily() {
return colFamily;
}
public long fileId() {
return fileId;
}
// Build full filenames from those components
public Path getMapFilePath() {
return new Path(HStoreFile.getMapDir(dir, regionName, colFamily),
HSTORE_DATFILE_PREFIX + fileId);
}
public Path getInfoFilePath() {
return new Path(HStoreFile.getInfoDir(dir, regionName, colFamily),
HSTORE_INFOFILE_PREFIX + fileId);
}
// Static methods to build partial paths to internal directories. Useful for
// HStore construction and log-rebuilding.
public static Path getMapDir(Path dir, Text regionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
new Path(colFamily.toString(), HSTORE_DATFILE_DIR)));
}
public static Path getInfoDir(Path dir, Text regionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
new Path(colFamily.toString(), HSTORE_INFO_DIR)));
}
public static Path getHStoreDir(Path dir, Text regionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
colFamily.toString()));
}
public static Path getHRegionDir(Path dir, Text regionName) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName));
}
/**
* Obtain a brand-new randomly-named HStoreFile. Checks the existing
* filesystem if the file already exists.
*/
static HStoreFile obtainNewHStoreFile(Configuration conf, Path dir,
Text regionName, Text colFamily, FileSystem fs) throws IOException {
Path mapdir = HStoreFile.getMapDir(dir, regionName, colFamily);
long fileId = Math.abs(rand.nextLong());
Path testpath1 = new Path(mapdir, HSTORE_DATFILE_PREFIX + fileId);
Path testpath2 = new Path(mapdir, HSTORE_INFOFILE_PREFIX + fileId);
while(fs.exists(testpath1) || fs.exists(testpath2)) {
fileId = Math.abs(rand.nextLong());
testpath1 = new Path(mapdir, HSTORE_DATFILE_PREFIX + fileId);
testpath2 = new Path(mapdir, HSTORE_INFOFILE_PREFIX + fileId);
}
return new HStoreFile(conf, dir, regionName, colFamily, fileId);
}
/**
* Create a series of HStoreFiles loaded from the given directory.
*
* There must be a matching 'mapdir' and 'loginfo' pair of files.
* If only one exists, we'll delete it.
*/
static Vector<HStoreFile> loadHStoreFiles(Configuration conf, Path dir,
Text regionName, Text colFamily, FileSystem fs) throws IOException {
Vector<HStoreFile> results = new Vector<HStoreFile>();
Path mapdir = HStoreFile.getMapDir(dir, regionName, colFamily);
Path datfiles[] = fs.listPaths(mapdir);
for(int i = 0; i < datfiles.length; i++) {
String name = datfiles[i].getName();
if(name.startsWith(HSTORE_DATFILE_PREFIX)) {
Long fileId = Long.parseLong(name.substring(HSTORE_DATFILE_PREFIX.length()));
HStoreFile curfile = new HStoreFile(conf, dir, regionName, colFamily, fileId);
Path mapfile = curfile.getMapFilePath();
Path infofile = curfile.getInfoFilePath();
if(fs.exists(infofile)) {
results.add(curfile);
} else {
fs.delete(mapfile);
}
}
}
Path infodir = HStoreFile.getInfoDir(dir, regionName, colFamily);
Path infofiles[] = fs.listPaths(infodir);
for(int i = 0; i < infofiles.length; i++) {
String name = infofiles[i].getName();
if(name.startsWith(HSTORE_INFOFILE_PREFIX)) {
long fileId = Long.parseLong(name.substring(HSTORE_INFOFILE_PREFIX.length()));
HStoreFile curfile = new HStoreFile(conf, dir, regionName, colFamily, fileId);
Path mapfile = curfile.getMapFilePath();
if(! fs.exists(mapfile)) {
fs.delete(curfile.getInfoFilePath());
}
}
}
return results;
}
//////////////////////////////////////////////////////////////////////////////
// File handling
//////////////////////////////////////////////////////////////////////////////
/**
* Break this HStoreFile file into two new parts, which live in different
* brand-new HRegions.
*/
public void splitStoreFile(Text midKey, HStoreFile dstA, HStoreFile dstB,
FileSystem fs, Configuration conf) throws IOException {
// Copy the appropriate tuples to one MapFile or the other.
MapFile.Reader in = new MapFile.Reader(fs, getMapFilePath().toString(), conf);
try {
MapFile.Writer outA = new MapFile.Writer(conf, fs,
dstA.getMapFilePath().toString(), HStoreKey.class, BytesWritable.class);
try {
MapFile.Writer outB = new MapFile.Writer(conf, fs,
dstB.getMapFilePath().toString(), HStoreKey.class, BytesWritable.class);
try {
HStoreKey readkey = new HStoreKey();
BytesWritable readval = new BytesWritable();
while(in.next(readkey, readval)) {
Text key = readkey.getRow();
if(key.compareTo(midKey) < 0) {
outA.append(readkey, readval);
} else {
outB.append(readkey, readval);
}
}
} finally {
outB.close();
}
} finally {
outA.close();
}
} finally {
in.close();
}
// Build an InfoFile for each output
long seqid = loadInfo(fs);
dstA.writeInfo(fs, seqid);
dstB.writeInfo(fs, seqid);
}
/**
* Write to this HStoreFile with all the contents of the given source HStoreFiles.
* We are merging multiple regions into a single new one.
*/
public void mergeStoreFiles(Vector<HStoreFile> srcFiles, FileSystem fs,
Configuration conf) throws IOException {
// Copy all the source MapFile tuples into this HSF's MapFile
MapFile.Writer out = new MapFile.Writer(conf, fs, getMapFilePath().toString(),
HStoreKey.class, BytesWritable.class);
try {
for(Iterator<HStoreFile> it = srcFiles.iterator(); it.hasNext(); ) {
HStoreFile src = it.next();
MapFile.Reader in = new MapFile.Reader(fs, src.getMapFilePath().toString(), conf);
try {
HStoreKey readkey = new HStoreKey();
BytesWritable readval = new BytesWritable();
while(in.next(readkey, readval)) {
out.append(readkey, readval);
}
} finally {
in.close();
}
}
} finally {
out.close();
}
// Build a unified InfoFile from the source InfoFiles.
long unifiedSeqId = -1;
for(Iterator<HStoreFile> it = srcFiles.iterator(); it.hasNext(); ) {
HStoreFile hsf = it.next();
long curSeqId = hsf.loadInfo(fs);
if(curSeqId > unifiedSeqId) {
unifiedSeqId = curSeqId;
}
}
writeInfo(fs, unifiedSeqId);
}
/** Read in an info file, give it a unique ID. */
public long loadInfo(FileSystem fs) throws IOException {
Path p = getInfoFilePath();
DataInputStream in = new DataInputStream(fs.open(p));
try {
byte flag = in.readByte();
if(flag == INFO_SEQ_NUM) {
return in.readLong();
} else {
throw new IOException("Cannot process log file: " + p);
}
} finally {
in.close();
}
}
/** Write the file-identifier to disk */
public void writeInfo(FileSystem fs, long infonum) throws IOException {
Path p = getInfoFilePath();
DataOutputStream out = new DataOutputStream(fs.create(p));
try {
out.writeByte(INFO_SEQ_NUM);
out.writeLong(infonum);
} finally {
out.close();
}
}
//////////////////////////////////////////////////////////////////////////////
// Writable
//////////////////////////////////////////////////////////////////////////////
public void write(DataOutput out) throws IOException {
out.writeUTF(dir.toString());
regionName.write(out);
colFamily.write(out);
out.writeLong(fileId);
}
public void readFields(DataInput in) throws IOException {
this.dir = new Path(in.readUTF());
this.regionName.readFields(in);
this.colFamily.readFields(in);
this.fileId = in.readLong();
}
//////////////////////////////////////////////////////////////////////////////
// Comparable
//////////////////////////////////////////////////////////////////////////////
public int compareTo(Object o) {
HStoreFile other = (HStoreFile) o;
int result = this.dir.compareTo(other.dir);
if(result == 0) {
this.regionName.compareTo(other.regionName);
}
if(result == 0) {
result = this.colFamily.compareTo(other.colFamily);
}
if(result == 0) {
if(this.fileId < other.fileId) {
result = -1;
} else if(this.fileId > other.fileId) {
result = 1;
}
}
return result;
}
public boolean equals(Object o) {
return this.compareTo(o) == 0;
}
}

View File

@ -0,0 +1,158 @@
/**
* 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 org.apache.hadoop.io.*;
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(":");
if(colpos < 0) {
throw new IllegalArgumentException("Illegal column name has no family indicator: " + column);
}
return new Text(column.substring(0, colpos));
}
Text row;
Text column;
long timestamp;
public HStoreKey() {
this.row = new Text();
this.column = new Text();
this.timestamp = Long.MAX_VALUE;
}
public HStoreKey(Text row) {
this.row = new Text(row);
this.column = new Text();
this.timestamp = Long.MAX_VALUE;
}
public HStoreKey(Text row, long timestamp) {
this.row = new Text(row);
this.column = new Text();
this.timestamp = timestamp;
}
public HStoreKey(Text row, Text column) {
this.row = new Text(row);
this.column = new Text(column);
this.timestamp = Long.MAX_VALUE;
}
public HStoreKey(Text row, Text column, long timestamp) {
this.row = new Text(row);
this.column = new Text(column);
this.timestamp = timestamp;
}
public void setRow(Text newrow) {
this.row.set(newrow);
}
public void setColumn(Text newcol) {
this.column.set(newcol);
}
public void setVersion(long timestamp) {
this.timestamp = timestamp;
}
public Text getRow() {
return row;
}
public Text getColumn() {
return column;
}
public long getTimestamp() {
return timestamp;
}
public boolean matchesRowCol(HStoreKey other) {
if(this.row.compareTo(other.row) == 0 &&
this.column.compareTo(other.column) == 0) {
return true;
} else {
return false;
}
}
public boolean matchesWithoutColumn(HStoreKey other) {
if((this.row.compareTo(other.row) == 0) &&
(this.timestamp >= other.getTimestamp())) {
return true;
} else {
return false;
}
}
public String toString() {
return row.toString() + "/" + column.toString() + "/" + timestamp;
}
//////////////////////////////////////////////////////////////////////////////
// Comparable
//////////////////////////////////////////////////////////////////////////////
public int compareTo(Object o) {
HStoreKey other = (HStoreKey) o;
int result = this.row.compareTo(other.row);
if(result == 0) {
result = this.column.compareTo(other.column);
if(result == 0) {
if(this.timestamp < other.timestamp) {
result = 1;
} else if(this.timestamp > other.timestamp) {
result = -1;
}
}
}
return result;
}
//////////////////////////////////////////////////////////////////////////////
// Writable
//////////////////////////////////////////////////////////////////////////////
public void write(DataOutput out) throws IOException {
row.write(out);
column.write(out);
out.writeLong(timestamp);
}
public void readFields(DataInput in) throws IOException {
row.readFields(in);
column.readFields(in);
timestamp = in.readLong();
}
}

View File

@ -0,0 +1,123 @@
/**
* 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 org.apache.hadoop.io.*;
import java.io.*;
import java.util.*;
/*******************************************************************************
* HTableDescriptor contains various facts about an HTable, like its columns,
* column families, etc.
******************************************************************************/
public class HTableDescriptor implements WritableComparable {
Text name;
int maxVersions;
TreeSet<Text> families = new TreeSet<Text>();
public HTableDescriptor() {
this.name = new Text();
this.families.clear();
}
public HTableDescriptor(String name, int maxVersions) {
this.name = new Text(name);
this.maxVersions = maxVersions;
}
public Text getName() {
return name;
}
public int getMaxVersions() {
return maxVersions;
}
/** Add a column */
public void addFamily(Text family) {
families.add(family);
}
/** Do we contain a given column? */
public boolean hasFamily(Text family) {
if(families.contains(family)) {
return true;
} else {
return false;
}
}
/** All the column families in this table. */
public TreeSet<Text> families() {
return families;
}
//////////////////////////////////////////////////////////////////////////////
// Writable
//////////////////////////////////////////////////////////////////////////////
public void write(DataOutput out) throws IOException {
name.write(out);
out.writeInt(maxVersions);
out.writeInt(families.size());
for(Iterator<Text> it = families.iterator(); it.hasNext(); ) {
it.next().write(out);
}
}
public void readFields(DataInput in) throws IOException {
this.name.readFields(in);
this.maxVersions = in.readInt();
int numCols = in.readInt();
families.clear();
for(int i = 0; i < numCols; i++) {
Text t = new Text();
t.readFields(in);
families.add(t);
}
}
//////////////////////////////////////////////////////////////////////////////
// Comparable
//////////////////////////////////////////////////////////////////////////////
public int compareTo(Object o) {
HTableDescriptor htd = (HTableDescriptor) o;
int result = name.compareTo(htd.name);
if(result == 0) {
result = maxVersions - htd.maxVersions;
}
if(result == 0) {
result = families.size() - htd.families.size();
}
if(result == 0) {
Iterator<Text> it2 = htd.families.iterator();
for(Iterator<Text> it = families.iterator(); it.hasNext(); ) {
Text family1 = it.next();
Text family2 = it2.next();
result = family1.compareTo(family2);
if(result != 0) {
return result;
}
}
}
return result;
}
}

View File

@ -0,0 +1,60 @@
/**
* 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 org.apache.hadoop.io.*;
import java.io.*;
/*******************************************************************************
* LabelledData is just a data pair.
* It includes a Text label and some associated data.
******************************************************************************/
public class LabelledData implements Writable {
Text label;
BytesWritable data;
public LabelledData() {
this.label = new Text();
this.data = new BytesWritable();
}
public LabelledData(Text label, byte[] data) {
this.label.set(label);
this.data = new BytesWritable(data);
}
public Text getLabel() {
return label;
}
public BytesWritable getDat() {
return data;
}
//////////////////////////////////////////////////////////////////////////////
// Writable
//////////////////////////////////////////////////////////////////////////////
public void write(DataOutput out) throws IOException {
label.write(out);
data.write(out);
}
public void readFields(DataInput in) throws IOException {
label.readFields(in);
data.readFields(in);
}
}

View File

@ -0,0 +1,41 @@
/**
* 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;
/*******************************************************************************
* LeaseListener is a small class meant to be overridden by users of the Leases
* class.
*
* It receives events from the Leases class about the status of its accompanying
* lease. Users of the Leases class can use a LeaseListener subclass to, for
* example, clean up resources after a lease has expired.
******************************************************************************/
public abstract class LeaseListener {
public LeaseListener() {
}
public void leaseRenewed() {
}
/** When the user cancels a lease, this method is called. */
public void leaseCancelled() {
}
/** When a lease expires, this method is called. */
public void leaseExpired() {
}
}

View File

@ -0,0 +1,220 @@
/**
* 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 org.apache.hadoop.io.*;
import java.io.*;
import java.util.*;
/*******************************************************************************
* Leases
*
* There are several server classes in HBase that need to track external clients
* that occasionally send heartbeats.
*
* These external clients hold resources in the server class. Those resources
* need to be released if the external client fails to send a heartbeat after
* some interval of time passes.
*
* The Leases class is a general reusable class for this kind of pattern.
*
* An instance of the Leases class will create a thread to do its dirty work.
* You should close() the instance if you want to clean up the thread properly.
******************************************************************************/
public class Leases {
long leasePeriod;
long leaseCheckFrequency;
LeaseMonitor leaseMonitor;
Thread leaseMonitorThread;
TreeMap<Text, Lease> leases = new TreeMap<Text, Lease>();
TreeSet<Lease> sortedLeases = new TreeSet<Lease>();
boolean running = true;
/** Indicate the length of the lease, in milliseconds */
public Leases(long leasePeriod, long leaseCheckFrequency) {
this.leasePeriod = leasePeriod;
this.leaseMonitor = new LeaseMonitor();
this.leaseMonitorThread = new Thread(leaseMonitor);
leaseMonitorThread.start();
}
/**
* Shut down this Leases outfit. All pending leases will be destroyed,
* without any cancellation calls.
*/
public void close() {
this.running = false;
try {
this.leaseMonitorThread.join();
} catch (InterruptedException iex) {
}
synchronized(leases) {
synchronized(sortedLeases) {
leases.clear();
sortedLeases.clear();
}
}
}
/** A client obtains a lease... */
public void createLease(Text holderId, Text resourceId, LeaseListener listener) throws IOException {
synchronized(leases) {
synchronized(sortedLeases) {
Lease lease = new Lease(holderId, resourceId, listener);
Text leaseId = lease.getLeaseId();
if(leases.get(leaseId) != null) {
throw new IOException("Impossible state for createLease(): Lease for holderId " + holderId + " and resourceId " + resourceId + " is still held.");
}
leases.put(leaseId, lease);
sortedLeases.add(lease);
}
}
}
/** A client renews a lease... */
public void renewLease(Text holderId, Text resourceId) throws IOException {
synchronized(leases) {
synchronized(sortedLeases) {
Text leaseId = createLeaseId(holderId, resourceId);
Lease lease = leases.get(leaseId);
if(lease == null) {
// It's possible that someone tries to renew the lease, but
// it just expired a moment ago. So fail.
throw new IOException("Cannot renew lease is not held (holderId=" + holderId + ", resourceId=" + resourceId + ")");
}
sortedLeases.remove(lease);
lease.renew();
sortedLeases.add(lease);
}
}
}
/** A client explicitly cancels a lease. The lease-cleanup method is not called. */
public void cancelLease(Text holderId, Text resourceId) throws IOException {
synchronized(leases) {
synchronized(sortedLeases) {
Text leaseId = createLeaseId(holderId, resourceId);
Lease lease = leases.get(leaseId);
if(lease == null) {
// It's possible that someone tries to renew the lease, but
// it just expired a moment ago. So fail.
throw new IOException("Cannot cancel lease that is not held (holderId=" + holderId + ", resourceId=" + resourceId + ")");
}
sortedLeases.remove(lease);
leases.remove(leaseId);
lease.cancelled();
}
}
}
/** LeaseMonitor is a thread that expires Leases that go on too long. */
class LeaseMonitor implements Runnable {
public void run() {
while(running) {
synchronized(leases) {
synchronized(sortedLeases) {
Lease top;
while((sortedLeases.size() > 0)
&& ((top = sortedLeases.first()) != null)) {
if(top.shouldExpire()) {
leases.remove(top.getLeaseId());
sortedLeases.remove(top);
top.expired();
} else {
break;
}
}
}
}
try {
Thread.sleep(leaseCheckFrequency);
} catch (InterruptedException ie) {
}
}
}
}
/** Create a lease id out of the holder and resource ids. */
Text createLeaseId(Text holderId, Text resourceId) {
return new Text("_" + holderId + "/" + resourceId + "_");
}
/** This class tracks a single Lease. */
class Lease implements Comparable {
Text holderId;
Text resourceId;
LeaseListener listener;
long lastUpdate;
public Lease(Text holderId, Text resourceId, LeaseListener listener) {
this.holderId = holderId;
this.resourceId = resourceId;
this.listener = listener;
renew();
}
public Text getLeaseId() {
return createLeaseId(holderId, resourceId);
}
public boolean shouldExpire() {
return (System.currentTimeMillis() - lastUpdate > leasePeriod);
}
public void renew() {
this.lastUpdate = System.currentTimeMillis();
listener.leaseRenewed();
}
public void cancelled() {
listener.leaseCancelled();
}
public void expired() {
listener.leaseExpired();
}
//////////////////////////////////////////////////////////////////////////////
// Comparable
//////////////////////////////////////////////////////////////////////////////
public int compareTo(Object o) {
Lease other = (Lease) o;
if(this.lastUpdate < other.lastUpdate) {
return -1;
} else if(this.lastUpdate > other.lastUpdate) {
return 1;
} else {
return this.getLeaseId().compareTo(other.getLeaseId());
}
}
}
}

View File

@ -0,0 +1,58 @@
/**
* 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 org.apache.log4j.Level;
/**
* Retrieve environment variables that control debugging and logging environment
*/
public class Environment {
public static boolean debugging = false;
public static Level logLevel = Level.INFO;
private Environment() {}; // Not instantiable
public static void getenv() {
String value = null;
value = System.getenv("DEBUGGING");
if(value != null && value.equalsIgnoreCase("TRUE")) {
debugging = true;
}
value = System.getenv("LOGGING_LEVEL");
if(value != null && value.length() != 0) {
if(value.equalsIgnoreCase("ALL")) {
logLevel = Level.ALL;
} else if(value.equalsIgnoreCase("DEBUG")) {
logLevel = Level.DEBUG;
} else if(value.equalsIgnoreCase("ERROR")) {
logLevel = Level.ERROR;
} else if(value.equalsIgnoreCase("FATAL")) {
logLevel = Level.FATAL;
} else if(value.equalsIgnoreCase("INFO")) {
logLevel = Level.INFO;
} else if(value.equalsIgnoreCase("OFF")) {
logLevel = Level.OFF;
} else if(value.equalsIgnoreCase("TRACE")) {
logLevel = Level.TRACE;
} else if(value.equalsIgnoreCase("WARN")) {
logLevel = Level.WARN;
}
}
}
}

View File

@ -0,0 +1,676 @@
/**
* 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 junit.framework.TestCase;
import junit.framework.Test;
import junit.framework.TestSuite;
import java.io.File;
import java.io.IOException;
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.Text;
import org.apache.log4j.Logger;
import org.apache.log4j.Level;
import org.apache.log4j.PatternLayout;
/**
* Basic stand-alone testing of HRegion.
*
* A lot of the meta information for an HRegion now lives inside other
* HRegions or in the HBaseMaster, so only basic testing is possible.
*/
public class TestHRegion extends TestCase {
/** Constructor */
public TestHRegion(String name) {
super(name);
}
/** Test suite so that all tests get run */
public static Test suite() {
TestSuite suite = new TestSuite();
suite.addTest(new TestHRegion("testSetup"));
suite.addTest(new TestHRegion("testBasic"));
suite.addTest(new TestHRegion("testScan"));
suite.addTest(new TestHRegion("testBatchWrite"));
suite.addTest(new TestHRegion("testSplitAndMerge"));
suite.addTest(new TestHRegion("testRead"));
suite.addTest(new TestHRegion("testCleanup"));
return suite;
}
private static final int FIRST_ROW = 0;
private static final int N_ROWS = 1000000;
private static final int NUM_VALS = 1000;
private static final Text CONTENTS_BASIC = new Text("contents:basic");
private static final String CONTENTSTR = "contentstr";
private static final String ANCHORNUM = "anchor:anchornum-";
private static final String ANCHORSTR = "anchorstr";
private static final Text CONTENTS_BODY = new Text("contents:body");
private static final Text CONTENTS_FIRSTCOL = new Text("contents:firstcol");
private static final Text ANCHOR_SECONDCOL = new Text("anchor:secondcol");
private static boolean initialized = false;
private static boolean failures = false;
private static Configuration conf = null;
private static MiniDFSCluster cluster = null;
private static FileSystem fs = null;
private static Path parentdir = null;
private static Path newlogdir = null;
private static Path oldlogfile = null;
private static HLog log = null;
private static HTableDescriptor desc = null;
private static HRegion region = null;
private static int numInserted = 0;
// Set up environment, start mini cluster, etc.
public void testSetup() throws IOException {
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 Configuration();
Environment.getenv();
if(Environment.debugging) {
Logger rootLogger = Logger.getRootLogger();
rootLogger.setLevel(Level.WARN);
PatternLayout consoleLayout
= (PatternLayout)rootLogger.getAppender("console").getLayout();
consoleLayout.setConversionPattern("%d %-5p [%t] %l: %m%n");
Logger.getLogger("org.apache.hadoop.hbase").setLevel(Environment.logLevel);
}
cluster = new MiniDFSCluster(65312, conf, 2, false);
fs = cluster.getFileSystem();
parentdir = new Path("/hbase");
fs.mkdirs(parentdir);
newlogdir = new Path(parentdir, "log");
oldlogfile = new Path(parentdir, "oldlogfile");
log = new HLog(fs, newlogdir, conf);
desc = new HTableDescriptor("test", 3);
desc.addFamily(new Text("contents"));
desc.addFamily(new Text("anchor"));
region = new HRegion(parentdir, log, fs, conf,
new HRegionInfo(1, desc, null, null), null, oldlogfile);
} catch(IOException 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 {
// Write out a bunch of values
for (int k = 0; 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.commit(writeid);
}
region.flushcache(false);
// Read them back in
Text collabel = null;
for (int k = 0; k < NUM_VALS; k++) {
Text rowlabel = new Text("row_" + k);
byte bodydata[] = region.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 = region.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);
/*
// Check to make sure that null values are actually null
for (int j = 0; j < Math.min(15, NUM_VALS); j++) {
if (k != j) {
collabel = new Text(ANCHORNUM + j);
byte results[] = region.get(rowlabel, collabel);
if (results != null) {
throw new IOException("Found incorrect value at [" + rowlabel + ", " + collabel + "] == " + new String(results).toString().trim());
}
}
}
*/
}
} catch(IOException e) {
failures = true;
throw e;
}
}
// Test scanners. Writes contents:firstcol and anchor:secondcol
public void testScan() throws IOException {
if(!initialized) {
throw new IllegalStateException();
}
Text cols[] = new Text[] {
CONTENTS_FIRSTCOL,
ANCHOR_SECONDCOL
};
// Test the Scanner!!!
String[] vals1 = new String[1000];
for(int k = 0; k < vals1.length; k++) {
vals1[k] = Integer.toString(k);
}
// 1. Insert a bunch of values
for(int k = 0; k < vals1.length / 2; k++) {
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.commit(lockid);
numInserted += 2;
}
// 2. Scan
HScannerInterface s = region.getScanner(cols, new Text());
int numFetched = 0;
try {
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);
int curval = Integer.parseInt(new String(val).trim());
for(int j = 0; j < cols.length; j++) {
if(col.compareTo(cols[j]) == 0) {
assertEquals("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp()
+ ", Value for " + col + " should be: " + k
+ ", but was fetched as: " + curval, k, curval);
numFetched++;
}
}
}
curVals.clear();
k++;
}
} finally {
s.close();
}
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
// 3. Flush to disk
region.flushcache(false);
// 4. Scan
s = region.getScanner(cols, new Text());
numFetched = 0;
try {
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);
int curval = Integer.parseInt(new String(val).trim());
for(int j = 0; j < cols.length; j++) {
if(col.compareTo(cols[j]) == 0) {
assertEquals("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp()
+ ", Value for " + col + " should be: " + k
+ ", but was fetched as: " + curval, k, curval);
numFetched++;
}
}
}
curVals.clear();
k++;
}
} finally {
s.close();
}
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
// 5. Insert more values
for(int k = vals1.length/2; k < vals1.length; k++) {
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.commit(lockid);
numInserted += 2;
}
// 6. Scan
s = region.getScanner(cols, new Text());
numFetched = 0;
try {
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);
int curval = Integer.parseInt(new String(val).trim());
for(int j = 0; j < cols.length; j++) {
if(col.compareTo(cols[j]) == 0) {
assertEquals("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp()
+ ", Value for " + col + " should be: " + k
+ ", but was fetched as: " + curval, k, curval);
numFetched++;
}
}
}
curVals.clear();
k++;
}
} finally {
s.close();
}
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
// 7. Flush to disk
region.flushcache(false);
// 8. Scan
s = region.getScanner(cols, new Text());
numFetched = 0;
try {
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);
int curval = Integer.parseInt(new String(val).trim());
for (int j = 0; j < cols.length; j++) {
if (col.compareTo(cols[j]) == 0) {
assertEquals("Value for " + col + " should be: " + k
+ ", but was fetched as: " + curval, curval, k);
numFetched++;
}
}
}
curVals.clear();
k++;
}
} finally {
s.close();
}
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
// 9. Scan with a starting point
s = region.getScanner(cols, new Text("row_vals1_500"));
numFetched = 0;
try {
HStoreKey curKey = new HStoreKey();
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
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());
for (int j = 0; j < cols.length; j++) {
if (col.compareTo(cols[j]) == 0) {
assertEquals("Value for " + col + " should be: " + k
+ ", but was fetched as: " + curval, curval, k);
numFetched++;
}
}
}
curVals.clear();
k++;
}
} finally {
s.close();
}
assertEquals("Should have fetched " + (numInserted / 2) + " values, but fetched " + numFetched, (numInserted / 2), numFetched);
}
// Do a large number of writes. Disabled if not debugging because it takes a
// long time to run.
// Creates contents:body
public void testBatchWrite() throws IOException {
if(!initialized || failures) {
throw new IllegalStateException();
}
if(! Environment.debugging) {
return;
}
try {
long totalFlush = 0;
long totalCompact = 0;
long totalLog = 0;
long startTime = System.currentTimeMillis();
// 1M writes
int valsize = 1000;
for (int k = FIRST_ROW; k < N_ROWS; k++) {
// Come up with a random 1000-byte string
String randstr1 = "" + System.currentTimeMillis();
StringBuffer buf1 = new StringBuffer("val_" + k + "__");
while (buf1.length() < valsize) {
buf1.append(randstr1);
}
// Write to the HRegion
long writeid = region.startUpdate(new Text("row_" + k));
region.put(writeid, CONTENTS_BODY, buf1.toString().getBytes());
region.commit(writeid);
if (k > 0 && k % (N_ROWS / 100) == 0) {
System.out.println("Flushing write #" + k);
long flushStart = System.currentTimeMillis();
region.flushcache(false);
long flushEnd = System.currentTimeMillis();
totalFlush += (flushEnd - flushStart);
if (k % (N_ROWS / 10) == 0) {
System.out.print("Rolling log...");
long logStart = System.currentTimeMillis();
log.rollWriter();
long logEnd = System.currentTimeMillis();
totalLog += (logEnd - logStart);
System.out.println(" elapsed time: " + ((logEnd - logStart) / 1000.0));
}
}
}
long startCompact = System.currentTimeMillis();
if(region.compactStores()) {
totalCompact = System.currentTimeMillis() - startCompact;
System.out.println("Region compacted - elapsedTime: " + (totalCompact / 1000.0));
} else {
System.out.println("No compaction required.");
}
long endTime = System.currentTimeMillis();
long totalElapsed = (endTime - startTime);
System.out.println();
System.out.println("Batch-write complete.");
System.out.println("Wrote " + N_ROWS + " rows, each of ~" + valsize + " bytes");
System.out.println("Total flush-time: " + (totalFlush / 1000.0));
System.out.println("Total compact-time: " + (totalCompact / 1000.0));
System.out.println("Total log-time: " + (totalLog / 1000.0));
System.out.println("Total time elapsed: " + (totalElapsed / 1000.0));
System.out.println("Total time, rows/second: " + (N_ROWS / (totalElapsed / 1000.0)));
System.out.println("Adjusted time (not including flush, compact, or log): " + ((totalElapsed - totalFlush - totalCompact - totalLog) / 1000.0));
System.out.println("Adjusted time, rows/second: " + (N_ROWS / ((totalElapsed - totalFlush - totalCompact - totalLog) / 1000.0)));
} catch(IOException e) {
failures = true;
throw e;
}
}
// NOTE: This test depends on testBatchWrite succeeding
public void testSplitAndMerge() throws IOException {
if(!initialized || failures) {
throw new IllegalStateException();
}
try {
Text midKey = new Text();
if(region.needsSplit(midKey)) {
System.out.println("Needs split");
}
// Split it anyway
Text midkey = new Text("row_" + (Environment.debugging ? (N_ROWS / 2) : (NUM_VALS/2)));
Path oldRegionPath = region.getRegionDir();
HRegion subregions[] = region.closeAndSplit(midkey);
assertEquals("Number of subregions", subregions.length, 2);
// Now merge it back together
Path oldRegion1 = subregions[0].getRegionDir();
Path oldRegion2 = subregions[1].getRegionDir();
region = HRegion.closeAndMerge(subregions[0], subregions[1]);
fs.delete(oldRegionPath);
fs.delete(oldRegion1);
fs.delete(oldRegion2);
} catch(IOException e) {
failures = true;
throw e;
}
}
// This test verifies that everything is still there after splitting and merging
public void testRead() throws IOException {
if(!initialized || failures) {
throw new IllegalStateException();
}
// First verify the data written by testBasic()
Text[] cols = new Text[] {
new Text(ANCHORNUM + "[0-9]+"),
new Text(CONTENTS_BASIC)
};
HScannerInterface 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[]>();
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);
} finally {
s.close();
}
// Verify testScan data
cols = new Text[] {
CONTENTS_FIRSTCOL,
ANCHOR_SECONDCOL
};
s = region.getScanner(cols, new Text());
try {
int numFetched = 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);
int curval = Integer.parseInt(new String(val).trim());
for (int j = 0; j < cols.length; j++) {
if (col.compareTo(cols[j]) == 0) {
assertEquals("Value for " + col + " should be: " + k
+ ", but was fetched as: " + curval, curval, k);
numFetched++;
}
}
}
curVals.clear();
k++;
}
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
} finally {
s.close();
}
// Verify testBatchWrite data
if(Environment.debugging) {
s = region.getScanner(new Text[] { CONTENTS_BODY }, new Text());
try {
int numFetched = 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);
assertTrue(col.compareTo(CONTENTS_BODY) == 0);
assertNotNull(val);
numFetched++;
}
curVals.clear();
k++;
}
assertEquals("Inserted " + N_ROWS + " values, but fetched " + numFetched, N_ROWS, numFetched);
} finally {
s.close();
}
}
// Test a scanner which only specifies the column family name
cols = new Text[] {
new Text("anchor:")
};
s = region.getScanner(cols, new Text());
try {
int fetched = 0;
HStoreKey curKey = new HStoreKey();
TreeMap<Text, byte[]> curVals = new TreeMap<Text, byte[]>();
while(s.next(curKey, curVals)) {
for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
it.next();
fetched++;
}
curVals.clear();
}
assertEquals("Inserted " + (NUM_VALS + numInserted/2) + " values, but fetched " + fetched, (NUM_VALS + numInserted/2), fetched);
} finally {
s.close();
}
}
private static void deleteFile(File f) {
if(f.isDirectory()) {
File[] children = f.listFiles();
for(int i = 0; i < children.length; i++) {
deleteFile(children[i]);
}
}
f.delete();
}
public void testCleanup() throws IOException {
if(!initialized) {
throw new IllegalStateException();
}
// Shut down the mini cluster
cluster.shutdown();
// Delete all the DFS files
deleteFile(new File(System.getProperty("test.build.data"), "dfs"));
}
}