HADOOP-1420, HADOOP-1423. Findbugs changes, remove reference to removed class HLocking that broke main build.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@540973 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jim Kellerman 2007-05-23 15:14:00 +00:00
parent a658989730
commit 19099c98b1
16 changed files with 181 additions and 139 deletions

View File

@ -12,3 +12,5 @@ Trunk (unreleased changes)
tolerant. tolerant.
6. HADOOP-1418. HBase miscellaneous: unit test for HClient, client to do 6. HADOOP-1418. HBase miscellaneous: unit test for HClient, client to do
'Performance Evaluation', etc. 'Performance Evaluation', etc.
7. HADOOP-1420, HADOOP-1423. Findbugs changes, remove reference to removed
class HLocking.

View File

@ -50,7 +50,7 @@ public abstract class HAbstractScanner implements HInternalScannerInterface {
// 2. Match on the column family + column key regex // 2. Match on the column family + column key regex
// 3. Simple match: compare column family + column key literally // 3. Simple match: compare column family + column key literally
private class ColumnMatcher { private static class ColumnMatcher {
private boolean wildCardmatch; private boolean wildCardmatch;
private MATCH_TYPE matchType; private MATCH_TYPE matchType;
private String family; private String family;

View File

@ -18,7 +18,6 @@ package org.apache.hadoop.hbase;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Iterator;
import java.util.NoSuchElementException; import java.util.NoSuchElementException;
import java.util.Random; import java.util.Random;
import java.util.TreeMap; import java.util.TreeMap;
@ -50,7 +49,7 @@ public class HClient implements HConstants {
private HMasterInterface master; private HMasterInterface master;
private final Configuration conf; private final Configuration conf;
private class TableInfo { private static class TableInfo {
public HRegionInfo regionInfo; public HRegionInfo regionInfo;
public HServerAddress serverAddress; public HServerAddress serverAddress;
@ -133,7 +132,6 @@ public class HClient implements HConstants {
try { try {
Thread.sleep(this.clientTimeout); Thread.sleep(this.clientTimeout);
} catch(InterruptedException e) { } catch(InterruptedException e) {
} }
} }
@ -179,9 +177,9 @@ public class HClient implements HConstants {
TreeMap<Text, TableInfo> metaServers = TreeMap<Text, TableInfo> metaServers =
this.tablesToServers.get(META_TABLE_NAME); this.tablesToServers.get(META_TABLE_NAME);
if(metaServers == null) { // Don't know where the meta is if (metaServers == null) { // Don't know where the meta is
loadMetaFromRoot(tableName); loadMetaFromRoot(tableName);
if(tableName.equals(META_TABLE_NAME) || tableName.equals(ROOT_TABLE_NAME)) { if (tableName.equals(META_TABLE_NAME) || tableName.equals(ROOT_TABLE_NAME)) {
// All we really wanted was the meta or root table // All we really wanted was the meta or root table
return; return;
} }
@ -192,32 +190,19 @@ public class HClient implements HConstants {
for(int tries = 0; for(int tries = 0;
this.tableServers.size() == 0 && tries < this.numRetries; this.tableServers.size() == 0 && tries < this.numRetries;
tries++) { tries++) {
Text firstMetaRegion = (metaServers.containsKey(tableName))?
Text firstMetaRegion = null; tableName: metaServers.headMap(tableName).lastKey();
if(metaServers.containsKey(tableName)) { for(TableInfo t: metaServers.tailMap(firstMetaRegion).values()) {
firstMetaRegion = tableName;
} else {
firstMetaRegion = metaServers.headMap(tableName).lastKey();
}
for(Iterator<TableInfo> i
= metaServers.tailMap(firstMetaRegion).values().iterator();
i.hasNext(); ) {
TableInfo t = i.next();
scanOneMetaRegion(t, tableName); scanOneMetaRegion(t, tableName);
} }
if(this.tableServers.size() == 0) { if (this.tableServers.size() == 0) {
// Table not assigned. Sleep and try again // Table not assigned. Sleep and try again
if (LOG.isDebugEnabled()) {
if(LOG.isDebugEnabled()) {
LOG.debug("Sleeping. Table " + tableName LOG.debug("Sleeping. Table " + tableName
+ " not currently being served."); + " not currently being served.");
} }
try { try {
Thread.sleep(this.clientTimeout); Thread.sleep(this.clientTimeout);
} catch(InterruptedException e) { } catch(InterruptedException e) {
} }
if(LOG.isDebugEnabled()) { if(LOG.isDebugEnabled()) {
@ -225,7 +210,7 @@ public class HClient implements HConstants {
} }
} }
} }
if(this.tableServers.size() == 0) { if (this.tableServers.size() == 0) {
throw new IOException("failed to scan " + META_TABLE_NAME + " after " throw new IOException("failed to scan " + META_TABLE_NAME + " after "
+ this.numRetries + " retries"); + this.numRetries + " retries");
} }
@ -976,7 +961,9 @@ public class HClient implements HConstants {
printUsage(); printUsage();
break; break;
} }
} catch (Exception e) { } catch (IOException e) {
e.printStackTrace();
} catch (RuntimeException e) {
e.printStackTrace(); e.printStackTrace();
} }
@ -988,4 +975,4 @@ public class HClient implements HConstants {
int errCode = (new HClient(c)).doCommandLine(args); int errCode = (new HClient(c)).doCommandLine(args);
System.exit(errCode); System.exit(errCode);
} }
} }

View File

@ -72,14 +72,13 @@ public class HLog implements HConstants {
boolean insideCacheFlush = false; boolean insideCacheFlush = false;
TreeMap<Text, Long> regionToLastFlush = new TreeMap<Text, Long>(); TreeMap<Text, Long> regionToLastFlush = new TreeMap<Text, Long>();
long oldestOutstandingSeqNum = -1;
boolean closed = false; boolean closed = false;
transient long logSeqNum = 0; transient long logSeqNum = 0;
long filenum = 0; long filenum = 0;
transient int numEntries = 0; transient int numEntries = 0;
Integer rollLock = new Integer(0); Integer rollLock = 0;
/** /**
* Bundle up a bunch of log files (which are no longer being written to), * Bundle up a bunch of log files (which are no longer being written to),
@ -319,10 +318,10 @@ public class HLog implements HConstants {
} }
int counter = 0; int counter = 0;
for (Text column: columns.keySet()) { for (Map.Entry<Text, BytesWritable> es: columns.entrySet()) {
HLogKey logKey = HLogKey logKey =
new HLogKey(regionName, tableName, row, seqNum[counter++]); new HLogKey(regionName, tableName, row, seqNum[counter++]);
HLogEdit logEdit = new HLogEdit(column, columns.get(column), timestamp); HLogEdit logEdit = new HLogEdit(es.getKey(), es.getValue(), timestamp);
writer.append(logKey, logEdit); writer.append(logKey, logEdit);
numEntries++; numEntries++;
} }

View File

@ -72,6 +72,19 @@ public class HLogKey implements WritableComparable {
return getTablename().toString() + " " + getRegionName().toString() + " " + return getTablename().toString() + " " + getRegionName().toString() + " " +
getRow().toString() + " " + getLogSeqNum(); getRow().toString() + " " + getLogSeqNum();
} }
@Override
public boolean equals(Object obj) {
return compareTo(obj) == 0;
}
@Override
public int hashCode() {
int result = this.regionName.hashCode();
result ^= this.row.hashCode();
result ^= Long.valueOf(this.logSeqNum).hashCode();
return result;
}
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////
// Comparable // Comparable

View File

@ -294,9 +294,9 @@ public class HMaster implements HConstants, HMasterInterface,
private RootScanner rootScanner; private RootScanner rootScanner;
private Thread rootScannerThread; private Thread rootScannerThread;
private Integer rootScannerLock = new Integer(0); private Integer rootScannerLock = 0;
private class MetaRegion { private static class MetaRegion {
public HServerAddress server; public HServerAddress server;
public Text regionName; public Text regionName;
public Text startKey; public Text startKey;
@ -426,7 +426,7 @@ public class HMaster implements HConstants, HMasterInterface,
private MetaScanner metaScanner; private MetaScanner metaScanner;
private Thread metaScannerThread; private Thread metaScannerThread;
private Integer metaScannerLock = new Integer(0); private Integer metaScannerLock = 0;
// The 'unassignedRegions' table maps from a region name to a HRegionInfo record, // The 'unassignedRegions' table maps from a region name to a HRegionInfo record,
// which includes the region's table, its id, and its start/end keys. // which includes the region's table, its id, and its start/end keys.
@ -1136,7 +1136,6 @@ public class HMaster implements HConstants, HMasterInterface,
} }
} }
scannerId = -1L;
} }
// Remove server from root/meta entries // Remove server from root/meta entries
@ -1739,7 +1738,7 @@ public class HMaster implements HConstants, HMasterInterface,
private String server; private String server;
public ServerExpirer(String server) { public ServerExpirer(String server) {
this.server = new String(server); this.server = server;
} }
public void leaseExpired() { public void leaseExpired() {

View File

@ -22,6 +22,7 @@ import org.apache.commons.logging.LogFactory;
import java.io.*; import java.io.*;
import java.util.*; import java.util.*;
import java.util.concurrent.locks.ReentrantReadWriteLock;
/******************************************************************************* /*******************************************************************************
* The HMemcache holds in-memory modifications to the HRegion. This is really a * The HMemcache holds in-memory modifications to the HRegion. This is really a
@ -38,9 +39,10 @@ public class HMemcache {
TreeMap<HStoreKey, BytesWritable> snapshot = null; TreeMap<HStoreKey, BytesWritable> snapshot = null;
HLocking lock = new HLocking(); ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
public HMemcache() { public HMemcache() {
super();
} }
public static class Snapshot { public static class Snapshot {
@ -48,6 +50,7 @@ public class HMemcache {
public long sequenceId = 0; public long sequenceId = 0;
public Snapshot() { public Snapshot() {
super();
} }
} }
@ -67,7 +70,7 @@ public class HMemcache {
public Snapshot snapshotMemcacheForLog(HLog log) throws IOException { public Snapshot snapshotMemcacheForLog(HLog log) throws IOException {
Snapshot retval = new Snapshot(); Snapshot retval = new Snapshot();
this.lock.obtainWriteLock(); this.lock.writeLock().lock();
try { try {
if(snapshot != null) { if(snapshot != null) {
throw new IOException("Snapshot in progress!"); throw new IOException("Snapshot in progress!");
@ -96,7 +99,7 @@ public class HMemcache {
return retval; return retval;
} finally { } finally {
this.lock.releaseWriteLock(); this.lock.writeLock().unlock();
} }
} }
@ -106,7 +109,7 @@ public class HMemcache {
* Modifying the structure means we need to obtain a writelock. * Modifying the structure means we need to obtain a writelock.
*/ */
public void deleteSnapshot() throws IOException { public void deleteSnapshot() throws IOException {
this.lock.obtainWriteLock(); this.lock.writeLock().lock();
try { try {
if(snapshot == null) { if(snapshot == null) {
@ -132,7 +135,7 @@ public class HMemcache {
} }
} finally { } finally {
this.lock.releaseWriteLock(); this.lock.writeLock().unlock();
} }
} }
@ -142,18 +145,14 @@ public class HMemcache {
* Operation uses a write lock. * Operation uses a write lock.
*/ */
public void add(Text row, TreeMap<Text, BytesWritable> columns, long timestamp) { public void add(Text row, TreeMap<Text, BytesWritable> columns, long timestamp) {
this.lock.obtainWriteLock(); this.lock.writeLock().lock();
try { try {
for(Iterator<Text> it = columns.keySet().iterator(); it.hasNext(); ) { for (Map.Entry<Text, BytesWritable> es: columns.entrySet()) {
Text column = it.next(); HStoreKey key = new HStoreKey(row, es.getKey(), timestamp);
BytesWritable val = columns.get(column); memcache.put(key, es.getValue());
HStoreKey key = new HStoreKey(row, column, timestamp);
memcache.put(key, val);
} }
} finally { } finally {
this.lock.releaseWriteLock(); this.lock.writeLock().unlock();
} }
} }
@ -164,7 +163,7 @@ public class HMemcache {
*/ */
public BytesWritable[] get(HStoreKey key, int numVersions) { public BytesWritable[] get(HStoreKey key, int numVersions) {
Vector<BytesWritable> results = new Vector<BytesWritable>(); Vector<BytesWritable> results = new Vector<BytesWritable>();
this.lock.obtainReadLock(); this.lock.readLock().lock();
try { try {
Vector<BytesWritable> result = get(memcache, key, numVersions-results.size()); Vector<BytesWritable> result = get(memcache, key, numVersions-results.size());
results.addAll(0, result); results.addAll(0, result);
@ -178,15 +177,10 @@ public class HMemcache {
results.addAll(results.size(), result); results.addAll(results.size(), result);
} }
if(results.size() == 0) { return (results.size() == 0)?
return null; null: results.toArray(new BytesWritable[results.size()]);
} else {
return results.toArray(new BytesWritable[results.size()]);
}
} finally { } finally {
this.lock.releaseReadLock(); this.lock.readLock().unlock();
} }
} }
@ -196,9 +190,9 @@ public class HMemcache {
* *
* The returned object should map column names to byte arrays (byte[]). * The returned object should map column names to byte arrays (byte[]).
*/ */
public TreeMap<Text, BytesWritable> getFull(HStoreKey key) throws IOException { public TreeMap<Text, BytesWritable> getFull(HStoreKey key) {
TreeMap<Text, BytesWritable> results = new TreeMap<Text, BytesWritable>(); TreeMap<Text, BytesWritable> results = new TreeMap<Text, BytesWritable>();
this.lock.obtainReadLock(); this.lock.readLock().lock();
try { try {
internalGetFull(memcache, key, results); internalGetFull(memcache, key, results);
for(int i = history.size()-1; i >= 0; i--) { for(int i = history.size()-1; i >= 0; i--) {
@ -208,25 +202,22 @@ public class HMemcache {
return results; return results;
} finally { } finally {
this.lock.releaseReadLock(); this.lock.readLock().unlock();
} }
} }
void internalGetFull(TreeMap<HStoreKey, BytesWritable> map, HStoreKey key, void internalGetFull(TreeMap<HStoreKey, BytesWritable> map, HStoreKey key,
TreeMap<Text, BytesWritable> results) { TreeMap<Text, BytesWritable> results) {
SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(key); SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(key);
for (Map.Entry<HStoreKey, BytesWritable> es: tailMap.entrySet()) {
for(Iterator<HStoreKey> it = tailMap.keySet().iterator(); it.hasNext(); ) { HStoreKey itKey = es.getKey();
HStoreKey itKey = it.next();
Text itCol = itKey.getColumn(); Text itCol = itKey.getColumn();
if (results.get(itCol) == null
if(results.get(itCol) == null
&& key.matchesWithoutColumn(itKey)) { && key.matchesWithoutColumn(itKey)) {
BytesWritable val = tailMap.get(itKey); BytesWritable val = tailMap.get(itKey);
results.put(itCol, val); results.put(itCol, val);
} else if(key.getRow().compareTo(itKey.getRow()) > 0) { } else if (key.getRow().compareTo(itKey.getRow()) > 0) {
break; break;
} }
} }
@ -246,15 +237,14 @@ public class HMemcache {
HStoreKey curKey = new HStoreKey(key.getRow(), key.getColumn(), key.getTimestamp()); HStoreKey curKey = new HStoreKey(key.getRow(), key.getColumn(), key.getTimestamp());
SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(curKey); SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(curKey);
for(Iterator<HStoreKey> it = tailMap.keySet().iterator(); it.hasNext(); ) { for (Map.Entry<HStoreKey, BytesWritable> es: tailMap.entrySet()) {
HStoreKey itKey = it.next(); HStoreKey itKey = es.getKey();
if (itKey.matchesRowCol(curKey)) {
if(itKey.matchesRowCol(curKey)) {
result.add(tailMap.get(itKey)); result.add(tailMap.get(itKey));
curKey.setVersion(itKey.getTimestamp() - 1); curKey.setVersion(itKey.getTimestamp() - 1);
} }
if(numVersions > 0 && result.size() >= numVersions) { if (numVersions > 0 && result.size() >= numVersions) {
break; break;
} }
} }
@ -285,7 +275,7 @@ public class HMemcache {
super(timestamp, targetCols); super(timestamp, targetCols);
lock.obtainReadLock(); lock.readLock().lock();
try { try {
this.backingMaps = new TreeMap[history.size() + 1]; this.backingMaps = new TreeMap[history.size() + 1];
@ -377,7 +367,7 @@ public class HMemcache {
} }
} finally { } finally {
lock.releaseReadLock(); lock.readLock().unlock();
scannerClosed = true; scannerClosed = true;
} }
} }

View File

@ -171,12 +171,11 @@ public class HRegion implements HConstants {
LOG.debug("merging stores"); LOG.debug("merging stores");
} }
for(Iterator<Text> it = filesToMerge.keySet().iterator(); it.hasNext(); ) { for (Map.Entry<Text, Vector<HStoreFile>> es: filesToMerge.entrySet()) {
Text colFamily = it.next(); Text colFamily = es.getKey();
Vector<HStoreFile> srcFiles = filesToMerge.get(colFamily); Vector<HStoreFile> srcFiles = es.getValue();
HStoreFile dst = new HStoreFile(conf, merges, newRegionInfo.regionName, HStoreFile dst = new HStoreFile(conf, merges, newRegionInfo.regionName,
colFamily, Math.abs(rand.nextLong())); colFamily, Math.abs(rand.nextLong()));
dst.mergeStoreFiles(srcFiles, fs, conf); dst.mergeStoreFiles(srcFiles, fs, conf);
alreadyMerged.addAll(srcFiles); alreadyMerged.addAll(srcFiles);
} }
@ -226,12 +225,11 @@ public class HRegion implements HConstants {
LOG.debug("merging changes since start of merge"); LOG.debug("merging changes since start of merge");
} }
for(Iterator<Text> it = filesToMerge.keySet().iterator(); it.hasNext(); ) { for (Map.Entry<Text, Vector<HStoreFile>> es : filesToMerge.entrySet()) {
Text colFamily = it.next(); Text colFamily = es.getKey();
Vector<HStoreFile> srcFiles = filesToMerge.get(colFamily); Vector<HStoreFile> srcFiles = es.getValue();
HStoreFile dst = new HStoreFile(conf, merges, newRegionInfo.regionName, HStoreFile dst = new HStoreFile(conf, merges,
colFamily, Math.abs(rand.nextLong())); newRegionInfo.regionName, colFamily, Math.abs(rand.nextLong()));
dst.mergeStoreFiles(srcFiles, fs, conf); dst.mergeStoreFiles(srcFiles, fs, conf);
} }
@ -268,7 +266,7 @@ public class HRegion implements HConstants {
HRegionInfo regionInfo; HRegionInfo regionInfo;
Path regiondir; Path regiondir;
class WriteState { static class WriteState {
public volatile boolean writesOngoing; public volatile boolean writesOngoing;
public volatile boolean writesEnabled; public volatile boolean writesEnabled;
public volatile boolean closed; public volatile boolean closed;
@ -1248,7 +1246,7 @@ public class HRegion implements HConstants {
/******************************************************************************* /*******************************************************************************
* HScanner is an iterator through a bunch of rows in an HRegion. * HScanner is an iterator through a bunch of rows in an HRegion.
******************************************************************************/ ******************************************************************************/
private class HScanner implements HInternalScannerInterface { private static class HScanner implements HInternalScannerInterface {
private HInternalScannerInterface[] scanners; private HInternalScannerInterface[] scanners;
private TreeMap<Text, BytesWritable>[] resultSets; private TreeMap<Text, BytesWritable>[] resultSets;
private HStoreKey[] keys; private HStoreKey[] keys;

View File

@ -64,7 +64,7 @@ public class HRegionServer
private long splitOrCompactCheckFrequency; private long splitOrCompactCheckFrequency;
private SplitOrCompactChecker splitOrCompactChecker; private SplitOrCompactChecker splitOrCompactChecker;
private Thread splitOrCompactCheckerThread; private Thread splitOrCompactCheckerThread;
private Integer splitOrCompactLock = new Integer(0); private Integer splitOrCompactLock = 0;
private class SplitOrCompactChecker implements Runnable, RegionUnavailableListener { private class SplitOrCompactChecker implements Runnable, RegionUnavailableListener {
private HClient client = new HClient(conf); private HClient client = new HClient(conf);
@ -222,7 +222,7 @@ public class HRegionServer
private Flusher cacheFlusher; private Flusher cacheFlusher;
private Thread cacheFlusherThread; private Thread cacheFlusherThread;
private Integer cacheFlusherLock = new Integer(0); private Integer cacheFlusherLock = 0;
private class Flusher implements Runnable { private class Flusher implements Runnable {
public void run() { public void run() {
while(! stopRequested) { while(! stopRequested) {
@ -291,7 +291,7 @@ public class HRegionServer
private HLog log; private HLog log;
private LogRoller logRoller; private LogRoller logRoller;
private Thread logRollerThread; private Thread logRollerThread;
private Integer logRollerLock = new Integer(0); private Integer logRollerLock = 0;
private class LogRoller implements Runnable { private class LogRoller implements Runnable {
public void run() { public void run() {
while(! stopRequested) { while(! stopRequested) {
@ -388,7 +388,7 @@ public class HRegionServer
try { try {
// Server to handle client requests // Server to handle client requests
this.server = RPC.getServer(this, address.getBindAddress().toString(), this.server = RPC.getServer(this, address.getBindAddress(),
address.getPort(), conf.getInt("hbase.regionserver.handler.count", 10), address.getPort(), conf.getInt("hbase.regionserver.handler.count", 10),
false, conf); false, conf);
@ -509,10 +509,11 @@ public class HRegionServer
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Sleep"); LOG.debug("Sleep");
} }
synchronized(this) { synchronized (this) {
try { try {
Thread.sleep(waitTime); wait(waitTime);
} catch(InterruptedException iex) { } catch (InterruptedException e1) {
// Go back up to the while test if stop has been requested.
} }
} }
} }
@ -588,10 +589,11 @@ public class HRegionServer
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Sleep"); LOG.debug("Sleep");
} }
synchronized(this) { synchronized (this) {
try { try {
Thread.sleep(waitTime); wait(waitTime);
} catch(InterruptedException iex) { } catch(InterruptedException iex) {
// On interrupt we go around to the while test of stopRequested
} }
} }
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
@ -927,10 +929,8 @@ public class HRegionServer
TreeMap<Text, BytesWritable> map = region.getFull(row); TreeMap<Text, BytesWritable> map = region.getFull(row);
LabelledData result[] = new LabelledData[map.size()]; LabelledData result[] = new LabelledData[map.size()];
int counter = 0; int counter = 0;
for(Iterator<Text> it = map.keySet().iterator(); it.hasNext(); ) { for (Map.Entry<Text, BytesWritable> es: map.entrySet()) {
Text colname = it.next(); result[counter++] = new LabelledData(es.getKey(), es.getValue());
BytesWritable val = map.get(colname);
result[counter++] = new LabelledData(colname, val);
} }
return result; return result;
} }
@ -939,7 +939,7 @@ public class HRegionServer
* Start an update to the HBase. This also creates a lease associated with * Start an update to the HBase. This also creates a lease associated with
* the caller. * the caller.
*/ */
private class RegionListener extends LeaseListener { private static class RegionListener extends LeaseListener {
private HRegion localRegion; private HRegion localRegion;
private long localLockId; private long localLockId;
@ -1180,4 +1180,4 @@ public class HRegionServer
printUsageAndExit(); printUsageAndExit();
} }
} }
} }

View File

@ -19,6 +19,7 @@ import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.regex.Matcher; import java.util.regex.Matcher;
import java.util.regex.Pattern; import java.util.regex.Pattern;
@ -181,12 +182,14 @@ public class HRegiondirReader {
// Every line starts with row name followed by column name // Every line starts with row name followed by column name
// followed by cell content. // followed by cell content.
while(scanner.next(key, results)) { while(scanner.next(key, results)) {
for (Text colname: results.keySet()) { for (Map.Entry<Text, BytesWritable> es: results.entrySet()) {
Text colname = es.getKey();
BytesWritable colvalue = es.getValue();
Object value = null; Object value = null;
byte[] bytes = new byte[results.get(colname).getSize()]; byte[] bytes = new byte[colvalue.getSize()];
if (colname.toString().equals("info:regioninfo")) { if (colname.toString().equals("info:regioninfo")) {
// Then bytes are instance of an HRegionInfo. // Then bytes are instance of an HRegionInfo.
System.arraycopy(results.get(colname).get(), 0, bytes, 0, bytes.length); System.arraycopy(colvalue, 0, bytes, 0, bytes.length);
value = new HRegionInfo(bytes); value = new HRegionInfo(bytes);
} else { } else {
value = new String(bytes, HConstants.UTF8_ENCODING); value = new String(bytes, HConstants.UTF8_ENCODING);

View File

@ -35,8 +35,8 @@ public class HServerAddress implements Writable {
public HServerAddress(InetSocketAddress address) { public HServerAddress(InetSocketAddress address) {
this.address = address; this.address = address;
this.stringValue = new String(address.getAddress().getHostAddress() this.stringValue = address.getAddress().getHostAddress() + ":" +
+ ":" + address.getPort()); address.getPort();
} }
public HServerAddress(String hostAndPort) { public HServerAddress(String hostAndPort) {
@ -47,19 +47,19 @@ public class HServerAddress implements Writable {
String host = hostAndPort.substring(0, colonIndex); String host = hostAndPort.substring(0, colonIndex);
int port = Integer.valueOf(hostAndPort.substring(colonIndex + 1)); int port = Integer.valueOf(hostAndPort.substring(colonIndex + 1));
this.address = new InetSocketAddress(host, port); this.address = new InetSocketAddress(host, port);
this.stringValue = new String(hostAndPort); this.stringValue = hostAndPort;
} }
public HServerAddress(String bindAddress, int port) { public HServerAddress(String bindAddress, int port) {
this.address = new InetSocketAddress(bindAddress, port); this.address = new InetSocketAddress(bindAddress, port);
this.stringValue = new String(bindAddress + ":" + port); this.stringValue = bindAddress + ":" + port;
} }
public HServerAddress(HServerAddress other) { public HServerAddress(HServerAddress other) {
String bindAddress = other.getBindAddress(); String bindAddress = other.getBindAddress();
int port = other.getPort(); int port = other.getPort();
address = new InetSocketAddress(bindAddress, port); address = new InetSocketAddress(bindAddress, port);
stringValue = new String(bindAddress + ":" + port); stringValue = bindAddress + ":" + port;
} }
public String getBindAddress() { public String getBindAddress() {

View File

@ -61,8 +61,8 @@ public class HStore {
Path compactdir; Path compactdir;
Path loginfodir; Path loginfodir;
Integer compactLock = new Integer(0); Integer compactLock = 0;
Integer flushLock = new Integer(0); Integer flushLock = 0;
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
@ -225,13 +225,10 @@ public class HStore {
if(LOG.isDebugEnabled()) { if(LOG.isDebugEnabled()) {
LOG.debug("starting map readers"); LOG.debug("starting map readers");
} }
for(Iterator<Long> it = mapFiles.keySet().iterator(); it.hasNext(); ) { for(Map.Entry<Long, HStoreFile> e: mapFiles.entrySet()) {
Long key = it.next().longValue();
HStoreFile hsf = mapFiles.get(key);
// TODO - is this really necessary? Don't I do this inside compact()? // TODO - is this really necessary? Don't I do this inside compact()?
maps.put(key, maps.put(e.getKey(),
new MapFile.Reader(fs, hsf.getMapFilePath().toString(), conf)); new MapFile.Reader(fs, e.getValue().getMapFilePath().toString(), conf));
} }
LOG.info("HStore online for " + this.regionName + "/" + this.colFamily); LOG.info("HStore online for " + this.regionName + "/" + this.colFamily);
@ -239,19 +236,16 @@ public class HStore {
/** Turn off all the MapFile readers */ /** Turn off all the MapFile readers */
public void close() throws IOException { public void close() throws IOException {
this.lock.writeLock().lock();
LOG.info("closing HStore for " + this.regionName + "/" + this.colFamily); LOG.info("closing HStore for " + this.regionName + "/" + this.colFamily);
this.lock.writeLock().lock();
try { try {
for(Iterator<MapFile.Reader> it = maps.values().iterator(); it.hasNext(); ) { for (MapFile.Reader map: maps.values()) {
MapFile.Reader map = it.next();
map.close(); map.close();
} }
maps.clear(); maps.clear();
mapFiles.clear(); mapFiles.clear();
LOG.info("HStore closed for " + this.regionName + "/" + this.colFamily); LOG.info("HStore closed for " + this.regionName + "/" + this.colFamily);
} finally { } finally {
this.lock.writeLock().unlock(); this.lock.writeLock().unlock();
} }
@ -300,10 +294,10 @@ public class HStore {
HStoreKey.class, BytesWritable.class); HStoreKey.class, BytesWritable.class);
try { try {
for (HStoreKey curkey: inputCache.keySet()) { for (Map.Entry<HStoreKey, BytesWritable> es: inputCache.entrySet()) {
if(this.colFamily.equals(HStoreKey.extractFamily(curkey.getColumn()))) { HStoreKey curkey = es.getKey();
BytesWritable val = inputCache.get(curkey); if (this.colFamily.equals(HStoreKey.extractFamily(curkey.getColumn()))) {
out.append(curkey, val); out.append(curkey, es.getValue());
} }
} }
if(LOG.isDebugEnabled()) { if(LOG.isDebugEnabled()) {
@ -631,8 +625,9 @@ public class HStore {
// 1. Acquiring the write-lock // 1. Acquiring the write-lock
this.lock.writeLock().lock();
Path curCompactStore = HStoreFile.getHStoreDir(compactdir, regionName, colFamily); Path curCompactStore = HStoreFile.getHStoreDir(compactdir, regionName, colFamily);
this.lock.writeLock().lock();
try { try {
Path doneFile = new Path(curCompactStore, COMPACTION_DONE); Path doneFile = new Path(curCompactStore, COMPACTION_DONE);
if(! fs.exists(doneFile)) { if(! fs.exists(doneFile)) {
@ -918,10 +913,10 @@ public class HStore {
/** Generate a random unique filename suffix */ /** Generate a random unique filename suffix */
String obtainFileLabel(Path prefix) throws IOException { String obtainFileLabel(Path prefix) throws IOException {
String testsuffix = String.valueOf(Math.abs(rand.nextInt())); String testsuffix = String.valueOf(rand.nextInt(Integer.MAX_VALUE));
Path testpath = new Path(prefix.toString() + testsuffix); Path testpath = new Path(prefix.toString() + testsuffix);
while(fs.exists(testpath)) { while(fs.exists(testpath)) {
testsuffix = String.valueOf(Math.abs(rand.nextInt())); testsuffix = String.valueOf(rand.nextInt(Integer.MAX_VALUE));
testpath = new Path(prefix.toString() + testsuffix); testpath = new Path(prefix.toString() + testsuffix);
} }
return testsuffix; return testsuffix;

View File

@ -326,6 +326,19 @@ public class HStoreFile implements HConstants, WritableComparable {
out.close(); out.close();
} }
} }
public boolean equals(Object o) {
return this.compareTo(o) == 0;
}
@Override
public int hashCode() {
int result = this.dir.hashCode();
result ^= this.regionName.hashCode();
result ^= this.colFamily.hashCode();
result ^= Long.valueOf(this.fileId).hashCode();
return result;
}
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////
// Writable // Writable
@ -368,11 +381,4 @@ public class HStoreFile implements HConstants, WritableComparable {
} }
return result; return result;
} }
}
public boolean equals(Object o) {
return this.compareTo(o) == 0;
}
}

View File

@ -15,6 +15,8 @@
*/ */
package org.apache.hadoop.hbase; package org.apache.hadoop.hbase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.*; import org.apache.hadoop.io.*;
import java.io.*; import java.io.*;
@ -23,6 +25,8 @@ import java.io.*;
* A Key for a stored row * A Key for a stored row
******************************************************************************/ ******************************************************************************/
public class HStoreKey implements WritableComparable { public class HStoreKey implements WritableComparable {
private final Log LOG = LogFactory.getLog(this.getClass().getName());
public static Text extractFamily(Text col) throws IOException { public static Text extractFamily(Text col) throws IOException {
String column = col.toString(); String column = col.toString();
int colpos = column.indexOf(":"); int colpos = column.indexOf(":");
@ -128,6 +132,7 @@ public class HStoreKey implements WritableComparable {
extractFamily(other.getColumn())) == 0; extractFamily(other.getColumn())) == 0;
} catch(IOException e) { } catch(IOException e) {
LOG.error(e);
} }
return status; return status;
} }
@ -135,6 +140,19 @@ public class HStoreKey implements WritableComparable {
public String toString() { public String toString() {
return row.toString() + "/" + column.toString() + "/" + timestamp; return row.toString() + "/" + column.toString() + "/" + timestamp;
} }
@Override
public boolean equals(Object obj) {
return compareTo(obj) == 0;
}
@Override
public int hashCode() {
int result = this.row.hashCode();
result ^= this.column.hashCode();
result ^= Long.valueOf(this.timestamp).hashCode();
return result;
}
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////
// Comparable // Comparable

View File

@ -124,6 +124,24 @@ public class HTableDescriptor implements WritableComparable {
", maxVersions: " + this.maxVersions + ", families: " + this.families; ", maxVersions: " + this.maxVersions + ", families: " + this.families;
} }
@Override
public boolean equals(Object obj) {
return compareTo(obj) == 0;
}
@Override
public int hashCode() {
// TODO: Cache.
int result = this.name.hashCode();
result ^= Integer.valueOf(this.maxVersions).hashCode();
if (this.families != null && this.families.size() > 0) {
for (Text family: this.families) {
result ^= family.hashCode();
}
}
return result;
}
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////
// Writable // Writable
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////

View File

@ -71,6 +71,7 @@ public class Leases {
this.leaseMonitorThread.interrupt(); this.leaseMonitorThread.interrupt();
this.leaseMonitorThread.join(); this.leaseMonitorThread.join();
} catch (InterruptedException iex) { } catch (InterruptedException iex) {
// Ignore
} }
synchronized(leases) { synchronized(leases) {
synchronized(sortedLeases) { synchronized(sortedLeases) {
@ -166,6 +167,7 @@ public class Leases {
try { try {
Thread.sleep(leaseCheckFrequency); Thread.sleep(leaseCheckFrequency);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
// Ignore
} }
} }
} }
@ -211,6 +213,18 @@ public class Leases {
listener.leaseExpired(); listener.leaseExpired();
} }
@Override
public boolean equals(Object obj) {
return compareTo(obj) == 0;
}
@Override
public int hashCode() {
int result = this.getLeaseId().hashCode();
result ^= Long.valueOf(this.lastUpdate).hashCode();
return result;
}
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////
// Comparable // Comparable
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////