Author: jimk
Date: Sat Jul 19 12:18:15 2008
New Revision: 678208
URL: http://svn.apache.org/viewvc?rev=678208&view=rev
Log:
HBASE-679 Regionserver addresses are still not right in the new tables page
Passed all regression tests and PerformanceEvaluation running with multiple region servers.
table.jsp now displays the correct information
HRegionInfo:
- removed getTableNameFromRegionName and parseMetaRegionRow we have the information in the
meta table, just use it.
HServerInfo:
- I had originally made some changes here but removed them. The only remaining changes are
javadoc
MetaScanner:
- build region name using ZEROES instead of NINES. When you scan you need a row name that
sorts before the first row rather than after the last row.
- scan using COLUMN_FAMILY_ARRAY instead of COL_REGIONINFO_ARRAY. This way you also get the
server name and start code
- change api for MetaScannerVisitor so that processRow only gets the RowResult. If you have
the RowResult you have everything you need.
HConnectionManager:
- change listTables' MetaScannerVisitor to conform to new processRow api
HTable:
- change getStartKeys' MetaScannerVisitor to conform to new processRow api
- getRegionsInfo: use new processRow api, and get the server address out of the RowResult,
rather than relying on the one that is cached in HConnectionManager
ScannerCallable:
- make constructor public, add javadoc
HMaster
- change createTable to get the HRegionInfo out of the RowResult and compare table name from
HRegionInfo with that in the HRegionInfo of the table being created, instead of parsing the
table name out of the row
TestHTable
- modify test's MetaScannerVisitor to conform to the new processRow api
TestHRegionInfo
- remove testParse as parse method no longer exists
table.jsp
- change catch of IOException to catch Exception and print stack trace. At least you'll be
able to see why the server crashes if it does.
Modified:
hadoop/hbase/trunk/CHANGES.txt
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionInfo.java
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HServerInfo.java
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/MetaScanner.java
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/HMaster.java
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTable.java
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
hadoop/hbase/trunk/src/webapps/master/table.jsp
Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=678208&r1=678207&r2=678208&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Sat Jul 19 12:18:15 2008
@@ -194,7 +194,8 @@
HBASE-42 Set region split size on table creation (Andrew Purtell via Stack)
HBASE-43 Add a read-only attribute to columns (Andrew Purtell via Stack)
HBASE-424 Should be able to enable/disable .META. table
-
+ HBASE-679 Regionserver addresses are still not right in the new tables page
+
IMPROVEMENTS
HBASE-559 MR example job to count table rows
HBASE-596 DemoClient.py (Ivan Begtin via Stack)
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionInfo.java?rev=678208&r1=678207&r2=678208&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionInfo.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionInfo.java Sat Jul 19 12:18:15
2008
@@ -22,8 +22,6 @@
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JenkinsHash;
@@ -57,62 +55,6 @@
public static final HRegionInfo FIRST_META_REGIONINFO =
new HRegionInfo(1L, HTableDescriptor.META_TABLEDESC);
- /**
- * Extracts table name prefix from a metaregion row name.
- * @param regionName A metaregion row name.
- * @return The table prefix of a region name.
- */
- public static byte [] getTableNameFromRegionName(final byte [] regionName) {
- return parseMetaRegionRow(regionName).get(0);
- }
-
- /**
- * Parses passed metaregion row into its constituent parts.
- * Presumes region names are ASCII characters only.
- * @param regionName A metaregion row name.
- * @return A list where first element is the tablename, second the row
- * portion, and the third the id.
- */
- public static List<byte []> parseMetaRegionRow(final byte [] regionName) {
- int offset = -1;
- for (int i = 0; i < regionName.length; i++) {
- if (regionName[i] == DELIMITER) {
- offset = i;
- break;
- }
- }
- if (offset == -1) {
- throw new IllegalArgumentException(Bytes.toString(regionName) +
- " does not contain '" + DELIMITER + "' character");
- }
- byte [] tableName = new byte[offset];
- System.arraycopy(regionName, 0, tableName, 0, offset);
- // Now move in from the tail till we hit DELIMITER to find the id
- offset = -1;
- for (int i = regionName.length - 1; i > tableName.length; i--) {
- if (regionName[i] == DELIMITER) {
- offset = i;
- break;
- }
- }
- if (offset == -1) {
- throw new IllegalArgumentException(Bytes.toString(regionName) +
- " does not have parseable tail");
- }
- byte [] row = new byte[offset - (tableName.length + 1)];
- System.arraycopy(regionName, tableName.length + 1, row, 0,
- offset - (tableName.length + 1));
- byte [] id = new byte[regionName.length - (offset + 1)];
- System.arraycopy(regionName, offset + 1, id, 0,
- regionName.length - (offset + 1));
- // Now make up an array to hold the three parse pieces.
- List<byte []> result = new ArrayList<byte []>(3);
- result.add(tableName);
- result.add(row);
- result.add(id);
- return result;
- }
-
private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
private boolean offLine = false;
private long regionId = -1;
@@ -395,7 +337,9 @@
public int hashCode() {
return this.hashCode;
}
-
+
+ /** @return the object version number */
+ @Override
public byte getVersion() {
return VERSION;
}
@@ -404,9 +348,8 @@
// Writable
//
- /**
- * {@inheritDoc}
- */
+ /** {@inheritDoc} */
+ @Override
public void write(DataOutput out) throws IOException {
super.write(out);
Bytes.writeByteArray(out, endKey);
@@ -419,9 +362,8 @@
out.writeInt(hashCode);
}
- /**
- * {@inheritDoc}
- */
+ /** {@inheritDoc} */
+ @Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
this.endKey = Bytes.readByteArray(in);
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HServerInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HServerInfo.java?rev=678208&r1=678207&r2=678208&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HServerInfo.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HServerInfo.java Sat Jul 19 12:18:15
2008
@@ -122,11 +122,13 @@
+ ", load: (" + this.load.toString() + ")";
}
+ /** {@inheritDoc} */
@Override
public boolean equals(Object obj) {
return compareTo(obj) == 0;
}
+ /** {@inheritDoc} */
@Override
public int hashCode() {
int result = this.serverAddress.hashCode();
@@ -137,6 +139,8 @@
// Writable
+
+ /** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
this.serverAddress.readFields(in);
this.startCode = in.readLong();
@@ -144,6 +148,7 @@
this.infoPort = in.readInt();
}
+ /** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
this.serverAddress.write(out);
out.writeLong(this.startCode);
@@ -151,6 +156,7 @@
out.writeInt(this.infoPort);
}
+ /** {@inheritDoc} */
public int compareTo(Object o) {
HServerInfo that = (HServerInfo)o;
int result = getServerAddress().compareTo(that.getServerAddress());
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=678208&r1=678207&r2=678208&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java Sat
Jul 19 12:18:15 2008
@@ -285,10 +285,9 @@
MetaScannerVisitor visitor = new MetaScannerVisitor() {
/** {@inheritDoc} */
- public boolean processRow(
- @SuppressWarnings("unused") RowResult rowResult,
- @SuppressWarnings("unused") HRegionLocation regionLocation,
- HRegionInfo info) {
+ public boolean processRow(RowResult rowResult) throws IOException {
+ HRegionInfo info = Writables.getHRegionInfo(
+ rowResult.get(COL_REGIONINFO));
// Only examine the rows where the startKey is zero length
if (info.getStartKey().length == 0) {
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java?rev=678208&r1=678207&r2=678208&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java Sat Jul 19 12:18:15
2008
@@ -42,6 +42,7 @@
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.Text;
/**
@@ -258,14 +259,10 @@
final List<byte[]> keyList = new ArrayList<byte[]>();
MetaScannerVisitor visitor = new MetaScannerVisitor() {
- @SuppressWarnings("unused")
- public boolean processRow(@SuppressWarnings("unused") RowResult rowResult,
- @SuppressWarnings("unused") HRegionLocation regionLocation,
- HRegionInfo info)
- throws IOException {
- if (!(Bytes.equals(info.getTableDesc().getName(), getTableName()))) {
- return false;
- }
+ public boolean processRow(RowResult rowResult) throws IOException {
+ HRegionInfo info = Writables.getHRegionInfo(
+ rowResult.get(HConstants.COL_REGIONINFO));
+
if (!(info.isOffline() || info.isSplit())) {
keyList.add(info.getStartKey());
}
@@ -288,15 +285,23 @@
new TreeMap<HRegionInfo, HServerAddress>();
MetaScannerVisitor visitor = new MetaScannerVisitor() {
- public boolean processRow(@SuppressWarnings("unused") RowResult rowResult,
- HRegionLocation regionLocation, HRegionInfo hri) {
+ public boolean processRow(RowResult rowResult) throws IOException {
+ HRegionInfo info = Writables.getHRegionInfo(
+ rowResult.get(HConstants.COL_REGIONINFO));
- HRegionInfo info = new UnmodifyableHRegionInfo(hri);
if (!(Bytes.equals(info.getTableDesc().getName(), getTableName()))) {
return false;
}
+
+ HServerAddress server = new HServerAddress();
+ Cell c = rowResult.get(HConstants.COL_SERVER);
+ if (c != null && c.getValue() != null && c.getValue().length >
0) {
+ String address = Bytes.toString(c.getValue());
+ server = new HServerAddress(address);
+ }
+
if (!(info.isOffline() || info.isSplit())) {
- regionMap.put(info, regionLocation.getServerAddress());
+ regionMap.put(new UnmodifyableHRegionInfo(info), server);
}
return true;
}
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/MetaScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/MetaScanner.java?rev=678208&r1=678207&r2=678208&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/MetaScanner.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/MetaScanner.java Sat Jul 19
12:18:15 2008
@@ -1,15 +1,12 @@
package org.apache.hadoop.hbase.client;
import java.io.IOException;
-import java.util.List;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Writables;
/**
* Scanner class that contains the <code>.META.</code> table scanning logic
@@ -45,30 +42,24 @@
MetaScannerVisitor visitor, byte[] tableName)
throws IOException {
HConnection connection = HConnectionManager.getConnection(configuration);
- boolean toContinue = true;
- byte [] startRow = Bytes.equals(tableName, EMPTY_START_ROW)? tableName:
- HRegionInfo.createRegionName(tableName, null, NINES);
+ byte [] startRow = tableName == null || tableName.length == 0 ?
+ HConstants.EMPTY_START_ROW :
+ HRegionInfo.createRegionName(tableName, null, ZEROES);
// Scan over each meta region
do {
ScannerCallable callable = new ScannerCallable(connection,
- META_TABLE_NAME, COL_REGIONINFO_ARRAY, tableName, LATEST_TIMESTAMP,
- null);
+ META_TABLE_NAME, COLUMN_FAMILY_ARRAY, startRow, LATEST_TIMESTAMP, null);
+ // Open scanner
+ connection.getRegionServerWithRetries(callable);
try {
- // Open scanner
- connection.getRegionServerWithRetries(callable);
- while (toContinue) {
- RowResult rowResult = connection.getRegionServerWithRetries(callable);
- if (rowResult == null || rowResult.size() == 0) {
+ RowResult r = null;
+ do {
+ r = connection.getRegionServerWithRetries(callable);
+ if (r == null || r.size() == 0) {
break;
}
- HRegionInfo info = Writables.getHRegionInfo(rowResult
- .get(COL_REGIONINFO));
- List<byte []> parse = HRegionInfo.parseMetaRegionRow(info.getRegionName());
- HRegionLocation regionLocation =
- connection.locateRegion(parse.get(0), parse.get(1));
- toContinue = visitor.processRow(rowResult, regionLocation, info);
- }
+ } while(visitor.processRow(r));
// Advance the startRow to the end key of the current region
startRow = callable.getHRegionInfo().getEndKey();
} finally {
@@ -89,12 +80,9 @@
* unnecessary for some reason.
*
* @param rowResult
- * @param regionLocation
- * @param info
* @return A boolean to know if it should continue to loop in the region
* @throws IOException
*/
- public boolean processRow(RowResult rowResult,
- HRegionLocation regionLocation, HRegionInfo info) throws IOException;
+ public boolean processRow(RowResult rowResult) throws IOException;
}
}
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java?rev=678208&r1=678207&r2=678208&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java Sat Jul
19 12:18:15 2008
@@ -39,7 +39,15 @@
private final long timestamp;
private final RowFilterInterface filter;
- protected ScannerCallable (HConnection connection, byte [] tableName, byte [][] columns,
+ /**
+ * @param connection
+ * @param tableName
+ * @param columns
+ * @param startRow
+ * @param timestamp
+ * @param filter
+ */
+ public ScannerCallable (HConnection connection, byte [] tableName, byte [][] columns,
byte [] startRow, long timestamp, RowFilterInterface filter) {
super(connection, tableName, startRow);
this.columns = columns;
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/HMaster.java?rev=678208&r1=678207&r2=678208&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/HMaster.java Sat Jul 19 12:18:15
2008
@@ -627,14 +627,10 @@
tableName, LATEST_TIMESTAMP, null);
try {
RowResult data = srvr.next(scannerid);
-
- // Test data and that the row for the data is for our table. If table
- // does not exist, scanner will return row after where our table would
- // be inserted if it exists so look for exact match on table name.
if (data != null && data.size() > 0) {
- byte [] tn = HRegionInfo.getTableNameFromRegionName(data.getRow());
- if (Bytes.equals(tn, tableName)) {
- // Then a region for this table already exists. Ergo table exists.
+ HRegionInfo info = Writables.getHRegionInfo(data.get(COL_REGIONINFO));
+ if (Bytes.equals(info.getTableDesc().getName(), tableName)) {
+ // A region for this table already exists. Ergo table exists.
throw new TableExistsException(Bytes.toString(tableName));
}
}
Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTable.java?rev=678208&r1=678207&r2=678208&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTable.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTable.java Sat Jul 19
12:18:15 2008
@@ -33,9 +33,9 @@
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HRegionLocation;
/**
* Tests HTable
@@ -137,7 +137,7 @@
try {
// make a modifiable descriptor
- HTableDescriptor desc = new HTableDescriptor(a.getMetadata());
+ HTableDescriptor desc = new HTableDescriptor(a.getTableDescriptor());
// offline the table
admin.disableTable(tableAname);
// add a user attribute to HTD
@@ -152,12 +152,13 @@
// Use a metascanner to avoid client API caching (HConnection has a
// metadata cache)
- MetaScanner.MetaScannerVisitor visitor = new MetaScanner.MetaScannerVisitor() {
- public boolean processRow(
- @SuppressWarnings("unused") RowResult rowResult,
- HRegionLocation regionLocation,
- HRegionInfo info) {
- LOG.info("visiting " + regionLocation.toString());
+ MetaScanner.MetaScannerVisitor visitor =
+ new MetaScanner.MetaScannerVisitor() {
+ public boolean processRow(RowResult rowResult) throws IOException {
+ HRegionInfo info = Writables.getHRegionInfo(
+ rowResult.get(HConstants.COL_REGIONINFO));
+
+ LOG.info("visiting " + info.toString());
HTableDescriptor desc = info.getTableDesc();
if (Bytes.compareTo(desc.getName(), tableAname) == 0) {
// check HTD attribute
Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java?rev=678208&r1=678207&r2=678208&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
(original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
Sat Jul 19 12:18:15 2008
@@ -19,25 +19,11 @@
*/
package org.apache.hadoop.hbase.regionserver;
-import java.util.List;
-
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.util.Bytes;
public class TestHRegionInfo extends HBaseTestCase {
- public void testParse() throws Exception {
- String tableName = getName();
- String row = getName();
- long id = 12345;
- List<byte []> parse =
- HRegionInfo.parseMetaRegionRow(Bytes.toBytes(tableName + "," + row +
- "," + Long.toString(id)));
- assertEquals(Bytes.toString(parse.get(0)), tableName);
- assertEquals(Bytes.toString(parse.get(1)), row);
- assertEquals(Long.parseLong(Bytes.toString(parse.get(2))), id);
- }
-
public void testCreateHRegionInfoName() throws Exception {
String tableName = "tablename";
final byte [] tn = Bytes.toBytes(tableName);
Modified: hadoop/hbase/trunk/src/webapps/master/table.jsp
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/webapps/master/table.jsp?rev=678208&r1=678207&r2=678208&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/webapps/master/table.jsp (original)
+++ hadoop/hbase/trunk/src/webapps/master/table.jsp Sat Jul 19 12:18:15 2008
@@ -70,7 +70,9 @@
</table>
<% }
}
- catch(IOException ioe) { }
+ catch(Exception ex) {
+ ex.printStackTrace();
+ }
}%>
</body>
</html>
|