You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2007/06/19 00:59:16 UTC
svn commit: r548523 [1/2] - in /lucene/hadoop/trunk/src/contrib/hbase: ./
src/java/org/apache/hadoop/hbase/ src/test/org/apache/hadoop/hbase/
Author: cutting
Date: Mon Jun 18 15:59:14 2007
New Revision: 548523
URL: http://svn.apache.org/viewvc?view=rev&rev=548523
Log:
HADOOP-1498. Replace boxed types with primitives in many places. Contributed by stack.
Modified:
lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HInternalScannerInterface.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogKey.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterInterface.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterRegionInterface.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HScannerInterface.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HServerAddress.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreKey.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/KeyedData.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/LeaseListener.java
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/Leases.java
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestGet.java
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHLog.java
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHMemcache.java
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java
Modified: lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt Mon Jun 18 15:59:14 2007
@@ -34,4 +34,4 @@
19. HADOOP-1415 Integrate BSD licensed bloom filter implementation.
20. HADOOP-1465 Add cluster stop/start scripts for hbase
21. HADOOP-1415 Provide configurable per-column bloom filters - part 2.
-
+ 22. HADOOP-1498. Replace boxed types with primitives in many places.
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java Mon Jun 18 15:59:14 2007
@@ -47,7 +47,9 @@
public static final int RETOUCHED_BLOOMFILTER = 3;
/** Default constructor - used in conjunction with Writable */
- public BloomFilterDescriptor() {}
+ public BloomFilterDescriptor() {
+ super();
+ }
/**
* @param type The kind of bloom filter to use.
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java Mon Jun 18 15:59:14 2007
@@ -18,29 +18,26 @@
import java.io.IOException;
import java.util.TreeMap;
import java.util.Vector;
-
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
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 HInternalScannerInterface {
final Log LOG = LogFactory.getLog(this.getClass().getName());
// Pattern to determine if a column key is a regex
-
- static Pattern isRegexPattern = Pattern.compile("^.*[\\\\+|^&*$\\[\\]\\}{)(]+.*$");
+ static Pattern isRegexPattern =
+ Pattern.compile("^.*[\\\\+|^&*$\\[\\]\\}{)(]+.*$");
// The kind of match we are doing on a column:
-
private static enum MATCH_TYPE {
/** Just check the column family name */
FAMILY_ONLY,
@@ -55,7 +52,6 @@
// 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 static class ColumnMatcher {
private boolean wildCardmatch;
private MATCH_TYPE matchType;
@@ -63,33 +59,24 @@
private Pattern columnMatcher;
private Text col;
- ColumnMatcher(Text col) throws IOException {
- String column = col.toString();
+ ColumnMatcher(final Text col) throws IOException {
+ Text qualifier = HStoreKey.extractQualifier(col);
try {
- int colpos = column.indexOf(":");
- if(colpos == -1) {
- throw new InvalidColumnNameException("Column name has no family indicator.");
- }
-
- String columnkey = column.substring(colpos + 1);
-
- if(columnkey == null || columnkey.length() == 0) {
+ if(qualifier == null || qualifier.getLength() == 0) {
this.matchType = MATCH_TYPE.FAMILY_ONLY;
- this.family = column.substring(0, colpos);
+ this.family = HStoreKey.extractFamily(col).toString();
this.wildCardmatch = true;
-
- } else if(isRegexPattern.matcher(columnkey).matches()) {
+ } else if(isRegexPattern.matcher(qualifier.toString()).matches()) {
this.matchType = MATCH_TYPE.REGEX;
- this.columnMatcher = Pattern.compile(column);
+ this.columnMatcher = Pattern.compile(col.toString());
this.wildCardmatch = true;
-
} else {
this.matchType = MATCH_TYPE.SIMPLE;
this.col = col;
this.wildCardmatch = false;
}
} catch(Exception e) {
- throw new IOException("Column: " + column + ": " + e.getMessage());
+ throw new IOException("Column: " + col + ": " + e.getMessage());
}
}
@@ -119,8 +106,10 @@
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
+ // Keys retrieved from the sources
+ protected HStoreKey keys[];
+ // Values that correspond to those keys
+ protected byte [][] vals;
protected long timestamp; // The timestamp to match entries against
private boolean wildcardMatch;
@@ -218,7 +207,7 @@
*
* @see org.apache.hadoop.hbase.HScannerInterface#next(org.apache.hadoop.hbase.HStoreKey, java.util.TreeMap)
*/
- public boolean next(HStoreKey key, TreeMap<Text, BytesWritable> results)
+ public boolean next(HStoreKey key, TreeMap<Text, byte []> results)
throws IOException {
// Find the next row label (and timestamp)
Text chosenRow = null;
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java Mon Jun 18 15:59:14 2007
@@ -30,11 +30,10 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.io.retry.RetryProxy;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
@@ -285,10 +284,8 @@
}
boolean found = false;
for(int j = 0; j < values.length; j++) {
- if(values[j].getKey().getColumn().equals(COL_REGIONINFO)) {
- byte[] bytes = new byte[values[j].getData().getSize()];
- System.arraycopy(values[j].getData().get(), 0, bytes, 0, bytes.length);
- inbuf.reset(bytes, bytes.length);
+ if (values[j].getKey().getColumn().equals(COL_REGIONINFO)) {
+ inbuf.reset(values[j].getData(), values[j].getData().length);
info.readFields(inbuf);
if(info.tableDesc.getName().equals(tableName)) {
found = true;
@@ -398,9 +395,7 @@
valuesfound += 1;
for(int j = 0; j < values.length; j++) {
if(values[j].getKey().getColumn().equals(COL_REGIONINFO)) {
- byte[] bytes = new byte[values[j].getData().getSize()];
- System.arraycopy(values[j].getData().get(), 0, bytes, 0, bytes.length);
- inbuf.reset(bytes, bytes.length);
+ inbuf.reset(values[j].getData(), values[j].getData().length);
info.readFields(inbuf);
isenabled = !info.offLine;
break;
@@ -483,9 +478,7 @@
valuesfound += 1;
for(int j = 0; j < values.length; j++) {
if(values[j].getKey().getColumn().equals(COL_REGIONINFO)) {
- byte[] bytes = new byte[values[j].getData().getSize()];
- System.arraycopy(values[j].getData().get(), 0, bytes, 0, bytes.length);
- inbuf.reset(bytes, bytes.length);
+ inbuf.reset(values[j].getData(), values[j].getData().length);
info.readFields(inbuf);
disabled = info.offLine;
break;
@@ -737,8 +730,8 @@
* @throws IOException
*/
private TreeMap<Text, RegionLocation> scanOneMetaRegion(final RegionLocation t,
- final Text tableName) throws IOException {
-
+ final Text tableName)
+ throws IOException {
HRegionInterface server = getHRegionConnection(t.serverAddress);
TreeMap<Text, RegionLocation> servers = new TreeMap<Text, RegionLocation>();
for(int tries = 0; servers.size() == 0 && tries < this.numRetries;
@@ -772,9 +765,7 @@
byte[] bytes = null;
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
for(int i = 0; i < values.length; i++) {
- bytes = new byte[values[i].getData().getSize()];
- System.arraycopy(values[i].getData().get(), 0, bytes, 0, bytes.length);
- results.put(values[i].getKey().getColumn(), bytes);
+ results.put(values[i].getKey().getColumn(), values[i].getData());
}
regionInfo = new HRegionInfo();
bytes = results.get(COL_REGIONINFO);
@@ -900,8 +891,7 @@
}
for(int i = 0; i < values.length; i++) {
if(values[i].getKey().getColumn().equals(COL_REGIONINFO)) {
- byte[] bytes = values[i].getData().get();
- inbuf.reset(bytes, bytes.length);
+ inbuf.reset(values[i].getData(), values[i].getData().length);
HRegionInfo info = new HRegionInfo();
info.readFields(inbuf);
@@ -967,22 +957,19 @@
/**
* Get a single value for the specified row and column
*
- * @param row - row key
- * @param column - column name
- * @return - value for specified row/column
+ * @param row row key
+ * @param column column name
+ * @return value for specified row/column
* @throws IOException
*/
public byte[] get(Text row, Text column) throws IOException {
RegionLocation info = null;
- BytesWritable value = null;
-
+ byte [] value = null;
for(int tries = 0; tries < numRetries && info == null; tries++) {
info = getRegionLocation(row);
-
try {
- value = getHRegionConnection(info.serverAddress).get(
- info.regionInfo.regionName, row, column);
-
+ value = getHRegionConnection(info.serverAddress).
+ get(info.regionInfo.regionName, row, column);
} catch(NotServingRegionException e) {
if(tries == numRetries - 1) {
// No more tries
@@ -992,13 +979,7 @@
info = null;
}
}
-
- if(value != null) {
- byte[] bytes = new byte[value.getSize()];
- System.arraycopy(value.get(), 0, bytes, 0, bytes.length);
- return bytes;
- }
- return null;
+ return value;
}
/**
@@ -1012,15 +993,12 @@
*/
public byte[][] get(Text row, Text column, int numVersions) throws IOException {
RegionLocation info = null;
- BytesWritable[] values = null;
-
+ byte [][] values = null;
for(int tries = 0; tries < numRetries && info == null; tries++) {
info = getRegionLocation(row);
-
try {
values = getHRegionConnection(info.serverAddress).get(
- info.regionInfo.regionName, row, column, numVersions);
-
+ info.regionInfo.regionName, row, column, numVersions);
} catch(NotServingRegionException e) {
if(tries == numRetries - 1) {
// No more tries
@@ -1034,9 +1012,7 @@
if(values != null) {
ArrayList<byte[]> bytes = new ArrayList<byte[]>();
for(int i = 0 ; i < values.length; i++) {
- byte[] value = new byte[values[i].getSize()];
- System.arraycopy(values[i].get(), 0, value, 0, value.length);
- bytes.add(value);
+ bytes.add(values[i]);
}
return bytes.toArray(new byte[values.length][]);
}
@@ -1057,14 +1033,12 @@
public byte[][] get(Text row, Text column, long timestamp, int numVersions)
throws IOException {
RegionLocation info = null;
- BytesWritable[] values = null;
-
+ byte [][] values = null;
for(int tries = 0; tries < numRetries && info == null; tries++) {
info = getRegionLocation(row);
-
try {
- values = getHRegionConnection(info.serverAddress).get(
- info.regionInfo.regionName, row, column, timestamp, numVersions);
+ values = getHRegionConnection(info.serverAddress).
+ get(info.regionInfo.regionName, row, column, timestamp, numVersions);
} catch(NotServingRegionException e) {
if(tries == numRetries - 1) {
@@ -1079,9 +1053,7 @@
if(values != null) {
ArrayList<byte[]> bytes = new ArrayList<byte[]>();
for(int i = 0 ; i < values.length; i++) {
- byte[] value = new byte[values[i].getSize()];
- System.arraycopy(values[i].get(), 0, value, 0, value.length);
- bytes.add(value);
+ bytes.add(values[i]);
}
return bytes.toArray(new byte[values.length][]);
}
@@ -1118,9 +1090,7 @@
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
if(value != null && value.length != 0) {
for(int i = 0; i < value.length; i++) {
- byte[] bytes = new byte[value[i].getData().getSize()];
- System.arraycopy(value[i].getData().get(), 0, bytes, 0, bytes.length);
- results.put(value[i].getKey().getColumn(), bytes);
+ results.put(value[i].getKey().getColumn(), value[i].getData());
}
}
return results;
@@ -1242,7 +1212,7 @@
public void put(long lockid, Text column, byte val[]) throws IOException {
try {
this.currentServer.put(this.currentRegion, this.clientid, lockid, column,
- new BytesWritable(val));
+ val);
} catch(IOException e) {
try {
this.currentServer.abort(this.currentRegion, this.clientid, lockid);
@@ -1432,9 +1402,7 @@
key.setRow(values[i].getKey().getRow());
key.setVersion(values[i].getKey().getTimestamp());
key.setColumn(EMPTY_COLUMN);
- byte[] bytes = new byte[values[i].getData().getSize()];
- System.arraycopy(values[i].getData().get(), 0, bytes, 0, bytes.length);
- results.put(values[i].getKey().getColumn(), bytes);
+ results.put(values[i].getKey().getColumn(), values[i].getData());
}
}
return values == null ? false : values.length != 0;
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java Mon Jun 18 15:59:14 2007
@@ -15,7 +15,7 @@
*/
package org.apache.hadoop.hbase;
-import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.io.Text;
/**
@@ -106,11 +106,10 @@
static final String UTF8_ENCODING = "UTF-8";
/** Value stored for a deleted item */
- static final BytesWritable DELETE_BYTES =
- new BytesWritable("HBASE::DELETEVAL".getBytes());
+ static final ImmutableBytesWritable DELETE_BYTES =
+ new ImmutableBytesWritable("HBASE::DELETEVAL".getBytes());
/** Value written to HLog on a complete cache flush */
- static final BytesWritable COMPLETE_CACHEFLUSH =
- new BytesWritable("HBASE::CACHEFLUSH".getBytes());
-
+ static final ImmutableBytesWritable COMPLETE_CACHEFLUSH =
+ new ImmutableBytesWritable("HBASE::CACHEFLUSH".getBytes());
}
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HInternalScannerInterface.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HInternalScannerInterface.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HInternalScannerInterface.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HInternalScannerInterface.java Mon Jun 18 15:59:14 2007
@@ -18,27 +18,29 @@
import java.io.IOException;
import java.util.TreeMap;
-import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.Text;
/**
* Internally, we need to be able to determine if the scanner is doing wildcard
* column matches (when only a column family is specified or if a column regex
- * is specified) or if multiple members of the same column family were specified.
- *
- * If so, we need to ignore the timestamp to ensure that we get all the family
- * members, as they may have been last updated at different times.
- *
+ * is specified) or if multiple members of the same column family were
+ * specified. If so, we need to ignore the timestamp to ensure that we get all
+ * the family members, as they may have been last updated at different times.
* This interface exposes two APIs for querying the scanner.
*/
public interface HInternalScannerInterface {
- public boolean next(HStoreKey key, TreeMap<Text, BytesWritable> results) throws IOException;
+ public boolean next(HStoreKey key, TreeMap<Text, byte []> results)
+ throws IOException;
+
+ /**
+ * Close the scanner.
+ */
public void close();
+
/** Returns true if the scanner is matching a column family or regex */
public boolean isWildcardScanner();
/** Returns true if the scanner is matching multiple column family members */
public boolean isMultipleMatchScanner();
-
-}
+}
\ No newline at end of file
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java Mon Jun 18 15:59:14 2007
@@ -78,7 +78,7 @@
long filenum = 0;
transient int numEntries = 0;
- Integer rollLock = 0;
+ Integer rollLock = new Integer(0);
/**
* Split up a bunch of log files, that are no longer being written to,
@@ -205,6 +205,7 @@
try {
wait();
} catch (InterruptedException ie) {
+ // continue;
}
}
@@ -282,8 +283,8 @@
* 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));
+ Path computeFilename(final long fn) {
+ return new Path(dir, HLOG_DATFILE + String.format("%1$03d", fn));
}
/**
@@ -333,7 +334,7 @@
* @throws IOException
*/
synchronized void append(Text regionName, Text tableName, Text row,
- TreeMap<Text, BytesWritable> columns, long timestamp)
+ TreeMap<Text, byte []> columns, long timestamp)
throws IOException {
if(closed) {
throw new IOException("Cannot append; log is closed");
@@ -350,7 +351,7 @@
}
int counter = 0;
- for (Map.Entry<Text, BytesWritable> es: columns.entrySet()) {
+ for (Map.Entry<Text, byte []> es: columns.entrySet()) {
HLogKey logKey =
new HLogKey(regionName, tableName, row, seqNum[counter++]);
HLogEdit logEdit = new HLogEdit(es.getKey(), es.getValue(), timestamp);
@@ -401,6 +402,7 @@
try {
wait();
} catch (InterruptedException ie) {
+ // continue
}
}
insideCacheFlush = true;
@@ -427,7 +429,7 @@
}
writer.append(new HLogKey(regionName, tableName, HLog.METAROW, logSeqId),
- new HLogEdit(HLog.METACOLUMN, COMPLETE_CACHEFLUSH,
+ new HLogEdit(HLog.METACOLUMN, COMPLETE_CACHEFLUSH.get(),
System.currentTimeMillis()));
numEntries++;
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java Mon Jun 18 15:59:14 2007
@@ -27,14 +27,15 @@
* This just indicates the column and value.
******************************************************************************/
public class HLogEdit implements Writable {
- Text column = new Text();
- BytesWritable val = new BytesWritable();
- long timestamp;
+ private Text column = new Text();
+ private byte [] val;
+ private long timestamp;
public HLogEdit() {
+ super();
}
- public HLogEdit(Text column, BytesWritable bval, long timestamp) {
+ public HLogEdit(Text column, byte [] bval, long timestamp) {
this.column.set(column);
this.val = bval;
this.timestamp = timestamp;
@@ -44,7 +45,7 @@
return this.column;
}
- public BytesWritable getVal() {
+ public byte [] getVal() {
return this.val;
}
@@ -55,7 +56,7 @@
@Override
public String toString() {
return getColumn().toString() + " " + this.getTimestamp() + " " +
- new String(getVal().get()).trim();
+ new String(getVal()).trim();
}
//////////////////////////////////////////////////////////////////////////////
@@ -64,13 +65,15 @@
public void write(DataOutput out) throws IOException {
this.column.write(out);
- this.val.write(out);
+ out.writeShort(this.val.length);
+ out.write(this.val);
out.writeLong(timestamp);
}
public void readFields(DataInput in) throws IOException {
this.column.readFields(in);
- this.val.readFields(in);
+ this.val = new byte[in.readShort()];
+ in.readFully(this.val);
this.timestamp = in.readLong();
}
}
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogKey.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogKey.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogKey.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogKey.java Mon Jun 18 15:59:14 2007
@@ -34,6 +34,7 @@
/** Create an empty key useful when deserializing */
public HLogKey() {
+ super();
}
/**
@@ -47,6 +48,7 @@
* @param logSeqNum - log sequence number
*/
public HLogKey(Text regionName, Text tablename, Text row, long logSeqNum) {
+ // TODO: Is this copy of the instances necessary? They are expensive.
this.regionName.set(regionName);
this.tablename.set(tablename);
this.row.set(row);
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java Mon Jun 18 15:59:14 2007
@@ -37,7 +37,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RPC;
@@ -179,10 +178,7 @@
}
for (int i = 0; i < values.length; i++) {
- byte[] bytes = new byte[values[i].getData().getSize()];
- System.arraycopy(values[i].getData().get(), 0, bytes, 0,
- bytes.length);
- results.put(values[i].getKey().getColumn(), bytes);
+ results.put(values[i].getKey().getColumn(), values[i].getData());
}
HRegionInfo info = HRegion.getRegionInfo(results);
@@ -272,7 +268,7 @@
// The current assignment is no good; load the region.
unassignedRegions.put(info.regionName, info);
- assignAttempts.put(info.regionName, 0L);
+ assignAttempts.put(info.regionName, Long.valueOf(0L));
}
}
}
@@ -333,7 +329,7 @@
private RootScanner rootScanner;
private Thread rootScannerThread;
- Integer rootScannerLock = 0;
+ Integer rootScannerLock = new Integer(0);
@SuppressWarnings("unchecked")
static class MetaRegion implements Comparable {
@@ -492,7 +488,7 @@
MetaScanner metaScanner;
private Thread metaScannerThread;
- Integer metaScannerLock = 0;
+ Integer metaScannerLock = new Integer(0);
/**
* The 'unassignedRegions' table maps from a region name to a HRegionInfo
@@ -642,7 +638,8 @@
this.pendingRegions =
Collections.synchronizedSortedSet(new TreeSet<Text>());
- this.assignAttempts.put(HGlobals.rootRegionInfo.regionName, 0L);
+ this.assignAttempts.put(HGlobals.rootRegionInfo.regionName,
+ Long.valueOf(0L));
this.killList =
Collections.synchronizedSortedMap(
@@ -655,9 +652,7 @@
Collections.synchronizedSortedSet(new TreeSet<Text>());
// We're almost open for business
-
this.closed = false;
-
LOG.info("HMaster initialized on " + this.address.toString());
}
@@ -815,7 +810,9 @@
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.HMasterRegionInterface#regionServerStartup(org.apache.hadoop.hbase.HServerInfo)
*/
- public void regionServerStartup(HServerInfo serverInfo) throws IOException {
+ @SuppressWarnings("unused")
+ public void regionServerStartup(HServerInfo serverInfo)
+ throws IOException {
String s = serverInfo.getServerAddress().toString().trim();
HServerInfo storedInfo = null;
LOG.info("received start message from: " + s);
@@ -834,11 +831,15 @@
// Either way, record the new server
serversToServerInfo.put(s, serverInfo);
if(!closed) {
- Text serverLabel = new Text(s);
+ long serverLabel = getServerLabel(s);
LOG.debug("Created lease for " + serverLabel);
serverLeases.createLease(serverLabel, serverLabel, new ServerExpirer(s));
}
}
+
+ private long getServerLabel(final String s) {
+ return s.hashCode();
+ }
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.HMasterRegionInterface#regionServerReport(org.apache.hadoop.hbase.HServerInfo, org.apache.hadoop.hbase.HMsg[])
@@ -846,7 +847,7 @@
public HMsg[] regionServerReport(HServerInfo serverInfo, HMsg msgs[])
throws IOException {
String s = serverInfo.getServerAddress().toString().trim();
- Text serverLabel = new Text(s);
+ long serverLabel = getServerLabel(s);
if (closed) {
// Cancel the server's lease
@@ -874,7 +875,7 @@
allMetaRegionsScanned = false;
}
unassignedRegions.put(info.regionName, info);
- assignAttempts.put(info.regionName, 0L);
+ assignAttempts.put(info.regionName, Long.valueOf(0L));
}
// We don't need to return anything to the server because it isn't
@@ -934,7 +935,8 @@
}
/** cancel a server's lease */
- private void cancelLease(String serverName, Text serverLabel) throws IOException {
+ private void cancelLease(final String serverName, final long serverLabel)
+ throws IOException {
if (serversToServerInfo.remove(serverName) != null) {
// Only cancel lease once.
// This method can be called a couple of times during shutdown.
@@ -1035,7 +1037,7 @@
if(region.regionName.compareTo(HGlobals.rootRegionInfo.regionName) == 0) { // Root region
rootRegionLocation = null;
unassignedRegions.put(region.regionName, region);
- assignAttempts.put(region.regionName, 0L);
+ assignAttempts.put(region.regionName, Long.valueOf(0L));
} else {
boolean reassignRegion = true;
@@ -1115,7 +1117,7 @@
returnMsgs.add(new HMsg(HMsg.MSG_REGION_OPEN, regionInfo));
- assignAttempts.put(curRegionName, now);
+ assignAttempts.put(curRegionName, Long.valueOf(now));
counter++;
}
@@ -1214,7 +1216,6 @@
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
Text row = null;
- byte[] bytes = null;
for(int i = 0; i < values.length; i++) {
if(row == null) {
row = values[i].getKey().getRow();
@@ -1225,12 +1226,10 @@
+ row + ", currentRow=" + values[i].getKey().getRow());
}
}
- bytes = new byte[values[i].getData().getSize()];
- System.arraycopy(values[i].getData().get(), 0, bytes, 0, bytes.length);
- results.put(values[i].getKey().getColumn(), bytes);
+ results.put(values[i].getKey().getColumn(), values[i].getData());
}
- bytes = results.get(COL_SERVER);
+ byte [] bytes = results.get(COL_SERVER);
String serverName = null;
if(bytes == null || bytes.length == 0) {
// No server
@@ -1335,21 +1334,18 @@
}
// Remove server from root/meta entries
-
for(int i = 0; i < toDoList.size(); i++) {
ToDoEntry e = toDoList.get(i);
long lockid = server.startUpdate(regionName, clientId, e.row);
if(e.deleteRegion) {
server.delete(regionName, clientId, lockid, COL_REGIONINFO);
-
} else if(e.regionOffline) {
e.info.offLine = true;
ByteArrayOutputStream byteValue = new ByteArrayOutputStream();
DataOutputStream s = new DataOutputStream(byteValue);
e.info.write(s);
-
server.put(regionName, clientId, lockid, COL_REGIONINFO,
- new BytesWritable(byteValue.toByteArray()));
+ byteValue.toByteArray());
}
server.delete(regionName, clientId, lockid, COL_SERVER);
server.delete(regionName, clientId, lockid, COL_STARTCODE);
@@ -1363,7 +1359,7 @@
HRegionInfo regionInfo = e.getValue();
unassignedRegions.put(region, regionInfo);
- assignAttempts.put(region, 0L);
+ assignAttempts.put(region, Long.valueOf(0L));
}
}
@@ -1384,7 +1380,8 @@
rootRegionLocation = null;
unassignedRegions.put(HGlobals.rootRegionInfo.regionName,
HGlobals.rootRegionInfo);
- assignAttempts.put(HGlobals.rootRegionInfo.regionName, 0L);
+ assignAttempts.put(HGlobals.rootRegionInfo.regionName,
+ Long.valueOf(0L));
}
// Scan the ROOT region
@@ -1525,7 +1522,7 @@
regionInfo.write(s);
server.put(metaRegionName, clientId, lockid, COL_REGIONINFO,
- new BytesWritable(byteValue.toByteArray()));
+ byteValue.toByteArray());
}
server.delete(metaRegionName, clientId, lockid, COL_SERVER);
server.delete(metaRegionName, clientId, lockid, COL_STARTCODE);
@@ -1546,7 +1543,7 @@
}
unassignedRegions.put(regionInfo.regionName, regionInfo);
- assignAttempts.put(regionInfo.regionName, 0L);
+ assignAttempts.put(regionInfo.regionName, Long.valueOf(0L));
} else if(deleteRegion) {
try {
@@ -1569,36 +1566,27 @@
private class PendingOpenReport extends PendingOperation {
private boolean rootRegion;
private Text regionName;
- private BytesWritable serverAddress;
- private BytesWritable startCode;
+ private byte [] serverAddress;
+ private byte [] startCode;
PendingOpenReport(HServerInfo info, HRegionInfo region) {
- if(region.tableDesc.getName().equals(META_TABLE_NAME)) {
-
+ if (region.tableDesc.getName().equals(META_TABLE_NAME)) {
// The region which just came on-line is a META region.
// We need to look in the ROOT region for its information.
-
this.rootRegion = true;
-
} else {
-
// Just an ordinary region. Look for it in the META table.
-
this.rootRegion = false;
}
this.regionName = region.regionName;
-
try {
- this.serverAddress = new BytesWritable(
- info.getServerAddress().toString().getBytes(UTF8_ENCODING));
-
- this.startCode = new BytesWritable(
- String.valueOf(info.getStartCode()).getBytes(UTF8_ENCODING));
-
+ this.serverAddress = info.getServerAddress().toString().
+ getBytes(UTF8_ENCODING);
+ this.startCode = String.valueOf(info.getStartCode()).
+ getBytes(UTF8_ENCODING);
} catch(UnsupportedEncodingException e) {
LOG.error(e);
}
-
}
@Override
@@ -1614,7 +1602,7 @@
if(LOG.isDebugEnabled()) {
LOG.debug(regionName + " open on "
- + new String(serverAddress.get(), UTF8_ENCODING));
+ + new String(this.serverAddress, UTF8_ENCODING));
}
// Register the newly-available Region's location.
@@ -1708,33 +1696,25 @@
}
// 1. Check to see if table already exists
-
- MetaRegion m = null;
- if(knownMetaRegions.containsKey(newRegion.regionName)) {
- m = knownMetaRegions.get(newRegion.regionName);
-
- } else {
- m = knownMetaRegions.get(
+ MetaRegion m = (knownMetaRegions.containsKey(newRegion.regionName))?
+ knownMetaRegions.get(newRegion.regionName):
+ knownMetaRegions.get(
knownMetaRegions.headMap(newRegion.regionName).lastKey());
- }
Text metaRegionName = m.regionName;
HRegionInterface server = client.getHRegionConnection(m.server);
-
-
- BytesWritable bytes = server.get(metaRegionName, desc.getName(), COL_REGIONINFO);
- if(bytes != null && bytes.getSize() != 0) {
- byte[] infoBytes = bytes.get();
+ byte [] infoBytes =
+ server.get(metaRegionName, desc.getName(), COL_REGIONINFO);
+ if (infoBytes != null && infoBytes.length != 0) {
DataInputBuffer inbuf = new DataInputBuffer();
inbuf.reset(infoBytes, infoBytes.length);
HRegionInfo info = new HRegionInfo();
info.readFields(inbuf);
- if(info.tableDesc.getName().compareTo(desc.getName()) == 0) {
+ if (info.tableDesc.getName().compareTo(desc.getName()) == 0) {
throw new IOException("table already exists");
}
}
// 2. Create the HRegion
-
HRegion r = HRegion.createHRegion(newRegion.regionId, desc, this.dir,
this.conf);
@@ -1748,8 +1728,8 @@
long clientId = rand.nextLong();
long lockid = server.startUpdate(metaRegionName, clientId, regionName);
- server.put(metaRegionName, clientId, lockid, COL_REGIONINFO,
- new BytesWritable(byteValue.toByteArray()));
+ server.put(metaRegionName, clientId, lockid, COL_REGIONINFO,
+ byteValue.toByteArray());
server.commit(metaRegionName, clientId, lockid);
// 4. Close the new region to flush it to disk
@@ -1759,7 +1739,7 @@
// 5. Get it assigned to a server
unassignedRegions.put(regionName, info);
- assignAttempts.put(regionName, 0L);
+ assignAttempts.put(regionName, Long.valueOf(0L));
break;
} catch(NotServingRegionException e) {
@@ -1887,30 +1867,26 @@
}
boolean haveRegionInfo = false;
for(int i = 0; i < values.length; i++) {
- bytes = new byte[values[i].getData().getSize()];
- if(bytes.length == 0) {
+ if(values[i].getData().length == 0) {
break;
}
- System.arraycopy(values[i].getData().get(), 0, bytes, 0, bytes.length);
-
Text column = values[i].getKey().getColumn();
if(column.equals(COL_REGIONINFO)) {
haveRegionInfo = true;
- inbuf.reset(bytes, bytes.length);
+ inbuf.reset(values[i].getData(),
+ values[i].getData().length);
info.readFields(inbuf);
-
} else if(column.equals(COL_SERVER)) {
try {
- serverName = new String(bytes, UTF8_ENCODING);
-
+ serverName =
+ new String(values[i].getData(), UTF8_ENCODING);
} catch(UnsupportedEncodingException e) {
assert(false);
}
-
} else if(column.equals(COL_STARTCODE)) {
try {
- startCode = Long.valueOf(new String(bytes, UTF8_ENCODING));
-
+ startCode = Long.valueOf(new String(values[i].getData(),
+ UTF8_ENCODING)).longValue();
} catch(UnsupportedEncodingException e) {
assert(false);
}
@@ -2115,8 +2091,7 @@
i.write(s);
server.put(regionName, clientId, lockid, COL_REGIONINFO,
- new BytesWritable(byteValue.toByteArray()));
-
+ byteValue.toByteArray());
}
}
@@ -2180,29 +2155,24 @@
}
protected void updateRegionInfo(HRegionInterface server, Text regionName,
- HRegionInfo i) throws IOException {
-
+ HRegionInfo i)
+ throws IOException {
ByteArrayOutputStream byteValue = new ByteArrayOutputStream();
DataOutputStream s = new DataOutputStream(byteValue);
i.write(s);
-
long lockid = -1L;
long clientId = rand.nextLong();
try {
lockid = server.startUpdate(regionName, clientId, i.regionName);
server.put(regionName, clientId, lockid, COL_REGIONINFO,
- new BytesWritable(byteValue.toByteArray()));
-
+ byteValue.toByteArray());
server.commit(regionName, clientId, lockid);
lockid = -1L;
-
if(LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: " + i.regionName);
}
-
} catch(NotServingRegionException e) {
throw e;
-
} catch(IOException e) {
LOG.error("column update failed in row: " + i.regionName);
LOG.error(e);
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterInterface.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterInterface.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterInterface.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterInterface.java Mon Jun 18 15:59:14 2007
@@ -57,4 +57,4 @@
//////////////////////////////////////////////////////////////////////////////
public HServerAddress findRootRegion();
-}
+}
\ No newline at end of file
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterRegionInterface.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterRegionInterface.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterRegionInterface.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterRegionInterface.java Mon Jun 18 15:59:14 2007
@@ -25,5 +25,6 @@
public interface HMasterRegionInterface extends VersionedProtocol {
public static final long versionID = 1L;
public void regionServerStartup(HServerInfo info) throws IOException;
- public HMsg[] regionServerReport(HServerInfo info, HMsg msgs[]) throws IOException;
+ public HMsg[] regionServerReport(HServerInfo info, HMsg msgs[])
+ throws IOException;
}
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java Mon Jun 18 15:59:14 2007
@@ -16,7 +16,9 @@
package org.apache.hadoop.hbase;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.Iterator;
+import java.util.List;
import java.util.Map;
import java.util.SortedMap;
import java.util.TreeMap;
@@ -24,7 +26,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.io.Text;
/**
@@ -34,13 +36,13 @@
public class HMemcache {
private final Log LOG = LogFactory.getLog(this.getClass().getName());
- TreeMap<HStoreKey, BytesWritable> memcache
- = new TreeMap<HStoreKey, BytesWritable>();
+ TreeMap<HStoreKey, byte []> memcache =
+ new TreeMap<HStoreKey, byte []>();
- Vector<TreeMap<HStoreKey, BytesWritable>> history
- = new Vector<TreeMap<HStoreKey, BytesWritable>>();
+ Vector<TreeMap<HStoreKey, byte []>> history
+ = new Vector<TreeMap<HStoreKey, byte []>>();
- TreeMap<HStoreKey, BytesWritable> snapshot = null;
+ TreeMap<HStoreKey, byte []> snapshot = null;
final HLocking lock = new HLocking();
@@ -49,7 +51,7 @@
}
public static class Snapshot {
- public TreeMap<HStoreKey, BytesWritable> memcacheSnapshot = null;
+ public TreeMap<HStoreKey, byte []> memcacheSnapshot = null;
public long sequenceId = 0;
public Snapshot() {
@@ -92,7 +94,7 @@
retval.memcacheSnapshot = memcache;
this.snapshot = memcache;
history.add(memcache);
- memcache = new TreeMap<HStoreKey, BytesWritable>();
+ memcache = new TreeMap<HStoreKey, byte []>();
retval.sequenceId = log.startCacheFlush();
if(LOG.isDebugEnabled()) {
@@ -122,21 +124,18 @@
LOG.debug("deleting snapshot");
}
- for(Iterator<TreeMap<HStoreKey, BytesWritable>> it = history.iterator();
+ for(Iterator<TreeMap<HStoreKey, byte []>> it = history.iterator();
it.hasNext(); ) {
-
- TreeMap<HStoreKey, BytesWritable> cur = it.next();
- if(snapshot == cur) {
+ TreeMap<HStoreKey, byte []> cur = it.next();
+ if (snapshot == cur) {
it.remove();
break;
}
}
this.snapshot = null;
-
if(LOG.isDebugEnabled()) {
LOG.debug("snapshot deleted");
}
-
} finally {
this.lock.releaseWriteLock();
}
@@ -144,14 +143,16 @@
/**
* Store a value.
- *
* Operation uses a write lock.
+ * @param row
+ * @param columns
+ * @param timestamp
*/
- public void add(final Text row, final TreeMap<Text, BytesWritable> columns,
+ public void add(final Text row, final TreeMap<Text, byte []> columns,
final long timestamp) {
this.lock.obtainWriteLock();
try {
- for (Map.Entry<Text, BytesWritable> es: columns.entrySet()) {
+ for (Map.Entry<Text, byte []> es: columns.entrySet()) {
HStoreKey key = new HStoreKey(row, es.getKey(), timestamp);
memcache.put(key, es.getValue());
}
@@ -162,45 +163,47 @@
/**
* Look back through all the backlog TreeMaps to find the target.
- *
- * We only need a readlock here.
+ * @param key
+ * @param numVersions
+ * @return An array of byte arrays orderded by timestamp.
*/
- public BytesWritable[] get(HStoreKey key, int numVersions) {
- Vector<BytesWritable> results = new Vector<BytesWritable>();
+ public byte [][] get(final HStoreKey key, final int numVersions) {
+ List<byte []> results = new ArrayList<byte[]>();
this.lock.obtainReadLock();
try {
- Vector<BytesWritable> result = get(memcache, key, numVersions-results.size());
+ ArrayList<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) {
+ 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());
+ result = get(history.elementAt(i), key, numVersions - results.size());
results.addAll(results.size(), result);
}
-
return (results.size() == 0)?
- null: results.toArray(new BytesWritable[results.size()]);
+ null: ImmutableBytesWritable.toArray(results);
} finally {
this.lock.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[]).
+ * @param key
+ * @return All columns for given key.
*/
- public TreeMap<Text, BytesWritable> getFull(HStoreKey key) {
- TreeMap<Text, BytesWritable> results = new TreeMap<Text, BytesWritable>();
+ public TreeMap<Text, byte []> getFull(HStoreKey key) {
+ TreeMap<Text, byte []> results = new TreeMap<Text, byte []>();
this.lock.obtainReadLock();
try {
internalGetFull(memcache, key, results);
- for(int i = history.size()-1; i >= 0; i--) {
- TreeMap<HStoreKey, BytesWritable> cur = history.elementAt(i);
+ for (int i = history.size()-1; i >= 0; i--) {
+ TreeMap<HStoreKey, byte []> cur = history.elementAt(i);
internalGetFull(cur, key, results);
}
return results;
@@ -210,17 +213,16 @@
}
}
- void internalGetFull(TreeMap<HStoreKey, BytesWritable> map, HStoreKey key,
- TreeMap<Text, BytesWritable> results) {
- SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(key);
- for (Map.Entry<HStoreKey, BytesWritable> es: tailMap.entrySet()) {
+ void internalGetFull(TreeMap<HStoreKey, byte []> map, HStoreKey key,
+ TreeMap<Text, byte []> results) {
+ SortedMap<HStoreKey, byte []> tailMap = map.tailMap(key);
+ for (Map.Entry<HStoreKey, byte []> es: tailMap.entrySet()) {
HStoreKey itKey = es.getKey();
Text itCol = itKey.getColumn();
if (results.get(itCol) == null
&& key.matchesWithoutColumn(itKey)) {
- BytesWritable val = tailMap.get(itKey);
+ byte [] val = tailMap.get(itKey);
results.put(itCol, val);
-
} else if (key.getRow().compareTo(itKey.getRow()) > 0) {
break;
}
@@ -235,18 +237,23 @@
*
* TODO - This is kinda slow. We need a data structure that allows for
* proximity-searches, not just precise-matches.
- */
- Vector<BytesWritable> get(TreeMap<HStoreKey, BytesWritable> map, HStoreKey key, int numVersions) {
- Vector<BytesWritable> result = new Vector<BytesWritable>();
- HStoreKey curKey = new HStoreKey(key.getRow(), key.getColumn(), key.getTimestamp());
- SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(curKey);
- for (Map.Entry<HStoreKey, BytesWritable> es: tailMap.entrySet()) {
+ * @param map
+ * @param key
+ * @param numVersions
+ * @return Ordered list of items found in passed <code>map</code>
+ */
+ ArrayList<byte []> get(final TreeMap<HStoreKey, byte []> map,
+ final HStoreKey key, final int numVersions) {
+ ArrayList<byte []> result = new ArrayList<byte []>();
+ HStoreKey curKey =
+ new HStoreKey(key.getRow(), key.getColumn(), key.getTimestamp());
+ SortedMap<HStoreKey, byte []> tailMap = map.tailMap(curKey);
+ for (Map.Entry<HStoreKey, byte []> es: tailMap.entrySet()) {
HStoreKey itKey = es.getKey();
if (itKey.matchesRowCol(curKey)) {
result.add(tailMap.get(itKey));
curKey.setVersion(itKey.getTimestamp() - 1);
}
-
if (numVersions > 0 && result.size() >= numVersions) {
break;
}
@@ -269,8 +276,8 @@
//////////////////////////////////////////////////////////////////////////////
class HMemcacheScanner extends HAbstractScanner {
- TreeMap<HStoreKey, BytesWritable> backingMaps[];
- Iterator<HStoreKey> keyIterators[];
+ final TreeMap<HStoreKey, byte []> backingMaps[];
+ final Iterator<HStoreKey> keyIterators[];
@SuppressWarnings("unchecked")
public HMemcacheScanner(long timestamp, Text targetCols[], Text firstRow)
@@ -292,7 +299,7 @@
this.keyIterators = new Iterator[backingMaps.length];
this.keys = new HStoreKey[backingMaps.length];
- this.vals = new BytesWritable[backingMaps.length];
+ this.vals = new byte[backingMaps.length][];
// Generate list of iterators
HStoreKey firstKey = new HStoreKey(firstRow);
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java Mon Jun 18 15:59:14 2007
@@ -27,7 +27,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.Text;
@@ -39,7 +38,9 @@
static final Log LOG = LogFactory.getLog(HMerge.class);
static final Text[] META_COLS = {COL_REGIONINFO};
- private HMerge() {} // Not instantiable
+ private HMerge() {
+ // Not instantiable
+ }
/**
* Scans the table and merges two adjacent regions if they are small. This
@@ -317,7 +318,7 @@
private static class OfflineMerger extends Merger {
private Path dir;
private TreeSet<HRegionInfo> metaRegions;
- private TreeMap<Text, BytesWritable> results;
+ private TreeMap<Text, byte []> results;
OfflineMerger(Configuration conf, FileSystem fs, Text tableName)
throws IOException {
@@ -325,7 +326,7 @@
super(conf, fs, tableName);
this.dir = new Path(conf.get(HBASE_DIR, DEFAULT_HBASE_DIR));
this.metaRegions = new TreeSet<HRegionInfo>();
- this.results = new TreeMap<Text, BytesWritable>();
+ this.results = new TreeMap<Text, byte []>();
// Scan root region to find all the meta regions
@@ -337,10 +338,8 @@
try {
while(rootScanner.next(key, results)) {
- for(BytesWritable b: results.values()) {
- byte[] bytes = new byte[b.getSize()];
- System.arraycopy(b.get(), 0, bytes, 0, bytes.length);
- in.reset(bytes, bytes.length);
+ for(byte [] b: results.values()) {
+ in.reset(b, b.length);
info.readFields(in);
metaRegions.add(info);
results.clear();
@@ -405,8 +404,7 @@
long lockid = -1L;
try {
lockid = root.startUpdate(newRegion.getRegionName());
- root.put(lockid, COL_REGIONINFO,
- new BytesWritable(byteValue.toByteArray()));
+ root.put(lockid, COL_REGIONINFO, byteValue.toByteArray());
root.commit(lockid);
lockid = -1L;
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java Mon Jun 18 15:59:14 2007
@@ -238,11 +238,11 @@
// Members
//////////////////////////////////////////////////////////////////////////////
- TreeMap<Text, Long> rowsToLocks = new TreeMap<Text, Long>();
- TreeMap<Long, Text> locksToRows = new TreeMap<Long, Text>();
- TreeMap<Text, HStore> stores = new TreeMap<Text, HStore>();
- Map<Long, TreeMap<Text, BytesWritable>> targetColumns
- = new HashMap<Long, TreeMap<Text, BytesWritable>>();
+ Map<Text, Long> rowsToLocks = new HashMap<Text, Long>();
+ Map<Long, Text> locksToRows = new HashMap<Long, Text>();
+ Map<Text, HStore> stores = new HashMap<Text, HStore>();
+ Map<Long, TreeMap<Text, byte []>> targetColumns
+ = new HashMap<Long, TreeMap<Text, byte []>>();
HMemcache memcache;
@@ -826,7 +826,7 @@
}
HMemcache.Snapshot retval = memcache.snapshotMemcacheForLog(log);
- TreeMap<HStoreKey, BytesWritable> memcacheSnapshot = retval.memcacheSnapshot;
+ TreeMap<HStoreKey, byte []> memcacheSnapshot = retval.memcacheSnapshot;
if(memcacheSnapshot == null) {
for(HStore hstore: stores.values()) {
Vector<HStoreFile> hstoreFiles = hstore.getAllMapFiles();
@@ -885,31 +885,28 @@
//////////////////////////////////////////////////////////////////////////////
/** Fetch a single data item. */
- BytesWritable get(Text row, Text column) throws IOException {
- BytesWritable[] results = get(row, column, Long.MAX_VALUE, 1);
- return (results == null)? null: results[0];
+ byte [] get(Text row, Text column) throws IOException {
+ byte [][] results = get(row, column, Long.MAX_VALUE, 1);
+ return (results == null || results.length == 0)? null: results[0];
}
/** Fetch multiple versions of a single data item */
- BytesWritable[] get(Text row, Text column, int numVersions) throws IOException {
+ byte [][] get(Text row, Text column, int numVersions) throws IOException {
return get(row, column, Long.MAX_VALUE, numVersions);
}
/** Fetch multiple versions of a single data item, with timestamp. */
- BytesWritable[] get(Text row, Text column, long timestamp, int numVersions)
- throws IOException {
-
+ byte [][] get(Text row, Text column, long timestamp, int numVersions)
+ throws IOException {
if(writestate.closed) {
throw new IOException("HRegion is closed.");
}
// Make sure this is a valid row and valid column
-
checkRow(row);
checkColumn(column);
// Obtain the row-lock
-
obtainRowLock(row);
try {
// Obtain the -col results
@@ -921,13 +918,12 @@
}
/** Private implementation: get the value for the indicated HStoreKey */
- private BytesWritable[] get(HStoreKey key, int numVersions) throws IOException {
+ private byte [][] get(HStoreKey key, int numVersions) throws IOException {
lock.obtainReadLock();
try {
// Check the memcache
-
- BytesWritable[] result = memcache.get(key, numVersions);
+ byte [][] result = memcache.get(key, numVersions);
if(result != null) {
return result;
}
@@ -957,19 +953,17 @@
* determine which column groups are useful for that row. That would let us
* avoid a bunch of disk activity.
*/
- TreeMap<Text, BytesWritable> getFull(Text row) throws IOException {
+ TreeMap<Text, byte []> getFull(Text row) throws IOException {
HStoreKey key = new HStoreKey(row, System.currentTimeMillis());
lock.obtainReadLock();
try {
- TreeMap<Text, BytesWritable> memResult = memcache.getFull(key);
- for(Iterator<Text> it = stores.keySet().iterator(); it.hasNext(); ) {
- Text colFamily = it.next();
+ TreeMap<Text, byte []> memResult = memcache.getFull(key);
+ for (Text colFamily: stores.keySet()) {
HStore targetStore = stores.get(colFamily);
targetStore.getFull(key, memResult);
}
return memResult;
-
} finally {
lock.releaseReadLock();
}
@@ -1035,9 +1029,8 @@
* This method really just tests the input, then calls an internal localput()
* method.
*/
- void put(long lockid, Text targetCol, BytesWritable val) throws IOException {
- if(val.getSize() == DELETE_BYTES.getSize()
- && val.compareTo(DELETE_BYTES) == 0) {
+ void put(long lockid, Text targetCol, byte [] val) throws IOException {
+ if (DELETE_BYTES.compareTo(val) == 0) {
throw new IOException("Cannot insert value: " + val);
}
localput(lockid, targetCol, val);
@@ -1047,7 +1040,7 @@
* Delete a value or write a value. This is a just a convenience method for put().
*/
void delete(long lockid, Text targetCol) throws IOException {
- localput(lockid, targetCol, DELETE_BYTES);
+ localput(lockid, targetCol, DELETE_BYTES.get());
}
/**
@@ -1063,7 +1056,7 @@
* @throws IOException
*/
void localput(final long lockid, final Text targetCol,
- final BytesWritable val)
+ final byte [] val)
throws IOException {
checkColumn(targetCol);
@@ -1083,9 +1076,9 @@
lockid + " unexpected aborted by another thread");
}
- TreeMap<Text, BytesWritable> targets = this.targetColumns.get(lockid);
+ TreeMap<Text, byte []> targets = this.targetColumns.get(lockid);
if (targets == null) {
- targets = new TreeMap<Text, BytesWritable>();
+ targets = new TreeMap<Text, byte []>();
this.targetColumns.put(lockid, targets);
}
targets.put(targetCol, val);
@@ -1144,8 +1137,7 @@
synchronized(row) {
// Add updates to the log and add values to the memcache.
long commitTimestamp = System.currentTimeMillis();
- TreeMap<Text, BytesWritable> columns =
- this.targetColumns.get(lockid);
+ TreeMap<Text, byte []> columns = this.targetColumns.get(lockid);
if (columns != null && columns.size() > 0) {
log.append(regionInfo.regionName, regionInfo.tableDesc.getName(),
row, columns, commitTimestamp);
@@ -1267,7 +1259,7 @@
*/
private static class HScanner implements HInternalScannerInterface {
private HInternalScannerInterface[] scanners;
- private TreeMap<Text, BytesWritable>[] resultSets;
+ private TreeMap<Text, byte []>[] resultSets;
private HStoreKey[] keys;
private boolean wildcardMatch;
private boolean multipleMatchers;
@@ -1323,7 +1315,7 @@
}
for(int i = 0; i < scanners.length; i++) {
keys[i] = new HStoreKey();
- resultSets[i] = new TreeMap<Text, BytesWritable>();
+ resultSets[i] = new TreeMap<Text, byte []>();
if(scanners[i] != null && !scanners[i].next(keys[i], resultSets[i])) {
closeScanner(i);
}
@@ -1351,7 +1343,7 @@
*
* @see org.apache.hadoop.hbase.HInternalScannerInterface#next(org.apache.hadoop.hbase.HStoreKey, java.util.TreeMap)
*/
- public boolean next(HStoreKey key, TreeMap<Text, BytesWritable> results)
+ public boolean next(HStoreKey key, TreeMap<Text, byte []> results)
throws IOException {
// Find the lowest-possible key.
Text chosenRow = null;
@@ -1393,7 +1385,7 @@
// values with older ones. So now we only insert
// a result if the map does not contain the key.
- for(Map.Entry<Text, BytesWritable> e: resultSets[i].entrySet()) {
+ for(Map.Entry<Text, byte []> e: resultSets[i].entrySet()) {
if(!results.containsKey(e.getKey())) {
results.put(e.getKey(), e.getValue());
insertedItem = true;
@@ -1504,7 +1496,7 @@
ByteArrayOutputStream bytes = new ByteArrayOutputStream();
DataOutputStream s = new DataOutputStream(bytes);
r.getRegionInfo().write(s);
- meta.put(writeid, COL_REGIONINFO, new BytesWritable(bytes.toByteArray()));
+ meta.put(writeid, COL_REGIONINFO, bytes.toByteArray());
meta.commit(writeid);
}
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java Mon Jun 18 15:59:14 2007
@@ -15,12 +15,11 @@
*/
package org.apache.hadoop.hbase;
-import org.apache.hadoop.io.BytesWritable;
+import java.io.IOException;
+
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.VersionedProtocol;
-import java.io.*;
-
/*******************************************************************************
* Clients interact with HRegionServers using
* a handle to the HRegionInterface.
@@ -36,57 +35,62 @@
* @return - HRegionInfo object for region
* @throws NotServingRegionException
*/
- public HRegionInfo getRegionInfo(final Text regionName) throws NotServingRegionException;
+ public HRegionInfo getRegionInfo(final Text regionName)
+ throws NotServingRegionException;
/**
* Retrieve a single value from the specified region for the specified row
* and column keys
*
- * @param regionName - name of region
- * @param row - row key
- * @param column - column key
- * @return - value for that region/row/column
+ * @param regionName name of region
+ * @param row row key
+ * @param column column key
+ * @return alue for that region/row/column
* @throws IOException
*/
- public BytesWritable get(final Text regionName, final Text row, final Text column) throws IOException;
+ public byte [] get(final Text regionName, final Text row, final Text column)
+ throws IOException;
/**
* Get the specified number of versions of the specified row and column
*
- * @param regionName - region name
- * @param row - row key
- * @param column - column key
- * @param numVersions - number of versions to return
- * @return - array of values
+ * @param regionName region name
+ * @param row row key
+ * @param column column key
+ * @param numVersions number of versions to return
+ * @return array of values
* @throws IOException
*/
- public BytesWritable[] get(final Text regionName, final Text row,
- final Text column, final int numVersions) throws IOException;
+ public byte [][] get(final Text regionName, final Text row,
+ final Text column, final int numVersions)
+ throws IOException;
/**
* Get the specified number of versions of the specified row and column with
* the specified timestamp.
*
- * @param regionName - region name
- * @param row - row key
- * @param column - column key
- * @param timestamp - timestamp
- * @param numVersions - number of versions to return
- * @return - array of values
+ * @param regionName region name
+ * @param row row key
+ * @param column column key
+ * @param timestamp timestamp
+ * @param numVersions number of versions to return
+ * @return array of values
* @throws IOException
*/
- public BytesWritable[] get(final Text regionName, final Text row, final Text column,
- final long timestamp, final int numVersions) throws IOException;
+ public byte [][] get(final Text regionName, final Text row,
+ final Text column, final long timestamp, final int numVersions)
+ throws IOException;
/**
* Get all the data for the specified row
*
- * @param regionName - region name
- * @param row - row key
- * @return - array of values
+ * @param regionName region name
+ * @param row row key
+ * @return array of values
* @throws IOException
*/
- public KeyedData[] getRow(final Text regionName, final Text row) throws IOException;
+ public KeyedData[] getRow(final Text regionName, final Text row)
+ throws IOException;
//////////////////////////////////////////////////////////////////////////////
// Start an atomic row insertion/update. No changes are committed until the
@@ -110,67 +114,72 @@
* The client can gain extra time with a call to renewLease().
* Start an atomic row insertion or update
*
- * @param regionName - region name
- * @param clientid - a unique value to identify the client
- * @param row - Name of row to start update against.
+ * @param regionName region name
+ * @param clientid a unique value to identify the client
+ * @param row Name of row to start update against.
* @return Row lockid.
* @throws IOException
*/
public long startUpdate(final Text regionName, final long clientid,
- final Text row) throws IOException;
+ final Text row)
+ throws IOException;
/**
* Change a value for the specified column
*
- * @param regionName - region name
- * @param clientid - a unique value to identify the client
- * @param lockid - lock id returned from startUpdate
- * @param column - column whose value is being set
- * @param val - new value for column
+ * @param regionName region name
+ * @param clientid a unique value to identify the client
+ * @param lockid lock id returned from startUpdate
+ * @param column column whose value is being set
+ * @param val new value for column
* @throws IOException
*/
public void put(final Text regionName, final long clientid, final long lockid,
- final Text column, final BytesWritable val) throws IOException;
+ final Text column, final byte [] val)
+ throws IOException;
/**
* Delete the value for a column
*
- * @param regionName - region name
- * @param clientid - a unique value to identify the client
- * @param lockid - lock id returned from startUpdate
- * @param column - name of column whose value is to be deleted
+ * @param regionName region name
+ * @param clientid a unique value to identify the client
+ * @param lockid lock id returned from startUpdate
+ * @param column name of column whose value is to be deleted
* @throws IOException
*/
- public void delete(final Text regionName, final long clientid, final long lockid,
- final Text column) throws IOException;
+ public void delete(final Text regionName, final long clientid,
+ final long lockid, final Text column)
+ throws IOException;
/**
* Abort a row mutation
*
- * @param regionName - region name
- * @param clientid - a unique value to identify the client
- * @param lockid - lock id returned from startUpdate
+ * @param regionName region name
+ * @param clientid a unique value to identify the client
+ * @param lockid lock id returned from startUpdate
* @throws IOException
*/
public void abort(final Text regionName, final long clientid,
- final long lockid) throws IOException;
+ final long lockid)
+ throws IOException;
/**
* Finalize a row mutation
*
- * @param regionName - region name
- * @param clientid - a unique value to identify the client
- * @param lockid - lock id returned from startUpdate
+ * @param regionName region name
+ * @param clientid a unique value to identify the client
+ * @param lockid lock id returned from startUpdate
* @throws IOException
*/
public void commit(final Text regionName, final long clientid,
- final long lockid) throws IOException;
+ final long lockid)
+ throws IOException;
/**
* Renew lease on update
*
- * @param lockid - lock id returned from startUpdate
- * @param clientid - a unique value to identify the client
+ * @param lockid lock id returned from startUpdate
+ * @param clientid a unique value to identify the client
* @throws IOException
*/
public void renewLease(long lockid, long clientid) throws IOException;
@@ -182,20 +191,21 @@
/**
* Opens a remote scanner.
*
- * @param regionName - name of region to scan
- * @param columns - columns to scan
- * @param startRow - starting row to scan
+ * @param regionName name of region to scan
+ * @param columns columns to scan
+ * @param startRow starting row to scan
*
- * @return scannerId - scanner identifier used in other calls
+ * @return scannerId scanner identifier used in other calls
* @throws IOException
*/
- public long openScanner(Text regionName, Text[] columns, Text startRow) throws IOException;
+ public long openScanner(Text regionName, Text[] columns, Text startRow)
+ throws IOException;
/**
* Get the next set of values
*
- * @param scannerId - clientId passed to openScanner
- * @return - array of values
+ * @param scannerId clientId passed to openScanner
+ * @return array of values
* @throws IOException
*/
public KeyedData[] next(long scannerId) throws IOException;
@@ -203,7 +213,7 @@
/**
* Close a scanner
*
- * @param scannerId - the scanner id returned by openScanner
+ * @param scannerId the scanner id returned by openScanner
* @throws IOException
*/
public void close(long scannerId) throws IOException;
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java Mon Jun 18 15:59:14 2007
@@ -33,7 +33,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.retry.RetryProxy;
import org.apache.hadoop.ipc.RPC;
@@ -396,8 +395,6 @@
Collections.synchronizedSortedMap(new TreeMap<Text, HRegion>());
this.outboundMsgs = new Vector<HMsg>();
- this.scanners =
- Collections.synchronizedMap(new TreeMap<Text, HInternalScannerInterface>());
// Config'ed params
this.numRetries = conf.getInt("hbase.client.retries.number", 2);
@@ -914,27 +911,26 @@
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.HRegionInterface#get(org.apache.hadoop.io.Text, org.apache.hadoop.io.Text, org.apache.hadoop.io.Text)
*/
- public BytesWritable get(final Text regionName, final Text row,
- final Text column) throws IOException {
-
+ public byte [] get(final Text regionName, final Text row,
+ final Text column)
+ throws IOException {
return getRegion(regionName).get(row, column);
}
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.HRegionInterface#get(org.apache.hadoop.io.Text, org.apache.hadoop.io.Text, org.apache.hadoop.io.Text, int)
*/
- public BytesWritable[] get(final Text regionName, final Text row,
- final Text column, final int numVersions) throws IOException {
-
+ public byte [][] get(final Text regionName, final Text row,
+ final Text column, final int numVersions)
+ throws IOException {
return getRegion(regionName).get(row, column, numVersions);
}
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.HRegionInterface#get(org.apache.hadoop.io.Text, org.apache.hadoop.io.Text, org.apache.hadoop.io.Text, long, int)
*/
- public BytesWritable[] get(final Text regionName, final Text row, final Text column,
+ public byte [][] get(final Text regionName, final Text row, final Text column,
final long timestamp, final int numVersions) throws IOException {
-
return getRegion(regionName).get(row, column, timestamp, numVersions);
}
@@ -943,10 +939,10 @@
*/
public KeyedData[] getRow(final Text regionName, final Text row) throws IOException {
HRegion region = getRegion(regionName);
- TreeMap<Text, BytesWritable> map = region.getFull(row);
+ TreeMap<Text, byte[]> map = region.getFull(row);
KeyedData result[] = new KeyedData[map.size()];
int counter = 0;
- for (Map.Entry<Text, BytesWritable> es: map.entrySet()) {
+ for (Map.Entry<Text, byte []> es: map.entrySet()) {
result[counter++] =
new KeyedData(new HStoreKey(row, es.getKey()), es.getValue());
}
@@ -957,30 +953,28 @@
* @see org.apache.hadoop.hbase.HRegionInterface#next(long)
*/
public KeyedData[] next(final long scannerId)
- throws IOException {
-
- Text scannerName = new Text(String.valueOf(scannerId));
+ throws IOException {
+ String scannerName = String.valueOf(scannerId);
HInternalScannerInterface s = scanners.get(scannerName);
if (s == null) {
throw new UnknownScannerException("Name: " + scannerName);
}
- leases.renewLease(scannerName, scannerName);
+ leases.renewLease(scannerId, scannerId);
// Collect values to be returned here
ArrayList<KeyedData> values = new ArrayList<KeyedData>();
- TreeMap<Text, BytesWritable> results = new TreeMap<Text, BytesWritable>();
+ TreeMap<Text, byte []> results = new TreeMap<Text, byte []>();
// Keep getting rows until we find one that has at least one non-deleted column value
HStoreKey key = new HStoreKey();
while (s.next(key, results)) {
- for(Map.Entry<Text, BytesWritable> e: results.entrySet()) {
+ for(Map.Entry<Text, byte []> e: results.entrySet()) {
HStoreKey k = new HStoreKey(key.getRow(), e.getKey(), key.getTimestamp());
- BytesWritable val = e.getValue();
- if(val.getSize() == DELETE_BYTES.getSize()
- && val.compareTo(DELETE_BYTES) == 0) {
+ byte [] val = e.getValue();
+ if (DELETE_BYTES.compareTo(val) == 0) {
// Column value is deleted. Don't return it.
if (LOG.isDebugEnabled()) {
LOG.debug("skipping deleted value for key: " + k.toString());
@@ -1011,10 +1005,8 @@
throws IOException {
HRegion region = getRegion(regionName);
long lockid = region.startUpdate(row);
- this.leases.createLease(new Text(String.valueOf(clientid)),
- new Text(String.valueOf(lockid)),
- new RegionListener(region, lockid));
-
+ this.leases.createLease(clientid, lockid,
+ new RegionListener(region, lockid));
return lockid;
}
@@ -1041,11 +1033,11 @@
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.HRegionInterface#put(org.apache.hadoop.io.Text, long, long, org.apache.hadoop.io.Text, org.apache.hadoop.io.BytesWritable)
*/
- public void put(Text regionName, long clientid, long lockid, Text column,
- BytesWritable val) throws IOException {
+ public void put(final Text regionName, final long clientid,
+ final long lockid, final Text column, final byte [] val)
+ throws IOException {
HRegion region = getRegion(regionName, true);
- leases.renewLease(new Text(String.valueOf(clientid)),
- new Text(String.valueOf(lockid)));
+ leases.renewLease(clientid, lockid);
region.put(lockid, column, val);
}
@@ -1053,10 +1045,9 @@
* @see org.apache.hadoop.hbase.HRegionInterface#delete(org.apache.hadoop.io.Text, long, long, org.apache.hadoop.io.Text)
*/
public void delete(Text regionName, long clientid, long lockid, Text column)
- throws IOException {
+ throws IOException {
HRegion region = getRegion(regionName);
- leases.renewLease(new Text(String.valueOf(clientid)),
- new Text(String.valueOf(lockid)));
+ leases.renewLease(clientid, lockid);
region.delete(lockid, column);
}
@@ -1064,10 +1055,9 @@
* @see org.apache.hadoop.hbase.HRegionInterface#abort(org.apache.hadoop.io.Text, long, long)
*/
public void abort(Text regionName, long clientid, long lockid)
- throws IOException {
+ throws IOException {
HRegion region = getRegion(regionName, true);
- leases.cancelLease(new Text(String.valueOf(clientid)),
- new Text(String.valueOf(lockid)));
+ leases.cancelLease(clientid, lockid);
region.abort(lockid);
}
@@ -1077,8 +1067,7 @@
public void commit(Text regionName, long clientid, long lockid)
throws IOException {
HRegion region = getRegion(regionName, true);
- leases.cancelLease(new Text(String.valueOf(clientid)),
- new Text(String.valueOf(lockid)));
+ leases.cancelLease(clientid, lockid);
region.commit(lockid);
}
@@ -1086,8 +1075,7 @@
* @see org.apache.hadoop.hbase.HRegionInterface#renewLease(long, long)
*/
public void renewLease(long lockid, long clientid) throws IOException {
- leases.renewLease(new Text(String.valueOf(clientid)),
- new Text(String.valueOf(lockid)));
+ leases.renewLease(clientid, lockid);
}
/**
@@ -1139,29 +1127,31 @@
// remote scanner interface
//////////////////////////////////////////////////////////////////////////////
- Map<Text, HInternalScannerInterface> scanners;
+ Map<String, HInternalScannerInterface> scanners =
+ Collections.synchronizedMap(new HashMap<String,
+ HInternalScannerInterface>());
/**
* Instantiated as a scanner lease.
* If the lease times out, the scanner is closed
*/
private class ScannerListener implements LeaseListener {
- private Text scannerName;
+ private final String scannerName;
- ScannerListener(Text scannerName) {
- this.scannerName = scannerName;
+ ScannerListener(final String n) {
+ this.scannerName = n;
}
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.LeaseListener#leaseExpired()
*/
public void leaseExpired() {
- LOG.info("Scanner " + scannerName + " lease expired");
+ LOG.info("Scanner " + this.scannerName + " lease expired");
HInternalScannerInterface s = null;
synchronized(scanners) {
- s = scanners.remove(scannerName);
+ s = scanners.remove(this.scannerName);
}
- if(s != null) {
+ if (s != null) {
s.close();
}
}
@@ -1177,11 +1167,11 @@
try {
HInternalScannerInterface s = r.getScanner(cols, firstRow);
scannerId = rand.nextLong();
- Text scannerName = new Text(String.valueOf(scannerId));
+ String scannerName = String.valueOf(scannerId);
synchronized(scanners) {
scanners.put(scannerName, s);
}
- leases.createLease(scannerName, scannerName,
+ leases.createLease(scannerId, scannerId,
new ScannerListener(scannerName));
} catch(IOException e) {
LOG.error(e);
@@ -1193,8 +1183,8 @@
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.HRegionInterface#close(long)
*/
- public void close(long scannerId) throws IOException {
- Text scannerName = new Text(String.valueOf(scannerId));
+ public void close(final long scannerId) throws IOException {
+ String scannerName = String.valueOf(scannerId);
HInternalScannerInterface s = null;
synchronized(scanners) {
s = scanners.remove(scannerName);
@@ -1203,7 +1193,7 @@
throw new UnknownScannerException(scannerName.toString());
}
s.close();
- leases.cancelLease(scannerName, scannerName);
+ leases.cancelLease(scannerId, scannerId);
}
private static void printUsageAndExit() {
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java Mon Jun 18 15:59:14 2007
@@ -28,7 +28,6 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.Text;
/**
@@ -168,7 +167,7 @@
Text [] families = info.tableDesc.families().keySet().toArray(new Text [] {});
HInternalScannerInterface scanner = r.getScanner(families, new Text());
HStoreKey key = new HStoreKey();
- TreeMap<Text, BytesWritable> results = new TreeMap<Text, BytesWritable>();
+ TreeMap<Text, byte []> results = new TreeMap<Text, byte []>();
// Print out table header line.
String s = info.startKey.toString();
String startKey = (s == null || s.length() <= 0)? "<>": s;
@@ -184,19 +183,15 @@
// Every line starts with row name followed by column name
// followed by cell content.
while(scanner.next(key, results)) {
- for (Map.Entry<Text, BytesWritable> es: results.entrySet()) {
- Text colname = es.getKey();
- BytesWritable colvalue = es.getValue();
+ for (Map.Entry<Text, byte []> es: results.entrySet()) {
+ Text colname = es.getKey();
+ byte [] colvalue = es.getValue();
Object value = null;
- byte[] bytes = new byte[colvalue.getSize()];
if (colname.toString().equals("info:regioninfo")) {
- // Then bytes are instance of an HRegionInfo.
- System.arraycopy(colvalue, 0, bytes, 0, bytes.length);
- value = new HRegionInfo(bytes);
+ value = new HRegionInfo(colvalue);
} else {
- value = new String(bytes, HConstants.UTF8_ENCODING);
+ value = new String(colvalue, HConstants.UTF8_ENCODING);
}
-
System.out.println(" " + key + ", " + colname.toString() + ": \"" +
value.toString() + "\"");
}
Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HScannerInterface.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HScannerInterface.java?view=diff&rev=548523&r1=548522&r2=548523
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HScannerInterface.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HScannerInterface.java Mon Jun 18 15:59:14 2007
@@ -24,6 +24,7 @@
* 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 boolean next(HStoreKey key, TreeMap<Text, byte[]> results)
+ throws IOException;
public void close() throws IOException;
}
Re: svn commit: r548523 [1/2] - in /lucene/hadoop/trunk/src/contrib/hbase:
./ src/java/org/apache/hadoop/hbase/ src/test/org/apache/hadoop/hbase/
Posted by Michael Stack <st...@duboce.net>.
Thanks Nigel. HADOOP-1498 added a file. Looks like it wasn't svn added
after patch application. I made a new issue, HADOOP-1503, with the
missing file added in case that helps resolve the broken build. Seems
to fix things when I run it locally.
St.Ack
Nigel Daley wrote:
> This commit seems to have broken trunk.
> http://lucene.zones.apache.org:8080/hudson/job/Hadoop-Nightly/127/console
>
> Cheers,
> Nige
>
> On Jun 18, 2007, at 3:59 PM, cutting@apache.org wrote:
>
>> Author: cutting
>> Date: Mon Jun 18 15:59:14 2007
>> New Revision: 548523
>>
>> URL: http://svn.apache.org/viewvc?view=rev&rev=548523
>> Log:
>> HADOOP-1498. Replace boxed types with primitives in many places.
>> Contributed by stack.
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HInternalScannerInterface.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogKey.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterInterface.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterRegionInterface.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HScannerInterface.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HServerAddress.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreKey.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/KeyedData.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/LeaseListener.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/Leases.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestGet.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHLog.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHMemcache.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java
>>
>>
>> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java
>>
>>
>> Modified: lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> --- lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt (original)
>> +++ lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt Mon Jun 18
>> 15:59:14 2007
>> @@ -34,4 +34,4 @@
>> 19. HADOOP-1415 Integrate BSD licensed bloom filter implementation.
>> 20. HADOOP-1465 Add cluster stop/start scripts for hbase
>> 21. HADOOP-1415 Provide configurable per-column bloom filters -
>> part 2.
>> -
>> + 22. HADOOP-1498. Replace boxed types with primitives in many places.
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java
>> Mon Jun 18 15:59:14 2007
>> @@ -47,7 +47,9 @@
>> public static final int RETOUCHED_BLOOMFILTER = 3;
>>
>> /** Default constructor - used in conjunction with Writable */
>> - public BloomFilterDescriptor() {}
>> + public BloomFilterDescriptor() {
>> + super();
>> + }
>>
>> /**
>> * @param type The kind of bloom filter to use.
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java
>> Mon Jun 18 15:59:14 2007
>> @@ -18,29 +18,26 @@
>> import java.io.IOException;
>> import java.util.TreeMap;
>> import java.util.Vector;
>> -
>> import java.util.regex.Pattern;
>>
>> import org.apache.commons.logging.Log;
>> import org.apache.commons.logging.LogFactory;
>> 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
>> HInternalScannerInterface {
>> final Log LOG = LogFactory.getLog(this.getClass().getName());
>>
>> // Pattern to determine if a column key is a regex
>> -
>> - static Pattern isRegexPattern =
>> Pattern.compile("^.*[\\\\+|^&*$\\[\\]\\}{)(]+.*$");
>> + static Pattern isRegexPattern =
>> + Pattern.compile("^.*[\\\\+|^&*$\\[\\]\\}{)(]+.*$");
>>
>> // The kind of match we are doing on a column:
>> -
>> private static enum MATCH_TYPE {
>> /** Just check the column family name */
>> FAMILY_ONLY,
>> @@ -55,7 +52,6 @@
>> // 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 static class ColumnMatcher {
>> private boolean wildCardmatch;
>> private MATCH_TYPE matchType;
>> @@ -63,33 +59,24 @@
>> private Pattern columnMatcher;
>> private Text col;
>>
>> - ColumnMatcher(Text col) throws IOException {
>> - String column = col.toString();
>> + ColumnMatcher(final Text col) throws IOException {
>> + Text qualifier = HStoreKey.extractQualifier(col);
>> try {
>> - int colpos = column.indexOf(":");
>> - if(colpos == -1) {
>> - throw new InvalidColumnNameException("Column name has no
>> family indicator.");
>> - }
>> -
>> - String columnkey = column.substring(colpos + 1);
>> -
>> - if(columnkey == null || columnkey.length() == 0) {
>> + if(qualifier == null || qualifier.getLength() == 0) {
>> this.matchType = MATCH_TYPE.FAMILY_ONLY;
>> - this.family = column.substring(0, colpos);
>> + this.family = HStoreKey.extractFamily(col).toString();
>> this.wildCardmatch = true;
>> -
>> - } else if(isRegexPattern.matcher(columnkey).matches()) {
>> + } else
>> if(isRegexPattern.matcher(qualifier.toString()).matches()) {
>> this.matchType = MATCH_TYPE.REGEX;
>> - this.columnMatcher = Pattern.compile(column);
>> + this.columnMatcher = Pattern.compile(col.toString());
>> this.wildCardmatch = true;
>> -
>> } else {
>> this.matchType = MATCH_TYPE.SIMPLE;
>> this.col = col;
>> this.wildCardmatch = false;
>> }
>> } catch(Exception e) {
>> - throw new IOException("Column: " + column + ": " +
>> e.getMessage());
>> + throw new IOException("Column: " + col + ": " +
>> e.getMessage());
>> }
>> }
>>
>> @@ -119,8 +106,10 @@
>>
>> 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
>> + // Keys retrieved from the sources
>> + protected HStoreKey keys[];
>> + // Values that correspond to those keys
>> + protected byte [][] vals;
>>
>> protected long timestamp; //
>> The timestamp to match entries against
>> private boolean wildcardMatch;
>> @@ -218,7 +207,7 @@
>> *
>> * @see
>> org.apache.hadoop.hbase.HScannerInterface#next(org.apache.hadoop.hbase.HStoreKey,
>> java.util.TreeMap)
>> */
>> - public boolean next(HStoreKey key, TreeMap<Text, BytesWritable>
>> results)
>> + public boolean next(HStoreKey key, TreeMap<Text, byte []> results)
>> throws IOException {
>> // Find the next row label (and timestamp)
>> Text chosenRow = null;
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java
>> Mon Jun 18 15:59:14 2007
>> @@ -30,11 +30,10 @@
>> import org.apache.commons.logging.Log;
>> import org.apache.commons.logging.LogFactory;
>> import org.apache.hadoop.conf.Configuration;
>> -import org.apache.hadoop.io.BytesWritable;
>> import org.apache.hadoop.io.DataInputBuffer;
>> import org.apache.hadoop.io.Text;
>> -import org.apache.hadoop.io.retry.RetryPolicy;
>> import org.apache.hadoop.io.retry.RetryPolicies;
>> +import org.apache.hadoop.io.retry.RetryPolicy;
>> import org.apache.hadoop.io.retry.RetryProxy;
>> import org.apache.hadoop.ipc.RPC;
>> import org.apache.hadoop.ipc.RemoteException;
>> @@ -285,10 +284,8 @@
>> }
>> boolean found = false;
>> for(int j = 0; j < values.length; j++) {
>> - if(values[j].getKey().getColumn().equals(COL_REGIONINFO)) {
>> - byte[] bytes = new byte[values[j].getData().getSize()];
>> - System.arraycopy(values[j].getData().get(), 0, bytes, 0,
>> bytes.length);
>> - inbuf.reset(bytes, bytes.length);
>> + if (values[j].getKey().getColumn().equals(COL_REGIONINFO)) {
>> + inbuf.reset(values[j].getData(),
>> values[j].getData().length);
>> info.readFields(inbuf);
>> if(info.tableDesc.getName().equals(tableName)) {
>> found = true;
>> @@ -398,9 +395,7 @@
>> valuesfound += 1;
>> for(int j = 0; j < values.length; j++) {
>> if(values[j].getKey().getColumn().equals(COL_REGIONINFO)) {
>> - byte[] bytes = new byte[values[j].getData().getSize()];
>> - System.arraycopy(values[j].getData().get(), 0, bytes,
>> 0, bytes.length);
>> - inbuf.reset(bytes, bytes.length);
>> + inbuf.reset(values[j].getData(),
>> values[j].getData().length);
>> info.readFields(inbuf);
>> isenabled = !info.offLine;
>> break;
>> @@ -483,9 +478,7 @@
>> valuesfound += 1;
>> for(int j = 0; j < values.length; j++) {
>> if(values[j].getKey().getColumn().equals(COL_REGIONINFO)) {
>> - byte[] bytes = new byte[values[j].getData().getSize()];
>> - System.arraycopy(values[j].getData().get(), 0, bytes,
>> 0, bytes.length);
>> - inbuf.reset(bytes, bytes.length);
>> + inbuf.reset(values[j].getData(),
>> values[j].getData().length);
>> info.readFields(inbuf);
>> disabled = info.offLine;
>> break;
>> @@ -737,8 +730,8 @@
>> * @throws IOException
>> */
>> private TreeMap<Text, RegionLocation> scanOneMetaRegion(final
>> RegionLocation t,
>> - final Text tableName) throws IOException {
>> -
>> + final Text tableName)
>> + throws IOException {
>> HRegionInterface server = getHRegionConnection(t.serverAddress);
>> TreeMap<Text, RegionLocation> servers = new TreeMap<Text,
>> RegionLocation>();
>> for(int tries = 0; servers.size() == 0 && tries < this.numRetries;
>> @@ -772,9 +765,7 @@
>> byte[] bytes = null;
>> TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
>> for(int i = 0; i < values.length; i++) {
>> - bytes = new byte[values[i].getData().getSize()];
>> - System.arraycopy(values[i].getData().get(), 0, bytes, 0,
>> bytes.length);
>> - results.put(values[i].getKey().getColumn(), bytes);
>> + results.put(values[i].getKey().getColumn(),
>> values[i].getData());
>> }
>> regionInfo = new HRegionInfo();
>> bytes = results.get(COL_REGIONINFO);
>> @@ -900,8 +891,7 @@
>> }
>> for(int i = 0; i < values.length; i++) {
>> if(values[i].getKey().getColumn().equals(COL_REGIONINFO)) {
>> - byte[] bytes = values[i].getData().get();
>> - inbuf.reset(bytes, bytes.length);
>> + inbuf.reset(values[i].getData(),
>> values[i].getData().length);
>> HRegionInfo info = new HRegionInfo();
>> info.readFields(inbuf);
>>
>> @@ -967,22 +957,19 @@
>> /**
>> * Get a single value for the specified row and column
>> *
>> - * @param row - row key
>> - * @param column - column name
>> - * @return - value for specified row/column
>> + * @param row row key
>> + * @param column column name
>> + * @return value for specified row/column
>> * @throws IOException
>> */
>> public byte[] get(Text row, Text column) throws IOException {
>> RegionLocation info = null;
>> - BytesWritable value = null;
>> -
>> + byte [] value = null;
>> for(int tries = 0; tries < numRetries && info == null; tries++) {
>> info = getRegionLocation(row);
>> -
>> try {
>> - value = getHRegionConnection(info.serverAddress).get(
>> - info.regionInfo.regionName, row, column);
>> -
>> + value = getHRegionConnection(info.serverAddress).
>> + get(info.regionInfo.regionName, row, column);
>> } catch(NotServingRegionException e) {
>> if(tries == numRetries - 1) {
>> // No more tries
>> @@ -992,13 +979,7 @@
>> info = null;
>> }
>> }
>> -
>> - if(value != null) {
>> - byte[] bytes = new byte[value.getSize()];
>> - System.arraycopy(value.get(), 0, bytes, 0, bytes.length);
>> - return bytes;
>> - }
>> - return null;
>> + return value;
>> }
>>
>> /**
>> @@ -1012,15 +993,12 @@
>> */
>> public byte[][] get(Text row, Text column, int numVersions) throws
>> IOException {
>> RegionLocation info = null;
>> - BytesWritable[] values = null;
>> -
>> + byte [][] values = null;
>> for(int tries = 0; tries < numRetries && info == null; tries++) {
>> info = getRegionLocation(row);
>> -
>> try {
>> values = getHRegionConnection(info.serverAddress).get(
>> - info.regionInfo.regionName, row, column, numVersions);
>> -
>> + info.regionInfo.regionName, row, column, numVersions);
>> } catch(NotServingRegionException e) {
>> if(tries == numRetries - 1) {
>> // No more tries
>> @@ -1034,9 +1012,7 @@
>> if(values != null) {
>> ArrayList<byte[]> bytes = new ArrayList<byte[]>();
>> for(int i = 0 ; i < values.length; i++) {
>> - byte[] value = new byte[values[i].getSize()];
>> - System.arraycopy(values[i].get(), 0, value, 0, value.length);
>> - bytes.add(value);
>> + bytes.add(values[i]);
>> }
>> return bytes.toArray(new byte[values.length][]);
>> }
>> @@ -1057,14 +1033,12 @@
>> public byte[][] get(Text row, Text column, long timestamp, int
>> numVersions)
>> throws IOException {
>> RegionLocation info = null;
>> - BytesWritable[] values = null;
>> -
>> + byte [][] values = null;
>> for(int tries = 0; tries < numRetries && info == null; tries++) {
>> info = getRegionLocation(row);
>> -
>> try {
>> - values = getHRegionConnection(info.serverAddress).get(
>> - info.regionInfo.regionName, row, column, timestamp,
>> numVersions);
>> + values = getHRegionConnection(info.serverAddress).
>> + get(info.regionInfo.regionName, row, column, timestamp,
>> numVersions);
>>
>> } catch(NotServingRegionException e) {
>> if(tries == numRetries - 1) {
>> @@ -1079,9 +1053,7 @@
>> if(values != null) {
>> ArrayList<byte[]> bytes = new ArrayList<byte[]>();
>> for(int i = 0 ; i < values.length; i++) {
>> - byte[] value = new byte[values[i].getSize()];
>> - System.arraycopy(values[i].get(), 0, value, 0, value.length);
>> - bytes.add(value);
>> + bytes.add(values[i]);
>> }
>> return bytes.toArray(new byte[values.length][]);
>> }
>> @@ -1118,9 +1090,7 @@
>> TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
>> if(value != null && value.length != 0) {
>> for(int i = 0; i < value.length; i++) {
>> - byte[] bytes = new byte[value[i].getData().getSize()];
>> - System.arraycopy(value[i].getData().get(), 0, bytes, 0,
>> bytes.length);
>> - results.put(value[i].getKey().getColumn(), bytes);
>> + results.put(value[i].getKey().getColumn(), value[i].getData());
>> }
>> }
>> return results;
>> @@ -1242,7 +1212,7 @@
>> public void put(long lockid, Text column, byte val[]) throws
>> IOException {
>> try {
>> this.currentServer.put(this.currentRegion, this.clientid,
>> lockid, column,
>> - new BytesWritable(val));
>> + val);
>> } catch(IOException e) {
>> try {
>> this.currentServer.abort(this.currentRegion, this.clientid,
>> lockid);
>> @@ -1432,9 +1402,7 @@
>> key.setRow(values[i].getKey().getRow());
>> key.setVersion(values[i].getKey().getTimestamp());
>> key.setColumn(EMPTY_COLUMN);
>> - byte[] bytes = new byte[values[i].getData().getSize()];
>> - System.arraycopy(values[i].getData().get(), 0, bytes, 0,
>> bytes.length);
>> - results.put(values[i].getKey().getColumn(), bytes);
>> + results.put(values[i].getKey().getColumn(),
>> values[i].getData());
>> }
>> }
>> return values == null ? false : values.length != 0;
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java
>> Mon Jun 18 15:59:14 2007
>> @@ -15,7 +15,7 @@
>> */
>> package org.apache.hadoop.hbase;
>>
>> -import org.apache.hadoop.io.BytesWritable;
>> +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
>> import org.apache.hadoop.io.Text;
>>
>> /**
>> @@ -106,11 +106,10 @@
>> static final String UTF8_ENCODING = "UTF-8";
>>
>> /** Value stored for a deleted item */
>> - static final BytesWritable DELETE_BYTES =
>> - new BytesWritable("HBASE::DELETEVAL".getBytes());
>> + static final ImmutableBytesWritable DELETE_BYTES =
>> + new ImmutableBytesWritable("HBASE::DELETEVAL".getBytes());
>>
>> /** Value written to HLog on a complete cache flush */
>> - static final BytesWritable COMPLETE_CACHEFLUSH =
>> - new BytesWritable("HBASE::CACHEFLUSH".getBytes());
>> -
>> + static final ImmutableBytesWritable COMPLETE_CACHEFLUSH =
>> + new ImmutableBytesWritable("HBASE::CACHEFLUSH".getBytes());
>> }
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HInternalScannerInterface.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HInternalScannerInterface.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HInternalScannerInterface.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HInternalScannerInterface.java
>> Mon Jun 18 15:59:14 2007
>> @@ -18,27 +18,29 @@
>> import java.io.IOException;
>> import java.util.TreeMap;
>>
>> -import org.apache.hadoop.io.BytesWritable;
>> import org.apache.hadoop.io.Text;
>>
>> /**
>> * Internally, we need to be able to determine if the scanner is
>> doing wildcard
>> * column matches (when only a column family is specified or if a
>> column regex
>> - * is specified) or if multiple members of the same column family
>> were specified.
>> - *
>> - * If so, we need to ignore the timestamp to ensure that we get all
>> the family
>> - * members, as they may have been last updated at different times.
>> - *
>> + * is specified) or if multiple members of the same column family were
>> + * specified. If so, we need to ignore the timestamp to ensure that
>> we get all
>> + * the family members, as they may have been last updated at
>> different times.
>> * This interface exposes two APIs for querying the scanner.
>> */
>> public interface HInternalScannerInterface {
>>
>> - public boolean next(HStoreKey key, TreeMap<Text, BytesWritable>
>> results) throws IOException;
>> + public boolean next(HStoreKey key, TreeMap<Text, byte []> results)
>> + throws IOException;
>> +
>> + /**
>> + * Close the scanner.
>> + */
>> public void close();
>> +
>> /** Returns true if the scanner is matching a column family or
>> regex */
>> public boolean isWildcardScanner();
>>
>> /** Returns true if the scanner is matching multiple column family
>> members */
>> public boolean isMultipleMatchScanner();
>> -
>> -}
>> +}
>> \ No newline at end of file
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java
>> Mon Jun 18 15:59:14 2007
>> @@ -78,7 +78,7 @@
>> long filenum = 0;
>> transient int numEntries = 0;
>>
>> - Integer rollLock = 0;
>> + Integer rollLock = new Integer(0);
>>
>> /**
>> * Split up a bunch of log files, that are no longer being written
>> to,
>> @@ -205,6 +205,7 @@
>> try {
>> wait();
>> } catch (InterruptedException ie) {
>> + // continue;
>> }
>> }
>>
>> @@ -282,8 +283,8 @@
>> * 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));
>> + Path computeFilename(final long fn) {
>> + return new Path(dir, HLOG_DATFILE + String.format("%1$03d", fn));
>> }
>>
>> /**
>> @@ -333,7 +334,7 @@
>> * @throws IOException
>> */
>> synchronized void append(Text regionName, Text tableName, Text row,
>> - TreeMap<Text, BytesWritable> columns, long timestamp)
>> + TreeMap<Text, byte []> columns, long timestamp)
>> throws IOException {
>> if(closed) {
>> throw new IOException("Cannot append; log is closed");
>> @@ -350,7 +351,7 @@
>> }
>>
>> int counter = 0;
>> - for (Map.Entry<Text, BytesWritable> es: columns.entrySet()) {
>> + for (Map.Entry<Text, byte []> es: columns.entrySet()) {
>> HLogKey logKey =
>> new HLogKey(regionName, tableName, row, seqNum[counter++]);
>> HLogEdit logEdit = new HLogEdit(es.getKey(), es.getValue(),
>> timestamp);
>> @@ -401,6 +402,7 @@
>> try {
>> wait();
>> } catch (InterruptedException ie) {
>> + // continue
>> }
>> }
>> insideCacheFlush = true;
>> @@ -427,7 +429,7 @@
>> }
>>
>> writer.append(new HLogKey(regionName, tableName, HLog.METAROW,
>> logSeqId),
>> - new HLogEdit(HLog.METACOLUMN, COMPLETE_CACHEFLUSH,
>> + new HLogEdit(HLog.METACOLUMN, COMPLETE_CACHEFLUSH.get(),
>> System.currentTimeMillis()));
>> numEntries++;
>>
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java
>> Mon Jun 18 15:59:14 2007
>> @@ -27,14 +27,15 @@
>> * This just indicates the column and value.
>>
>> ******************************************************************************/
>>
>> public class HLogEdit implements Writable {
>> - Text column = new Text();
>> - BytesWritable val = new BytesWritable();
>> - long timestamp;
>> + private Text column = new Text();
>> + private byte [] val;
>> + private long timestamp;
>>
>> public HLogEdit() {
>> + super();
>> }
>>
>> - public HLogEdit(Text column, BytesWritable bval, long timestamp) {
>> + public HLogEdit(Text column, byte [] bval, long timestamp) {
>> this.column.set(column);
>> this.val = bval;
>> this.timestamp = timestamp;
>> @@ -44,7 +45,7 @@
>> return this.column;
>> }
>>
>> - public BytesWritable getVal() {
>> + public byte [] getVal() {
>> return this.val;
>> }
>>
>> @@ -55,7 +56,7 @@
>> @Override
>> public String toString() {
>> return getColumn().toString() + " " + this.getTimestamp() + " " +
>> - new String(getVal().get()).trim();
>> + new String(getVal()).trim();
>> }
>>
>>
>> //////////////////////////////////////////////////////////////////////////////
>>
>> @@ -64,13 +65,15 @@
>>
>> public void write(DataOutput out) throws IOException {
>> this.column.write(out);
>> - this.val.write(out);
>> + out.writeShort(this.val.length);
>> + out.write(this.val);
>> out.writeLong(timestamp);
>> }
>>
>> public void readFields(DataInput in) throws IOException {
>> this.column.readFields(in);
>> - this.val.readFields(in);
>> + this.val = new byte[in.readShort()];
>> + in.readFully(this.val);
>> this.timestamp = in.readLong();
>> }
>> }
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogKey.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogKey.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogKey.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogKey.java
>> Mon Jun 18 15:59:14 2007
>> @@ -34,6 +34,7 @@
>>
>> /** Create an empty key useful when deserializing */
>> public HLogKey() {
>> + super();
>> }
>>
>> /**
>> @@ -47,6 +48,7 @@
>> * @param logSeqNum - log sequence number
>> */
>> public HLogKey(Text regionName, Text tablename, Text row, long
>> logSeqNum) {
>> + // TODO: Is this copy of the instances necessary? They are
>> expensive.
>> this.regionName.set(regionName);
>> this.tablename.set(tablename);
>> this.row.set(row);
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
>> Mon Jun 18 15:59:14 2007
>> @@ -37,7 +37,6 @@
>> import org.apache.hadoop.conf.Configuration;
>> import org.apache.hadoop.fs.FileSystem;
>> import org.apache.hadoop.fs.Path;
>> -import org.apache.hadoop.io.BytesWritable;
>> import org.apache.hadoop.io.DataInputBuffer;
>> import org.apache.hadoop.io.Text;
>> import org.apache.hadoop.ipc.RPC;
>> @@ -179,10 +178,7 @@
>> }
>>
>> for (int i = 0; i < values.length; i++) {
>> - byte[] bytes = new byte[values[i].getData().getSize()];
>> - System.arraycopy(values[i].getData().get(), 0, bytes, 0,
>> - bytes.length);
>> - results.put(values[i].getKey().getColumn(), bytes);
>> + results.put(values[i].getKey().getColumn(),
>> values[i].getData());
>> }
>>
>> HRegionInfo info = HRegion.getRegionInfo(results);
>> @@ -272,7 +268,7 @@
>> // The current assignment is no good; load the region.
>>
>> unassignedRegions.put(info.regionName, info);
>> - assignAttempts.put(info.regionName, 0L);
>> + assignAttempts.put(info.regionName, Long.valueOf(0L));
>> }
>> }
>> }
>> @@ -333,7 +329,7 @@
>>
>> private RootScanner rootScanner;
>> private Thread rootScannerThread;
>> - Integer rootScannerLock = 0;
>> + Integer rootScannerLock = new Integer(0);
>>
>> @SuppressWarnings("unchecked")
>> static class MetaRegion implements Comparable {
>> @@ -492,7 +488,7 @@
>>
>> MetaScanner metaScanner;
>> private Thread metaScannerThread;
>> - Integer metaScannerLock = 0;
>> + Integer metaScannerLock = new Integer(0);
>>
>> /**
>> * The 'unassignedRegions' table maps from a region name to a
>> HRegionInfo
>> @@ -642,7 +638,8 @@
>> this.pendingRegions =
>> Collections.synchronizedSortedSet(new TreeSet<Text>());
>>
>> - this.assignAttempts.put(HGlobals.rootRegionInfo.regionName, 0L);
>> + this.assignAttempts.put(HGlobals.rootRegionInfo.regionName,
>> + Long.valueOf(0L));
>>
>> this.killList =
>> Collections.synchronizedSortedMap(
>> @@ -655,9 +652,7 @@
>> Collections.synchronizedSortedSet(new TreeSet<Text>());
>>
>> // We're almost open for business
>> -
>> this.closed = false;
>> -
>> LOG.info("HMaster initialized on " + this.address.toString());
>> }
>>
>> @@ -815,7 +810,9 @@
>> /* (non-Javadoc)
>> * @see
>> org.apache.hadoop.hbase.HMasterRegionInterface#regionServerStartup(org.apache.hadoop.hbase.HServerInfo)
>>
>> */
>> - public void regionServerStartup(HServerInfo serverInfo) throws
>> IOException {
>> + @SuppressWarnings("unused")
>> + public void regionServerStartup(HServerInfo serverInfo)
>> + throws IOException {
>> String s = serverInfo.getServerAddress().toString().trim();
>> HServerInfo storedInfo = null;
>> LOG.info("received start message from: " + s);
>> @@ -834,11 +831,15 @@
>> // Either way, record the new server
>> serversToServerInfo.put(s, serverInfo);
>> if(!closed) {
>> - Text serverLabel = new Text(s);
>> + long serverLabel = getServerLabel(s);
>> LOG.debug("Created lease for " + serverLabel);
>> serverLeases.createLease(serverLabel, serverLabel, new
>> ServerExpirer(s));
>> }
>> }
>> +
>> + private long getServerLabel(final String s) {
>> + return s.hashCode();
>> + }
>>
>> /* (non-Javadoc)
>> * @see
>> org.apache.hadoop.hbase.HMasterRegionInterface#regionServerReport(org.apache.hadoop.hbase.HServerInfo,
>> org.apache.hadoop.hbase.HMsg[])
>> @@ -846,7 +847,7 @@
>> public HMsg[] regionServerReport(HServerInfo serverInfo, HMsg msgs[])
>> throws IOException {
>> String s = serverInfo.getServerAddress().toString().trim();
>> - Text serverLabel = new Text(s);
>> + long serverLabel = getServerLabel(s);
>>
>> if (closed) {
>> // Cancel the server's lease
>> @@ -874,7 +875,7 @@
>> allMetaRegionsScanned = false;
>> }
>> unassignedRegions.put(info.regionName, info);
>> - assignAttempts.put(info.regionName, 0L);
>> + assignAttempts.put(info.regionName, Long.valueOf(0L));
>> }
>>
>> // We don't need to return anything to the server because it
>> isn't
>> @@ -934,7 +935,8 @@
>> }
>>
>> /** cancel a server's lease */
>> - private void cancelLease(String serverName, Text serverLabel)
>> throws IOException {
>> + private void cancelLease(final String serverName, final long
>> serverLabel)
>> + throws IOException {
>> if (serversToServerInfo.remove(serverName) != null) {
>> // Only cancel lease once.
>> // This method can be called a couple of times during shutdown.
>> @@ -1035,7 +1037,7 @@
>>
>> if(region.regionName.compareTo(HGlobals.rootRegionInfo.regionName) ==
>> 0) { // Root region
>> rootRegionLocation = null;
>> unassignedRegions.put(region.regionName, region);
>> - assignAttempts.put(region.regionName, 0L);
>> + assignAttempts.put(region.regionName, Long.valueOf(0L));
>>
>> } else {
>> boolean reassignRegion = true;
>> @@ -1115,7 +1117,7 @@
>>
>> returnMsgs.add(new HMsg(HMsg.MSG_REGION_OPEN, regionInfo));
>>
>> - assignAttempts.put(curRegionName, now);
>> + assignAttempts.put(curRegionName, Long.valueOf(now));
>> counter++;
>> }
>>
>> @@ -1214,7 +1216,6 @@
>>
>> TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
>> Text row = null;
>> - byte[] bytes = null;
>> for(int i = 0; i < values.length; i++) {
>> if(row == null) {
>> row = values[i].getKey().getRow();
>> @@ -1225,12 +1226,10 @@
>> + row + ", currentRow=" +
>> values[i].getKey().getRow());
>> }
>> }
>> - bytes = new byte[values[i].getData().getSize()];
>> - System.arraycopy(values[i].getData().get(), 0, bytes, 0,
>> bytes.length);
>> - results.put(values[i].getKey().getColumn(), bytes);
>> + results.put(values[i].getKey().getColumn(),
>> values[i].getData());
>> }
>>
>> - bytes = results.get(COL_SERVER);
>> + byte [] bytes = results.get(COL_SERVER);
>> String serverName = null;
>> if(bytes == null || bytes.length == 0) {
>> // No server
>> @@ -1335,21 +1334,18 @@
>> }
>>
>> // Remove server from root/meta entries
>> -
>> for(int i = 0; i < toDoList.size(); i++) {
>> ToDoEntry e = toDoList.get(i);
>> long lockid = server.startUpdate(regionName, clientId, e.row);
>> if(e.deleteRegion) {
>> server.delete(regionName, clientId, lockid, COL_REGIONINFO);
>> -
>> } else if(e.regionOffline) {
>> e.info.offLine = true;
>> ByteArrayOutputStream byteValue = new
>> ByteArrayOutputStream();
>> DataOutputStream s = new DataOutputStream(byteValue);
>> e.info.write(s);
>> -
>> server.put(regionName, clientId, lockid, COL_REGIONINFO,
>> - new BytesWritable(byteValue.toByteArray()));
>> + byteValue.toByteArray());
>> }
>> server.delete(regionName, clientId, lockid, COL_SERVER);
>> server.delete(regionName, clientId, lockid, COL_STARTCODE);
>> @@ -1363,7 +1359,7 @@
>> HRegionInfo regionInfo = e.getValue();
>>
>> unassignedRegions.put(region, regionInfo);
>> - assignAttempts.put(region, 0L);
>> + assignAttempts.put(region, Long.valueOf(0L));
>> }
>> }
>>
>> @@ -1384,7 +1380,8 @@
>> rootRegionLocation = null;
>> unassignedRegions.put(HGlobals.rootRegionInfo.regionName,
>> HGlobals.rootRegionInfo);
>> - assignAttempts.put(HGlobals.rootRegionInfo.regionName, 0L);
>> + assignAttempts.put(HGlobals.rootRegionInfo.regionName,
>> + Long.valueOf(0L));
>> }
>>
>> // Scan the ROOT region
>> @@ -1525,7 +1522,7 @@
>> regionInfo.write(s);
>>
>> server.put(metaRegionName, clientId, lockid,
>> COL_REGIONINFO,
>> - new BytesWritable(byteValue.toByteArray()));
>> + byteValue.toByteArray());
>> }
>> server.delete(metaRegionName, clientId, lockid, COL_SERVER);
>> server.delete(metaRegionName, clientId, lockid,
>> COL_STARTCODE);
>> @@ -1546,7 +1543,7 @@
>> }
>>
>> unassignedRegions.put(regionInfo.regionName, regionInfo);
>> - assignAttempts.put(regionInfo.regionName, 0L);
>> + assignAttempts.put(regionInfo.regionName, Long.valueOf(0L));
>>
>> } else if(deleteRegion) {
>> try {
>> @@ -1569,36 +1566,27 @@
>> private class PendingOpenReport extends PendingOperation {
>> private boolean rootRegion;
>> private Text regionName;
>> - private BytesWritable serverAddress;
>> - private BytesWritable startCode;
>> + private byte [] serverAddress;
>> + private byte [] startCode;
>>
>> PendingOpenReport(HServerInfo info, HRegionInfo region) {
>> - if(region.tableDesc.getName().equals(META_TABLE_NAME)) {
>> -
>> + if (region.tableDesc.getName().equals(META_TABLE_NAME)) {
>> // The region which just came on-line is a META region.
>> // We need to look in the ROOT region for its information.
>> -
>> this.rootRegion = true;
>> -
>> } else {
>> -
>> // Just an ordinary region. Look for it in the META table.
>> -
>> this.rootRegion = false;
>> }
>> this.regionName = region.regionName;
>> -
>> try {
>> - this.serverAddress = new BytesWritable(
>> -
>> info.getServerAddress().toString().getBytes(UTF8_ENCODING));
>> -
>> - this.startCode = new BytesWritable(
>> -
>> String.valueOf(info.getStartCode()).getBytes(UTF8_ENCODING));
>> -
>> + this.serverAddress = info.getServerAddress().toString().
>> + getBytes(UTF8_ENCODING);
>> + this.startCode = String.valueOf(info.getStartCode()).
>> + getBytes(UTF8_ENCODING);
>> } catch(UnsupportedEncodingException e) {
>> LOG.error(e);
>> }
>> -
>> }
>>
>> @Override
>> @@ -1614,7 +1602,7 @@
>>
>> if(LOG.isDebugEnabled()) {
>> LOG.debug(regionName + " open on "
>> - + new String(serverAddress.get(), UTF8_ENCODING));
>> + + new String(this.serverAddress, UTF8_ENCODING));
>> }
>>
>> // Register the newly-available Region's location.
>> @@ -1708,33 +1696,25 @@
>> }
>>
>> // 1. Check to see if table already exists
>> -
>> - MetaRegion m = null;
>> - if(knownMetaRegions.containsKey(newRegion.regionName)) {
>> - m = knownMetaRegions.get(newRegion.regionName);
>> -
>> - } else {
>> - m = knownMetaRegions.get(
>> + MetaRegion m =
>> (knownMetaRegions.containsKey(newRegion.regionName))?
>> + knownMetaRegions.get(newRegion.regionName):
>> + knownMetaRegions.get(
>>
>> knownMetaRegions.headMap(newRegion.regionName).lastKey());
>> - }
>> Text metaRegionName = m.regionName;
>> HRegionInterface server =
>> client.getHRegionConnection(m.server);
>> -
>> -
>> - BytesWritable bytes = server.get(metaRegionName,
>> desc.getName(), COL_REGIONINFO);
>> - if(bytes != null && bytes.getSize() != 0) {
>> - byte[] infoBytes = bytes.get();
>> + byte [] infoBytes =
>> + server.get(metaRegionName, desc.getName(), COL_REGIONINFO);
>> + if (infoBytes != null && infoBytes.length != 0) {
>> DataInputBuffer inbuf = new DataInputBuffer();
>> inbuf.reset(infoBytes, infoBytes.length);
>> HRegionInfo info = new HRegionInfo();
>> info.readFields(inbuf);
>> - if(info.tableDesc.getName().compareTo(desc.getName()) == 0) {
>> + if (info.tableDesc.getName().compareTo(desc.getName()) ==
>> 0) {
>> throw new IOException("table already exists");
>> }
>> }
>>
>> // 2. Create the HRegion
>> -
>> HRegion r = HRegion.createHRegion(newRegion.regionId, desc,
>> this.dir,
>> this.conf);
>>
>> @@ -1748,8 +1728,8 @@
>>
>> long clientId = rand.nextLong();
>> long lockid = server.startUpdate(metaRegionName, clientId,
>> regionName);
>> - server.put(metaRegionName, clientId, lockid, COL_REGIONINFO,
>> - new BytesWritable(byteValue.toByteArray()));
>> + server.put(metaRegionName, clientId, lockid, COL_REGIONINFO,
>> + byteValue.toByteArray());
>> server.commit(metaRegionName, clientId, lockid);
>>
>> // 4. Close the new region to flush it to disk
>> @@ -1759,7 +1739,7 @@
>> // 5. Get it assigned to a server
>>
>> unassignedRegions.put(regionName, info);
>> - assignAttempts.put(regionName, 0L);
>> + assignAttempts.put(regionName, Long.valueOf(0L));
>> break;
>>
>> } catch(NotServingRegionException e) {
>> @@ -1887,30 +1867,26 @@
>> }
>> boolean haveRegionInfo = false;
>> for(int i = 0; i < values.length; i++) {
>> - bytes = new byte[values[i].getData().getSize()];
>> - if(bytes.length == 0) {
>> + if(values[i].getData().length == 0) {
>> break;
>> }
>> - System.arraycopy(values[i].getData().get(), 0,
>> bytes, 0, bytes.length);
>> -
>> Text column = values[i].getKey().getColumn();
>> if(column.equals(COL_REGIONINFO)) {
>> haveRegionInfo = true;
>> - inbuf.reset(bytes, bytes.length);
>> + inbuf.reset(values[i].getData(),
>> + values[i].getData().length);
>> info.readFields(inbuf);
>> -
>> } else if(column.equals(COL_SERVER)) {
>> try {
>> - serverName = new String(bytes, UTF8_ENCODING);
>> -
>> + serverName =
>> + new String(values[i].getData(),
>> UTF8_ENCODING);
>> } catch(UnsupportedEncodingException e) {
>> assert(false);
>> }
>> -
>> } else if(column.equals(COL_STARTCODE)) {
>> try {
>> - startCode = Long.valueOf(new String(bytes,
>> UTF8_ENCODING));
>> -
>> + startCode = Long.valueOf(new
>> String(values[i].getData(),
>> + UTF8_ENCODING)).longValue();
>> } catch(UnsupportedEncodingException e) {
>> assert(false);
>> }
>> @@ -2115,8 +2091,7 @@
>> i.write(s);
>>
>> server.put(regionName, clientId, lockid, COL_REGIONINFO,
>> - new BytesWritable(byteValue.toByteArray()));
>> -
>> + byteValue.toByteArray());
>> }
>> }
>>
>> @@ -2180,29 +2155,24 @@
>> }
>>
>> protected void updateRegionInfo(HRegionInterface server, Text
>> regionName,
>> - HRegionInfo i) throws IOException {
>> -
>> + HRegionInfo i)
>> + throws IOException {
>> ByteArrayOutputStream byteValue = new ByteArrayOutputStream();
>> DataOutputStream s = new DataOutputStream(byteValue);
>> i.write(s);
>> -
>> long lockid = -1L;
>> long clientId = rand.nextLong();
>> try {
>> lockid = server.startUpdate(regionName, clientId,
>> i.regionName);
>> server.put(regionName, clientId, lockid, COL_REGIONINFO,
>> - new BytesWritable(byteValue.toByteArray()));
>> -
>> + byteValue.toByteArray());
>> server.commit(regionName, clientId, lockid);
>> lockid = -1L;
>> -
>> if(LOG.isDebugEnabled()) {
>> LOG.debug("updated columns in row: " + i.regionName);
>> }
>> -
>> } catch(NotServingRegionException e) {
>> throw e;
>> -
>> } catch(IOException e) {
>> LOG.error("column update failed in row: " + i.regionName);
>> LOG.error(e);
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterInterface.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterInterface.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterInterface.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterInterface.java
>> Mon Jun 18 15:59:14 2007
>> @@ -57,4 +57,4 @@
>>
>> //////////////////////////////////////////////////////////////////////////////
>>
>>
>> public HServerAddress findRootRegion();
>> -}
>> +}
>> \ No newline at end of file
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterRegionInterface.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterRegionInterface.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterRegionInterface.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterRegionInterface.java
>> Mon Jun 18 15:59:14 2007
>> @@ -25,5 +25,6 @@
>> public interface HMasterRegionInterface extends VersionedProtocol {
>> public static final long versionID = 1L;
>> public void regionServerStartup(HServerInfo info) throws IOException;
>> - public HMsg[] regionServerReport(HServerInfo info, HMsg msgs[])
>> throws IOException;
>> + public HMsg[] regionServerReport(HServerInfo info, HMsg msgs[])
>> + throws IOException;
>> }
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java
>> Mon Jun 18 15:59:14 2007
>> @@ -16,7 +16,9 @@
>> package org.apache.hadoop.hbase;
>>
>> import java.io.IOException;
>> +import java.util.ArrayList;
>> import java.util.Iterator;
>> +import java.util.List;
>> import java.util.Map;
>> import java.util.SortedMap;
>> import java.util.TreeMap;
>> @@ -24,7 +26,7 @@
>>
>> import org.apache.commons.logging.Log;
>> import org.apache.commons.logging.LogFactory;
>> -import org.apache.hadoop.io.BytesWritable;
>> +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
>> import org.apache.hadoop.io.Text;
>>
>> /**
>> @@ -34,13 +36,13 @@
>> public class HMemcache {
>> private final Log LOG = LogFactory.getLog(this.getClass().getName());
>>
>> - TreeMap<HStoreKey, BytesWritable> memcache
>> - = new TreeMap<HStoreKey, BytesWritable>();
>> + TreeMap<HStoreKey, byte []> memcache =
>> + new TreeMap<HStoreKey, byte []>();
>>
>> - Vector<TreeMap<HStoreKey, BytesWritable>> history
>> - = new Vector<TreeMap<HStoreKey, BytesWritable>>();
>> + Vector<TreeMap<HStoreKey, byte []>> history
>> + = new Vector<TreeMap<HStoreKey, byte []>>();
>>
>> - TreeMap<HStoreKey, BytesWritable> snapshot = null;
>> + TreeMap<HStoreKey, byte []> snapshot = null;
>>
>> final HLocking lock = new HLocking();
>>
>> @@ -49,7 +51,7 @@
>> }
>>
>> public static class Snapshot {
>> - public TreeMap<HStoreKey, BytesWritable> memcacheSnapshot = null;
>> + public TreeMap<HStoreKey, byte []> memcacheSnapshot = null;
>> public long sequenceId = 0;
>>
>> public Snapshot() {
>> @@ -92,7 +94,7 @@
>> retval.memcacheSnapshot = memcache;
>> this.snapshot = memcache;
>> history.add(memcache);
>> - memcache = new TreeMap<HStoreKey, BytesWritable>();
>> + memcache = new TreeMap<HStoreKey, byte []>();
>> retval.sequenceId = log.startCacheFlush();
>>
>> if(LOG.isDebugEnabled()) {
>> @@ -122,21 +124,18 @@
>> LOG.debug("deleting snapshot");
>> }
>>
>> - for(Iterator<TreeMap<HStoreKey, BytesWritable>> it =
>> history.iterator();
>> + for(Iterator<TreeMap<HStoreKey, byte []>> it =
>> history.iterator();
>> it.hasNext(); ) {
>> -
>> - TreeMap<HStoreKey, BytesWritable> cur = it.next();
>> - if(snapshot == cur) {
>> + TreeMap<HStoreKey, byte []> cur = it.next();
>> + if (snapshot == cur) {
>> it.remove();
>> break;
>> }
>> }
>> this.snapshot = null;
>> -
>> if(LOG.isDebugEnabled()) {
>> LOG.debug("snapshot deleted");
>> }
>> -
>> } finally {
>> this.lock.releaseWriteLock();
>> }
>> @@ -144,14 +143,16 @@
>>
>> /**
>> * Store a value.
>> - *
>> * Operation uses a write lock.
>> + * @param row
>> + * @param columns
>> + * @param timestamp
>> */
>> - public void add(final Text row, final TreeMap<Text, BytesWritable>
>> columns,
>> + public void add(final Text row, final TreeMap<Text, byte []> columns,
>> final long timestamp) {
>> this.lock.obtainWriteLock();
>> try {
>> - for (Map.Entry<Text, BytesWritable> es: columns.entrySet()) {
>> + for (Map.Entry<Text, byte []> es: columns.entrySet()) {
>> HStoreKey key = new HStoreKey(row, es.getKey(), timestamp);
>> memcache.put(key, es.getValue());
>> }
>> @@ -162,45 +163,47 @@
>>
>> /**
>> * Look back through all the backlog TreeMaps to find the target.
>> - *
>> - * We only need a readlock here.
>> + * @param key
>> + * @param numVersions
>> + * @return An array of byte arrays orderded by timestamp.
>> */
>> - public BytesWritable[] get(HStoreKey key, int numVersions) {
>> - Vector<BytesWritable> results = new Vector<BytesWritable>();
>> + public byte [][] get(final HStoreKey key, final int numVersions) {
>> + List<byte []> results = new ArrayList<byte[]>();
>> this.lock.obtainReadLock();
>> try {
>> - Vector<BytesWritable> result = get(memcache, key,
>> numVersions-results.size());
>> + ArrayList<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) {
>> + 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());
>> + result = get(history.elementAt(i), key, numVersions -
>> results.size());
>> results.addAll(results.size(), result);
>> }
>> -
>> return (results.size() == 0)?
>> - null: results.toArray(new BytesWritable[results.size()]);
>> + null: ImmutableBytesWritable.toArray(results);
>> } finally {
>> this.lock.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[]).
>> + * @param key
>> + * @return All columns for given key.
>> */
>> - public TreeMap<Text, BytesWritable> getFull(HStoreKey key) {
>> - TreeMap<Text, BytesWritable> results = new TreeMap<Text,
>> BytesWritable>();
>> + public TreeMap<Text, byte []> getFull(HStoreKey key) {
>> + TreeMap<Text, byte []> results = new TreeMap<Text, byte []>();
>> this.lock.obtainReadLock();
>> try {
>> internalGetFull(memcache, key, results);
>> - for(int i = history.size()-1; i >= 0; i--) {
>> - TreeMap<HStoreKey, BytesWritable> cur = history.elementAt(i);
>> + for (int i = history.size()-1; i >= 0; i--) {
>> + TreeMap<HStoreKey, byte []> cur = history.elementAt(i);
>> internalGetFull(cur, key, results);
>> }
>> return results;
>> @@ -210,17 +213,16 @@
>> }
>> }
>>
>> - void internalGetFull(TreeMap<HStoreKey, BytesWritable> map,
>> HStoreKey key,
>> - TreeMap<Text, BytesWritable> results) {
>> - SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(key);
>> - for (Map.Entry<HStoreKey, BytesWritable> es: tailMap.entrySet()) {
>> + void internalGetFull(TreeMap<HStoreKey, byte []> map, HStoreKey key,
>> + TreeMap<Text, byte []> results) {
>> + SortedMap<HStoreKey, byte []> tailMap = map.tailMap(key);
>> + for (Map.Entry<HStoreKey, byte []> es: tailMap.entrySet()) {
>> HStoreKey itKey = es.getKey();
>> Text itCol = itKey.getColumn();
>> if (results.get(itCol) == null
>> && key.matchesWithoutColumn(itKey)) {
>> - BytesWritable val = tailMap.get(itKey);
>> + byte [] val = tailMap.get(itKey);
>> results.put(itCol, val);
>> -
>> } else if (key.getRow().compareTo(itKey.getRow()) > 0) {
>> break;
>> }
>> @@ -235,18 +237,23 @@
>> *
>> * TODO - This is kinda slow. We need a data structure that
>> allows for
>> * proximity-searches, not just precise-matches.
>> - */
>> - Vector<BytesWritable> get(TreeMap<HStoreKey, BytesWritable> map,
>> HStoreKey key, int numVersions) {
>> - Vector<BytesWritable> result = new Vector<BytesWritable>();
>> - HStoreKey curKey = new HStoreKey(key.getRow(), key.getColumn(),
>> key.getTimestamp());
>> - SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(curKey);
>> - for (Map.Entry<HStoreKey, BytesWritable> es: tailMap.entrySet()) {
>> + * @param map
>> + * @param key
>> + * @param numVersions
>> + * @return Ordered list of items found in passed <code>map</code>
>> + */
>> + ArrayList<byte []> get(final TreeMap<HStoreKey, byte []> map,
>> + final HStoreKey key, final int numVersions) {
>> + ArrayList<byte []> result = new ArrayList<byte []>();
>> + HStoreKey curKey =
>> + new HStoreKey(key.getRow(), key.getColumn(), key.getTimestamp());
>> + SortedMap<HStoreKey, byte []> tailMap = map.tailMap(curKey);
>> + for (Map.Entry<HStoreKey, byte []> es: tailMap.entrySet()) {
>> HStoreKey itKey = es.getKey();
>> if (itKey.matchesRowCol(curKey)) {
>> result.add(tailMap.get(itKey));
>> curKey.setVersion(itKey.getTimestamp() - 1);
>> }
>> -
>> if (numVersions > 0 && result.size() >= numVersions) {
>> break;
>> }
>> @@ -269,8 +276,8 @@
>>
>> //////////////////////////////////////////////////////////////////////////////
>>
>>
>> class HMemcacheScanner extends HAbstractScanner {
>> - TreeMap<HStoreKey, BytesWritable> backingMaps[];
>> - Iterator<HStoreKey> keyIterators[];
>> + final TreeMap<HStoreKey, byte []> backingMaps[];
>> + final Iterator<HStoreKey> keyIterators[];
>>
>> @SuppressWarnings("unchecked")
>> public HMemcacheScanner(long timestamp, Text targetCols[], Text
>> firstRow)
>> @@ -292,7 +299,7 @@
>>
>> this.keyIterators = new Iterator[backingMaps.length];
>> this.keys = new HStoreKey[backingMaps.length];
>> - this.vals = new BytesWritable[backingMaps.length];
>> + this.vals = new byte[backingMaps.length][];
>>
>> // Generate list of iterators
>> HStoreKey firstKey = new HStoreKey(firstRow);
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java
>> Mon Jun 18 15:59:14 2007
>> @@ -27,7 +27,6 @@
>> import org.apache.hadoop.conf.Configuration;
>> import org.apache.hadoop.fs.FileSystem;
>> import org.apache.hadoop.fs.Path;
>> -import org.apache.hadoop.io.BytesWritable;
>> import org.apache.hadoop.io.DataInputBuffer;
>> import org.apache.hadoop.io.Text;
>>
>> @@ -39,7 +38,9 @@
>> static final Log LOG = LogFactory.getLog(HMerge.class);
>> static final Text[] META_COLS = {COL_REGIONINFO};
>>
>> - private HMerge() {} // Not instantiable
>> + private HMerge() {
>> + // Not instantiable
>> + }
>>
>> /**
>> * Scans the table and merges two adjacent regions if they are
>> small. This
>> @@ -317,7 +318,7 @@
>> private static class OfflineMerger extends Merger {
>> private Path dir;
>> private TreeSet<HRegionInfo> metaRegions;
>> - private TreeMap<Text, BytesWritable> results;
>> + private TreeMap<Text, byte []> results;
>>
>> OfflineMerger(Configuration conf, FileSystem fs, Text tableName)
>> throws IOException {
>> @@ -325,7 +326,7 @@
>> super(conf, fs, tableName);
>> this.dir = new Path(conf.get(HBASE_DIR, DEFAULT_HBASE_DIR));
>> this.metaRegions = new TreeSet<HRegionInfo>();
>> - this.results = new TreeMap<Text, BytesWritable>();
>> + this.results = new TreeMap<Text, byte []>();
>>
>> // Scan root region to find all the meta regions
>>
>> @@ -337,10 +338,8 @@
>>
>> try {
>> while(rootScanner.next(key, results)) {
>> - for(BytesWritable b: results.values()) {
>> - byte[] bytes = new byte[b.getSize()];
>> - System.arraycopy(b.get(), 0, bytes, 0, bytes.length);
>> - in.reset(bytes, bytes.length);
>> + for(byte [] b: results.values()) {
>> + in.reset(b, b.length);
>> info.readFields(in);
>> metaRegions.add(info);
>> results.clear();
>> @@ -405,8 +404,7 @@
>> long lockid = -1L;
>> try {
>> lockid = root.startUpdate(newRegion.getRegionName());
>> - root.put(lockid, COL_REGIONINFO,
>> - new BytesWritable(byteValue.toByteArray()));
>> + root.put(lockid, COL_REGIONINFO, byteValue.toByteArray());
>> root.commit(lockid);
>> lockid = -1L;
>>
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
>> Mon Jun 18 15:59:14 2007
>> @@ -238,11 +238,11 @@
>> // Members
>>
>> //////////////////////////////////////////////////////////////////////////////
>>
>>
>> - TreeMap<Text, Long> rowsToLocks = new TreeMap<Text, Long>();
>> - TreeMap<Long, Text> locksToRows = new TreeMap<Long, Text>();
>> - TreeMap<Text, HStore> stores = new TreeMap<Text, HStore>();
>> - Map<Long, TreeMap<Text, BytesWritable>> targetColumns
>> - = new HashMap<Long, TreeMap<Text, BytesWritable>>();
>> + Map<Text, Long> rowsToLocks = new HashMap<Text, Long>();
>> + Map<Long, Text> locksToRows = new HashMap<Long, Text>();
>> + Map<Text, HStore> stores = new HashMap<Text, HStore>();
>> + Map<Long, TreeMap<Text, byte []>> targetColumns
>> + = new HashMap<Long, TreeMap<Text, byte []>>();
>>
>> HMemcache memcache;
>>
>> @@ -826,7 +826,7 @@
>> }
>>
>> HMemcache.Snapshot retval = memcache.snapshotMemcacheForLog(log);
>> - TreeMap<HStoreKey, BytesWritable> memcacheSnapshot =
>> retval.memcacheSnapshot;
>> + TreeMap<HStoreKey, byte []> memcacheSnapshot =
>> retval.memcacheSnapshot;
>> if(memcacheSnapshot == null) {
>> for(HStore hstore: stores.values()) {
>> Vector<HStoreFile> hstoreFiles = hstore.getAllMapFiles();
>> @@ -885,31 +885,28 @@
>>
>> //////////////////////////////////////////////////////////////////////////////
>>
>>
>> /** Fetch a single data item. */
>> - BytesWritable get(Text row, Text column) throws IOException {
>> - BytesWritable[] results = get(row, column, Long.MAX_VALUE, 1);
>> - return (results == null)? null: results[0];
>> + byte [] get(Text row, Text column) throws IOException {
>> + byte [][] results = get(row, column, Long.MAX_VALUE, 1);
>> + return (results == null || results.length == 0)? null: results[0];
>> }
>>
>> /** Fetch multiple versions of a single data item */
>> - BytesWritable[] get(Text row, Text column, int numVersions) throws
>> IOException {
>> + byte [][] get(Text row, Text column, int numVersions) throws
>> IOException {
>> return get(row, column, Long.MAX_VALUE, numVersions);
>> }
>>
>> /** Fetch multiple versions of a single data item, with timestamp. */
>> - BytesWritable[] get(Text row, Text column, long timestamp, int
>> numVersions)
>> - throws IOException {
>> -
>> + byte [][] get(Text row, Text column, long timestamp, int numVersions)
>> + throws IOException {
>> if(writestate.closed) {
>> throw new IOException("HRegion is closed.");
>> }
>>
>> // Make sure this is a valid row and valid column
>> -
>> checkRow(row);
>> checkColumn(column);
>>
>> // Obtain the row-lock
>> -
>> obtainRowLock(row);
>> try {
>> // Obtain the -col results
>> @@ -921,13 +918,12 @@
>> }
>>
>> /** Private implementation: get the value for the indicated
>> HStoreKey */
>> - private BytesWritable[] get(HStoreKey key, int numVersions) throws
>> IOException {
>> + private byte [][] get(HStoreKey key, int numVersions) throws
>> IOException {
>>
>> lock.obtainReadLock();
>> try {
>> // Check the memcache
>> -
>> - BytesWritable[] result = memcache.get(key, numVersions);
>> + byte [][] result = memcache.get(key, numVersions);
>> if(result != null) {
>> return result;
>> }
>> @@ -957,19 +953,17 @@
>> * determine which column groups are useful for that row. That
>> would let us
>> * avoid a bunch of disk activity.
>> */
>> - TreeMap<Text, BytesWritable> getFull(Text row) throws IOException {
>> + TreeMap<Text, byte []> getFull(Text row) throws IOException {
>> HStoreKey key = new HStoreKey(row, System.currentTimeMillis());
>>
>> lock.obtainReadLock();
>> try {
>> - TreeMap<Text, BytesWritable> memResult = memcache.getFull(key);
>> - for(Iterator<Text> it = stores.keySet().iterator();
>> it.hasNext(); ) {
>> - Text colFamily = it.next();
>> + TreeMap<Text, byte []> memResult = memcache.getFull(key);
>> + for (Text colFamily: stores.keySet()) {
>> HStore targetStore = stores.get(colFamily);
>> targetStore.getFull(key, memResult);
>> }
>> return memResult;
>> -
>> } finally {
>> lock.releaseReadLock();
>> }
>> @@ -1035,9 +1029,8 @@
>> * This method really just tests the input, then calls an internal
>> localput()
>> * method.
>> */
>> - void put(long lockid, Text targetCol, BytesWritable val) throws
>> IOException {
>> - if(val.getSize() == DELETE_BYTES.getSize()
>> - && val.compareTo(DELETE_BYTES) == 0) {
>> + void put(long lockid, Text targetCol, byte [] val) throws
>> IOException {
>> + if (DELETE_BYTES.compareTo(val) == 0) {
>> throw new IOException("Cannot insert value: " + val);
>> }
>> localput(lockid, targetCol, val);
>> @@ -1047,7 +1040,7 @@
>> * Delete a value or write a value. This is a just a convenience
>> method for put().
>> */
>> void delete(long lockid, Text targetCol) throws IOException {
>> - localput(lockid, targetCol, DELETE_BYTES);
>> + localput(lockid, targetCol, DELETE_BYTES.get());
>> }
>>
>> /**
>> @@ -1063,7 +1056,7 @@
>> * @throws IOException
>> */
>> void localput(final long lockid, final Text targetCol,
>> - final BytesWritable val)
>> + final byte [] val)
>> throws IOException {
>> checkColumn(targetCol);
>>
>> @@ -1083,9 +1076,9 @@
>> lockid + " unexpected aborted by another thread");
>> }
>>
>> - TreeMap<Text, BytesWritable> targets =
>> this.targetColumns.get(lockid);
>> + TreeMap<Text, byte []> targets = this.targetColumns.get(lockid);
>> if (targets == null) {
>> - targets = new TreeMap<Text, BytesWritable>();
>> + targets = new TreeMap<Text, byte []>();
>> this.targetColumns.put(lockid, targets);
>> }
>> targets.put(targetCol, val);
>> @@ -1144,8 +1137,7 @@
>> synchronized(row) {
>> // Add updates to the log and add values to the memcache.
>> long commitTimestamp = System.currentTimeMillis();
>> - TreeMap<Text, BytesWritable> columns =
>> - this.targetColumns.get(lockid);
>> + TreeMap<Text, byte []> columns = this.targetColumns.get(lockid);
>> if (columns != null && columns.size() > 0) {
>> log.append(regionInfo.regionName,
>> regionInfo.tableDesc.getName(),
>> row, columns, commitTimestamp);
>> @@ -1267,7 +1259,7 @@
>> */
>> private static class HScanner implements HInternalScannerInterface {
>> private HInternalScannerInterface[] scanners;
>> - private TreeMap<Text, BytesWritable>[] resultSets;
>> + private TreeMap<Text, byte []>[] resultSets;
>> private HStoreKey[] keys;
>> private boolean wildcardMatch;
>> private boolean multipleMatchers;
>> @@ -1323,7 +1315,7 @@
>> }
>> for(int i = 0; i < scanners.length; i++) {
>> keys[i] = new HStoreKey();
>> - resultSets[i] = new TreeMap<Text, BytesWritable>();
>> + resultSets[i] = new TreeMap<Text, byte []>();
>> if(scanners[i] != null && !scanners[i].next(keys[i],
>> resultSets[i])) {
>> closeScanner(i);
>> }
>> @@ -1351,7 +1343,7 @@
>> *
>> * @see
>> org.apache.hadoop.hbase.HInternalScannerInterface#next(org.apache.hadoop.hbase.HStoreKey,
>> java.util.TreeMap)
>> */
>> - public boolean next(HStoreKey key, TreeMap<Text, BytesWritable>
>> results)
>> + public boolean next(HStoreKey key, TreeMap<Text, byte []> results)
>> throws IOException {
>> // Find the lowest-possible key.
>> Text chosenRow = null;
>> @@ -1393,7 +1385,7 @@
>> // values with older ones. So now we only insert
>> // a result if the map does not contain the key.
>>
>> - for(Map.Entry<Text, BytesWritable> e:
>> resultSets[i].entrySet()) {
>> + for(Map.Entry<Text, byte []> e: resultSets[i].entrySet()) {
>> if(!results.containsKey(e.getKey())) {
>> results.put(e.getKey(), e.getValue());
>> insertedItem = true;
>> @@ -1504,7 +1496,7 @@
>> ByteArrayOutputStream bytes = new ByteArrayOutputStream();
>> DataOutputStream s = new DataOutputStream(bytes);
>> r.getRegionInfo().write(s);
>> - meta.put(writeid, COL_REGIONINFO, new
>> BytesWritable(bytes.toByteArray()));
>> + meta.put(writeid, COL_REGIONINFO, bytes.toByteArray());
>> meta.commit(writeid);
>> }
>>
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
>> Mon Jun 18 15:59:14 2007
>> @@ -15,12 +15,11 @@
>> */
>> package org.apache.hadoop.hbase;
>>
>> -import org.apache.hadoop.io.BytesWritable;
>> +import java.io.IOException;
>> +
>> import org.apache.hadoop.io.Text;
>> import org.apache.hadoop.ipc.VersionedProtocol;
>>
>> -import java.io.*;
>> -
>> /*******************************************************************************
>>
>> * Clients interact with HRegionServers using
>> * a handle to the HRegionInterface.
>> @@ -36,57 +35,62 @@
>> * @return - HRegionInfo object for region
>> * @throws NotServingRegionException
>> */
>> - public HRegionInfo getRegionInfo(final Text regionName) throws
>> NotServingRegionException;
>> + public HRegionInfo getRegionInfo(final Text regionName)
>> + throws NotServingRegionException;
>>
>> /**
>> * Retrieve a single value from the specified region for the
>> specified row
>> * and column keys
>> *
>> - * @param regionName - name of region
>> - * @param row - row key
>> - * @param column - column key
>> - * @return - value for that region/row/column
>> + * @param regionName name of region
>> + * @param row row key
>> + * @param column column key
>> + * @return alue for that region/row/column
>> * @throws IOException
>> */
>> - public BytesWritable get(final Text regionName, final Text row,
>> final Text column) throws IOException;
>> + public byte [] get(final Text regionName, final Text row, final
>> Text column)
>> + throws IOException;
>>
>> /**
>> * Get the specified number of versions of the specified row and
>> column
>> *
>> - * @param regionName - region name
>> - * @param row - row key
>> - * @param column - column key
>> - * @param numVersions - number of versions to return
>> - * @return - array of values
>> + * @param regionName region name
>> + * @param row row key
>> + * @param column column key
>> + * @param numVersions number of versions to return
>> + * @return array of values
>> * @throws IOException
>> */
>> - public BytesWritable[] get(final Text regionName, final Text row,
>> - final Text column, final int numVersions) throws IOException;
>> + public byte [][] get(final Text regionName, final Text row,
>> + final Text column, final int numVersions)
>> + throws IOException;
>>
>> /**
>> * Get the specified number of versions of the specified row and
>> column with
>> * the specified timestamp.
>> *
>> - * @param regionName - region name
>> - * @param row - row key
>> - * @param column - column key
>> - * @param timestamp - timestamp
>> - * @param numVersions - number of versions to return
>> - * @return - array of values
>> + * @param regionName region name
>> + * @param row row key
>> + * @param column column key
>> + * @param timestamp timestamp
>> + * @param numVersions number of versions to return
>> + * @return array of values
>> * @throws IOException
>> */
>> - public BytesWritable[] get(final Text regionName, final Text row,
>> final Text column,
>> - final long timestamp, final int numVersions) throws IOException;
>> + public byte [][] get(final Text regionName, final Text row,
>> + final Text column, final long timestamp, final int numVersions)
>> + throws IOException;
>>
>> /**
>> * Get all the data for the specified row
>> *
>> - * @param regionName - region name
>> - * @param row - row key
>> - * @return - array of values
>> + * @param regionName region name
>> + * @param row row key
>> + * @return array of values
>> * @throws IOException
>> */
>> - public KeyedData[] getRow(final Text regionName, final Text row)
>> throws IOException;
>> + public KeyedData[] getRow(final Text regionName, final Text row)
>> + throws IOException;
>>
>>
>> //////////////////////////////////////////////////////////////////////////////
>>
>> // Start an atomic row insertion/update. No changes are committed
>> until the
>> @@ -110,67 +114,72 @@
>> * The client can gain extra time with a call to renewLease().
>> * Start an atomic row insertion or update
>> *
>> - * @param regionName - region name
>> - * @param clientid - a unique value to identify the client
>> - * @param row - Name of row to start update against.
>> + * @param regionName region name
>> + * @param clientid a unique value to identify the client
>> + * @param row Name of row to start update against.
>> * @return Row lockid.
>> * @throws IOException
>> */
>> public long startUpdate(final Text regionName, final long clientid,
>> - final Text row) throws IOException;
>> + final Text row)
>> + throws IOException;
>>
>> /**
>> * Change a value for the specified column
>> *
>> - * @param regionName - region name
>> - * @param clientid - a unique value to identify the client
>> - * @param lockid - lock id returned from startUpdate
>> - * @param column - column whose value is being set
>> - * @param val - new value for column
>> + * @param regionName region name
>> + * @param clientid a unique value to identify the client
>> + * @param lockid lock id returned from startUpdate
>> + * @param column column whose value is being set
>> + * @param val new value for column
>> * @throws IOException
>> */
>> public void put(final Text regionName, final long clientid, final
>> long lockid,
>> - final Text column, final BytesWritable val) throws IOException;
>> + final Text column, final byte [] val)
>> + throws IOException;
>>
>> /**
>> * Delete the value for a column
>> *
>> - * @param regionName - region name
>> - * @param clientid - a unique value to identify the client
>> - * @param lockid - lock id returned from startUpdate
>> - * @param column - name of column whose value is to
>> be deleted
>> + * @param regionName region name
>> + * @param clientid a unique value to identify the client
>> + * @param lockid lock id returned from startUpdate
>> + * @param column name of column whose value is to be deleted
>> * @throws IOException
>> */
>> - public void delete(final Text regionName, final long clientid,
>> final long lockid,
>> - final Text column) throws IOException;
>> + public void delete(final Text regionName, final long clientid,
>> + final long lockid, final Text column)
>> + throws IOException;
>>
>> /**
>> * Abort a row mutation
>> *
>> - * @param regionName - region name
>> - * @param clientid - a unique value to identify the client
>> - * @param lockid - lock id returned from startUpdate
>> + * @param regionName region name
>> + * @param clientid a unique value to identify the client
>> + * @param lockid lock id returned from startUpdate
>> * @throws IOException
>> */
>> public void abort(final Text regionName, final long clientid,
>> - final long lockid) throws IOException;
>> + final long lockid)
>> + throws IOException;
>>
>> /**
>> * Finalize a row mutation
>> *
>> - * @param regionName - region name
>> - * @param clientid - a unique value to identify the client
>> - * @param lockid - lock id returned from startUpdate
>> + * @param regionName region name
>> + * @param clientid a unique value to identify the client
>> + * @param lockid lock id returned from startUpdate
>> * @throws IOException
>> */
>> public void commit(final Text regionName, final long clientid,
>> - final long lockid) throws IOException;
>> + final long lockid)
>> + throws IOException;
>>
>> /**
>> * Renew lease on update
>> *
>> - * @param lockid - lock id returned from startUpdate
>> - * @param clientid - a unique value to identify the client
>> + * @param lockid lock id returned from startUpdate
>> + * @param clientid a unique value to identify the client
>> * @throws IOException
>> */
>> public void renewLease(long lockid, long clientid) throws
>> IOException;
>> @@ -182,20 +191,21 @@
>> /**
>> * Opens a remote scanner.
>> *
>> - * @param regionName - name of region to scan
>> - * @param columns - columns to scan
>> - * @param startRow - starting row to scan
>> + * @param regionName name of region to scan
>> + * @param columns columns to scan
>> + * @param startRow starting row to scan
>> *
>> - * @return scannerId - scanner identifier used in other calls
>> + * @return scannerId scanner identifier used in other calls
>> * @throws IOException
>> */
>> - public long openScanner(Text regionName, Text[] columns, Text
>> startRow) throws IOException;
>> + public long openScanner(Text regionName, Text[] columns, Text
>> startRow)
>> + throws IOException;
>>
>> /**
>> * Get the next set of values
>> *
>> - * @param scannerId - clientId passed to openScanner
>> - * @return - array of values
>> + * @param scannerId clientId passed to openScanner
>> + * @return array of values
>> * @throws IOException
>> */
>> public KeyedData[] next(long scannerId) throws IOException;
>> @@ -203,7 +213,7 @@
>> /**
>> * Close a scanner
>> *
>> - * @param scannerId - the scanner id returned by openScanner
>> + * @param scannerId the scanner id returned by openScanner
>> * @throws IOException
>> */
>> public void close(long scannerId) throws IOException;
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
>> Mon Jun 18 15:59:14 2007
>> @@ -33,7 +33,6 @@
>> import org.apache.hadoop.conf.Configuration;
>> import org.apache.hadoop.fs.FileSystem;
>> import org.apache.hadoop.fs.Path;
>> -import org.apache.hadoop.io.BytesWritable;
>> import org.apache.hadoop.io.Text;
>> import org.apache.hadoop.io.retry.RetryProxy;
>> import org.apache.hadoop.ipc.RPC;
>> @@ -396,8 +395,6 @@
>> Collections.synchronizedSortedMap(new TreeMap<Text, HRegion>());
>>
>> this.outboundMsgs = new Vector<HMsg>();
>> - this.scanners =
>> - Collections.synchronizedMap(new TreeMap<Text,
>> HInternalScannerInterface>());
>>
>> // Config'ed params
>> this.numRetries = conf.getInt("hbase.client.retries.number", 2);
>> @@ -914,27 +911,26 @@
>> /* (non-Javadoc)
>> * @see
>> org.apache.hadoop.hbase.HRegionInterface#get(org.apache.hadoop.io.Text,
>> org.apache.hadoop.io.Text, org.apache.hadoop.io.Text)
>> */
>> - public BytesWritable get(final Text regionName, final Text row,
>> - final Text column) throws IOException {
>> -
>> + public byte [] get(final Text regionName, final Text row,
>> + final Text column)
>> + throws IOException {
>> return getRegion(regionName).get(row, column);
>> }
>>
>> /* (non-Javadoc)
>> * @see
>> org.apache.hadoop.hbase.HRegionInterface#get(org.apache.hadoop.io.Text,
>> org.apache.hadoop.io.Text, org.apache.hadoop.io.Text, int)
>> */
>> - public BytesWritable[] get(final Text regionName, final Text row,
>> - final Text column, final int numVersions) throws IOException {
>> -
>> + public byte [][] get(final Text regionName, final Text row,
>> + final Text column, final int numVersions)
>> + throws IOException {
>> return getRegion(regionName).get(row, column, numVersions);
>> }
>>
>> /* (non-Javadoc)
>> * @see
>> org.apache.hadoop.hbase.HRegionInterface#get(org.apache.hadoop.io.Text,
>> org.apache.hadoop.io.Text, org.apache.hadoop.io.Text, long, int)
>> */
>> - public BytesWritable[] get(final Text regionName, final Text row,
>> final Text column,
>> + public byte [][] get(final Text regionName, final Text row, final
>> Text column,
>> final long timestamp, final int numVersions) throws IOException {
>> -
>> return getRegion(regionName).get(row, column, timestamp,
>> numVersions);
>> }
>>
>> @@ -943,10 +939,10 @@
>> */
>> public KeyedData[] getRow(final Text regionName, final Text row)
>> throws IOException {
>> HRegion region = getRegion(regionName);
>> - TreeMap<Text, BytesWritable> map = region.getFull(row);
>> + TreeMap<Text, byte[]> map = region.getFull(row);
>> KeyedData result[] = new KeyedData[map.size()];
>> int counter = 0;
>> - for (Map.Entry<Text, BytesWritable> es: map.entrySet()) {
>> + for (Map.Entry<Text, byte []> es: map.entrySet()) {
>> result[counter++] =
>> new KeyedData(new HStoreKey(row, es.getKey()), es.getValue());
>> }
>> @@ -957,30 +953,28 @@
>> * @see org.apache.hadoop.hbase.HRegionInterface#next(long)
>> */
>> public KeyedData[] next(final long scannerId)
>> - throws IOException {
>> -
>> - Text scannerName = new Text(String.valueOf(scannerId));
>> + throws IOException {
>> + String scannerName = String.valueOf(scannerId);
>> HInternalScannerInterface s = scanners.get(scannerName);
>> if (s == null) {
>> throw new UnknownScannerException("Name: " + scannerName);
>> }
>> - leases.renewLease(scannerName, scannerName);
>> + leases.renewLease(scannerId, scannerId);
>>
>> // Collect values to be returned here
>>
>> ArrayList<KeyedData> values = new ArrayList<KeyedData>();
>>
>> - TreeMap<Text, BytesWritable> results = new TreeMap<Text,
>> BytesWritable>();
>> + TreeMap<Text, byte []> results = new TreeMap<Text, byte []>();
>>
>> // Keep getting rows until we find one that has at least one
>> non-deleted column value
>>
>> HStoreKey key = new HStoreKey();
>> while (s.next(key, results)) {
>> - for(Map.Entry<Text, BytesWritable> e: results.entrySet()) {
>> + for(Map.Entry<Text, byte []> e: results.entrySet()) {
>> HStoreKey k = new HStoreKey(key.getRow(), e.getKey(),
>> key.getTimestamp());
>> - BytesWritable val = e.getValue();
>> - if(val.getSize() == DELETE_BYTES.getSize()
>> - && val.compareTo(DELETE_BYTES) == 0) {
>> + byte [] val = e.getValue();
>> + if (DELETE_BYTES.compareTo(val) == 0) {
>> // Column value is deleted. Don't return it.
>> if (LOG.isDebugEnabled()) {
>> LOG.debug("skipping deleted value for key: " +
>> k.toString());
>> @@ -1011,10 +1005,8 @@
>> throws IOException {
>> HRegion region = getRegion(regionName);
>> long lockid = region.startUpdate(row);
>> - this.leases.createLease(new Text(String.valueOf(clientid)),
>> - new Text(String.valueOf(lockid)),
>> - new RegionListener(region, lockid));
>> -
>> + this.leases.createLease(clientid, lockid,
>> + new RegionListener(region, lockid));
>> return lockid;
>> }
>>
>> @@ -1041,11 +1033,11 @@
>> /* (non-Javadoc)
>> * @see
>> org.apache.hadoop.hbase.HRegionInterface#put(org.apache.hadoop.io.Text,
>> long, long, org.apache.hadoop.io.Text,
>> org.apache.hadoop.io.BytesWritable)
>> */
>> - public void put(Text regionName, long clientid, long lockid, Text
>> column,
>> - BytesWritable val) throws IOException {
>> + public void put(final Text regionName, final long clientid,
>> + final long lockid, final Text column, final byte [] val)
>> + throws IOException {
>> HRegion region = getRegion(regionName, true);
>> - leases.renewLease(new Text(String.valueOf(clientid)),
>> - new Text(String.valueOf(lockid)));
>> + leases.renewLease(clientid, lockid);
>> region.put(lockid, column, val);
>> }
>>
>> @@ -1053,10 +1045,9 @@
>> * @see
>> org.apache.hadoop.hbase.HRegionInterface#delete(org.apache.hadoop.io.Text,
>> long, long, org.apache.hadoop.io.Text)
>> */
>> public void delete(Text regionName, long clientid, long lockid,
>> Text column)
>> - throws IOException {
>> + throws IOException {
>> HRegion region = getRegion(regionName);
>> - leases.renewLease(new Text(String.valueOf(clientid)),
>> - new Text(String.valueOf(lockid)));
>> + leases.renewLease(clientid, lockid);
>> region.delete(lockid, column);
>> }
>>
>> @@ -1064,10 +1055,9 @@
>> * @see
>> org.apache.hadoop.hbase.HRegionInterface#abort(org.apache.hadoop.io.Text,
>> long, long)
>> */
>> public void abort(Text regionName, long clientid, long lockid)
>> - throws IOException {
>> + throws IOException {
>> HRegion region = getRegion(regionName, true);
>> - leases.cancelLease(new Text(String.valueOf(clientid)),
>> - new Text(String.valueOf(lockid)));
>> + leases.cancelLease(clientid, lockid);
>> region.abort(lockid);
>> }
>>
>> @@ -1077,8 +1067,7 @@
>> public void commit(Text regionName, long clientid, long lockid)
>> throws IOException {
>> HRegion region = getRegion(regionName, true);
>> - leases.cancelLease(new Text(String.valueOf(clientid)),
>> - new Text(String.valueOf(lockid)));
>> + leases.cancelLease(clientid, lockid);
>> region.commit(lockid);
>> }
>>
>> @@ -1086,8 +1075,7 @@
>> * @see org.apache.hadoop.hbase.HRegionInterface#renewLease(long,
>> long)
>> */
>> public void renewLease(long lockid, long clientid) throws
>> IOException {
>> - leases.renewLease(new Text(String.valueOf(clientid)),
>> - new Text(String.valueOf(lockid)));
>> + leases.renewLease(clientid, lockid);
>> }
>>
>> /**
>> @@ -1139,29 +1127,31 @@
>> // remote scanner interface
>>
>> //////////////////////////////////////////////////////////////////////////////
>>
>>
>> - Map<Text, HInternalScannerInterface> scanners;
>> + Map<String, HInternalScannerInterface> scanners =
>> + Collections.synchronizedMap(new HashMap<String,
>> + HInternalScannerInterface>());
>>
>> /**
>> * Instantiated as a scanner lease.
>> * If the lease times out, the scanner is closed
>> */
>> private class ScannerListener implements LeaseListener {
>> - private Text scannerName;
>> + private final String scannerName;
>>
>> - ScannerListener(Text scannerName) {
>> - this.scannerName = scannerName;
>> + ScannerListener(final String n) {
>> + this.scannerName = n;
>> }
>>
>> /* (non-Javadoc)
>> * @see org.apache.hadoop.hbase.LeaseListener#leaseExpired()
>> */
>> public void leaseExpired() {
>> - LOG.info("Scanner " + scannerName + " lease expired");
>> + LOG.info("Scanner " + this.scannerName + " lease expired");
>> HInternalScannerInterface s = null;
>> synchronized(scanners) {
>> - s = scanners.remove(scannerName);
>> + s = scanners.remove(this.scannerName);
>> }
>> - if(s != null) {
>> + if (s != null) {
>> s.close();
>> }
>> }
>> @@ -1177,11 +1167,11 @@
>> try {
>> HInternalScannerInterface s = r.getScanner(cols, firstRow);
>> scannerId = rand.nextLong();
>> - Text scannerName = new Text(String.valueOf(scannerId));
>> + String scannerName = String.valueOf(scannerId);
>> synchronized(scanners) {
>> scanners.put(scannerName, s);
>> }
>> - leases.createLease(scannerName, scannerName,
>> + leases.createLease(scannerId, scannerId,
>> new ScannerListener(scannerName));
>> } catch(IOException e) {
>> LOG.error(e);
>> @@ -1193,8 +1183,8 @@
>> /* (non-Javadoc)
>> * @see org.apache.hadoop.hbase.HRegionInterface#close(long)
>> */
>> - public void close(long scannerId) throws IOException {
>> - Text scannerName = new Text(String.valueOf(scannerId));
>> + public void close(final long scannerId) throws IOException {
>> + String scannerName = String.valueOf(scannerId);
>> HInternalScannerInterface s = null;
>> synchronized(scanners) {
>> s = scanners.remove(scannerName);
>> @@ -1203,7 +1193,7 @@
>> throw new UnknownScannerException(scannerName.toString());
>> }
>> s.close();
>> - leases.cancelLease(scannerName, scannerName);
>> + leases.cancelLease(scannerId, scannerId);
>> }
>>
>> private static void printUsageAndExit() {
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java
>> Mon Jun 18 15:59:14 2007
>> @@ -28,7 +28,6 @@
>> import org.apache.hadoop.fs.FileSystem;
>> import org.apache.hadoop.fs.Path;
>> import org.apache.hadoop.fs.PathFilter;
>> -import org.apache.hadoop.io.BytesWritable;
>> import org.apache.hadoop.io.Text;
>>
>> /**
>> @@ -168,7 +167,7 @@
>> Text [] families =
>> info.tableDesc.families().keySet().toArray(new Text [] {});
>> HInternalScannerInterface scanner = r.getScanner(families, new
>> Text());
>> HStoreKey key = new HStoreKey();
>> - TreeMap<Text, BytesWritable> results = new TreeMap<Text,
>> BytesWritable>();
>> + TreeMap<Text, byte []> results = new TreeMap<Text, byte []>();
>> // Print out table header line.
>> String s = info.startKey.toString();
>> String startKey = (s == null || s.length() <= 0)? "<>": s;
>> @@ -184,19 +183,15 @@
>> // Every line starts with row name followed by column name
>> // followed by cell content.
>> while(scanner.next(key, results)) {
>> - for (Map.Entry<Text, BytesWritable> es: results.entrySet()) {
>> - Text colname = es.getKey();
>> - BytesWritable colvalue = es.getValue();
>> + for (Map.Entry<Text, byte []> es: results.entrySet()) {
>> + Text colname = es.getKey();
>> + byte [] colvalue = es.getValue();
>> Object value = null;
>> - byte[] bytes = new byte[colvalue.getSize()];
>> if (colname.toString().equals("info:regioninfo")) {
>> - // Then bytes are instance of an HRegionInfo.
>> - System.arraycopy(colvalue, 0, bytes, 0, bytes.length);
>> - value = new HRegionInfo(bytes);
>> + value = new HRegionInfo(colvalue);
>> } else {
>> - value = new String(bytes, HConstants.UTF8_ENCODING);
>> + value = new String(colvalue, HConstants.UTF8_ENCODING);
>> }
>> -
>> System.out.println(" " + key + ", " + colname.toString() +
>> ": \"" +
>> value.toString() + "\"");
>> }
>>
>> Modified:
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HScannerInterface.java
>>
>> URL:
>> http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HScannerInterface.java?view=diff&rev=548523&r1=548522&r2=548523
>>
>> ==============================================================================
>>
>> ---
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HScannerInterface.java
>> (original)
>> +++
>> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HScannerInterface.java
>> Mon Jun 18 15:59:14 2007
>> @@ -24,6 +24,7 @@
>> * 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 boolean next(HStoreKey key, TreeMap<Text, byte[]> results)
>> + throws IOException;
>> public void close() throws IOException;
>> }
>>
>>
>
Re: svn commit: r548523 [1/2] - in /lucene/hadoop/trunk/src/contrib/hbase: ./ src/java/org/apache/hadoop/hbase/ src/test/org/apache/hadoop/hbase/
Posted by Nigel Daley <nd...@yahoo-inc.com>.
This commit seems to have broken trunk.
http://lucene.zones.apache.org:8080/hudson/job/Hadoop-Nightly/127/
console
Cheers,
Nige
On Jun 18, 2007, at 3:59 PM, cutting@apache.org wrote:
> Author: cutting
> Date: Mon Jun 18 15:59:14 2007
> New Revision: 548523
>
> URL: http://svn.apache.org/viewvc?view=rev&rev=548523
> Log:
> HADOOP-1498. Replace boxed types with primitives in many places.
> Contributed by stack.
>
> Modified:
> lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/BloomFilterDescriptor.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HAbstractScanner.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HClient.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HConstants.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HInternalScannerInterface.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HLog.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HLogEdit.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HLogKey.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMaster.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMasterInterface.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMasterRegionInterface.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMemcache.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMerge.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegion.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegionInterface.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegionServer.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegiondirReader.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HScannerInterface.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HServerAddress.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HStore.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HStoreFile.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HStoreKey.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/KeyedData.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/LeaseListener.java
> lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/Leases.java
> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/
> hadoop/hbase/AbstractMergeTestBase.java
> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/
> hadoop/hbase/TestGet.java
> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/
> hadoop/hbase/TestHBaseCluster.java
> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/
> hadoop/hbase/TestHLog.java
> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/
> hadoop/hbase/TestHMemcache.java
> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/
> hadoop/hbase/TestHRegion.java
> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/
> hadoop/hbase/TestScanner.java
> lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/
> hadoop/hbase/TestScanner2.java
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/CHANGES.txt?view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt Mon Jun 18
> 15:59:14 2007
> @@ -34,4 +34,4 @@
> 19. HADOOP-1415 Integrate BSD licensed bloom filter implementation.
> 20. HADOOP-1465 Add cluster stop/start scripts for hbase
> 21. HADOOP-1415 Provide configurable per-column bloom filters -
> part 2.
> -
> + 22. HADOOP-1498. Replace boxed types with primitives in many places.
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/BloomFilterDescriptor.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/BloomFilterDescriptor.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/BloomFilterDescriptor.java Mon Jun 18 15:59:14 2007
> @@ -47,7 +47,9 @@
> public static final int RETOUCHED_BLOOMFILTER = 3;
>
> /** Default constructor - used in conjunction with Writable */
> - public BloomFilterDescriptor() {}
> + public BloomFilterDescriptor() {
> + super();
> + }
>
> /**
> * @param type The kind of bloom filter to use.
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HAbstractScanner.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HAbstractScanner.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HAbstractScanner.java Mon Jun 18 15:59:14 2007
> @@ -18,29 +18,26 @@
> import java.io.IOException;
> import java.util.TreeMap;
> import java.util.Vector;
> -
> import java.util.regex.Pattern;
>
> import org.apache.commons.logging.Log;
> import org.apache.commons.logging.LogFactory;
> 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
> HInternalScannerInterface {
> final Log LOG = LogFactory.getLog(this.getClass().getName());
>
> // Pattern to determine if a column key is a regex
> -
> - static Pattern isRegexPattern = Pattern.compile("^.*[\\\\+|^&*$\\
> [\\]\\}{)(]+.*$");
> + static Pattern isRegexPattern =
> + Pattern.compile("^.*[\\\\+|^&*$\\[\\]\\}{)(]+.*$");
>
> // The kind of match we are doing on a column:
> -
> private static enum MATCH_TYPE {
> /** Just check the column family name */
> FAMILY_ONLY,
> @@ -55,7 +52,6 @@
> // 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 static class ColumnMatcher {
> private boolean wildCardmatch;
> private MATCH_TYPE matchType;
> @@ -63,33 +59,24 @@
> private Pattern columnMatcher;
> private Text col;
>
> - ColumnMatcher(Text col) throws IOException {
> - String column = col.toString();
> + ColumnMatcher(final Text col) throws IOException {
> + Text qualifier = HStoreKey.extractQualifier(col);
> try {
> - int colpos = column.indexOf(":");
> - if(colpos == -1) {
> - throw new InvalidColumnNameException("Column name has no
> family indicator.");
> - }
> -
> - String columnkey = column.substring(colpos + 1);
> -
> - if(columnkey == null || columnkey.length() == 0) {
> + if(qualifier == null || qualifier.getLength() == 0) {
> this.matchType = MATCH_TYPE.FAMILY_ONLY;
> - this.family = column.substring(0, colpos);
> + this.family = HStoreKey.extractFamily(col).toString();
> this.wildCardmatch = true;
> -
> - } else if(isRegexPattern.matcher(columnkey).matches()) {
> + } else if(isRegexPattern.matcher(qualifier.toString
> ()).matches()) {
> this.matchType = MATCH_TYPE.REGEX;
> - this.columnMatcher = Pattern.compile(column);
> + this.columnMatcher = Pattern.compile(col.toString());
> this.wildCardmatch = true;
> -
> } else {
> this.matchType = MATCH_TYPE.SIMPLE;
> this.col = col;
> this.wildCardmatch = false;
> }
> } catch(Exception e) {
> - throw new IOException("Column: " + column + ": " +
> e.getMessage());
> + throw new IOException("Column: " + col + ": " +
> e.getMessage());
> }
> }
>
> @@ -119,8 +106,10 @@
>
> 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
> + // Keys retrieved from the sources
> + protected HStoreKey keys[];
> + // Values that correspond to those keys
> + protected byte [][] vals;
>
> protected long timestamp; //
> The timestamp to match entries against
> private boolean wildcardMatch;
> @@ -218,7 +207,7 @@
> *
> * @see org.apache.hadoop.hbase.HScannerInterface#next
> (org.apache.hadoop.hbase.HStoreKey, java.util.TreeMap)
> */
> - public boolean next(HStoreKey key, TreeMap<Text, BytesWritable>
> results)
> + public boolean next(HStoreKey key, TreeMap<Text, byte []> results)
> throws IOException {
> // Find the next row label (and timestamp)
> Text chosenRow = null;
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HClient.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HClient.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HClient.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HClient.java Mon Jun 18 15:59:14 2007
> @@ -30,11 +30,10 @@
> import org.apache.commons.logging.Log;
> import org.apache.commons.logging.LogFactory;
> import org.apache.hadoop.conf.Configuration;
> -import org.apache.hadoop.io.BytesWritable;
> import org.apache.hadoop.io.DataInputBuffer;
> import org.apache.hadoop.io.Text;
> -import org.apache.hadoop.io.retry.RetryPolicy;
> import org.apache.hadoop.io.retry.RetryPolicies;
> +import org.apache.hadoop.io.retry.RetryPolicy;
> import org.apache.hadoop.io.retry.RetryProxy;
> import org.apache.hadoop.ipc.RPC;
> import org.apache.hadoop.ipc.RemoteException;
> @@ -285,10 +284,8 @@
> }
> boolean found = false;
> for(int j = 0; j < values.length; j++) {
> - if(values[j].getKey().getColumn().equals(COL_REGIONINFO)) {
> - byte[] bytes = new byte[values[j].getData().getSize()];
> - System.arraycopy(values[j].getData().get(), 0, bytes,
> 0, bytes.length);
> - inbuf.reset(bytes, bytes.length);
> + if (values[j].getKey().getColumn().equals
> (COL_REGIONINFO)) {
> + inbuf.reset(values[j].getData(), values[j].getData
> ().length);
> info.readFields(inbuf);
> if(info.tableDesc.getName().equals(tableName)) {
> found = true;
> @@ -398,9 +395,7 @@
> valuesfound += 1;
> for(int j = 0; j < values.length; j++) {
> if(values[j].getKey().getColumn().equals
> (COL_REGIONINFO)) {
> - byte[] bytes = new byte[values[j].getData().getSize()];
> - System.arraycopy(values[j].getData().get(), 0,
> bytes, 0, bytes.length);
> - inbuf.reset(bytes, bytes.length);
> + inbuf.reset(values[j].getData(), values[j].getData
> ().length);
> info.readFields(inbuf);
> isenabled = !info.offLine;
> break;
> @@ -483,9 +478,7 @@
> valuesfound += 1;
> for(int j = 0; j < values.length; j++) {
> if(values[j].getKey().getColumn().equals
> (COL_REGIONINFO)) {
> - byte[] bytes = new byte[values[j].getData().getSize()];
> - System.arraycopy(values[j].getData().get(), 0,
> bytes, 0, bytes.length);
> - inbuf.reset(bytes, bytes.length);
> + inbuf.reset(values[j].getData(), values[j].getData
> ().length);
> info.readFields(inbuf);
> disabled = info.offLine;
> break;
> @@ -737,8 +730,8 @@
> * @throws IOException
> */
> private TreeMap<Text, RegionLocation> scanOneMetaRegion(final
> RegionLocation t,
> - final Text tableName) throws IOException {
> -
> + final Text tableName)
> + throws IOException {
> HRegionInterface server = getHRegionConnection(t.serverAddress);
> TreeMap<Text, RegionLocation> servers = new TreeMap<Text,
> RegionLocation>();
> for(int tries = 0; servers.size() == 0 && tries <
> this.numRetries;
> @@ -772,9 +765,7 @@
> byte[] bytes = null;
> TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>
> ();
> for(int i = 0; i < values.length; i++) {
> - bytes = new byte[values[i].getData().getSize()];
> - System.arraycopy(values[i].getData().get(), 0, bytes,
> 0, bytes.length);
> - results.put(values[i].getKey().getColumn(), bytes);
> + results.put(values[i].getKey().getColumn(), values
> [i].getData());
> }
> regionInfo = new HRegionInfo();
> bytes = results.get(COL_REGIONINFO);
> @@ -900,8 +891,7 @@
> }
> for(int i = 0; i < values.length; i++) {
> if(values[i].getKey().getColumn().equals
> (COL_REGIONINFO)) {
> - byte[] bytes = values[i].getData().get();
> - inbuf.reset(bytes, bytes.length);
> + inbuf.reset(values[i].getData(), values[i].getData
> ().length);
> HRegionInfo info = new HRegionInfo();
> info.readFields(inbuf);
>
> @@ -967,22 +957,19 @@
> /**
> * Get a single value for the specified row and column
> *
> - * @param row - row key
> - * @param column - column name
> - * @return - value for specified row/column
> + * @param row row key
> + * @param column column name
> + * @return value for specified row/column
> * @throws IOException
> */
> public byte[] get(Text row, Text column) throws IOException {
> RegionLocation info = null;
> - BytesWritable value = null;
> -
> + byte [] value = null;
> for(int tries = 0; tries < numRetries && info == null; tries++) {
> info = getRegionLocation(row);
> -
> try {
> - value = getHRegionConnection(info.serverAddress).get(
> - info.regionInfo.regionName, row, column);
> -
> + value = getHRegionConnection(info.serverAddress).
> + get(info.regionInfo.regionName, row, column);
> } catch(NotServingRegionException e) {
> if(tries == numRetries - 1) {
> // No more tries
> @@ -992,13 +979,7 @@
> info = null;
> }
> }
> -
> - if(value != null) {
> - byte[] bytes = new byte[value.getSize()];
> - System.arraycopy(value.get(), 0, bytes, 0, bytes.length);
> - return bytes;
> - }
> - return null;
> + return value;
> }
>
> /**
> @@ -1012,15 +993,12 @@
> */
> public byte[][] get(Text row, Text column, int numVersions)
> throws IOException {
> RegionLocation info = null;
> - BytesWritable[] values = null;
> -
> + byte [][] values = null;
> for(int tries = 0; tries < numRetries && info == null; tries++) {
> info = getRegionLocation(row);
> -
> try {
> values = getHRegionConnection(info.serverAddress).get(
> - info.regionInfo.regionName, row, column, numVersions);
> -
> + info.regionInfo.regionName, row, column, numVersions);
> } catch(NotServingRegionException e) {
> if(tries == numRetries - 1) {
> // No more tries
> @@ -1034,9 +1012,7 @@
> if(values != null) {
> ArrayList<byte[]> bytes = new ArrayList<byte[]>();
> for(int i = 0 ; i < values.length; i++) {
> - byte[] value = new byte[values[i].getSize()];
> - System.arraycopy(values[i].get(), 0, value, 0, value.length);
> - bytes.add(value);
> + bytes.add(values[i]);
> }
> return bytes.toArray(new byte[values.length][]);
> }
> @@ -1057,14 +1033,12 @@
> public byte[][] get(Text row, Text column, long timestamp, int
> numVersions)
> throws IOException {
> RegionLocation info = null;
> - BytesWritable[] values = null;
> -
> + byte [][] values = null;
> for(int tries = 0; tries < numRetries && info == null; tries++) {
> info = getRegionLocation(row);
> -
> try {
> - values = getHRegionConnection(info.serverAddress).get(
> - info.regionInfo.regionName, row, column, timestamp,
> numVersions);
> + values = getHRegionConnection(info.serverAddress).
> + get(info.regionInfo.regionName, row, column, timestamp,
> numVersions);
>
> } catch(NotServingRegionException e) {
> if(tries == numRetries - 1) {
> @@ -1079,9 +1053,7 @@
> if(values != null) {
> ArrayList<byte[]> bytes = new ArrayList<byte[]>();
> for(int i = 0 ; i < values.length; i++) {
> - byte[] value = new byte[values[i].getSize()];
> - System.arraycopy(values[i].get(), 0, value, 0, value.length);
> - bytes.add(value);
> + bytes.add(values[i]);
> }
> return bytes.toArray(new byte[values.length][]);
> }
> @@ -1118,9 +1090,7 @@
> TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
> if(value != null && value.length != 0) {
> for(int i = 0; i < value.length; i++) {
> - byte[] bytes = new byte[value[i].getData().getSize()];
> - System.arraycopy(value[i].getData().get(), 0, bytes, 0,
> bytes.length);
> - results.put(value[i].getKey().getColumn(), bytes);
> + results.put(value[i].getKey().getColumn(), value[i].getData
> ());
> }
> }
> return results;
> @@ -1242,7 +1212,7 @@
> public void put(long lockid, Text column, byte val[]) throws
> IOException {
> try {
> this.currentServer.put(this.currentRegion, this.clientid,
> lockid, column,
> - new BytesWritable(val));
> + val);
> } catch(IOException e) {
> try {
> this.currentServer.abort(this.currentRegion,
> this.clientid, lockid);
> @@ -1432,9 +1402,7 @@
> key.setRow(values[i].getKey().getRow());
> key.setVersion(values[i].getKey().getTimestamp());
> key.setColumn(EMPTY_COLUMN);
> - byte[] bytes = new byte[values[i].getData().getSize()];
> - System.arraycopy(values[i].getData().get(), 0, bytes, 0,
> bytes.length);
> - results.put(values[i].getKey().getColumn(), bytes);
> + results.put(values[i].getKey().getColumn(), values
> [i].getData());
> }
> }
> return values == null ? false : values.length != 0;
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HConstants.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HConstants.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HConstants.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HConstants.java Mon Jun 18 15:59:14 2007
> @@ -15,7 +15,7 @@
> */
> package org.apache.hadoop.hbase;
>
> -import org.apache.hadoop.io.BytesWritable;
> +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
> import org.apache.hadoop.io.Text;
>
> /**
> @@ -106,11 +106,10 @@
> static final String UTF8_ENCODING = "UTF-8";
>
> /** Value stored for a deleted item */
> - static final BytesWritable DELETE_BYTES =
> - new BytesWritable("HBASE::DELETEVAL".getBytes());
> + static final ImmutableBytesWritable DELETE_BYTES =
> + new ImmutableBytesWritable("HBASE::DELETEVAL".getBytes());
>
> /** Value written to HLog on a complete cache flush */
> - static final BytesWritable COMPLETE_CACHEFLUSH =
> - new BytesWritable("HBASE::CACHEFLUSH".getBytes());
> -
> + static final ImmutableBytesWritable COMPLETE_CACHEFLUSH =
> + new ImmutableBytesWritable("HBASE::CACHEFLUSH".getBytes());
> }
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HInternalScannerInterface.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/
> HInternalScannerInterface.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HInternalScannerInterface.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HInternalScannerInterface.java Mon Jun 18 15:59:14 2007
> @@ -18,27 +18,29 @@
> import java.io.IOException;
> import java.util.TreeMap;
>
> -import org.apache.hadoop.io.BytesWritable;
> import org.apache.hadoop.io.Text;
>
> /**
> * Internally, we need to be able to determine if the scanner is
> doing wildcard
> * column matches (when only a column family is specified or if a
> column regex
> - * is specified) or if multiple members of the same column family
> were specified.
> - *
> - * If so, we need to ignore the timestamp to ensure that we get
> all the family
> - * members, as they may have been last updated at different times.
> - *
> + * is specified) or if multiple members of the same column family
> were
> + * specified. If so, we need to ignore the timestamp to ensure
> that we get all
> + * the family members, as they may have been last updated at
> different times.
> * This interface exposes two APIs for querying the scanner.
> */
> public interface HInternalScannerInterface {
>
> - public boolean next(HStoreKey key, TreeMap<Text, BytesWritable>
> results) throws IOException;
> + public boolean next(HStoreKey key, TreeMap<Text, byte []> results)
> + throws IOException;
> +
> + /**
> + * Close the scanner.
> + */
> public void close();
> +
> /** Returns true if the scanner is matching a column family or
> regex */
> public boolean isWildcardScanner();
>
> /** Returns true if the scanner is matching multiple column
> family members */
> public boolean isMultipleMatchScanner();
> -
> -}
> +}
> \ No newline at end of file
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HLog.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HLog.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HLog.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HLog.java Mon Jun 18 15:59:14 2007
> @@ -78,7 +78,7 @@
> long filenum = 0;
> transient int numEntries = 0;
>
> - Integer rollLock = 0;
> + Integer rollLock = new Integer(0);
>
> /**
> * Split up a bunch of log files, that are no longer being
> written to,
> @@ -205,6 +205,7 @@
> try {
> wait();
> } catch (InterruptedException ie) {
> + // continue;
> }
> }
>
> @@ -282,8 +283,8 @@
> * 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));
> + Path computeFilename(final long fn) {
> + return new Path(dir, HLOG_DATFILE + String.format("%1$03d", fn));
> }
>
> /**
> @@ -333,7 +334,7 @@
> * @throws IOException
> */
> synchronized void append(Text regionName, Text tableName, Text row,
> - TreeMap<Text, BytesWritable> columns, long timestamp)
> + TreeMap<Text, byte []> columns, long timestamp)
> throws IOException {
> if(closed) {
> throw new IOException("Cannot append; log is closed");
> @@ -350,7 +351,7 @@
> }
>
> int counter = 0;
> - for (Map.Entry<Text, BytesWritable> es: columns.entrySet()) {
> + for (Map.Entry<Text, byte []> es: columns.entrySet()) {
> HLogKey logKey =
> new HLogKey(regionName, tableName, row, seqNum[counter++]);
> HLogEdit logEdit = new HLogEdit(es.getKey(), es.getValue(),
> timestamp);
> @@ -401,6 +402,7 @@
> try {
> wait();
> } catch (InterruptedException ie) {
> + // continue
> }
> }
> insideCacheFlush = true;
> @@ -427,7 +429,7 @@
> }
>
> writer.append(new HLogKey(regionName, tableName, HLog.METAROW,
> logSeqId),
> - new HLogEdit(HLog.METACOLUMN, COMPLETE_CACHEFLUSH,
> + new HLogEdit(HLog.METACOLUMN, COMPLETE_CACHEFLUSH.get(),
> System.currentTimeMillis()));
> numEntries++;
>
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HLogEdit.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HLogEdit.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HLogEdit.java Mon Jun 18 15:59:14 2007
> @@ -27,14 +27,15 @@
> * This just indicates the column and value.
>
> **********************************************************************
> ********/
> public class HLogEdit implements Writable {
> - Text column = new Text();
> - BytesWritable val = new BytesWritable();
> - long timestamp;
> + private Text column = new Text();
> + private byte [] val;
> + private long timestamp;
>
> public HLogEdit() {
> + super();
> }
>
> - public HLogEdit(Text column, BytesWritable bval, long timestamp) {
> + public HLogEdit(Text column, byte [] bval, long timestamp) {
> this.column.set(column);
> this.val = bval;
> this.timestamp = timestamp;
> @@ -44,7 +45,7 @@
> return this.column;
> }
>
> - public BytesWritable getVal() {
> + public byte [] getVal() {
> return this.val;
> }
>
> @@ -55,7 +56,7 @@
> @Override
> public String toString() {
> return getColumn().toString() + " " + this.getTimestamp() + " " +
> - new String(getVal().get()).trim();
> + new String(getVal()).trim();
> }
>
> ///////////////////////////////////////////////////////////////////
> ///////////
> @@ -64,13 +65,15 @@
>
> public void write(DataOutput out) throws IOException {
> this.column.write(out);
> - this.val.write(out);
> + out.writeShort(this.val.length);
> + out.write(this.val);
> out.writeLong(timestamp);
> }
>
> public void readFields(DataInput in) throws IOException {
> this.column.readFields(in);
> - this.val.readFields(in);
> + this.val = new byte[in.readShort()];
> + in.readFully(this.val);
> this.timestamp = in.readLong();
> }
> }
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HLogKey.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HLogKey.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HLogKey.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HLogKey.java Mon Jun 18 15:59:14 2007
> @@ -34,6 +34,7 @@
>
> /** Create an empty key useful when deserializing */
> public HLogKey() {
> + super();
> }
>
> /**
> @@ -47,6 +48,7 @@
> * @param logSeqNum - log sequence number
> */
> public HLogKey(Text regionName, Text tablename, Text row, long
> logSeqNum) {
> + // TODO: Is this copy of the instances necessary? They are
> expensive.
> this.regionName.set(regionName);
> this.tablename.set(tablename);
> this.row.set(row);
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMaster.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HMaster.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMaster.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMaster.java Mon Jun 18 15:59:14 2007
> @@ -37,7 +37,6 @@
> import org.apache.hadoop.conf.Configuration;
> import org.apache.hadoop.fs.FileSystem;
> import org.apache.hadoop.fs.Path;
> -import org.apache.hadoop.io.BytesWritable;
> import org.apache.hadoop.io.DataInputBuffer;
> import org.apache.hadoop.io.Text;
> import org.apache.hadoop.ipc.RPC;
> @@ -179,10 +178,7 @@
> }
>
> for (int i = 0; i < values.length; i++) {
> - byte[] bytes = new byte[values[i].getData().getSize()];
> - System.arraycopy(values[i].getData().get(), 0, bytes, 0,
> - bytes.length);
> - results.put(values[i].getKey().getColumn(), bytes);
> + results.put(values[i].getKey().getColumn(), values
> [i].getData());
> }
>
> HRegionInfo info = HRegion.getRegionInfo(results);
> @@ -272,7 +268,7 @@
> // The current assignment is no good; load the region.
>
> unassignedRegions.put(info.regionName, info);
> - assignAttempts.put(info.regionName, 0L);
> + assignAttempts.put(info.regionName, Long.valueOf(0L));
> }
> }
> }
> @@ -333,7 +329,7 @@
>
> private RootScanner rootScanner;
> private Thread rootScannerThread;
> - Integer rootScannerLock = 0;
> + Integer rootScannerLock = new Integer(0);
>
> @SuppressWarnings("unchecked")
> static class MetaRegion implements Comparable {
> @@ -492,7 +488,7 @@
>
> MetaScanner metaScanner;
> private Thread metaScannerThread;
> - Integer metaScannerLock = 0;
> + Integer metaScannerLock = new Integer(0);
>
> /**
> * The 'unassignedRegions' table maps from a region name to a
> HRegionInfo
> @@ -642,7 +638,8 @@
> this.pendingRegions =
> Collections.synchronizedSortedSet(new TreeSet<Text>());
>
> - this.assignAttempts.put(HGlobals.rootRegionInfo.regionName, 0L);
> + this.assignAttempts.put(HGlobals.rootRegionInfo.regionName,
> + Long.valueOf(0L));
>
> this.killList =
> Collections.synchronizedSortedMap(
> @@ -655,9 +652,7 @@
> Collections.synchronizedSortedSet(new TreeSet<Text>());
>
> // We're almost open for business
> -
> this.closed = false;
> -
> LOG.info("HMaster initialized on " + this.address.toString());
> }
>
> @@ -815,7 +810,9 @@
> /* (non-Javadoc)
> * @see
> org.apache.hadoop.hbase.HMasterRegionInterface#regionServerStartup
> (org.apache.hadoop.hbase.HServerInfo)
> */
> - public void regionServerStartup(HServerInfo serverInfo) throws
> IOException {
> + @SuppressWarnings("unused")
> + public void regionServerStartup(HServerInfo serverInfo)
> + throws IOException {
> String s = serverInfo.getServerAddress().toString().trim();
> HServerInfo storedInfo = null;
> LOG.info("received start message from: " + s);
> @@ -834,11 +831,15 @@
> // Either way, record the new server
> serversToServerInfo.put(s, serverInfo);
> if(!closed) {
> - Text serverLabel = new Text(s);
> + long serverLabel = getServerLabel(s);
> LOG.debug("Created lease for " + serverLabel);
> serverLeases.createLease(serverLabel, serverLabel, new
> ServerExpirer(s));
> }
> }
> +
> + private long getServerLabel(final String s) {
> + return s.hashCode();
> + }
>
> /* (non-Javadoc)
> * @see
> org.apache.hadoop.hbase.HMasterRegionInterface#regionServerReport
> (org.apache.hadoop.hbase.HServerInfo, org.apache.hadoop.hbase.HMsg[])
> @@ -846,7 +847,7 @@
> public HMsg[] regionServerReport(HServerInfo serverInfo, HMsg
> msgs[])
> throws IOException {
> String s = serverInfo.getServerAddress().toString().trim();
> - Text serverLabel = new Text(s);
> + long serverLabel = getServerLabel(s);
>
> if (closed) {
> // Cancel the server's lease
> @@ -874,7 +875,7 @@
> allMetaRegionsScanned = false;
> }
> unassignedRegions.put(info.regionName, info);
> - assignAttempts.put(info.regionName, 0L);
> + assignAttempts.put(info.regionName, Long.valueOf(0L));
> }
>
> // We don't need to return anything to the server because it
> isn't
> @@ -934,7 +935,8 @@
> }
>
> /** cancel a server's lease */
> - private void cancelLease(String serverName, Text serverLabel)
> throws IOException {
> + private void cancelLease(final String serverName, final long
> serverLabel)
> + throws IOException {
> if (serversToServerInfo.remove(serverName) != null) {
> // Only cancel lease once.
> // This method can be called a couple of times during shutdown.
> @@ -1035,7 +1037,7 @@
> if(region.regionName.compareTo
> (HGlobals.rootRegionInfo.regionName) == 0) { // Root region
> rootRegionLocation = null;
> unassignedRegions.put(region.regionName, region);
> - assignAttempts.put(region.regionName, 0L);
> + assignAttempts.put(region.regionName, Long.valueOf(0L));
>
> } else {
> boolean reassignRegion = true;
> @@ -1115,7 +1117,7 @@
>
> returnMsgs.add(new HMsg(HMsg.MSG_REGION_OPEN, regionInfo));
>
> - assignAttempts.put(curRegionName, now);
> + assignAttempts.put(curRegionName, Long.valueOf(now));
> counter++;
> }
>
> @@ -1214,7 +1216,6 @@
>
> TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>
> ();
> Text row = null;
> - byte[] bytes = null;
> for(int i = 0; i < values.length; i++) {
> if(row == null) {
> row = values[i].getKey().getRow();
> @@ -1225,12 +1226,10 @@
> + row + ", currentRow=" + values[i].getKey
> ().getRow());
> }
> }
> - bytes = new byte[values[i].getData().getSize()];
> - System.arraycopy(values[i].getData().get(), 0, bytes,
> 0, bytes.length);
> - results.put(values[i].getKey().getColumn(), bytes);
> + results.put(values[i].getKey().getColumn(), values
> [i].getData());
> }
>
> - bytes = results.get(COL_SERVER);
> + byte [] bytes = results.get(COL_SERVER);
> String serverName = null;
> if(bytes == null || bytes.length == 0) {
> // No server
> @@ -1335,21 +1334,18 @@
> }
>
> // Remove server from root/meta entries
> -
> for(int i = 0; i < toDoList.size(); i++) {
> ToDoEntry e = toDoList.get(i);
> long lockid = server.startUpdate(regionName, clientId,
> e.row);
> if(e.deleteRegion) {
> server.delete(regionName, clientId, lockid,
> COL_REGIONINFO);
> -
> } else if(e.regionOffline) {
> e.info.offLine = true;
> ByteArrayOutputStream byteValue = new
> ByteArrayOutputStream();
> DataOutputStream s = new DataOutputStream(byteValue);
> e.info.write(s);
> -
> server.put(regionName, clientId, lockid, COL_REGIONINFO,
> - new BytesWritable(byteValue.toByteArray()));
> + byteValue.toByteArray());
> }
> server.delete(regionName, clientId, lockid, COL_SERVER);
> server.delete(regionName, clientId, lockid, COL_STARTCODE);
> @@ -1363,7 +1359,7 @@
> HRegionInfo regionInfo = e.getValue();
>
> unassignedRegions.put(region, regionInfo);
> - assignAttempts.put(region, 0L);
> + assignAttempts.put(region, Long.valueOf(0L));
> }
> }
>
> @@ -1384,7 +1380,8 @@
> rootRegionLocation = null;
> unassignedRegions.put(HGlobals.rootRegionInfo.regionName,
> HGlobals.rootRegionInfo);
> - assignAttempts.put(HGlobals.rootRegionInfo.regionName, 0L);
> + assignAttempts.put(HGlobals.rootRegionInfo.regionName,
> + Long.valueOf(0L));
> }
>
> // Scan the ROOT region
> @@ -1525,7 +1522,7 @@
> regionInfo.write(s);
>
> server.put(metaRegionName, clientId, lockid,
> COL_REGIONINFO,
> - new BytesWritable(byteValue.toByteArray()));
> + byteValue.toByteArray());
> }
> server.delete(metaRegionName, clientId, lockid,
> COL_SERVER);
> server.delete(metaRegionName, clientId, lockid,
> COL_STARTCODE);
> @@ -1546,7 +1543,7 @@
> }
>
> unassignedRegions.put(regionInfo.regionName, regionInfo);
> - assignAttempts.put(regionInfo.regionName, 0L);
> + assignAttempts.put(regionInfo.regionName, Long.valueOf(0L));
>
> } else if(deleteRegion) {
> try {
> @@ -1569,36 +1566,27 @@
> private class PendingOpenReport extends PendingOperation {
> private boolean rootRegion;
> private Text regionName;
> - private BytesWritable serverAddress;
> - private BytesWritable startCode;
> + private byte [] serverAddress;
> + private byte [] startCode;
>
> PendingOpenReport(HServerInfo info, HRegionInfo region) {
> - if(region.tableDesc.getName().equals(META_TABLE_NAME)) {
> -
> + if (region.tableDesc.getName().equals(META_TABLE_NAME)) {
> // The region which just came on-line is a META region.
> // We need to look in the ROOT region for its information.
> -
> this.rootRegion = true;
> -
> } else {
> -
> // Just an ordinary region. Look for it in the META table.
> -
> this.rootRegion = false;
> }
> this.regionName = region.regionName;
> -
> try {
> - this.serverAddress = new BytesWritable(
> - info.getServerAddress().toString().getBytes
> (UTF8_ENCODING));
> -
> - this.startCode = new BytesWritable(
> - String.valueOf(info.getStartCode()).getBytes
> (UTF8_ENCODING));
> -
> + this.serverAddress = info.getServerAddress().toString().
> + getBytes(UTF8_ENCODING);
> + this.startCode = String.valueOf(info.getStartCode()).
> + getBytes(UTF8_ENCODING);
> } catch(UnsupportedEncodingException e) {
> LOG.error(e);
> }
> -
> }
>
> @Override
> @@ -1614,7 +1602,7 @@
>
> if(LOG.isDebugEnabled()) {
> LOG.debug(regionName + " open on "
> - + new String(serverAddress.get(), UTF8_ENCODING));
> + + new String(this.serverAddress, UTF8_ENCODING));
> }
>
> // Register the newly-available Region's location.
> @@ -1708,33 +1696,25 @@
> }
>
> // 1. Check to see if table already exists
> -
> - MetaRegion m = null;
> - if(knownMetaRegions.containsKey(newRegion.regionName)) {
> - m = knownMetaRegions.get(newRegion.regionName);
> -
> - } else {
> - m = knownMetaRegions.get(
> + MetaRegion m = (knownMetaRegions.containsKey
> (newRegion.regionName))?
> + knownMetaRegions.get(newRegion.regionName):
> + knownMetaRegions.get(
> knownMetaRegions.headMap
> (newRegion.regionName).lastKey());
> - }
> Text metaRegionName = m.regionName;
> HRegionInterface server = client.getHRegionConnection
> (m.server);
> -
> -
> - BytesWritable bytes = server.get(metaRegionName,
> desc.getName(), COL_REGIONINFO);
> - if(bytes != null && bytes.getSize() != 0) {
> - byte[] infoBytes = bytes.get();
> + byte [] infoBytes =
> + server.get(metaRegionName, desc.getName(), COL_REGIONINFO);
> + if (infoBytes != null && infoBytes.length != 0) {
> DataInputBuffer inbuf = new DataInputBuffer();
> inbuf.reset(infoBytes, infoBytes.length);
> HRegionInfo info = new HRegionInfo();
> info.readFields(inbuf);
> - if(info.tableDesc.getName().compareTo(desc.getName()) ==
> 0) {
> + if (info.tableDesc.getName().compareTo(desc.getName())
> == 0) {
> throw new IOException("table already exists");
> }
> }
>
> // 2. Create the HRegion
> -
> HRegion r = HRegion.createHRegion(newRegion.regionId,
> desc, this.dir,
> this.conf);
>
> @@ -1748,8 +1728,8 @@
>
> long clientId = rand.nextLong();
> long lockid = server.startUpdate(metaRegionName, clientId,
> regionName);
> - server.put(metaRegionName, clientId, lockid, COL_REGIONINFO,
> - new BytesWritable(byteValue.toByteArray()));
> + server.put(metaRegionName, clientId, lockid, COL_REGIONINFO,
> + byteValue.toByteArray());
> server.commit(metaRegionName, clientId, lockid);
>
> // 4. Close the new region to flush it to disk
> @@ -1759,7 +1739,7 @@
> // 5. Get it assigned to a server
>
> unassignedRegions.put(regionName, info);
> - assignAttempts.put(regionName, 0L);
> + assignAttempts.put(regionName, Long.valueOf(0L));
> break;
>
> } catch(NotServingRegionException e) {
> @@ -1887,30 +1867,26 @@
> }
> boolean haveRegionInfo = false;
> for(int i = 0; i < values.length; i++) {
> - bytes = new byte[values[i].getData().getSize()];
> - if(bytes.length == 0) {
> + if(values[i].getData().length == 0) {
> break;
> }
> - System.arraycopy(values[i].getData().get(), 0,
> bytes, 0, bytes.length);
> -
> Text column = values[i].getKey().getColumn();
> if(column.equals(COL_REGIONINFO)) {
> haveRegionInfo = true;
> - inbuf.reset(bytes, bytes.length);
> + inbuf.reset(values[i].getData(),
> + values[i].getData().length);
> info.readFields(inbuf);
> -
> } else if(column.equals(COL_SERVER)) {
> try {
> - serverName = new String(bytes,
> UTF8_ENCODING);
> -
> + serverName =
> + new String(values[i].getData(),
> UTF8_ENCODING);
> } catch(UnsupportedEncodingException e) {
> assert(false);
> }
> -
> } else if(column.equals(COL_STARTCODE)) {
> try {
> - startCode = Long.valueOf(new String(bytes,
> UTF8_ENCODING));
> -
> + startCode = Long.valueOf(new String(values
> [i].getData(),
> + UTF8_ENCODING)).longValue();
> } catch(UnsupportedEncodingException e) {
> assert(false);
> }
> @@ -2115,8 +2091,7 @@
> i.write(s);
>
> server.put(regionName, clientId, lockid, COL_REGIONINFO,
> - new BytesWritable(byteValue.toByteArray()));
> -
> + byteValue.toByteArray());
> }
> }
>
> @@ -2180,29 +2155,24 @@
> }
>
> protected void updateRegionInfo(HRegionInterface server, Text
> regionName,
> - HRegionInfo i) throws IOException {
> -
> + HRegionInfo i)
> + throws IOException {
> ByteArrayOutputStream byteValue = new ByteArrayOutputStream();
> DataOutputStream s = new DataOutputStream(byteValue);
> i.write(s);
> -
> long lockid = -1L;
> long clientId = rand.nextLong();
> try {
> lockid = server.startUpdate(regionName, clientId,
> i.regionName);
> server.put(regionName, clientId, lockid, COL_REGIONINFO,
> - new BytesWritable(byteValue.toByteArray()));
> -
> + byteValue.toByteArray());
> server.commit(regionName, clientId, lockid);
> lockid = -1L;
> -
> if(LOG.isDebugEnabled()) {
> LOG.debug("updated columns in row: " + i.regionName);
> }
> -
> } catch(NotServingRegionException e) {
> throw e;
> -
> } catch(IOException e) {
> LOG.error("column update failed in row: " + i.regionName);
> LOG.error(e);
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMasterInterface.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HMasterInterface.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMasterInterface.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMasterInterface.java Mon Jun 18 15:59:14 2007
> @@ -57,4 +57,4 @@
> ///////////////////////////////////////////////////////////////////
> ///////////
>
> public HServerAddress findRootRegion();
> -}
> +}
> \ No newline at end of file
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMasterRegionInterface.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HMasterRegionInterface.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMasterRegionInterface.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMasterRegionInterface.java Mon Jun 18 15:59:14 2007
> @@ -25,5 +25,6 @@
> public interface HMasterRegionInterface extends VersionedProtocol {
> public static final long versionID = 1L;
> public void regionServerStartup(HServerInfo info) throws
> IOException;
> - public HMsg[] regionServerReport(HServerInfo info, HMsg msgs[])
> throws IOException;
> + public HMsg[] regionServerReport(HServerInfo info, HMsg msgs[])
> + throws IOException;
> }
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMemcache.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HMemcache.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMemcache.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMemcache.java Mon Jun 18 15:59:14 2007
> @@ -16,7 +16,9 @@
> package org.apache.hadoop.hbase;
>
> import java.io.IOException;
> +import java.util.ArrayList;
> import java.util.Iterator;
> +import java.util.List;
> import java.util.Map;
> import java.util.SortedMap;
> import java.util.TreeMap;
> @@ -24,7 +26,7 @@
>
> import org.apache.commons.logging.Log;
> import org.apache.commons.logging.LogFactory;
> -import org.apache.hadoop.io.BytesWritable;
> +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
> import org.apache.hadoop.io.Text;
>
> /**
> @@ -34,13 +36,13 @@
> public class HMemcache {
> private final Log LOG = LogFactory.getLog(this.getClass().getName
> ());
>
> - TreeMap<HStoreKey, BytesWritable> memcache
> - = new TreeMap<HStoreKey, BytesWritable>();
> + TreeMap<HStoreKey, byte []> memcache =
> + new TreeMap<HStoreKey, byte []>();
>
> - Vector<TreeMap<HStoreKey, BytesWritable>> history
> - = new Vector<TreeMap<HStoreKey, BytesWritable>>();
> + Vector<TreeMap<HStoreKey, byte []>> history
> + = new Vector<TreeMap<HStoreKey, byte []>>();
>
> - TreeMap<HStoreKey, BytesWritable> snapshot = null;
> + TreeMap<HStoreKey, byte []> snapshot = null;
>
> final HLocking lock = new HLocking();
>
> @@ -49,7 +51,7 @@
> }
>
> public static class Snapshot {
> - public TreeMap<HStoreKey, BytesWritable> memcacheSnapshot = null;
> + public TreeMap<HStoreKey, byte []> memcacheSnapshot = null;
> public long sequenceId = 0;
>
> public Snapshot() {
> @@ -92,7 +94,7 @@
> retval.memcacheSnapshot = memcache;
> this.snapshot = memcache;
> history.add(memcache);
> - memcache = new TreeMap<HStoreKey, BytesWritable>();
> + memcache = new TreeMap<HStoreKey, byte []>();
> retval.sequenceId = log.startCacheFlush();
>
> if(LOG.isDebugEnabled()) {
> @@ -122,21 +124,18 @@
> LOG.debug("deleting snapshot");
> }
>
> - for(Iterator<TreeMap<HStoreKey, BytesWritable>> it =
> history.iterator();
> + for(Iterator<TreeMap<HStoreKey, byte []>> it =
> history.iterator();
> it.hasNext(); ) {
> -
> - TreeMap<HStoreKey, BytesWritable> cur = it.next();
> - if(snapshot == cur) {
> + TreeMap<HStoreKey, byte []> cur = it.next();
> + if (snapshot == cur) {
> it.remove();
> break;
> }
> }
> this.snapshot = null;
> -
> if(LOG.isDebugEnabled()) {
> LOG.debug("snapshot deleted");
> }
> -
> } finally {
> this.lock.releaseWriteLock();
> }
> @@ -144,14 +143,16 @@
>
> /**
> * Store a value.
> - *
> * Operation uses a write lock.
> + * @param row
> + * @param columns
> + * @param timestamp
> */
> - public void add(final Text row, final TreeMap<Text,
> BytesWritable> columns,
> + public void add(final Text row, final TreeMap<Text, byte []>
> columns,
> final long timestamp) {
> this.lock.obtainWriteLock();
> try {
> - for (Map.Entry<Text, BytesWritable> es: columns.entrySet()) {
> + for (Map.Entry<Text, byte []> es: columns.entrySet()) {
> HStoreKey key = new HStoreKey(row, es.getKey(), timestamp);
> memcache.put(key, es.getValue());
> }
> @@ -162,45 +163,47 @@
>
> /**
> * Look back through all the backlog TreeMaps to find the target.
> - *
> - * We only need a readlock here.
> + * @param key
> + * @param numVersions
> + * @return An array of byte arrays orderded by timestamp.
> */
> - public BytesWritable[] get(HStoreKey key, int numVersions) {
> - Vector<BytesWritable> results = new Vector<BytesWritable>();
> + public byte [][] get(final HStoreKey key, final int numVersions) {
> + List<byte []> results = new ArrayList<byte[]>();
> this.lock.obtainReadLock();
> try {
> - Vector<BytesWritable> result = get(memcache, key,
> numVersions-results.size());
> + ArrayList<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) {
> + 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());
> + result = get(history.elementAt(i), key, numVersions -
> results.size());
> results.addAll(results.size(), result);
> }
> -
> return (results.size() == 0)?
> - null: results.toArray(new BytesWritable[results.size()]);
> + null: ImmutableBytesWritable.toArray(results);
> } finally {
> this.lock.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[]).
> + * @param key
> + * @return All columns for given key.
> */
> - public TreeMap<Text, BytesWritable> getFull(HStoreKey key) {
> - TreeMap<Text, BytesWritable> results = new TreeMap<Text,
> BytesWritable>();
> + public TreeMap<Text, byte []> getFull(HStoreKey key) {
> + TreeMap<Text, byte []> results = new TreeMap<Text, byte []>();
> this.lock.obtainReadLock();
> try {
> internalGetFull(memcache, key, results);
> - for(int i = history.size()-1; i >= 0; i--) {
> - TreeMap<HStoreKey, BytesWritable> cur = history.elementAt(i);
> + for (int i = history.size()-1; i >= 0; i--) {
> + TreeMap<HStoreKey, byte []> cur = history.elementAt(i);
> internalGetFull(cur, key, results);
> }
> return results;
> @@ -210,17 +213,16 @@
> }
> }
>
> - void internalGetFull(TreeMap<HStoreKey, BytesWritable> map,
> HStoreKey key,
> - TreeMap<Text, BytesWritable> results) {
> - SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap(key);
> - for (Map.Entry<HStoreKey, BytesWritable> es: tailMap.entrySet
> ()) {
> + void internalGetFull(TreeMap<HStoreKey, byte []> map, HStoreKey
> key,
> + TreeMap<Text, byte []> results) {
> + SortedMap<HStoreKey, byte []> tailMap = map.tailMap(key);
> + for (Map.Entry<HStoreKey, byte []> es: tailMap.entrySet()) {
> HStoreKey itKey = es.getKey();
> Text itCol = itKey.getColumn();
> if (results.get(itCol) == null
> && key.matchesWithoutColumn(itKey)) {
> - BytesWritable val = tailMap.get(itKey);
> + byte [] val = tailMap.get(itKey);
> results.put(itCol, val);
> -
> } else if (key.getRow().compareTo(itKey.getRow()) > 0) {
> break;
> }
> @@ -235,18 +237,23 @@
> *
> * TODO - This is kinda slow. We need a data structure that
> allows for
> * proximity-searches, not just precise-matches.
> - */
> - Vector<BytesWritable> get(TreeMap<HStoreKey, BytesWritable> map,
> HStoreKey key, int numVersions) {
> - Vector<BytesWritable> result = new Vector<BytesWritable>();
> - HStoreKey curKey = new HStoreKey(key.getRow(), key.getColumn
> (), key.getTimestamp());
> - SortedMap<HStoreKey, BytesWritable> tailMap = map.tailMap
> (curKey);
> - for (Map.Entry<HStoreKey, BytesWritable> es: tailMap.entrySet
> ()) {
> + * @param map
> + * @param key
> + * @param numVersions
> + * @return Ordered list of items found in passed <code>map</code>
> + */
> + ArrayList<byte []> get(final TreeMap<HStoreKey, byte []> map,
> + final HStoreKey key, final int numVersions) {
> + ArrayList<byte []> result = new ArrayList<byte []>();
> + HStoreKey curKey =
> + new HStoreKey(key.getRow(), key.getColumn(), key.getTimestamp
> ());
> + SortedMap<HStoreKey, byte []> tailMap = map.tailMap(curKey);
> + for (Map.Entry<HStoreKey, byte []> es: tailMap.entrySet()) {
> HStoreKey itKey = es.getKey();
> if (itKey.matchesRowCol(curKey)) {
> result.add(tailMap.get(itKey));
> curKey.setVersion(itKey.getTimestamp() - 1);
> }
> -
> if (numVersions > 0 && result.size() >= numVersions) {
> break;
> }
> @@ -269,8 +276,8 @@
> ///////////////////////////////////////////////////////////////////
> ///////////
>
> class HMemcacheScanner extends HAbstractScanner {
> - TreeMap<HStoreKey, BytesWritable> backingMaps[];
> - Iterator<HStoreKey> keyIterators[];
> + final TreeMap<HStoreKey, byte []> backingMaps[];
> + final Iterator<HStoreKey> keyIterators[];
>
> @SuppressWarnings("unchecked")
> public HMemcacheScanner(long timestamp, Text targetCols[],
> Text firstRow)
> @@ -292,7 +299,7 @@
>
> this.keyIterators = new Iterator[backingMaps.length];
> this.keys = new HStoreKey[backingMaps.length];
> - this.vals = new BytesWritable[backingMaps.length];
> + this.vals = new byte[backingMaps.length][];
>
> // Generate list of iterators
> HStoreKey firstKey = new HStoreKey(firstRow);
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMerge.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HMerge.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMerge.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HMerge.java Mon Jun 18 15:59:14 2007
> @@ -27,7 +27,6 @@
> import org.apache.hadoop.conf.Configuration;
> import org.apache.hadoop.fs.FileSystem;
> import org.apache.hadoop.fs.Path;
> -import org.apache.hadoop.io.BytesWritable;
> import org.apache.hadoop.io.DataInputBuffer;
> import org.apache.hadoop.io.Text;
>
> @@ -39,7 +38,9 @@
> static final Log LOG = LogFactory.getLog(HMerge.class);
> static final Text[] META_COLS = {COL_REGIONINFO};
>
> - private HMerge() {} // Not instantiable
> + private HMerge() {
> + // Not instantiable
> + }
>
> /**
> * Scans the table and merges two adjacent regions if they are
> small. This
> @@ -317,7 +318,7 @@
> private static class OfflineMerger extends Merger {
> private Path dir;
> private TreeSet<HRegionInfo> metaRegions;
> - private TreeMap<Text, BytesWritable> results;
> + private TreeMap<Text, byte []> results;
>
> OfflineMerger(Configuration conf, FileSystem fs, Text tableName)
> throws IOException {
> @@ -325,7 +326,7 @@
> super(conf, fs, tableName);
> this.dir = new Path(conf.get(HBASE_DIR, DEFAULT_HBASE_DIR));
> this.metaRegions = new TreeSet<HRegionInfo>();
> - this.results = new TreeMap<Text, BytesWritable>();
> + this.results = new TreeMap<Text, byte []>();
>
> // Scan root region to find all the meta regions
>
> @@ -337,10 +338,8 @@
>
> try {
> while(rootScanner.next(key, results)) {
> - for(BytesWritable b: results.values()) {
> - byte[] bytes = new byte[b.getSize()];
> - System.arraycopy(b.get(), 0, bytes, 0, bytes.length);
> - in.reset(bytes, bytes.length);
> + for(byte [] b: results.values()) {
> + in.reset(b, b.length);
> info.readFields(in);
> metaRegions.add(info);
> results.clear();
> @@ -405,8 +404,7 @@
> long lockid = -1L;
> try {
> lockid = root.startUpdate(newRegion.getRegionName());
> - root.put(lockid, COL_REGIONINFO,
> - new BytesWritable(byteValue.toByteArray()));
> + root.put(lockid, COL_REGIONINFO, byteValue.toByteArray());
> root.commit(lockid);
> lockid = -1L;
>
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegion.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HRegion.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegion.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegion.java Mon Jun 18 15:59:14 2007
> @@ -238,11 +238,11 @@
> // Members
> ///////////////////////////////////////////////////////////////////
> ///////////
>
> - TreeMap<Text, Long> rowsToLocks = new TreeMap<Text, Long>();
> - TreeMap<Long, Text> locksToRows = new TreeMap<Long, Text>();
> - TreeMap<Text, HStore> stores = new TreeMap<Text, HStore>();
> - Map<Long, TreeMap<Text, BytesWritable>> targetColumns
> - = new HashMap<Long, TreeMap<Text, BytesWritable>>();
> + Map<Text, Long> rowsToLocks = new HashMap<Text, Long>();
> + Map<Long, Text> locksToRows = new HashMap<Long, Text>();
> + Map<Text, HStore> stores = new HashMap<Text, HStore>();
> + Map<Long, TreeMap<Text, byte []>> targetColumns
> + = new HashMap<Long, TreeMap<Text, byte []>>();
>
> HMemcache memcache;
>
> @@ -826,7 +826,7 @@
> }
>
> HMemcache.Snapshot retval = memcache.snapshotMemcacheForLog(log);
> - TreeMap<HStoreKey, BytesWritable> memcacheSnapshot =
> retval.memcacheSnapshot;
> + TreeMap<HStoreKey, byte []> memcacheSnapshot =
> retval.memcacheSnapshot;
> if(memcacheSnapshot == null) {
> for(HStore hstore: stores.values()) {
> Vector<HStoreFile> hstoreFiles = hstore.getAllMapFiles();
> @@ -885,31 +885,28 @@
> ///////////////////////////////////////////////////////////////////
> ///////////
>
> /** Fetch a single data item. */
> - BytesWritable get(Text row, Text column) throws IOException {
> - BytesWritable[] results = get(row, column, Long.MAX_VALUE, 1);
> - return (results == null)? null: results[0];
> + byte [] get(Text row, Text column) throws IOException {
> + byte [][] results = get(row, column, Long.MAX_VALUE, 1);
> + return (results == null || results.length == 0)? null: results
> [0];
> }
>
> /** Fetch multiple versions of a single data item */
> - BytesWritable[] get(Text row, Text column, int numVersions)
> throws IOException {
> + byte [][] get(Text row, Text column, int numVersions) throws
> IOException {
> return get(row, column, Long.MAX_VALUE, numVersions);
> }
>
> /** Fetch multiple versions of a single data item, with
> timestamp. */
> - BytesWritable[] get(Text row, Text column, long timestamp, int
> numVersions)
> - throws IOException {
> -
> + byte [][] get(Text row, Text column, long timestamp, int
> numVersions)
> + throws IOException {
> if(writestate.closed) {
> throw new IOException("HRegion is closed.");
> }
>
> // Make sure this is a valid row and valid column
> -
> checkRow(row);
> checkColumn(column);
>
> // Obtain the row-lock
> -
> obtainRowLock(row);
> try {
> // Obtain the -col results
> @@ -921,13 +918,12 @@
> }
>
> /** Private implementation: get the value for the indicated
> HStoreKey */
> - private BytesWritable[] get(HStoreKey key, int numVersions)
> throws IOException {
> + private byte [][] get(HStoreKey key, int numVersions) throws
> IOException {
>
> lock.obtainReadLock();
> try {
> // Check the memcache
> -
> - BytesWritable[] result = memcache.get(key, numVersions);
> + byte [][] result = memcache.get(key, numVersions);
> if(result != null) {
> return result;
> }
> @@ -957,19 +953,17 @@
> * determine which column groups are useful for that row. That
> would let us
> * avoid a bunch of disk activity.
> */
> - TreeMap<Text, BytesWritable> getFull(Text row) throws IOException {
> + TreeMap<Text, byte []> getFull(Text row) throws IOException {
> HStoreKey key = new HStoreKey(row, System.currentTimeMillis());
>
> lock.obtainReadLock();
> try {
> - TreeMap<Text, BytesWritable> memResult = memcache.getFull(key);
> - for(Iterator<Text> it = stores.keySet().iterator();
> it.hasNext(); ) {
> - Text colFamily = it.next();
> + TreeMap<Text, byte []> memResult = memcache.getFull(key);
> + for (Text colFamily: stores.keySet()) {
> HStore targetStore = stores.get(colFamily);
> targetStore.getFull(key, memResult);
> }
> return memResult;
> -
> } finally {
> lock.releaseReadLock();
> }
> @@ -1035,9 +1029,8 @@
> * This method really just tests the input, then calls an
> internal localput()
> * method.
> */
> - void put(long lockid, Text targetCol, BytesWritable val) throws
> IOException {
> - if(val.getSize() == DELETE_BYTES.getSize()
> - && val.compareTo(DELETE_BYTES) == 0) {
> + void put(long lockid, Text targetCol, byte [] val) throws
> IOException {
> + if (DELETE_BYTES.compareTo(val) == 0) {
> throw new IOException("Cannot insert value: " + val);
> }
> localput(lockid, targetCol, val);
> @@ -1047,7 +1040,7 @@
> * Delete a value or write a value. This is a just a convenience
> method for put().
> */
> void delete(long lockid, Text targetCol) throws IOException {
> - localput(lockid, targetCol, DELETE_BYTES);
> + localput(lockid, targetCol, DELETE_BYTES.get());
> }
>
> /**
> @@ -1063,7 +1056,7 @@
> * @throws IOException
> */
> void localput(final long lockid, final Text targetCol,
> - final BytesWritable val)
> + final byte [] val)
> throws IOException {
> checkColumn(targetCol);
>
> @@ -1083,9 +1076,9 @@
> lockid + " unexpected aborted by another thread");
> }
>
> - TreeMap<Text, BytesWritable> targets = this.targetColumns.get
> (lockid);
> + TreeMap<Text, byte []> targets = this.targetColumns.get
> (lockid);
> if (targets == null) {
> - targets = new TreeMap<Text, BytesWritable>();
> + targets = new TreeMap<Text, byte []>();
> this.targetColumns.put(lockid, targets);
> }
> targets.put(targetCol, val);
> @@ -1144,8 +1137,7 @@
> synchronized(row) {
> // Add updates to the log and add values to the memcache.
> long commitTimestamp = System.currentTimeMillis();
> - TreeMap<Text, BytesWritable> columns =
> - this.targetColumns.get(lockid);
> + TreeMap<Text, byte []> columns = this.targetColumns.get
> (lockid);
> if (columns != null && columns.size() > 0) {
> log.append(regionInfo.regionName,
> regionInfo.tableDesc.getName(),
> row, columns, commitTimestamp);
> @@ -1267,7 +1259,7 @@
> */
> private static class HScanner implements
> HInternalScannerInterface {
> private HInternalScannerInterface[] scanners;
> - private TreeMap<Text, BytesWritable>[] resultSets;
> + private TreeMap<Text, byte []>[] resultSets;
> private HStoreKey[] keys;
> private boolean wildcardMatch;
> private boolean multipleMatchers;
> @@ -1323,7 +1315,7 @@
> }
> for(int i = 0; i < scanners.length; i++) {
> keys[i] = new HStoreKey();
> - resultSets[i] = new TreeMap<Text, BytesWritable>();
> + resultSets[i] = new TreeMap<Text, byte []>();
> if(scanners[i] != null && !scanners[i].next(keys[i],
> resultSets[i])) {
> closeScanner(i);
> }
> @@ -1351,7 +1343,7 @@
> *
> * @see org.apache.hadoop.hbase.HInternalScannerInterface#next
> (org.apache.hadoop.hbase.HStoreKey, java.util.TreeMap)
> */
> - public boolean next(HStoreKey key, TreeMap<Text,
> BytesWritable> results)
> + public boolean next(HStoreKey key, TreeMap<Text, byte []>
> results)
> throws IOException {
> // Find the lowest-possible key.
> Text chosenRow = null;
> @@ -1393,7 +1385,7 @@
> // values with older ones. So now we only insert
> // a result if the map does not contain the key.
>
> - for(Map.Entry<Text, BytesWritable> e: resultSets
> [i].entrySet()) {
> + for(Map.Entry<Text, byte []> e: resultSets[i].entrySet
> ()) {
> if(!results.containsKey(e.getKey())) {
> results.put(e.getKey(), e.getValue());
> insertedItem = true;
> @@ -1504,7 +1496,7 @@
> ByteArrayOutputStream bytes = new ByteArrayOutputStream();
> DataOutputStream s = new DataOutputStream(bytes);
> r.getRegionInfo().write(s);
> - meta.put(writeid, COL_REGIONINFO, new BytesWritable
> (bytes.toByteArray()));
> + meta.put(writeid, COL_REGIONINFO, bytes.toByteArray());
> meta.commit(writeid);
> }
>
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegionInterface.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegionInterface.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegionInterface.java Mon Jun 18 15:59:14 2007
> @@ -15,12 +15,11 @@
> */
> package org.apache.hadoop.hbase;
>
> -import org.apache.hadoop.io.BytesWritable;
> +import java.io.IOException;
> +
> import org.apache.hadoop.io.Text;
> import org.apache.hadoop.ipc.VersionedProtocol;
>
> -import java.io.*;
> -
> /
> **********************************************************************
> *********
> * Clients interact with HRegionServers using
> * a handle to the HRegionInterface.
> @@ -36,57 +35,62 @@
> * @return - HRegionInfo object for
> region
> * @throws NotServingRegionException
> */
> - public HRegionInfo getRegionInfo(final Text regionName) throws
> NotServingRegionException;
> + public HRegionInfo getRegionInfo(final Text regionName)
> + throws NotServingRegionException;
>
> /**
> * Retrieve a single value from the specified region for the
> specified row
> * and column keys
> *
> - * @param regionName - name of region
> - * @param row - row key
> - * @param column - column key
> - * @return - value for that region/row/column
> + * @param regionName name of region
> + * @param row row key
> + * @param column column key
> + * @return alue for that region/row/column
> * @throws IOException
> */
> - public BytesWritable get(final Text regionName, final Text row,
> final Text column) throws IOException;
> + public byte [] get(final Text regionName, final Text row, final
> Text column)
> + throws IOException;
>
> /**
> * Get the specified number of versions of the specified row and
> column
> *
> - * @param regionName - region name
> - * @param row - row key
> - * @param column - column key
> - * @param numVersions - number of versions to return
> - * @return - array of values
> + * @param regionName region name
> + * @param row row key
> + * @param column column key
> + * @param numVersions number of versions to return
> + * @return array of values
> * @throws IOException
> */
> - public BytesWritable[] get(final Text regionName, final Text row,
> - final Text column, final int numVersions) throws IOException;
> + public byte [][] get(final Text regionName, final Text row,
> + final Text column, final int numVersions)
> + throws IOException;
>
> /**
> * Get the specified number of versions of the specified row and
> column with
> * the specified timestamp.
> *
> - * @param regionName - region name
> - * @param row - row key
> - * @param column - column key
> - * @param timestamp - timestamp
> - * @param numVersions - number of versions to return
> - * @return - array of values
> + * @param regionName region name
> + * @param row row key
> + * @param column column key
> + * @param timestamp timestamp
> + * @param numVersions number of versions to return
> + * @return array of values
> * @throws IOException
> */
> - public BytesWritable[] get(final Text regionName, final Text
> row, final Text column,
> - final long timestamp, final int numVersions) throws
> IOException;
> + public byte [][] get(final Text regionName, final Text row,
> + final Text column, final long timestamp, final int numVersions)
> + throws IOException;
>
> /**
> * Get all the data for the specified row
> *
> - * @param regionName - region name
> - * @param row - row key
> - * @return - array of values
> + * @param regionName region name
> + * @param row row key
> + * @return array of values
> * @throws IOException
> */
> - public KeyedData[] getRow(final Text regionName, final Text row)
> throws IOException;
> + public KeyedData[] getRow(final Text regionName, final Text row)
> + throws IOException;
>
> ///////////////////////////////////////////////////////////////////
> ///////////
> // Start an atomic row insertion/update. No changes are
> committed until the
> @@ -110,67 +114,72 @@
> * The client can gain extra time with a call to renewLease().
> * Start an atomic row insertion or update
> *
> - * @param regionName - region name
> - * @param clientid - a unique value to identify the client
> - * @param row - Name of row to start update against.
> + * @param regionName region name
> + * @param clientid a unique value to identify the client
> + * @param row Name of row to start update against.
> * @return Row lockid.
> * @throws IOException
> */
> public long startUpdate(final Text regionName, final long clientid,
> - final Text row) throws IOException;
> + final Text row)
> + throws IOException;
>
> /**
> * Change a value for the specified column
> *
> - * @param regionName - region name
> - * @param clientid - a unique value to identify the
> client
> - * @param lockid - lock id returned from startUpdate
> - * @param column - column whose value is being set
> - * @param val - new value for column
> + * @param regionName region name
> + * @param clientid a unique value to identify the client
> + * @param lockid lock id returned from startUpdate
> + * @param column column whose value is being set
> + * @param val new value for column
> * @throws IOException
> */
> public void put(final Text regionName, final long clientid,
> final long lockid,
> - final Text column, final BytesWritable val) throws IOException;
> + final Text column, final byte [] val)
> + throws IOException;
>
> /**
> * Delete the value for a column
> *
> - * @param regionName - region name
> - * @param clientid - a unique value to identify the
> client
> - * @param lockid - lock id returned from startUpdate
> - * @param column - name of column whose value is to
> be deleted
> + * @param regionName region name
> + * @param clientid a unique value to identify the client
> + * @param lockid lock id returned from startUpdate
> + * @param column name of column whose value is to be deleted
> * @throws IOException
> */
> - public void delete(final Text regionName, final long clientid,
> final long lockid,
> - final Text column) throws IOException;
> + public void delete(final Text regionName, final long clientid,
> + final long lockid, final Text column)
> + throws IOException;
>
> /**
> * Abort a row mutation
> *
> - * @param regionName - region name
> - * @param clientid - a unique value to identify the
> client
> - * @param lockid - lock id returned from startUpdate
> + * @param regionName region name
> + * @param clientid a unique value to identify the client
> + * @param lockid lock id returned from startUpdate
> * @throws IOException
> */
> public void abort(final Text regionName, final long clientid,
> - final long lockid) throws IOException;
> + final long lockid)
> + throws IOException;
>
> /**
> * Finalize a row mutation
> *
> - * @param regionName - region name
> - * @param clientid - a unique value to identify the
> client
> - * @param lockid - lock id returned from startUpdate
> + * @param regionName region name
> + * @param clientid a unique value to identify the client
> + * @param lockid lock id returned from startUpdate
> * @throws IOException
> */
> public void commit(final Text regionName, final long clientid,
> - final long lockid) throws IOException;
> + final long lockid)
> + throws IOException;
>
> /**
> * Renew lease on update
> *
> - * @param lockid - lock id returned from startUpdate
> - * @param clientid - a unique value to identify the
> client
> + * @param lockid lock id returned from startUpdate
> + * @param clientid a unique value to identify the client
> * @throws IOException
> */
> public void renewLease(long lockid, long clientid) throws
> IOException;
> @@ -182,20 +191,21 @@
> /**
> * Opens a remote scanner.
> *
> - * @param regionName - name of region to scan
> - * @param columns - columns to scan
> - * @param startRow - starting row to scan
> + * @param regionName name of region to scan
> + * @param columns columns to scan
> + * @param startRow starting row to scan
> *
> - * @return scannerId - scanner identifier used in other calls
> + * @return scannerId scanner identifier used in other calls
> * @throws IOException
> */
> - public long openScanner(Text regionName, Text[] columns, Text
> startRow) throws IOException;
> + public long openScanner(Text regionName, Text[] columns, Text
> startRow)
> + throws IOException;
>
> /**
> * Get the next set of values
> *
> - * @param scannerId - clientId passed to openScanner
> - * @return - array of values
> + * @param scannerId clientId passed to openScanner
> + * @return array of values
> * @throws IOException
> */
> public KeyedData[] next(long scannerId) throws IOException;
> @@ -203,7 +213,7 @@
> /**
> * Close a scanner
> *
> - * @param scannerId - the scanner id returned by openScanner
> + * @param scannerId the scanner id returned by openScanner
> * @throws IOException
> */
> public void close(long scannerId) throws IOException;
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegionServer.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegionServer.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegionServer.java Mon Jun 18 15:59:14 2007
> @@ -33,7 +33,6 @@
> import org.apache.hadoop.conf.Configuration;
> import org.apache.hadoop.fs.FileSystem;
> import org.apache.hadoop.fs.Path;
> -import org.apache.hadoop.io.BytesWritable;
> import org.apache.hadoop.io.Text;
> import org.apache.hadoop.io.retry.RetryProxy;
> import org.apache.hadoop.ipc.RPC;
> @@ -396,8 +395,6 @@
> Collections.synchronizedSortedMap(new TreeMap<Text, HRegion>
> ());
>
> this.outboundMsgs = new Vector<HMsg>();
> - this.scanners =
> - Collections.synchronizedMap(new TreeMap<Text,
> HInternalScannerInterface>());
>
> // Config'ed params
> this.numRetries = conf.getInt("hbase.client.retries.number", 2);
> @@ -914,27 +911,26 @@
> /* (non-Javadoc)
> * @see org.apache.hadoop.hbase.HRegionInterface#get
> (org.apache.hadoop.io.Text, org.apache.hadoop.io.Text,
> org.apache.hadoop.io.Text)
> */
> - public BytesWritable get(final Text regionName, final Text row,
> - final Text column) throws IOException {
> -
> + public byte [] get(final Text regionName, final Text row,
> + final Text column)
> + throws IOException {
> return getRegion(regionName).get(row, column);
> }
>
> /* (non-Javadoc)
> * @see org.apache.hadoop.hbase.HRegionInterface#get
> (org.apache.hadoop.io.Text, org.apache.hadoop.io.Text,
> org.apache.hadoop.io.Text, int)
> */
> - public BytesWritable[] get(final Text regionName, final Text row,
> - final Text column, final int numVersions) throws IOException {
> -
> + public byte [][] get(final Text regionName, final Text row,
> + final Text column, final int numVersions)
> + throws IOException {
> return getRegion(regionName).get(row, column, numVersions);
> }
>
> /* (non-Javadoc)
> * @see org.apache.hadoop.hbase.HRegionInterface#get
> (org.apache.hadoop.io.Text, org.apache.hadoop.io.Text,
> org.apache.hadoop.io.Text, long, int)
> */
> - public BytesWritable[] get(final Text regionName, final Text
> row, final Text column,
> + public byte [][] get(final Text regionName, final Text row,
> final Text column,
> final long timestamp, final int numVersions) throws
> IOException {
> -
> return getRegion(regionName).get(row, column, timestamp,
> numVersions);
> }
>
> @@ -943,10 +939,10 @@
> */
> public KeyedData[] getRow(final Text regionName, final Text row)
> throws IOException {
> HRegion region = getRegion(regionName);
> - TreeMap<Text, BytesWritable> map = region.getFull(row);
> + TreeMap<Text, byte[]> map = region.getFull(row);
> KeyedData result[] = new KeyedData[map.size()];
> int counter = 0;
> - for (Map.Entry<Text, BytesWritable> es: map.entrySet()) {
> + for (Map.Entry<Text, byte []> es: map.entrySet()) {
> result[counter++] =
> new KeyedData(new HStoreKey(row, es.getKey()), es.getValue
> ());
> }
> @@ -957,30 +953,28 @@
> * @see org.apache.hadoop.hbase.HRegionInterface#next(long)
> */
> public KeyedData[] next(final long scannerId)
> - throws IOException {
> -
> - Text scannerName = new Text(String.valueOf(scannerId));
> + throws IOException {
> + String scannerName = String.valueOf(scannerId);
> HInternalScannerInterface s = scanners.get(scannerName);
> if (s == null) {
> throw new UnknownScannerException("Name: " + scannerName);
> }
> - leases.renewLease(scannerName, scannerName);
> + leases.renewLease(scannerId, scannerId);
>
> // Collect values to be returned here
>
> ArrayList<KeyedData> values = new ArrayList<KeyedData>();
>
> - TreeMap<Text, BytesWritable> results = new TreeMap<Text,
> BytesWritable>();
> + TreeMap<Text, byte []> results = new TreeMap<Text, byte []>();
>
> // Keep getting rows until we find one that has at least one
> non-deleted column value
>
> HStoreKey key = new HStoreKey();
> while (s.next(key, results)) {
> - for(Map.Entry<Text, BytesWritable> e: results.entrySet()) {
> + for(Map.Entry<Text, byte []> e: results.entrySet()) {
> HStoreKey k = new HStoreKey(key.getRow(), e.getKey(),
> key.getTimestamp());
> - BytesWritable val = e.getValue();
> - if(val.getSize() == DELETE_BYTES.getSize()
> - && val.compareTo(DELETE_BYTES) == 0) {
> + byte [] val = e.getValue();
> + if (DELETE_BYTES.compareTo(val) == 0) {
> // Column value is deleted. Don't return it.
> if (LOG.isDebugEnabled()) {
> LOG.debug("skipping deleted value for key: " +
> k.toString());
> @@ -1011,10 +1005,8 @@
> throws IOException {
> HRegion region = getRegion(regionName);
> long lockid = region.startUpdate(row);
> - this.leases.createLease(new Text(String.valueOf(clientid)),
> - new Text(String.valueOf(lockid)),
> - new RegionListener(region, lockid));
> -
> + this.leases.createLease(clientid, lockid,
> + new RegionListener(region, lockid));
> return lockid;
> }
>
> @@ -1041,11 +1033,11 @@
> /* (non-Javadoc)
> * @see org.apache.hadoop.hbase.HRegionInterface#put
> (org.apache.hadoop.io.Text, long, long, org.apache.hadoop.io.Text,
> org.apache.hadoop.io.BytesWritable)
> */
> - public void put(Text regionName, long clientid, long lockid,
> Text column,
> - BytesWritable val) throws IOException {
> + public void put(final Text regionName, final long clientid,
> + final long lockid, final Text column, final byte [] val)
> + throws IOException {
> HRegion region = getRegion(regionName, true);
> - leases.renewLease(new Text(String.valueOf(clientid)),
> - new Text(String.valueOf(lockid)));
> + leases.renewLease(clientid, lockid);
> region.put(lockid, column, val);
> }
>
> @@ -1053,10 +1045,9 @@
> * @see org.apache.hadoop.hbase.HRegionInterface#delete
> (org.apache.hadoop.io.Text, long, long, org.apache.hadoop.io.Text)
> */
> public void delete(Text regionName, long clientid, long lockid,
> Text column)
> - throws IOException {
> + throws IOException {
> HRegion region = getRegion(regionName);
> - leases.renewLease(new Text(String.valueOf(clientid)),
> - new Text(String.valueOf(lockid)));
> + leases.renewLease(clientid, lockid);
> region.delete(lockid, column);
> }
>
> @@ -1064,10 +1055,9 @@
> * @see org.apache.hadoop.hbase.HRegionInterface#abort
> (org.apache.hadoop.io.Text, long, long)
> */
> public void abort(Text regionName, long clientid, long lockid)
> - throws IOException {
> + throws IOException {
> HRegion region = getRegion(regionName, true);
> - leases.cancelLease(new Text(String.valueOf(clientid)),
> - new Text(String.valueOf(lockid)));
> + leases.cancelLease(clientid, lockid);
> region.abort(lockid);
> }
>
> @@ -1077,8 +1067,7 @@
> public void commit(Text regionName, long clientid, long lockid)
> throws IOException {
> HRegion region = getRegion(regionName, true);
> - leases.cancelLease(new Text(String.valueOf(clientid)),
> - new Text(String.valueOf(lockid)));
> + leases.cancelLease(clientid, lockid);
> region.commit(lockid);
> }
>
> @@ -1086,8 +1075,7 @@
> * @see org.apache.hadoop.hbase.HRegionInterface#renewLease
> (long, long)
> */
> public void renewLease(long lockid, long clientid) throws
> IOException {
> - leases.renewLease(new Text(String.valueOf(clientid)),
> - new Text(String.valueOf(lockid)));
> + leases.renewLease(clientid, lockid);
> }
>
> /**
> @@ -1139,29 +1127,31 @@
> // remote scanner interface
> ///////////////////////////////////////////////////////////////////
> ///////////
>
> - Map<Text, HInternalScannerInterface> scanners;
> + Map<String, HInternalScannerInterface> scanners =
> + Collections.synchronizedMap(new HashMap<String,
> + HInternalScannerInterface>());
>
> /**
> * Instantiated as a scanner lease.
> * If the lease times out, the scanner is closed
> */
> private class ScannerListener implements LeaseListener {
> - private Text scannerName;
> + private final String scannerName;
>
> - ScannerListener(Text scannerName) {
> - this.scannerName = scannerName;
> + ScannerListener(final String n) {
> + this.scannerName = n;
> }
>
> /* (non-Javadoc)
> * @see org.apache.hadoop.hbase.LeaseListener#leaseExpired()
> */
> public void leaseExpired() {
> - LOG.info("Scanner " + scannerName + " lease expired");
> + LOG.info("Scanner " + this.scannerName + " lease expired");
> HInternalScannerInterface s = null;
> synchronized(scanners) {
> - s = scanners.remove(scannerName);
> + s = scanners.remove(this.scannerName);
> }
> - if(s != null) {
> + if (s != null) {
> s.close();
> }
> }
> @@ -1177,11 +1167,11 @@
> try {
> HInternalScannerInterface s = r.getScanner(cols, firstRow);
> scannerId = rand.nextLong();
> - Text scannerName = new Text(String.valueOf(scannerId));
> + String scannerName = String.valueOf(scannerId);
> synchronized(scanners) {
> scanners.put(scannerName, s);
> }
> - leases.createLease(scannerName, scannerName,
> + leases.createLease(scannerId, scannerId,
> new ScannerListener(scannerName));
> } catch(IOException e) {
> LOG.error(e);
> @@ -1193,8 +1183,8 @@
> /* (non-Javadoc)
> * @see org.apache.hadoop.hbase.HRegionInterface#close(long)
> */
> - public void close(long scannerId) throws IOException {
> - Text scannerName = new Text(String.valueOf(scannerId));
> + public void close(final long scannerId) throws IOException {
> + String scannerName = String.valueOf(scannerId);
> HInternalScannerInterface s = null;
> synchronized(scanners) {
> s = scanners.remove(scannerName);
> @@ -1203,7 +1193,7 @@
> throw new UnknownScannerException(scannerName.toString());
> }
> s.close();
> - leases.cancelLease(scannerName, scannerName);
> + leases.cancelLease(scannerId, scannerId);
> }
>
> private static void printUsageAndExit() {
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegiondirReader.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegiondirReader.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HRegiondirReader.java Mon Jun 18 15:59:14 2007
> @@ -28,7 +28,6 @@
> import org.apache.hadoop.fs.FileSystem;
> import org.apache.hadoop.fs.Path;
> import org.apache.hadoop.fs.PathFilter;
> -import org.apache.hadoop.io.BytesWritable;
> import org.apache.hadoop.io.Text;
>
> /**
> @@ -168,7 +167,7 @@
> Text [] families = info.tableDesc.families().keySet().toArray
> (new Text [] {});
> HInternalScannerInterface scanner = r.getScanner(families, new
> Text());
> HStoreKey key = new HStoreKey();
> - TreeMap<Text, BytesWritable> results = new TreeMap<Text,
> BytesWritable>();
> + TreeMap<Text, byte []> results = new TreeMap<Text, byte []>();
> // Print out table header line.
> String s = info.startKey.toString();
> String startKey = (s == null || s.length() <= 0)? "<>": s;
> @@ -184,19 +183,15 @@
> // Every line starts with row name followed by column name
> // followed by cell content.
> while(scanner.next(key, results)) {
> - for (Map.Entry<Text, BytesWritable> es: results.entrySet()) {
> - Text colname = es.getKey();
> - BytesWritable colvalue = es.getValue();
> + for (Map.Entry<Text, byte []> es: results.entrySet()) {
> + Text colname = es.getKey();
> + byte [] colvalue = es.getValue();
> Object value = null;
> - byte[] bytes = new byte[colvalue.getSize()];
> if (colname.toString().equals("info:regioninfo")) {
> - // Then bytes are instance of an HRegionInfo.
> - System.arraycopy(colvalue, 0, bytes, 0, bytes.length);
> - value = new HRegionInfo(bytes);
> + value = new HRegionInfo(colvalue);
> } else {
> - value = new String(bytes, HConstants.UTF8_ENCODING);
> + value = new String(colvalue, HConstants.UTF8_ENCODING);
> }
> -
> System.out.println(" " + key + ", " + colname.toString() +
> ": \"" +
> value.toString() + "\"");
> }
>
> Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HScannerInterface.java
> URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/
> hbase/src/java/org/apache/hadoop/hbase/HScannerInterface.java?
> view=diff&rev=548523&r1=548522&r2=548523
> ======================================================================
> ========
> --- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HScannerInterface.java (original)
> +++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/
> hadoop/hbase/HScannerInterface.java Mon Jun 18 15:59:14 2007
> @@ -24,6 +24,7 @@
> * 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 boolean next(HStoreKey key, TreeMap<Text, byte[]> results)
> + throws IOException;
> public void close() throws IOException;
> }
>
>