HADOOP-2056 A table with row keys containing colon fails to split regions

Many modifications required because region names (which contain start key) were used as file or directory names. Now the startKey is encoded in an order preserving Base64 dialect.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@586159 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jim Kellerman 2007-10-18 22:21:09 +00:00
parent 0b92e8826a
commit cc202c4b74
34 changed files with 622 additions and 774 deletions

View File

@ -13,6 +13,7 @@ Trunk (unreleased changes)
HADOOP-2059 In tests, exceptions in min dfs shutdown should not fail test
(e.g. nightly #272)
HADOOP-2064 TestSplit assertion and NPE failures (Patch build #952 and #953)
HADOOP-2056 A table with row keys containing colon fails to split regions
IMPROVEMENTS
HADOOP-2401 Add convenience put method that takes writable

View File

@ -192,7 +192,7 @@ public class HBaseAdmin implements HConstants {
info = (HRegionInfo) Writables.getWritable(
((ImmutableBytesWritable) e.getValue()).get(), info);
if (info.tableDesc.getName().equals(tableName)) {
if (info.getTableDesc().getName().equals(tableName)) {
found = true;
}
}
@ -280,7 +280,7 @@ public class HBaseAdmin implements HConstants {
info = (HRegionInfo) Writables.getWritable(
((ImmutableBytesWritable) e.getValue()).get(), info);
isenabled = !info.offLine;
isenabled = !info.isOffline();
break;
}
}
@ -380,7 +380,7 @@ public class HBaseAdmin implements HConstants {
info = (HRegionInfo) Writables.getWritable(
((ImmutableBytesWritable) e.getValue()).get(), info);
disabled = info.offLine;
disabled = info.isOffline();
break;
}
}

View File

@ -261,8 +261,8 @@ public class HConnectionManager implements HConstants {
((ImmutableBytesWritable) e.getValue()).get(), info);
// Only examine the rows where the startKey is zero length
if (info.startKey.getLength() == 0) {
uniqueTables.add(info.tableDesc);
if (info.getStartKey().getLength() == 0) {
uniqueTables.add(info.getTableDesc());
}
}
}
@ -603,7 +603,7 @@ public class HConnectionManager implements HConstants {
HRegionInterface rootRegion = getHRegionConnection(rootRegionLocation);
try {
rootRegion.getRegionInfo(HGlobals.rootRegionInfo.regionName);
rootRegion.getRegionInfo(HRegionInfo.rootRegionInfo.getRegionName());
break;
} catch (IOException e) {
if (tries == numRetries - 1) {
@ -640,7 +640,7 @@ public class HConnectionManager implements HConstants {
new TreeMap<Text, HRegionLocation>();
rootServer.put(EMPTY_START_ROW,
new HRegionLocation(HGlobals.rootRegionInfo, rootRegionLocation));
new HRegionLocation(HRegionInfo.rootRegionInfo, rootRegionLocation));
return rootServer;
}
@ -707,7 +707,7 @@ public class HConnectionManager implements HConstants {
HRegionInfo regionInfo = (HRegionInfo) Writables.getWritable(
results.get(COL_REGIONINFO), new HRegionInfo());
if (!regionInfo.tableDesc.getName().equals(tableName)) {
if (!regionInfo.getTableDesc().getName().equals(tableName)) {
// We're done
if (LOG.isDebugEnabled()) {
LOG.debug("Found " + servers.size() + " servers for table " +
@ -736,7 +736,7 @@ public class HConnectionManager implements HConstants {
}
String serverAddress = Writables.bytesToString(bytes);
servers.put(regionInfo.startKey, new HRegionLocation(
servers.put(regionInfo.getStartKey(), new HRegionLocation(
regionInfo, new HServerAddress(serverAddress)));
}
} catch (IOException e) {

View File

@ -46,7 +46,8 @@ public interface HConstants {
/** Default master address */
static final String DEFAULT_MASTER_ADDRESS = DEFAULT_HOST + ":60000";
/** default port for master web api */
static final int DEFAULT_MASTER_INFOPORT = 60010;
/** Parameter name for hbase.regionserver address. */
@ -54,7 +55,8 @@ public interface HConstants {
/** Default region server address */
static final String DEFAULT_REGIONSERVER_ADDRESS = DEFAULT_HOST + ":60020";
/** default port for region server web api */
static final int DEFAULT_REGIONSERVER_INFOPORT = 60030;
/** Parameter name for what region server interface to use. */

View File

@ -1,68 +0,0 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import java.io.UnsupportedEncodingException;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
/**
* Global values that require initialization that cannot be done in HConstants
*/
public class HGlobals implements HConstants {
/** table descriptor for root table */
public static HTableDescriptor rootTableDesc = null;
/** region info for the root region */
public static HRegionInfo rootRegionInfo = null;
/** table descriptor for meta table */
public static HTableDescriptor metaTableDesc = null;
/** Value stored for a deleted item */
public static ImmutableBytesWritable deleteBytes = null;
/** Value written to HLog on a complete cache flush */
public static ImmutableBytesWritable completeCacheFlush = null;
static {
rootTableDesc = new HTableDescriptor(ROOT_TABLE_NAME.toString());
rootTableDesc.addFamily(new HColumnDescriptor(COLUMN_FAMILY, 1,
HColumnDescriptor.CompressionType.NONE, false, Integer.MAX_VALUE, null));
rootRegionInfo = new HRegionInfo(0L, rootTableDesc, null, null);
metaTableDesc = new HTableDescriptor(META_TABLE_NAME.toString());
metaTableDesc.addFamily(new HColumnDescriptor(COLUMN_FAMILY, 1,
HColumnDescriptor.CompressionType.NONE, false, Integer.MAX_VALUE, null));
try {
deleteBytes =
new ImmutableBytesWritable("HBASE::DELETEVAL".getBytes(UTF8_ENCODING));
completeCacheFlush =
new ImmutableBytesWritable("HBASE::CACHEFLUSH".getBytes(UTF8_ENCODING));
} catch (UnsupportedEncodingException e) {
assert(false);
}
}
}

View File

@ -159,7 +159,9 @@ public class HLog implements HConstants {
SequenceFile.Writer w = logWriters.get(regionName);
if (w == null) {
Path logfile = new Path(HRegion.getRegionDir(rootDir,
regionName), HREGION_OLDLOGFILE_NAME);
HRegionInfo.rootRegionInfo.getEncodedName()),
HREGION_OLDLOGFILE_NAME);
if (LOG.isDebugEnabled()) {
LOG.debug("getting new log file writer for path " + logfile);
}
@ -498,7 +500,7 @@ public class HLog implements HConstants {
return;
}
this.writer.append(new HLogKey(regionName, tableName, HLog.METAROW, logSeqId),
new HLogEdit(HLog.METACOLUMN, HGlobals.completeCacheFlush.get(),
new HLogEdit(HLog.METACOLUMN, HLogEdit.completeCacheFlush.get(),
System.currentTimeMillis()));
this.numEntries++;
Long seq = this.lastSeqWritten.get(regionName);

View File

@ -19,6 +19,7 @@
*/
package org.apache.hadoop.hbase;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.io.*;
import java.io.*;
@ -30,7 +31,35 @@ import java.io.*;
* The table and row are already identified in HLogKey.
* This just indicates the column and value.
*/
public class HLogEdit implements Writable {
public class HLogEdit implements Writable, HConstants {
/** Value stored for a deleted item */
public static ImmutableBytesWritable deleteBytes = null;
/** Value written to HLog on a complete cache flush */
public static ImmutableBytesWritable completeCacheFlush = null;
static {
try {
deleteBytes =
new ImmutableBytesWritable("HBASE::DELETEVAL".getBytes(UTF8_ENCODING));
completeCacheFlush =
new ImmutableBytesWritable("HBASE::CACHEFLUSH".getBytes(UTF8_ENCODING));
} catch (UnsupportedEncodingException e) {
assert(false);
}
}
/**
* @param value
* @return True if an entry and its content is {@link #deleteBytes}.
*/
public static boolean isDeleted(final byte [] value) {
return (value == null)? false: deleteBytes.compareTo(value) == 0;
}
private Text column = new Text();
private byte [] val;
private long timestamp;

View File

@ -124,6 +124,8 @@ HMasterRegionInterface {
// Default access so accesible from unit tests. MASTER is name of the webapp
// and the attribute name used stuffing this instance into web context.
InfoServer infoServer;
/** Name of master server */
public static final String MASTER = "master";
/**
@ -294,7 +296,7 @@ HMasterRegionInterface {
return false;
}
if (!info.isOffline()) {
LOG.warn("Region is split but not offline: " + info.regionName);
LOG.warn("Region is split but not offline: " + info.getRegionName());
}
return true;
}
@ -324,7 +326,7 @@ HMasterRegionInterface {
if (!hasReferencesA && !hasReferencesB) {
LOG.info("Deleting region " + parent.getRegionName() +
" because daughter splits no longer hold references");
if (!HRegion.deleteRegion(fs, dir, parent.getRegionName())) {
if (!HRegion.deleteRegion(fs, dir, parent.getEncodedName())) {
LOG.warn("Deletion of " + parent.getRegionName() + " failed");
}
@ -368,7 +370,7 @@ HMasterRegionInterface {
}
for (Text family: split.getTableDesc().families().keySet()) {
Path p = HStoreFile.getMapDir(fs.makeQualified(dir),
split.getRegionName(), HStoreKey.extractFamily(family));
split.getEncodedName(), HStoreKey.extractFamily(family));
// Look for reference files. Call listPaths with an anonymous
// instance of PathFilter.
@ -408,22 +410,23 @@ HMasterRegionInterface {
final String serverName, final long startCode)
throws IOException {
// Skip region - if ...
if(info.offLine // offline
|| killedRegions.contains(info.regionName) // queued for offline
|| regionsToDelete.contains(info.regionName)) { // queued for delete
unassignedRegions.remove(info.regionName);
assignAttempts.remove(info.regionName);
if(info.isOffline() // offline
|| killedRegions.contains(info.getRegionName()) // queued for offline
|| regionsToDelete.contains(info.getRegionName())) { // queued for delete
unassignedRegions.remove(info.getRegionName());
assignAttempts.remove(info.getRegionName());
return;
}
HServerInfo storedInfo = null;
if (serverName.length() != 0) {
Map<Text, HRegionInfo> regionsToKill = killList.get(serverName);
if (regionsToKill != null &&
regionsToKill.containsKey(info.regionName)) {
regionsToKill.containsKey(info.getRegionName())) {
// Skip if region is on kill list
if(LOG.isDebugEnabled()) {
LOG.debug("not assigning region (on kill list): " + info.regionName);
LOG.debug("not assigning region (on kill list): " +
info.getRegionName());
}
return;
}
@ -432,14 +435,15 @@ HMasterRegionInterface {
}
}
if (LOG.isDebugEnabled()) {
LOG.debug("Checking " + info.regionName + " is assigned");
LOG.debug("Checking " + info.getRegionName() + " is assigned");
}
if (!(unassignedRegions.containsKey(info.regionName) ||
pendingRegions.contains(info.regionName))
if (!(unassignedRegions.containsKey(info.getRegionName()) ||
pendingRegions.contains(info.getRegionName()))
&& (storedInfo == null || storedInfo.getStartCode() != startCode)) {
// The current assignment is no good
if (LOG.isDebugEnabled()) {
LOG.debug("Current assignment of " + info.regionName + " is no good");
LOG.debug("Current assignment of " + info.getRegionName() +
" is no good");
}
// Recover the region server's log if there is one.
if (serverName.length() != 0) {
@ -464,8 +468,8 @@ HMasterRegionInterface {
}
}
// Now get the region assigned
unassignedRegions.put(info.regionName, info);
assignAttempts.put(info.regionName, Long.valueOf(0L));
unassignedRegions.put(info.getRegionName(), info);
assignAttempts.put(info.getRegionName(), Long.valueOf(0L));
}
}
}
@ -503,7 +507,7 @@ HMasterRegionInterface {
// Don't interrupt us while we're working
synchronized(rootScannerLock) {
scanRegion(new MetaRegion(rootRegionLocation.get(),
HGlobals.rootRegionInfo.regionName, null));
HRegionInfo.rootRegionInfo.getRegionName(), null));
}
succeeded = true;
break;
@ -552,6 +556,7 @@ HMasterRegionInterface {
private RootScanner rootScannerThread;
Integer rootScannerLock = new Integer(0);
/** Describes a meta region and its server */
@SuppressWarnings("unchecked")
public static class MetaRegion implements Comparable {
private HServerAddress server;
@ -575,6 +580,7 @@ HMasterRegionInterface {
}
}
/** {@inheritDoc} */
@Override
public String toString() {
return "regionname: " + this.regionName.toString() + ", startKey: <" +
@ -859,7 +865,7 @@ HMasterRegionInterface {
this.rand = new Random();
Path rootRegionDir =
HRegion.getRegionDir(dir, HGlobals.rootRegionInfo.regionName);
HRegion.getRegionDir(dir, HRegionInfo.rootRegionInfo.getEncodedName());
LOG.info("Root region dir: " + rootRegionDir.toString());
try {
@ -871,10 +877,10 @@ HMasterRegionInterface {
if (!fs.exists(rootRegionDir)) {
LOG.info("bootstrap: creating ROOT and first META regions");
try {
HRegion root = HRegion.createHRegion(HGlobals.rootRegionInfo, this.dir,
this.conf, null);
HRegion meta = HRegion.createHRegion(new HRegionInfo(1L,
HGlobals.metaTableDesc, null, null), this.dir, this.conf, null);
HRegion root = HRegion.createHRegion(HRegionInfo.rootRegionInfo,
this.dir, this.conf, null);
HRegion meta = HRegion.createHRegion(HRegionInfo.firstMetaRegionInfo,
this.dir, this.conf, null);
// Add first region from the META table to the ROOT region.
HRegion.addRegionToMETA(root, meta);
@ -961,9 +967,9 @@ HMasterRegionInterface {
*/
void unassignRootRegion() {
this.rootRegionLocation.set(null);
this.unassignedRegions.put(HGlobals.rootRegionInfo.regionName,
HGlobals.rootRegionInfo);
this.assignAttempts.put(HGlobals.rootRegionInfo.regionName,
this.unassignedRegions.put(HRegionInfo.rootRegionInfo.getRegionName(),
HRegionInfo.rootRegionInfo);
this.assignAttempts.put(HRegionInfo.rootRegionInfo.getRegionName(),
Long.valueOf(0L));
// TODO: If the old root region server had a log, it needs splitting.
}
@ -1300,14 +1306,14 @@ HMasterRegionInterface {
if (!closed.get()) {
for (int i = 1; i < msgs.length; i++) {
HRegionInfo info = msgs[i].getRegionInfo();
if (info.tableDesc.getName().equals(ROOT_TABLE_NAME)) {
if (info.getTableDesc().getName().equals(ROOT_TABLE_NAME)) {
rootRegionLocation.set(null);
} else if (info.tableDesc.getName().equals(META_TABLE_NAME)) {
} else if (info.getTableDesc().getName().equals(META_TABLE_NAME)) {
onlineMetaRegions.remove(info.getStartKey());
}
this.unassignedRegions.put(info.regionName, info);
this.assignAttempts.put(info.regionName, Long.valueOf(0L));
this.unassignedRegions.put(info.getRegionName(), info);
this.assignAttempts.put(info.getRegionName(), Long.valueOf(0L));
}
}
}
@ -1449,13 +1455,13 @@ HMasterRegionInterface {
switch (incomingMsgs[i].getMsg()) {
case HMsg.MSG_REPORT_OPEN:
HRegionInfo regionInfo = unassignedRegions.get(region.regionName);
HRegionInfo regionInfo = unassignedRegions.get(region.getRegionName());
if (regionInfo == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("region server " + info.getServerAddress().toString()
+ " should not have opened region " + region.regionName);
+ " should not have opened region " + region.getRegionName());
}
// This Region should not have been opened.
@ -1467,12 +1473,12 @@ HMasterRegionInterface {
} else {
LOG.info(info.getServerAddress().toString() + " serving " +
region.regionName);
region.getRegionName());
// Remove from unassigned list so we don't assign it to someone else
this.unassignedRegions.remove(region.regionName);
this.assignAttempts.remove(region.regionName);
if (region.regionName.compareTo(
HGlobals.rootRegionInfo.regionName) == 0) {
this.unassignedRegions.remove(region.getRegionName());
this.assignAttempts.remove(region.getRegionName());
if (region.getRegionName().compareTo(
HRegionInfo.rootRegionInfo.getRegionName()) == 0) {
// Store the Root Region location (in memory)
synchronized (rootRegionLocation) {
this.rootRegionLocation.
@ -1485,7 +1491,7 @@ HMasterRegionInterface {
// Note that the table has been assigned and is waiting for the meta
// table to be updated.
pendingRegions.add(region.regionName);
pendingRegions.add(region.getRegionName());
// Queue up an update to note the region location.
@ -1499,26 +1505,26 @@ HMasterRegionInterface {
case HMsg.MSG_REPORT_CLOSE:
LOG.info(info.getServerAddress().toString() + " no longer serving " +
region.regionName);
region.getRegionName());
if (region.regionName.compareTo(
HGlobals.rootRegionInfo.regionName) == 0) {
if (region.getRegionName().compareTo(
HRegionInfo.rootRegionInfo.getRegionName()) == 0) {
// Root region
rootRegionLocation.set(null);
unassignedRegions.put(region.regionName, region);
assignAttempts.put(region.regionName, Long.valueOf(0L));
unassignedRegions.put(region.getRegionName(), region);
assignAttempts.put(region.getRegionName(), Long.valueOf(0L));
} else {
boolean reassignRegion = true;
boolean deleteRegion = false;
if (killedRegions.remove(region.regionName)) {
if (killedRegions.remove(region.getRegionName())) {
reassignRegion = false;
}
if (regionsToDelete.remove(region.regionName)) {
if (regionsToDelete.remove(region.getRegionName())) {
reassignRegion = false;
deleteRegion = true;
}
@ -1527,8 +1533,8 @@ HMasterRegionInterface {
// could create a race with the pending close if it gets
// reassigned before the close is processed.
unassignedRegions.remove(region.regionName);
assignAttempts.remove(region.regionName);
unassignedRegions.remove(region.getRegionName());
assignAttempts.remove(region.getRegionName());
try {
msgQueue.put(new PendingCloseReport(region, reassignRegion,
@ -1551,10 +1557,11 @@ HMasterRegionInterface {
unassignedRegions.put(newRegionB.getRegionName(), newRegionB);
assignAttempts.put(newRegionB.getRegionName(), Long.valueOf(0L));
LOG.info("region " + region.regionName + " split. New regions are: "
+ newRegionA.regionName + ", " + newRegionB.regionName);
LOG.info("region " + region.getRegionName() +
" split. New regions are: " + newRegionA.getRegionName() + ", " +
newRegionB.getRegionName());
if (region.tableDesc.getName().equals(META_TABLE_NAME)) {
if (region.getTableDesc().getName().equals(META_TABLE_NAME)) {
// A meta region has split.
onlineMetaRegions.remove(region.getStartKey());
@ -1574,7 +1581,7 @@ HMasterRegionInterface {
if (regionsToKill != null) {
for (HRegionInfo i: regionsToKill.values()) {
returnMsgs.add(new HMsg(HMsg.MSG_REGION_CLOSE, i));
killedRegions.add(i.regionName);
killedRegions.add(i.getRegionName());
}
}
@ -1899,7 +1906,7 @@ HMasterRegionInterface {
serverName + "> (or server is null). Marking unassigned if " +
"meta and clearing pendingRegions");
if (info.tableDesc.getName().equals(META_TABLE_NAME)) {
if (info.getTableDesc().getName().equals(META_TABLE_NAME)) {
onlineMetaRegions.remove(info.getStartKey());
}
@ -1910,14 +1917,14 @@ HMasterRegionInterface {
HashMap<Text, HRegionInfo> regionsToKill =
killList.get(deadServerName);
if (regionsToKill.containsKey(info.regionName)) {
regionsToKill.remove(info.regionName);
if (regionsToKill.containsKey(info.getRegionName())) {
regionsToKill.remove(info.getRegionName());
killList.put(deadServerName, regionsToKill);
unassignedRegions.remove(info.regionName);
assignAttempts.remove(info.regionName);
if (regionsToDelete.contains(info.regionName)) {
unassignedRegions.remove(info.getRegionName());
assignAttempts.remove(info.getRegionName());
if (regionsToDelete.contains(info.getRegionName())) {
// Delete this region
regionsToDelete.remove(info.regionName);
regionsToDelete.remove(info.getRegionName());
todo.deleteRegion = true;
} else {
// Mark region offline
@ -1927,7 +1934,7 @@ HMasterRegionInterface {
} else {
// Get region reassigned
regions.put(info.regionName, info);
regions.put(info.getRegionName(), info);
// If it was pending, remove.
// Otherwise will obstruct its getting reassigned.
@ -1952,7 +1959,7 @@ HMasterRegionInterface {
if (e.deleteRegion) {
b.delete(lockid, COL_REGIONINFO);
} else if (e.regionOffline) {
e.info.offLine = true;
e.info.setOffline(true);
b.put(lockid, COL_REGIONINFO, Writables.getBytes(e.info));
}
b.delete(lockid, COL_SERVER);
@ -1997,10 +2004,10 @@ HMasterRegionInterface {
deadServer.equals(rootRegionLocation.get())) {
rootRegionLocation.set(null);
unassignedRegions.put(HGlobals.rootRegionInfo.regionName,
HGlobals.rootRegionInfo);
unassignedRegions.put(HRegionInfo.rootRegionInfo.getRegionName(),
HRegionInfo.rootRegionInfo);
assignAttempts.put(HGlobals.rootRegionInfo.regionName,
assignAttempts.put(HRegionInfo.rootRegionInfo.getRegionName(),
Long.valueOf(0L));
}
rootChecked = true;
@ -2032,11 +2039,13 @@ HMasterRegionInterface {
LOG.debug("process server shutdown scanning root region on " +
rootRegionLocation.get().getBindAddress());
}
scannerId = server.openScanner(HGlobals.rootRegionInfo.regionName,
COLUMN_FAMILY_ARRAY, EMPTY_START_ROW,
System.currentTimeMillis(), null);
scannerId =
server.openScanner(HRegionInfo.rootRegionInfo.getRegionName(),
COLUMN_FAMILY_ARRAY, EMPTY_START_ROW,
System.currentTimeMillis(), null);
scanMetaRegion(server, scannerId, HGlobals.rootRegionInfo.regionName);
scanMetaRegion(server, scannerId,
HRegionInfo.rootRegionInfo.getRegionName());
break;
} catch (IOException e) {
@ -2132,7 +2141,7 @@ HMasterRegionInterface {
// If the region closing down is a meta region then we need to update
// the ROOT table
if (this.regionInfo.tableDesc.getName().equals(META_TABLE_NAME)) {
if (this.regionInfo.getTableDesc().getName().equals(META_TABLE_NAME)) {
this.rootRegion = true;
} else {
@ -2152,7 +2161,7 @@ HMasterRegionInterface {
if (closed.get()) {
return true;
}
LOG.info("region closed: " + regionInfo.regionName);
LOG.info("region closed: " + regionInfo.getRegionName());
// Mark the Region as unavailable in the appropriate meta table
@ -2164,7 +2173,7 @@ HMasterRegionInterface {
// scanned
return false;
}
metaRegionName = HGlobals.rootRegionInfo.regionName;
metaRegionName = HRegionInfo.rootRegionInfo.getRegionName();
server = connection.getHRegionConnection(rootRegionLocation.get());
onlineMetaRegions.remove(regionInfo.getStartKey());
@ -2200,13 +2209,13 @@ HMasterRegionInterface {
try {
BatchUpdate b = new BatchUpdate(rand.nextLong());
long lockid = b.startUpdate(regionInfo.regionName);
long lockid = b.startUpdate(regionInfo.getRegionName());
if (deleteRegion) {
b.delete(lockid, COL_REGIONINFO);
} else if (!reassignRegion ) {
regionInfo.offLine = true;
regionInfo.setOffline(true);
b.put(lockid, COL_REGIONINFO, Writables.getBytes(regionInfo));
}
b.delete(lockid, COL_SERVER);
@ -2224,17 +2233,17 @@ HMasterRegionInterface {
}
if (reassignRegion) {
LOG.info("reassign region: " + regionInfo.regionName);
LOG.info("reassign region: " + regionInfo.getRegionName());
unassignedRegions.put(regionInfo.regionName, regionInfo);
assignAttempts.put(regionInfo.regionName, Long.valueOf(0L));
unassignedRegions.put(regionInfo.getRegionName(), regionInfo);
assignAttempts.put(regionInfo.getRegionName(), Long.valueOf(0L));
} else if (deleteRegion) {
try {
HRegion.deleteRegion(fs, dir, regionInfo.regionName);
HRegion.deleteRegion(fs, dir, regionInfo.getEncodedName());
} catch (IOException e) {
e = RemoteExceptionHandler.checkIOException(e);
LOG.error("failed delete region " + regionInfo.regionName, e);
LOG.error("failed delete region " + regionInfo.getRegionName(), e);
throw e;
}
}
@ -2258,12 +2267,13 @@ HMasterRegionInterface {
// If true, the region which just came on-line is a META region.
// We need to look in the ROOT region for its information. Otherwise,
// its just an ordinary region. Look for it in the META table.
this.rootRegion = region.tableDesc.getName().equals(META_TABLE_NAME);
this.rootRegion = region.getTableDesc().getName().equals(META_TABLE_NAME);
this.region = region;
this.serverAddress = info.getServerAddress();
this.startCode = Writables.longToBytes(info.getStartCode());
}
/** {@inheritDoc} */
@Override
public String toString() {
return "PendingOpenOperation from " + serverAddress.toString();
@ -2292,7 +2302,7 @@ HMasterRegionInterface {
}
return false;
}
metaRegionName = HGlobals.rootRegionInfo.regionName;
metaRegionName = HRegionInfo.rootRegionInfo.getRegionName();
server = connection.getHRegionConnection(rootRegionLocation.get());
} else {
if (!rootScanned ||
@ -2329,10 +2339,10 @@ HMasterRegionInterface {
Writables.stringToBytes(serverAddress.toString()));
b.put(lockid, COL_STARTCODE, startCode);
server.batchUpdate(metaRegionName, System.currentTimeMillis(), b);
if (region.tableDesc.getName().equals(META_TABLE_NAME)) {
if (region.getTableDesc().getName().equals(META_TABLE_NAME)) {
// It's a meta region.
MetaRegion m = new MetaRegion(this.serverAddress,
this.region.regionName, this.region.startKey);
this.region.getRegionName(), this.region.getRegionName());
if (!initialMetaScanComplete) {
// Put it on the queue to be scanned for the first time.
try {
@ -2345,7 +2355,7 @@ HMasterRegionInterface {
} else {
// Add it to the online meta regions
LOG.debug("Adding to onlineMetaRegions: " + m.toString());
onlineMetaRegions.put(this.region.startKey, m);
onlineMetaRegions.put(this.region.getRegionName(), m);
}
}
// If updated successfully, remove from pending list.
@ -2393,7 +2403,7 @@ HMasterRegionInterface {
if (!isMasterRunning()) {
throw new MasterNotRunningException();
}
HRegionInfo newRegion = new HRegionInfo(rand.nextLong(), desc, null, null);
HRegionInfo newRegion = new HRegionInfo(desc, null, null);
for (int tries = 0; tries < numRetries; tries++) {
try {
@ -2418,7 +2428,7 @@ HMasterRegionInterface {
private Set<Text> tableInCreation = new HashSet<Text>();
private void createTable(final HRegionInfo newRegion) throws IOException {
Text tableName = newRegion.tableDesc.getName();
Text tableName = newRegion.getTableDesc().getName();
synchronized (tableInCreation) {
if (tableInCreation.contains(tableName)) {
throw new TableExistsException("Table " + tableName + " in process "
@ -2432,8 +2442,8 @@ HMasterRegionInterface {
// for the table we want to create already exists, then table already
// created. Throw already-exists exception.
MetaRegion m = (onlineMetaRegions.containsKey(newRegion.regionName) ?
onlineMetaRegions.get(newRegion.regionName) :
MetaRegion m = (onlineMetaRegions.containsKey(newRegion.getRegionName()) ?
onlineMetaRegions.get(newRegion.getRegionName()) :
onlineMetaRegions.get(onlineMetaRegions.headMap(
newRegion.getTableDesc().getName()).lastKey()));
@ -2631,7 +2641,7 @@ HMasterRegionInterface {
throw new IOException(COL_REGIONINFO + " not found");
}
if (info.tableDesc.getName().compareTo(tableName) > 0) {
if (info.getTableDesc().getName().compareTo(tableName) > 0) {
break; // Beyond any more entries for this table
}
@ -2690,7 +2700,7 @@ HMasterRegionInterface {
}
protected boolean isEnabled(HRegionInfo info) {
return !info.offLine;
return !info.isOffline();
}
protected abstract void processScanItem(String serverName, long startCode,
@ -2738,7 +2748,7 @@ HMasterRegionInterface {
LOG.debug("processing unserved regions");
}
for (HRegionInfo i: unservedRegions) {
if (i.offLine && i.isSplit()) {
if (i.isOffline() && i.isSplit()) {
if (LOG.isDebugEnabled()) {
LOG.debug("Skipping region " + i.toString() + " because it is " +
"offline because it has been split");
@ -2749,28 +2759,28 @@ HMasterRegionInterface {
// Update meta table
if (LOG.isDebugEnabled()) {
LOG.debug("updating columns in row: " + i.regionName);
LOG.debug("updating columns in row: " + i.getRegionName());
}
BatchUpdate b = new BatchUpdate(rand.nextLong());
lockid = b.startUpdate(i.regionName);
lockid = b.startUpdate(i.getRegionName());
updateRegionInfo(b, i);
b.delete(lockid, COL_SERVER);
b.delete(lockid, COL_STARTCODE);
server.batchUpdate(m.getRegionName(), System.currentTimeMillis(), b);
if (LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: " + i.regionName);
LOG.debug("updated columns in row: " + i.getRegionName());
}
if (online) { // Bring offline regions on-line
if (!unassignedRegions.containsKey(i.regionName)) {
unassignedRegions.put(i.regionName, i);
assignAttempts.put(i.regionName, Long.valueOf(0L));
if (!unassignedRegions.containsKey(i.getRegionName())) {
unassignedRegions.put(i.getRegionName(), i);
assignAttempts.put(i.getRegionName(), Long.valueOf(0L));
}
} else { // Prevent region from getting assigned.
unassignedRegions.remove(i.regionName);
assignAttempts.remove(i.regionName);
unassignedRegions.remove(i.getRegionName());
assignAttempts.remove(i.getRegionName());
}
}
@ -2794,9 +2804,10 @@ HMasterRegionInterface {
}
for (HRegionInfo i: e.getValue()) {
if (LOG.isDebugEnabled()) {
LOG.debug("adding region " + i.regionName + " to local kill list");
LOG.debug("adding region " + i.getRegionName() +
" to local kill list");
}
localKillList.put(i.regionName, i);
localKillList.put(i.getRegionName(), i);
}
if (localKillList.size() > 0) {
if (LOG.isDebugEnabled()) {
@ -2812,7 +2823,7 @@ HMasterRegionInterface {
protected void updateRegionInfo(final BatchUpdate b, final HRegionInfo i)
throws IOException {
i.offLine = !online;
i.setOffline(!online);
b.put(lockid, COL_REGIONINFO, Writables.getBytes(i));
}
}
@ -2836,7 +2847,7 @@ HMasterRegionInterface {
for (HashSet<HRegionInfo> s: servedRegions.values()) {
for (HRegionInfo i: s) {
regionsToDelete.add(i.regionName);
regionsToDelete.add(i.getRegionName());
}
}
@ -2846,10 +2857,10 @@ HMasterRegionInterface {
// Delete the region
try {
HRegion.deleteRegion(fs, dir, i.regionName);
HRegion.deleteRegion(fs, dir, i.getEncodedName());
} catch (IOException e) {
LOG.error("failed to delete region " + i.regionName,
LOG.error("failed to delete region " + i.getRegionName(),
RemoteExceptionHandler.checkIOException(e));
}
}
@ -2885,11 +2896,11 @@ HMasterRegionInterface {
HRegionInfo i) throws IOException {
BatchUpdate b = new BatchUpdate(rand.nextLong());
long lockid = b.startUpdate(i.regionName);
long lockid = b.startUpdate(i.getRegionName());
b.put(lockid, COL_REGIONINFO, Writables.getBytes(i));
server.batchUpdate(regionName, System.currentTimeMillis(), b);
if (LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: " + i.regionName);
LOG.debug("updated columns in row: " + i.getRegionName());
}
}
}
@ -2908,13 +2919,13 @@ HMasterRegionInterface {
throws IOException {
for (HRegionInfo i: unservedRegions) {
i.tableDesc.families().remove(columnName);
i.getTableDesc().families().remove(columnName);
updateRegionInfo(server, m.getRegionName(), i);
// Delete the directories used by the column
fs.delete(HStoreFile.getMapDir(dir, i.regionName, columnName));
fs.delete(HStoreFile.getInfoDir(dir, i.regionName, columnName));
fs.delete(HStoreFile.getMapDir(dir, i.getEncodedName(), columnName));
fs.delete(HStoreFile.getInfoDir(dir, i.getEncodedName(), columnName));
}
}
}
@ -2938,7 +2949,7 @@ HMasterRegionInterface {
// When the region is brought on-line, it will find the column missing
// and create it.
i.tableDesc.addFamily(newColumn);
i.getTableDesc().addFamily(newColumn);
updateRegionInfo(server, m.getRegionName(), i);
}
}

View File

@ -267,7 +267,7 @@ public class HMemcache {
for (Map.Entry<HStoreKey, byte []> es: tailMap.entrySet()) {
HStoreKey itKey = es.getKey();
if (itKey.matchesRowCol(curKey)) {
if (!isDeleted(es.getValue())) {
if (!HLogEdit.isDeleted(es.getValue())) {
result.add(tailMap.get(itKey));
curKey.setVersion(itKey.getTimestamp() - 1);
}
@ -323,7 +323,7 @@ public class HMemcache {
if (!key.matchesRowCol(origin)) {
break;
}
if (!isDeleted(es.getValue())) {
if (!HLogEdit.isDeleted(es.getValue())) {
result.add(key);
if (versions != HConstants.ALL_VERSIONS && result.size() >= versions) {
// We have enough results. Return.
@ -341,15 +341,7 @@ public class HMemcache {
* the cell has been deleted.
*/
boolean isDeleted(final HStoreKey key) {
return isDeleted(this.memcache.get(key));
}
/**
* @param value
* @return True if an entry and its content is {@link HGlobals.deleteBytes}.
*/
boolean isDeleted(final byte [] value) {
return (value == null)? false: HGlobals.deleteBytes.compareTo(value) == 0;
return HLogEdit.isDeleted(this.memcache.get(key));
}
/**

View File

@ -225,8 +225,8 @@ class HMerge implements HConstants {
HRegionInfo region =
(HRegionInfo) Writables.getWritable(bytes, new HRegionInfo());
if(!region.offLine) {
throw new TableNotDisabledException("region " + region.regionName
if(!region.isOffline()) {
throw new TableNotDisabledException("region " + region.getRegionName()
+ " is not disabled");
}
return region;
@ -267,7 +267,7 @@ class HMerge implements HConstants {
oldRegion2
};
for(int r = 0; r < regionsToDelete.length; r++) {
if(regionsToDelete[r].equals(latestRegion.regionName)) {
if(regionsToDelete[r].equals(latestRegion.getRegionName())) {
latestRegion = null;
}
long lockid = -1L;
@ -290,7 +290,7 @@ class HMerge implements HConstants {
}
ByteArrayOutputStream byteValue = new ByteArrayOutputStream();
DataOutputStream s = new DataOutputStream(byteValue);
newRegion.getRegionInfo().offLine = true;
newRegion.getRegionInfo().setOffline(true);
newRegion.getRegionInfo().write(s);
long lockid = -1L;
try {
@ -326,7 +326,7 @@ class HMerge implements HConstants {
// Scan root region to find all the meta regions
HRegion root =
new HRegion(dir, hlog,fs, conf, HGlobals.rootRegionInfo, null);
new HRegion(dir, hlog,fs, conf, HRegionInfo.rootRegionInfo, null);
HInternalScannerInterface rootScanner =
root.getScanner(META_COLS, new Text(), System.currentTimeMillis(), null);
@ -362,7 +362,7 @@ class HMerge implements HConstants {
HRegion newRegion) throws IOException {
HRegion root =
new HRegion(dir, hlog, fs, conf, HGlobals.rootRegionInfo, null);
new HRegion(dir, hlog, fs, conf, HRegionInfo.rootRegionInfo, null);
Text[] regionsToDelete = {
oldRegion1,
@ -394,7 +394,7 @@ class HMerge implements HConstants {
}
ByteArrayOutputStream byteValue = new ByteArrayOutputStream();
DataOutputStream s = new DataOutputStream(byteValue);
newRegion.getRegionInfo().offLine = true;
newRegion.getRegionInfo().setOffline(true);
newRegion.getRegionInfo().write(s);
long lockid = -1L;
try {

View File

@ -120,9 +120,9 @@ public class HRegion implements HConstants {
fs.mkdirs(merges);
}
HRegionInfo newRegionInfo
= new HRegionInfo(Math.abs(rand.nextLong()), tabledesc, startKey, endKey);
Path newRegionDir = HRegion.getRegionDir(merges, newRegionInfo.regionName);
HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey);
Path newRegionDir =
HRegion.getRegionDir(merges, newRegionInfo.getEncodedName());
if(fs.exists(newRegionDir)) {
throw new IOException("Cannot merge; target file collision at " +
newRegionDir);
@ -138,7 +138,8 @@ public class HRegion implements HConstants {
for (Map.Entry<Text, Vector<HStoreFile>> es : byFamily.entrySet()) {
Text colFamily = es.getKey();
Vector<HStoreFile> srcFiles = es.getValue();
HStoreFile dst = new HStoreFile(conf, merges, newRegionInfo.regionName,
HStoreFile dst =
new HStoreFile(conf, merges, newRegionInfo.getEncodedName(),
colFamily, Math.abs(rand.nextLong()));
dst.mergeStoreFiles(srcFiles, fs, conf);
}
@ -253,7 +254,8 @@ public class HRegion implements HConstants {
// Declare the regionName. This is a unique string for the region, used to
// build a unique filename.
this.regiondir = HRegion.getRegionDir(rootDir, this.regionInfo.regionName);
this.regiondir =
HRegion.getRegionDir(rootDir, this.regionInfo.getEncodedName());
Path oldLogFile = new Path(regiondir, HREGION_OLDLOGFILE_NAME);
// Move prefab HStore files into place (if any). This picks up split files
@ -265,10 +267,10 @@ public class HRegion implements HConstants {
// Load in all the HStores.
long maxSeqId = -1;
for(Map.Entry<Text, HColumnDescriptor> e :
this.regionInfo.tableDesc.families().entrySet()) {
this.regionInfo.getTableDesc().families().entrySet()) {
Text colFamily = HStoreKey.extractFamily(e.getKey());
HStore store = new HStore(rootDir, this.regionInfo.regionName,
HStore store = new HStore(rootDir, this.regionInfo.getEncodedName(),
e.getValue(), fs, oldLogFile, conf);
stores.put(colFamily, store);
@ -306,7 +308,7 @@ public class HRegion implements HConstants {
// HRegion is ready to go!
this.writestate.compacting = false;
LOG.info("region " + this.regionInfo.regionName + " available");
LOG.info("region " + this.regionInfo.getRegionName() + " available");
}
/**
@ -317,7 +319,7 @@ public class HRegion implements HConstants {
return this.minSequenceId;
}
/** Returns a HRegionInfo object for this region */
/** @return a HRegionInfo object for this region */
public HRegionInfo getRegionInfo() {
return this.regionInfo;
}
@ -360,7 +362,7 @@ public class HRegion implements HConstants {
*/
Vector<HStoreFile> close(boolean abort) throws IOException {
if (isClosed()) {
LOG.info("region " + this.regionInfo.regionName + " already closed");
LOG.info("region " + this.regionInfo.getRegionName() + " already closed");
return null;
}
lock.obtainWriteLock();
@ -393,7 +395,7 @@ public class HRegion implements HConstants {
result.addAll(store.close());
}
this.closed.set(true);
LOG.info("closed " + this.regionInfo.regionName);
LOG.info("closed " + this.regionInfo.getRegionName());
return result;
} finally {
lock.releaseWriteLock();
@ -416,15 +418,15 @@ public class HRegion implements HConstants {
checkMidKey(midKey);
long startTime = System.currentTimeMillis();
Path splits = getSplitsDir();
HRegionInfo regionAInfo = new HRegionInfo(Math.abs(rand.nextLong()),
this.regionInfo.tableDesc, this.regionInfo.startKey, midKey);
Path dirA = getSplitRegionDir(splits, regionAInfo.regionName);
HRegionInfo regionAInfo = new HRegionInfo(this.regionInfo.getTableDesc(),
this.regionInfo.getStartKey(), midKey);
Path dirA = getSplitRegionDir(splits, regionAInfo.getEncodedName());
if(fs.exists(dirA)) {
throw new IOException("Cannot split; target file collision at " + dirA);
}
HRegionInfo regionBInfo = new HRegionInfo(Math.abs(rand.nextLong()),
this.regionInfo.tableDesc, midKey, null);
Path dirB = getSplitRegionDir(splits, regionBInfo.regionName);
HRegionInfo regionBInfo = new HRegionInfo(this.regionInfo.getTableDesc(),
midKey, null);
Path dirB = getSplitRegionDir(splits, regionBInfo.getEncodedName());
if(this.fs.exists(dirB)) {
throw new IOException("Cannot split; target file collision at " + dirB);
}
@ -455,18 +457,18 @@ public class HRegion implements HConstants {
for(HStoreFile h: hstoreFilesToSplit) {
// A reference to the bottom half of the hsf store file.
HStoreFile.Reference aReference = new HStoreFile.Reference(
getRegionName(), h.getFileId(), new HStoreKey(midKey),
this.regionInfo.getEncodedName(), h.getFileId(), new HStoreKey(midKey),
HStoreFile.Range.bottom);
HStoreFile a = new HStoreFile(this.conf, splits,
regionAInfo.regionName, h.getColFamily(), Math.abs(rand.nextLong()),
aReference);
regionAInfo.getEncodedName(), h.getColFamily(),
Math.abs(rand.nextLong()), aReference);
// Reference to top half of the hsf store file.
HStoreFile.Reference bReference = new HStoreFile.Reference(
getRegionName(), h.getFileId(), new HStoreKey(midKey),
this.regionInfo.getEncodedName(), h.getFileId(), new HStoreKey(midKey),
HStoreFile.Range.top);
HStoreFile b = new HStoreFile(this.conf, splits,
regionBInfo.regionName, h.getColFamily(), Math.abs(rand.nextLong()),
bReference);
regionBInfo.getEncodedName(), h.getColFamily(),
Math.abs(rand.nextLong()), bReference);
h.splitStoreFile(a, b, this.fs);
}
@ -482,25 +484,25 @@ public class HRegion implements HConstants {
LOG.debug("Cleaned up " + splits.toString() + " " + deleted);
}
HRegion regions[] = new HRegion [] {regionA, regionB};
LOG.info("Region split of " + this.regionInfo.regionName + " complete; " +
"new regions: " + regions[0].getRegionName() + ", " +
regions[1].getRegionName() + ". Split took " +
StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime));
LOG.info("Region split of " + this.regionInfo.getRegionName() +
" complete; " + "new regions: " + regions[0].getRegionName() + ", " +
regions[1].getRegionName() + ". Split took " +
StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime));
return regions;
}
private void checkMidKey(final Text midKey) throws IOException {
if(((this.regionInfo.startKey.getLength() != 0)
&& (this.regionInfo.startKey.compareTo(midKey) > 0))
|| ((this.regionInfo.endKey.getLength() != 0)
&& (this.regionInfo.endKey.compareTo(midKey) < 0))) {
if(((this.regionInfo.getStartKey().getLength() != 0)
&& (this.regionInfo.getStartKey().compareTo(midKey) > 0))
|| ((this.regionInfo.getEndKey().getLength() != 0)
&& (this.regionInfo.getEndKey().compareTo(midKey) < 0))) {
throw new IOException("Region splitkey must lie within region " +
"boundaries.");
}
}
private Path getSplitRegionDir(final Path splits, final Text regionName) {
return HRegion.getRegionDir(splits, regionName);
private Path getSplitRegionDir(final Path splits, final String region) {
return HRegion.getRegionDir(splits, region);
}
private Path getSplitsDir() throws IOException {
@ -517,22 +519,22 @@ public class HRegion implements HConstants {
/** @return start key for region */
public Text getStartKey() {
return this.regionInfo.startKey;
return this.regionInfo.getStartKey();
}
/** @return end key for region */
public Text getEndKey() {
return this.regionInfo.endKey;
return this.regionInfo.getEndKey();
}
/** @return region id */
public long getRegionId() {
return this.regionInfo.regionId;
return this.regionInfo.getRegionId();
}
/** @return region name */
public Text getRegionName() {
return this.regionInfo.regionName;
return this.regionInfo.getRegionName();
}
/** @return root directory path */
@ -542,7 +544,7 @@ public class HRegion implements HConstants {
/** @return HTableDescriptor for this region */
public HTableDescriptor getTableDesc() {
return this.regionInfo.tableDesc;
return this.regionInfo.getTableDesc();
}
/** @return HLog in use for this region */
@ -788,7 +790,7 @@ public class HRegion implements HConstants {
if(!shouldFlush) {
if(LOG.isDebugEnabled()) {
LOG.debug("NOT flushing memcache for region " +
this.regionInfo.regionName);
this.regionInfo.getRegionName());
}
return;
}
@ -837,7 +839,7 @@ public class HRegion implements HConstants {
if(LOG.isDebugEnabled()) {
startTime = System.currentTimeMillis();
LOG.debug("Started memcache flush for region " +
this.regionInfo.regionName + ". Size " +
this.regionInfo.getRegionName() + ". Size " +
StringUtils.humanReadableInt(this.memcache.getSize()));
}
@ -866,7 +868,7 @@ public class HRegion implements HConstants {
long logCacheFlushId = retval.sequenceId;
if(LOG.isDebugEnabled()) {
LOG.debug("Snapshotted memcache for region " +
this.regionInfo.regionName + " with sequence id " +
this.regionInfo.getRegionName() + " with sequence id " +
retval.sequenceId + " and entries " +
retval.memcacheSnapshot.size());
}
@ -893,8 +895,8 @@ public class HRegion implements HConstants {
// This tells future readers that the HStores were emitted correctly,
// and that all updates to the log for this regionName that have lower
// log-sequence-ids can be safely ignored.
this.log.completeCacheFlush(this.regionInfo.regionName,
regionInfo.tableDesc.getName(), logCacheFlushId);
this.log.completeCacheFlush(this.regionInfo.getRegionName(),
regionInfo.getTableDesc().getName(), logCacheFlushId);
} finally {
// C. Delete the now-irrelevant memcache snapshot; its contents have been
@ -909,7 +911,7 @@ public class HRegion implements HConstants {
}
if (LOG.isDebugEnabled()) {
LOG.debug("Finished memcache flush for region " +
this.regionInfo.regionName + " in " +
this.regionInfo.getRegionName() + " in " +
(System.currentTimeMillis() - startTime) + "ms");
}
}
@ -1207,7 +1209,7 @@ public class HRegion implements HConstants {
* @throws IOException
*/
public void put(long lockid, Text targetCol, byte [] val) throws IOException {
if (HGlobals.deleteBytes.compareTo(val) == 0) {
if (HLogEdit.isDeleted(val)) {
throw new IOException("Cannot insert value: " + val);
}
localput(lockid, targetCol, val);
@ -1221,7 +1223,7 @@ public class HRegion implements HConstants {
* @throws IOException
*/
public void delete(long lockid, Text targetCol) throws IOException {
localput(lockid, targetCol, HGlobals.deleteBytes.get());
localput(lockid, targetCol, HLogEdit.deleteBytes.get());
}
/**
@ -1258,7 +1260,7 @@ public class HRegion implements HConstants {
List<HStoreKey> keys = getKeys(origin, versions);
if (keys.size() > 0) {
TreeMap<Text, byte []> edits = new TreeMap<Text, byte []>();
edits.put(column, HGlobals.deleteBytes.get());
edits.put(column, HLogEdit.deleteBytes.get());
for (HStoreKey key: keys) {
update(row, key.getTimestamp(), edits);
}
@ -1400,7 +1402,7 @@ public class HRegion implements HConstants {
// Run updates one at a time so we can supply appropriate timestamp
long now = System.currentTimeMillis();
for (Map.Entry<Text, byte []>e: updatesByColumn.entrySet()) {
if (HGlobals.deleteBytes.equals(e.getValue())) {
if (HLogEdit.isDeleted(e.getValue())) {
// Its a delete. Delete latest. deleteMultiple calls update for us.
// Actually regets the row lock but since we already have it, should
// be fine.
@ -1431,8 +1433,8 @@ public class HRegion implements HConstants {
if (updatesByColumn == null || updatesByColumn.size() <= 0) {
return;
}
this.log.append(regionInfo.regionName, regionInfo.tableDesc.getName(),
row, updatesByColumn, timestamp);
this.log.append(regionInfo.getRegionName(),
regionInfo.getTableDesc().getName(), row, updatesByColumn, timestamp);
this.memcache.add(row, updatesByColumn, timestamp);
}
@ -1442,17 +1444,17 @@ public class HRegion implements HConstants {
/** Make sure this is a valid row for the HRegion */
void checkRow(Text row) throws IOException {
if(((regionInfo.startKey.getLength() == 0)
|| (regionInfo.startKey.compareTo(row) <= 0))
&& ((regionInfo.endKey.getLength() == 0)
|| (regionInfo.endKey.compareTo(row) > 0))) {
if(((regionInfo.getStartKey().getLength() == 0)
|| (regionInfo.getStartKey().compareTo(row) <= 0))
&& ((regionInfo.getEndKey().getLength() == 0)
|| (regionInfo.getEndKey().compareTo(row) > 0))) {
// all's well
} else {
throw new WrongRegionException("Requested row out of range for " +
"HRegion " + regionInfo.regionName + ", startKey='" +
regionInfo.startKey + "', endKey='" + regionInfo.endKey + "', row='" +
row + "'");
"HRegion " + regionInfo.getRegionName() + ", startKey='" +
regionInfo.getStartKey() + "', getEndKey()='" + regionInfo.getEndKey() +
"', row='" + row + "'");
}
}
@ -1463,10 +1465,10 @@ public class HRegion implements HConstants {
*/
void checkColumn(Text columnName) throws IOException {
Text family = new Text(HStoreKey.extractFamily(columnName) + ":");
if(! regionInfo.tableDesc.hasFamily(family)) {
if(! regionInfo.getTableDesc().hasFamily(family)) {
throw new IOException("Requested column family " + family
+ " does not exist in HRegion " + regionInfo.regionName
+ " for table " + regionInfo.tableDesc.getName());
+ " does not exist in HRegion " + regionInfo.getRegionName()
+ " for table " + regionInfo.getTableDesc().getName());
}
}
@ -1615,14 +1617,17 @@ public class HRegion implements HConstants {
}
}
/** @return true if the scanner is a wild card scanner */
public boolean isWildcardScanner() {
return wildcardMatch;
}
/** @return true if the scanner is a multiple match scanner */
public boolean isMultipleMatchScanner() {
return multipleMatchers;
}
/** {@inheritDoc} */
public boolean next(HStoreKey key, SortedMap<Text, byte[]> results)
throws IOException {
// Filtered flag is set by filters. If a cell has been 'filtered out'
@ -1690,7 +1695,7 @@ public class HRegion implements HConstants {
key.getTimestamp());
for (Map.Entry<Text, byte[]> e : resultSets[i].entrySet()) {
hsk.setColumn(e.getKey());
if (HGlobals.deleteBytes.equals(e.getValue())) {
if (HLogEdit.isDeleted(e.getValue())) {
if (!deletes.contains(hsk)) {
// Key changes as we cycle the for loop so add a copy to
// the set of deletes.
@ -1817,10 +1822,9 @@ public class HRegion implements HConstants {
*
* @throws IOException
*/
static HRegion createHRegion(final HRegionInfo info,
final Path rootDir, final Configuration conf, final Path initialFiles)
throws IOException {
Path regionDir = HRegion.getRegionDir(rootDir, info.regionName);
static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
final Configuration conf, final Path initialFiles) throws IOException {
Path regionDir = HRegion.getRegionDir(rootDir, info.getEncodedName());
FileSystem fs = FileSystem.get(conf);
fs.mkdirs(regionDir);
return new HRegion(rootDir,
@ -1851,13 +1855,13 @@ public class HRegion implements HConstants {
*
* @param fs the file system object
* @param baseDirectory base directory for HBase
* @param regionName name of the region to delete
* @param name region file name
* @throws IOException
* @return True if deleted.
*/
static boolean deleteRegion(FileSystem fs, Path baseDirectory,
Text regionName) throws IOException {
Path p = HRegion.getRegionDir(fs.makeQualified(baseDirectory), regionName);
static boolean deleteRegion(FileSystem fs, Path baseDirectory, String name)
throws IOException {
Path p = HRegion.getRegionDir(fs.makeQualified(baseDirectory), name);
return fs.delete(p);
}
@ -1865,10 +1869,10 @@ public class HRegion implements HConstants {
* Computes the Path of the HRegion
*
* @param dir hbase home directory
* @param regionName name of the region
* @param name region file name
* @return Path of HRegion directory
*/
public static Path getRegionDir(final Path dir, final Text regionName) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName));
public static Path getRegionDir(final Path dir, final String name) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + name));
}
}

View File

@ -26,6 +26,8 @@ import java.io.IOException;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.hbase.util.Base64;
/**
* HRegion information.
* Contains HRegion id, start and end keys, a reference to this
@ -33,7 +35,15 @@ import org.apache.hadoop.io.WritableComparable;
*/
public class HRegionInfo implements WritableComparable {
/** delimiter used between portions of a region name */
public static final char DELIMITER = ',';
public static final String DELIMITER = ",";
/** HRegionInfo for root region */
public static final HRegionInfo rootRegionInfo =
new HRegionInfo(0L, HTableDescriptor.rootTableDesc);
/** HRegionInfo for first meta region */
public static final HRegionInfo firstMetaRegionInfo =
new HRegionInfo(1L, HTableDescriptor.metaTableDesc);
/**
* Extracts table name prefix from a region name.
@ -42,61 +52,92 @@ public class HRegionInfo implements WritableComparable {
* @return The table prefix of a region name.
*/
public static Text getTableNameFromRegionName(final Text regionName) {
int index = -1;
byte [] bytes = regionName.getBytes();
for (int i = 0; i < bytes.length; i++) {
if (((char) bytes[i]) == DELIMITER) {
index = i;
break;
}
}
if (index == -1) {
int offset = regionName.find(DELIMITER);
if (offset == -1) {
throw new IllegalArgumentException(regionName.toString() + " does not " +
"contain " + DELIMITER + " character");
"contain '" + DELIMITER + "' character");
}
byte [] tableName = new byte[index];
System.arraycopy(bytes, 0, tableName, 0, index);
byte [] tableName = new byte[offset];
System.arraycopy(regionName.getBytes(), 0, tableName, 0, offset);
return new Text(tableName);
}
Text regionName;
long regionId;
Text startKey;
Text endKey;
boolean offLine;
boolean split;
HTableDescriptor tableDesc;
/**
* Converts an encoded region name to its unencoded form
*
* @param encodedName
* @return unencoded region name
*/
public static Text decodeRegionName(String encodedName) {
int offset = encodedName.indexOf(DELIMITER);
if (offset == -1) {
throw new IllegalArgumentException(
"encoded region name does not contain '" + DELIMITER + "': " +
encodedName);
}
String regionName = encodedName.substring(0, offset++);
String remainder = encodedName.substring(offset);
offset = remainder.indexOf(DELIMITER);
if (offset == -1) {
throw new IllegalArgumentException(
"improperly formatted encoded region name " + encodedName);
}
Text startKey = new Text();
if (offset != 0) {
startKey.set(Base64.decode(remainder.substring(0, offset), Base64.ORDERED));
}
offset += 1;
return new Text(regionName + DELIMITER + startKey.toString() + DELIMITER +
remainder.substring(offset));
}
private Text endKey;
private boolean offLine;
private long regionId;
private Text regionName;
private boolean split;
private Text startKey;
private HTableDescriptor tableDesc;
/** Used to construct the HRegionInfo for the root and first meta regions */
private HRegionInfo(long regionId, HTableDescriptor tableDesc) {
this.regionId = regionId;
this.tableDesc = tableDesc;
this.endKey = new Text();
this.offLine = false;
this.regionName = new Text(tableDesc.getName().toString() + DELIMITER +
DELIMITER + regionId);
this.split = false;
this.startKey = new Text();
}
/** Default constructor - creates empty object */
public HRegionInfo() {
this.regionId = 0;
this.tableDesc = new HTableDescriptor();
this.startKey = new Text();
this.endKey = new Text();
this.regionName = new Text();
this.offLine = false;
this.regionId = 0;
this.regionName = new Text();
this.split = false;
this.startKey = new Text();
this.tableDesc = new HTableDescriptor();
}
/**
* Construct HRegionInfo with explicit parameters
*
* @param regionId the region id
* @param tableDesc the table descriptor
* @param startKey first key in region
* @param endKey end of key range
* @throws IllegalArgumentException
*/
public HRegionInfo(long regionId, HTableDescriptor tableDesc, Text startKey,
Text endKey)
throws IllegalArgumentException {
this(regionId, tableDesc, startKey, endKey, false);
public HRegionInfo(HTableDescriptor tableDesc, Text startKey, Text endKey)
throws IllegalArgumentException {
this(tableDesc, startKey, endKey, false);
}
/**
* Construct HRegionInfo with explicit parameters
*
* @param regionId the region id
* @param tableDesc the table descriptor
* @param startKey first key in region
* @param endKey end of key range
@ -104,34 +145,33 @@ public class HRegionInfo implements WritableComparable {
* regions that may or may not hold references to this region.
* @throws IllegalArgumentException
*/
public HRegionInfo(long regionId, HTableDescriptor tableDesc, Text startKey,
Text endKey, final boolean split)
throws IllegalArgumentException {
this.regionId = regionId;
public HRegionInfo(HTableDescriptor tableDesc, Text startKey, Text endKey,
final boolean split) throws IllegalArgumentException {
if(tableDesc == null) {
throw new IllegalArgumentException("tableDesc cannot be null");
}
this.tableDesc = tableDesc;
this.startKey = new Text();
if(startKey != null) {
this.startKey.set(startKey);
}
this.endKey = new Text();
if(endKey != null) {
this.endKey.set(endKey);
}
this.regionName = new Text(tableDesc.getName().toString() + DELIMITER +
(startKey == null ? "" : startKey.toString()) + DELIMITER +
regionId);
this.offLine = false;
this.regionId = System.currentTimeMillis();
this.regionName = new Text(tableDesc.getName().toString() + DELIMITER +
(startKey == null ? "" : startKey.toString()) + DELIMITER +
regionId);
this.split = split;
this.startKey = new Text();
if(startKey != null) {
this.startKey.set(startKey);
}
this.tableDesc = tableDesc;
}
/** @return the endKey */
@ -149,6 +189,16 @@ public class HRegionInfo implements WritableComparable {
return regionName;
}
/**
* @return the encodedName
*/
public String getEncodedName() {
return tableDesc.getName().toString() + DELIMITER +
(startKey == null || startKey.getLength() == 0 ? "" :
Base64.encodeBytes(startKey.getBytes(), Base64.ORDERED)) + DELIMITER +
regionId;
}
/** @return the startKey */
public Text getStartKey(){
return startKey;
@ -166,6 +216,13 @@ public class HRegionInfo implements WritableComparable {
return this.split;
}
/**
* @param split set split status
*/
public void setSplit(boolean split) {
this.split = split;
}
/**
* @return True if this region is offline.
*/
@ -173,6 +230,13 @@ public class HRegionInfo implements WritableComparable {
return this.offLine;
}
/**
* @param offLine set online - offline status
*/
public void setOffline(boolean offLine) {
this.offLine = offLine;
}
/**
* {@inheritDoc}
*/
@ -215,26 +279,26 @@ public class HRegionInfo implements WritableComparable {
* {@inheritDoc}
*/
public void write(DataOutput out) throws IOException {
out.writeLong(regionId);
tableDesc.write(out);
startKey.write(out);
endKey.write(out);
regionName.write(out);
out.writeBoolean(offLine);
out.writeLong(regionId);
regionName.write(out);
out.writeBoolean(split);
startKey.write(out);
tableDesc.write(out);
}
/**
* {@inheritDoc}
*/
public void readFields(DataInput in) throws IOException {
this.regionId = in.readLong();
this.tableDesc.readFields(in);
this.startKey.readFields(in);
this.endKey.readFields(in);
this.regionName.readFields(in);
this.offLine = in.readBoolean();
this.regionId = in.readLong();
this.regionName.readFields(in);
this.split = in.readBoolean();
this.startKey.readFields(in);
this.tableDesc.readFields(in);
}
//

View File

@ -122,6 +122,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
// is name of the webapp and the attribute name used stuffing this instance
// into web context.
InfoServer infoServer;
/** region server process name */
public static final String REGIONSERVER = "regionserver";
// Check to see if regions should be split
@ -220,7 +222,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
// splitting a 'normal' region, and the ROOT table needs to be
// updated if we are splitting a META region.
HTable t = null;
if (region.getRegionInfo().tableDesc.getName().equals(META_TABLE_NAME)) {
if (region.getRegionInfo().getTableDesc().getName().equals(META_TABLE_NAME)) {
// We need to update the root region
if (this.root == null) {
this.root = new HTable(conf, ROOT_TABLE_NAME);
@ -238,8 +240,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
// Mark old region as offline and split in META.
// NOTE: there is no need for retry logic here. HTable does it for us.
long lockid = t.startUpdate(oldRegionInfo.getRegionName());
oldRegionInfo.offLine = true;
oldRegionInfo.split = true;
oldRegionInfo.setOffline(true);
oldRegionInfo.setSplit(true);
t.put(lockid, COL_REGIONINFO, Writables.getBytes(oldRegionInfo));
t.put(lockid, COL_SPLITA, Writables.getBytes(
newRegions[0].getRegionInfo()));
@ -927,7 +929,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
}
void openRegion(final HRegionInfo regionInfo) throws IOException {
HRegion region = onlineRegions.get(regionInfo.regionName);
HRegion region = onlineRegions.get(regionInfo.getRegionName());
if(region == null) {
region = new HRegion(new Path(this.conf.get(HConstants.HBASE_DIR)),
this.log, FileSystem.get(conf), conf, regionInfo, null);
@ -947,7 +949,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
this.lock.writeLock().lock();
HRegion region = null;
try {
region = onlineRegions.remove(hri.regionName);
region = onlineRegions.remove(hri.getRegionName());
} finally {
this.lock.writeLock().unlock();
}
@ -1154,6 +1156,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
// remote scanner interface
//
/** {@inheritDoc} */
public long openScanner(Text regionName, Text[] cols, Text firstRow,
final long timestamp, final RowFilterInterface filter)
throws IOException {
@ -1284,6 +1287,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
return Collections.unmodifiableSortedMap(this.onlineRegions);
}
/** @return the request count */
public AtomicInteger getRequestCount() {
return this.requestCount;
}

View File

@ -1,232 +0,0 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.hbase.util.Writables;
/**
* A standalone HRegion directory reader. Currently reads content on
* file system only.
* TODO: Add dumping of HStoreFile content and HLog.
*/
class HRegiondirReader {
private final Configuration conf;
private final Path parentdir;
static final Pattern REGION_NAME_PARSER =
Pattern.compile(HConstants.HREGIONDIR_PREFIX +
"([^_]+)_([^_]*)_([^_]*)");
private static final String USAGE = "Usage: " +
"java org.apache.hadoop.hbase.HRegionDirReader <regiondir> " +
"[<tablename>]";
private final List<HRegionInfo> infos;
HRegiondirReader(final HBaseConfiguration conf,
final String parentdirName)
throws IOException {
this.conf = conf;
FileSystem fs = FileSystem.get(conf);
this.parentdir = new Path(parentdirName);
if (!fs.exists(parentdir)) {
throw new FileNotFoundException(parentdirName);
}
if (!fs.getFileStatus(parentdir).isDir()) {
throw new IOException(parentdirName + " not a directory");
}
// Look for regions in parentdir.
Path [] regiondirs =
fs.listPaths(parentdir, new PathFilter() {
/* (non-Javadoc)
* @see org.apache.hadoop.fs.PathFilter#accept(org.apache.hadoop.fs.Path)
*/
public boolean accept(Path path) {
Matcher m = REGION_NAME_PARSER.matcher(path.getName());
return m != null && m.matches();
}
});
// Create list of HRegionInfos for all regions found in
// parentdir.
this.infos = new ArrayList<HRegionInfo>();
for (Path d: regiondirs) {
Matcher m = REGION_NAME_PARSER.matcher(d.getName());
if (m == null || !m.matches()) {
throw new IOException("Unparseable region dir name");
}
String tableName = m.group(1);
String endKey = m.group(2);
long regionid = Long.parseLong(m.group(3));
HTableDescriptor desc = getTableDescriptor(fs, d, tableName);
HRegionInfo info = new HRegionInfo(regionid, desc,
new Text(), (endKey == null || endKey.length() == 0)?
new Text(): new Text(endKey));
infos.add(info);
}
}
/**
* Returns a populated table descriptor.
* @param fs Current filesystem.
* @param d The regiondir for <code>tableName</code>
* @param tableName Name of this table.
* @return A HTableDescriptor populated with all known column
* families.
* @throws IOException
*/
private HTableDescriptor getTableDescriptor(final FileSystem fs,
final Path d, final String tableName)
throws IOException {
HTableDescriptor desc = new HTableDescriptor(tableName);
Text [] families = getFamilies(fs, d);
for (Text f: families) {
desc.addFamily(new HColumnDescriptor(f.toString()));
}
return desc;
}
/**
* Get column families for this region by looking at
* directory names under this region.
* This is a hack. HRegions only know what columns they have
* because they are told by passed-in metadata.
* @param regiondir
* @return Array of family names.
* @throws IOException
*/
private Text [] getFamilies(final FileSystem fs,
final Path regiondir)
throws IOException {
Path [] subdirs = fs.listPaths(regiondir, new PathFilter() {
public boolean accept(Path path) {
return !path.getName().equals("log");
}
});
List<Text> families = new ArrayList<Text>();
for (Path d: subdirs) {
// Convert names of subdirectories into column family names
// by adding the colon.
Text family = new Text(d.getName() + ":");
families.add(family);
}
return families.toArray(new Text [families.size()]);
}
List <HRegionInfo> getRegions() {
return this.infos;
}
HRegionInfo getRegionInfo(final Text tableName) {
HRegionInfo result = null;
for(HRegionInfo i: getRegions()) {
if(i.tableDesc.getName().equals(tableName)) {
result = i;
break;
}
}
if (result == null) {
throw new NullPointerException("No such table: " +
tableName);
}
return result;
}
private void dump(final Text tableName) throws IOException {
dump(getRegionInfo(tableName));
}
private void dump(final HRegionInfo info) throws IOException {
HRegion r = new HRegion(this.parentdir, null,
FileSystem.get(this.conf), conf, info, null);
Text [] families = info.tableDesc.families().keySet().toArray(
new Text [info.tableDesc.getFamilies().size()]);
HInternalScannerInterface scanner =
r.getScanner(families, new Text(), System.currentTimeMillis(), null);
HStoreKey key = new HStoreKey();
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;
s = info.endKey.toString();
String endKey = (s == null || s.length() <= 0)? "<>": s;
String tableName = info.tableDesc.getName().toString();
System.out.println("table: " + tableName +
", regionid: " + info.regionId +
", startkey: " + startKey +
", endkey: " + endKey);
// Now print rows. Offset by a space to distingush rows from
// table headers. TODO: Add in better formatting of output.
// Every line starts with row name followed by column name
// followed by cell content.
while(scanner.next(key, results)) {
for (Map.Entry<Text, byte []> es: results.entrySet()) {
Text colname = es.getKey();
byte [] colvalue = es.getValue();
Object value = null;
if (colname.toString().equals("info:regioninfo")) {
value = Writables.getWritable(colvalue, new HRegionInfo());
} else {
value = new String(colvalue, HConstants.UTF8_ENCODING);
}
System.out.println(" " + key + ", " + colname.toString() + ": \"" +
value.toString() + "\"");
}
}
}
/**
* @param args
* @throws IOException
*/
public static void main(String[] args) throws IOException {
if (args.length < 1) {
System.err.println(USAGE);
System.exit(-1);
}
HBaseConfiguration c = new HBaseConfiguration();
HRegiondirReader reader = new HRegiondirReader(c, args[0]);
if (args.length == 1) {
// Do all regions.
for(HRegionInfo info: reader.getRegions()) {
reader.dump(info);
}
} else {
for (int i = 1; i < args.length; i++) {
reader.dump(new Text(args[i]));
}
}
}
}

View File

@ -72,7 +72,7 @@ class HStore implements HConstants {
private static final String BLOOMFILTER_FILE_NAME = "filter";
Path dir;
Text regionName;
String encodedRegionName;
HColumnDescriptor family;
Text familyName;
SequenceFile.CompressionType compression;
@ -131,24 +131,23 @@ class HStore implements HConstants {
* file will be deleted (by whoever has instantiated the HStore).
*
* @param dir log file directory
* @param regionName name of region
* @param encodedRegionName filename friendly name of region
* @param family name of column family
* @param fs file system object
* @param reconstructionLog existing log file to apply if any
* @param conf configuration object
* @throws IOException
*/
HStore(Path dir, Text regionName, HColumnDescriptor family,
HStore(Path dir, String encodedRegionName, HColumnDescriptor family,
FileSystem fs, Path reconstructionLog, Configuration conf)
throws IOException {
this.dir = dir;
this.compactionDir = new Path(dir, "compaction.dir");
this.regionName = regionName;
this.encodedRegionName = encodedRegionName;
this.family = family;
this.familyName = HStoreKey.extractFamily(this.family.getName());
this.compression = SequenceFile.CompressionType.NONE;
this.storeName = this.regionName.toString() + "/" +
this.familyName.toString();
this.storeName = this.encodedRegionName + "/" + this.familyName.toString();
if(family.getCompression() != HColumnDescriptor.CompressionType.NONE) {
if(family.getCompression() == HColumnDescriptor.CompressionType.BLOCK) {
@ -163,15 +162,16 @@ class HStore implements HConstants {
this.fs = fs;
this.conf = conf;
this.mapdir = HStoreFile.getMapDir(dir, regionName, familyName);
this.mapdir = HStoreFile.getMapDir(dir, encodedRegionName, familyName);
fs.mkdirs(mapdir);
this.loginfodir = HStoreFile.getInfoDir(dir, regionName, familyName);
this.loginfodir = HStoreFile.getInfoDir(dir, encodedRegionName, familyName);
fs.mkdirs(loginfodir);
if(family.getBloomFilter() == null) {
this.filterDir = null;
this.bloomFilter = null;
} else {
this.filterDir = HStoreFile.getFilterDir(dir, regionName, familyName);
this.filterDir =
HStoreFile.getFilterDir(dir, encodedRegionName, familyName);
fs.mkdirs(filterDir);
loadOrCreateBloomFilter();
}
@ -187,8 +187,8 @@ class HStore implements HConstants {
// MapFiles are in a reliable state. Every entry in 'mapdir' must have a
// corresponding one in 'loginfodir'. Without a corresponding log info
// file, the entry in 'mapdir' must be deleted.
Collection<HStoreFile> hstoreFiles
= HStoreFile.loadHStoreFiles(conf, dir, regionName, familyName, fs);
Collection<HStoreFile> hstoreFiles = HStoreFile.loadHStoreFiles(conf, dir,
encodedRegionName, familyName, fs);
for(HStoreFile hsf: hstoreFiles) {
this.storefiles.put(Long.valueOf(hsf.loadInfo(fs)), hsf);
}
@ -265,6 +265,7 @@ class HStore implements HConstants {
SequenceFile.Reader login =
new SequenceFile.Reader(this.fs, reconstructionLog, this.conf);
try {
Text thisRegionName = HRegionInfo.decodeRegionName(encodedRegionName);
HLogKey key = new HLogKey();
HLogEdit val = new HLogEdit();
while (login.next(key, val)) {
@ -281,13 +282,13 @@ class HStore implements HConstants {
// METACOLUMN info such as HBASE::CACHEFLUSH entries
Text column = val.getColumn();
if (column.equals(HLog.METACOLUMN)
|| !key.getRegionName().equals(this.regionName)
|| !key.getRegionName().equals(thisRegionName)
|| !HStoreKey.extractFamily(column).equals(this.familyName)) {
if (LOG.isDebugEnabled()) {
LOG.debug("Passing on edit " + key.getRegionName() + ", " +
column.toString() + ": " +
new String(val.getVal(), UTF8_ENCODING) +
", my region: " + this.regionName + ", my column: " +
", my region: " + thisRegionName + ", my column: " +
this.familyName);
}
continue;
@ -452,7 +453,7 @@ class HStore implements HConstants {
synchronized(flushLock) {
// A. Write the TreeMap out to the disk
HStoreFile flushedFile = HStoreFile.obtainNewHStoreFile(conf, dir,
regionName, familyName, fs);
encodedRegionName, familyName, fs);
String name = flushedFile.toString();
MapFile.Writer out = flushedFile.getWriter(this.fs, this.compression,
this.bloomFilter);
@ -575,8 +576,8 @@ class HStore implements HConstants {
throws IOException {
long maxId = maxSeenSeqID;
synchronized(compactLock) {
Path curCompactStore =
HStoreFile.getHStoreDir(this.compactionDir, regionName, familyName);
Path curCompactStore = HStoreFile.getHStoreDir(this.compactionDir,
encodedRegionName, familyName);
if(LOG.isDebugEnabled()) {
LOG.debug("started compaction of " + storefiles.size() + " files in " +
curCompactStore.toString());
@ -590,8 +591,8 @@ class HStore implements HConstants {
}
try {
List<HStoreFile> toCompactFiles = getFilesToCompact();
HStoreFile compactedOutputFile =
new HStoreFile(conf, this.compactionDir, regionName, familyName, -1);
HStoreFile compactedOutputFile = new HStoreFile(conf,
this.compactionDir, encodedRegionName, familyName, -1);
if (toCompactFiles.size() < 1 ||
(toCompactFiles.size() == 1 &&
!toCompactFiles.get(0).isReference())) {
@ -943,7 +944,7 @@ class HStore implements HConstants {
// If a null value, shouldn't be in here. Mark it as deleted cell.
return true;
}
if (!HGlobals.deleteBytes.equals(value)) {
if (!HLogEdit.isDeleted(value)) {
return false;
}
// Cell has delete value. Save it into deletes.
@ -981,8 +982,8 @@ class HStore implements HConstants {
*/
void processReadyCompaction() throws IOException {
// 1. Acquiring the write-lock
Path curCompactStore =
HStoreFile.getHStoreDir(this.compactionDir, regionName, familyName);
Path curCompactStore = HStoreFile.getHStoreDir(this.compactionDir,
encodedRegionName, familyName);
this.lock.obtainWriteLock();
try {
Path doneFile = new Path(curCompactStore, COMPACTION_DONE);
@ -1010,12 +1011,12 @@ class HStore implements HConstants {
}
// 3. Moving the new MapFile into place.
HStoreFile compactedFile
= new HStoreFile(conf, this.compactionDir, regionName, familyName, -1);
HStoreFile compactedFile = new HStoreFile(conf, this.compactionDir,
encodedRegionName, familyName, -1);
// obtainNewHStoreFile does its best to generate a filename that does not
// currently exist.
HStoreFile finalCompactedFile
= HStoreFile.obtainNewHStoreFile(conf, dir, regionName, familyName, fs);
HStoreFile finalCompactedFile = HStoreFile.obtainNewHStoreFile(conf, dir,
encodedRegionName, familyName, fs);
if(LOG.isDebugEnabled()) {
LOG.debug("moving " + compactedFile.toString() + " in " +
this.compactionDir.toString() +
@ -1093,7 +1094,7 @@ class HStore implements HConstants {
Text readcol = readkey.getColumn();
if (results.get(readcol) == null
&& key.matchesWithoutColumn(readkey)) {
if(readval.equals(HGlobals.deleteBytes)) {
if(HLogEdit.isDeleted(readval.get())) {
break;
}
results.put(new Text(readcol), readval.get());

View File

@ -118,7 +118,7 @@ public class HStoreFile implements HConstants, WritableComparable {
private static Random rand = new Random();
private Path dir;
private Text regionName;
private String encodedRegionName;
private Text colFamily;
private long fileId;
private final Configuration conf;
@ -126,7 +126,7 @@ public class HStoreFile implements HConstants, WritableComparable {
/** Shutdown constructor used by Writable */
HStoreFile(Configuration conf) {
this(conf, new Path(Path.CUR_DIR), new Text(), new Text(), 0);
this(conf, new Path(Path.CUR_DIR), "", new Text(), 0);
}
/**
@ -137,25 +137,25 @@ public class HStoreFile implements HConstants, WritableComparable {
* @param colFamily name of the column family
* @param fileId file identifier
*/
HStoreFile(final Configuration conf, final Path dir, final Text regionName,
final Text colFamily, final long fileId) {
this(conf, dir, regionName, colFamily, fileId, null);
HStoreFile(final Configuration conf, final Path dir,
final String encodedRegionName, final Text colFamily, final long fileId) {
this(conf, dir, encodedRegionName, colFamily, fileId, null);
}
/**
* Constructor that fully initializes the object
* @param conf Configuration object
* @param dir directory path
* @param regionName name of the region
* @param encodedRegionName file name friendly name of the region
* @param colFamily name of the column family
* @param fileId file identifier
* @param ref Reference to another HStoreFile.
*/
HStoreFile(Configuration conf, Path dir, Text regionName,
HStoreFile(Configuration conf, Path dir, String encodedRegionName,
Text colFamily, long fileId, final Reference ref) {
this.conf = conf;
this.dir = dir;
this.regionName = new Text(regionName);
this.encodedRegionName = encodedRegionName;
this.colFamily = new Text(colFamily);
this.fileId = fileId;
// If a reference, construction does not write the pointer files. Thats
@ -168,14 +168,14 @@ public class HStoreFile implements HConstants, WritableComparable {
* Data structure to hold reference to a store file over in another region.
*/
static class Reference implements Writable {
Text regionName;
long fileid;
Range region;
HStoreKey midkey;
private String encodedRegionName;
private long fileid;
private Range region;
private HStoreKey midkey;
Reference(final Text rn, final long fid, final HStoreKey m,
Reference(final String ern, final long fid, final HStoreKey m,
final Range fr) {
this.regionName = rn;
this.encodedRegionName = ern;
this.fileid = fid;
this.region = fr;
this.midkey = m;
@ -197,21 +197,21 @@ public class HStoreFile implements HConstants, WritableComparable {
return this.midkey;
}
Text getRegionName() {
return this.regionName;
String getEncodedRegionName() {
return this.encodedRegionName;
}
/** {@inheritDoc} */
@Override
public String toString() {
return this.regionName + "/" + this.fileid + "/" + this.region;
return this.encodedRegionName + "/" + this.fileid + "/" + this.region;
}
// Make it serializable.
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
this.regionName.write(out);
out.writeUTF(this.encodedRegionName);
out.writeLong(this.fileid);
// Write true if we're doing top of the file.
out.writeBoolean(isTopFileRegion(this.region));
@ -220,8 +220,7 @@ public class HStoreFile implements HConstants, WritableComparable {
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
this.regionName = new Text();
this.regionName.readFields(in);
this.encodedRegionName = in.readUTF();
this.fileid = in.readLong();
boolean tmp = in.readBoolean();
// If true, set region to top.
@ -244,8 +243,8 @@ public class HStoreFile implements HConstants, WritableComparable {
return this.reference;
}
Text getRegionName() {
return this.regionName;
String getEncodedRegionName() {
return this.encodedRegionName;
}
/** @return the column family */
@ -262,43 +261,45 @@ public class HStoreFile implements HConstants, WritableComparable {
/** @return path for MapFile */
Path getMapFilePath() {
return isReference()?
getMapFilePath(this.regionName, this.fileId,
this.reference.getRegionName()):
getMapFilePath(this.regionName, this.fileId);
getMapFilePath(this.encodedRegionName, this.fileId,
this.reference.getEncodedRegionName()):
getMapFilePath(this.encodedRegionName, this.fileId);
}
private Path getMapFilePath(final Reference r) {
return r == null?
getMapFilePath():
getMapFilePath(r.getRegionName(), r.getFileId());
getMapFilePath(r.getEncodedRegionName(), r.getFileId());
}
private Path getMapFilePath(final Text name, final long fid) {
return new Path(HStoreFile.getMapDir(dir, name, colFamily),
private Path getMapFilePath(final String encodedName, final long fid) {
return new Path(HStoreFile.getMapDir(dir, encodedName, colFamily),
createHStoreFilename(fid, null));
}
private Path getMapFilePath(final Text name, final long fid, final Text rn) {
return new Path(HStoreFile.getMapDir(dir, name, colFamily),
createHStoreFilename(fid, rn));
private Path getMapFilePath(final String encodedName, final long fid,
final String ern) {
return new Path(HStoreFile.getMapDir(dir, encodedName, colFamily),
createHStoreFilename(fid, ern));
}
/** @return path for info file */
Path getInfoFilePath() {
return isReference()?
getInfoFilePath(this.regionName, this.fileId,
this.reference.getRegionName()):
getInfoFilePath(this.regionName, this.fileId);
getInfoFilePath(this.encodedRegionName, this.fileId,
this.reference.getEncodedRegionName()):
getInfoFilePath(this.encodedRegionName, this.fileId);
}
private Path getInfoFilePath(final Text name, final long fid) {
return new Path(HStoreFile.getInfoDir(dir, name, colFamily),
private Path getInfoFilePath(final String encodedName, final long fid) {
return new Path(HStoreFile.getInfoDir(dir, encodedName, colFamily),
createHStoreFilename(fid, null));
}
private Path getInfoFilePath(final Text name, final long fid, final Text rn) {
return new Path(HStoreFile.getInfoDir(dir, name, colFamily),
createHStoreFilename(fid, rn));
private Path getInfoFilePath(final String encodedName, final long fid,
final String ern) {
return new Path(HStoreFile.getInfoDir(dir, encodedName, colFamily),
createHStoreFilename(fid, ern));
}
// Static methods to build partial paths to internal directories. Useful for
@ -308,35 +309,35 @@ public class HStoreFile implements HConstants, WritableComparable {
}
private static String createHStoreFilename(final long fid,
final Text regionName) {
final String encodedRegionName) {
return Long.toString(fid) +
((regionName != null)? "." + regionName.toString(): "");
((encodedRegionName != null) ? "." + encodedRegionName : "");
}
private static String createHStoreInfoFilename(final long fid) {
return createHStoreFilename(fid, null);
}
static Path getMapDir(Path dir, Text regionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
static Path getMapDir(Path dir, String encodedRegionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + encodedRegionName,
new Path(colFamily.toString(), HSTORE_DATFILE_DIR)));
}
/** @return the info directory path */
static Path getInfoDir(Path dir, Text regionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
static Path getInfoDir(Path dir, String encodedRegionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + encodedRegionName,
new Path(colFamily.toString(), HSTORE_INFO_DIR)));
}
/** @return the bloom filter directory path */
static Path getFilterDir(Path dir, Text regionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
static Path getFilterDir(Path dir, String encodedRegionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + encodedRegionName,
new Path(colFamily.toString(), HSTORE_FILTER_DIR)));
}
/** @return the HStore directory path */
static Path getHStoreDir(Path dir, Text regionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
static Path getHStoreDir(Path dir, String encodedRegionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + encodedRegionName,
colFamily.toString()));
}
@ -347,9 +348,10 @@ public class HStoreFile implements HConstants, WritableComparable {
* will keep generating names until it generates a name that does not exist.
*/
static HStoreFile obtainNewHStoreFile(Configuration conf, Path dir,
Text regionName, Text colFamily, FileSystem fs) throws IOException {
String encodedRegionName, Text colFamily, FileSystem fs)
throws IOException {
Path mapdir = HStoreFile.getMapDir(dir, regionName, colFamily);
Path mapdir = HStoreFile.getMapDir(dir, encodedRegionName, colFamily);
long fileId = Math.abs(rand.nextLong());
Path testpath1 = new Path(mapdir, createHStoreFilename(fileId));
@ -359,7 +361,7 @@ public class HStoreFile implements HConstants, WritableComparable {
testpath1 = new Path(mapdir, createHStoreFilename(fileId));
testpath2 = new Path(mapdir, createHStoreInfoFilename(fileId));
}
return new HStoreFile(conf, dir, regionName, colFamily, fileId);
return new HStoreFile(conf, dir, encodedRegionName, colFamily, fileId);
}
/*
@ -376,11 +378,11 @@ public class HStoreFile implements HConstants, WritableComparable {
* @throws IOException
*/
static Vector<HStoreFile> loadHStoreFiles(Configuration conf, Path dir,
Text regionName, Text colFamily, FileSystem fs)
String encodedRegionName, Text colFamily, FileSystem fs)
throws IOException {
// Look first at info files. If a reference, these contain info we need
// to create the HStoreFile.
Path infodir = HStoreFile.getInfoDir(dir, regionName, colFamily);
Path infodir = HStoreFile.getInfoDir(dir, encodedRegionName, colFamily);
Path infofiles[] = fs.listPaths(new Path[] {infodir});
Vector<HStoreFile> results = new Vector<HStoreFile>(infofiles.length);
Vector<Path> mapfiles = new Vector<Path>(infofiles.length);
@ -392,10 +394,10 @@ public class HStoreFile implements HConstants, WritableComparable {
HStoreFile curfile = null;
if (isReference) {
Reference reference = readSplitInfo(infofiles[i], fs);
curfile = new HStoreFile(conf, dir, regionName, colFamily, fid,
curfile = new HStoreFile(conf, dir, encodedRegionName, colFamily, fid,
reference);
} else {
curfile = new HStoreFile(conf, dir, regionName, colFamily, fid);
curfile = new HStoreFile(conf, dir, encodedRegionName, colFamily, fid);
}
Path mapfile = curfile.getMapFilePath();
if (!fs.exists(mapfile)) {
@ -415,7 +417,7 @@ public class HStoreFile implements HConstants, WritableComparable {
mapfiles.add(qualified);
}
Path mapdir = HStoreFile.getMapDir(dir, regionName, colFamily);
Path mapdir = HStoreFile.getMapDir(dir, encodedRegionName, colFamily);
// List paths by experience returns fully qualified names -- at least when
// running on a mini hdfs cluster.
Path datfiles[] = fs.listPaths(new Path[] {mapdir});
@ -486,14 +488,13 @@ public class HStoreFile implements HConstants, WritableComparable {
* @param fs
* @throws IOException
*/
private void writeSplitInfo(final FileSystem fs)
throws IOException {
private void writeSplitInfo(final FileSystem fs) throws IOException {
Path p = getInfoFilePath();
if (fs.exists(p)) {
throw new IOException("File already exists " + p.toString());
}
FSDataOutputStream out = fs.create(p);
getReference().getRegionName().write(out);
out.writeUTF(getReference().getEncodedRegionName());
getReference().getMidkey().write(out);
out.writeLong(getReference().getFileId());
out.writeBoolean(isTopFileRegion(getReference().getFileRegion()));
@ -506,8 +507,7 @@ public class HStoreFile implements HConstants, WritableComparable {
static Reference readSplitInfo(final Path p, final FileSystem fs)
throws IOException {
FSDataInputStream in = fs.open(p);
Text rn = new Text();
rn.readFields(in);
String rn = in.readUTF();
HStoreKey midkey = new HStoreKey();
midkey.readFields(in);
long fid = in.readLong();
@ -580,10 +580,10 @@ public class HStoreFile implements HConstants, WritableComparable {
* @throws IOException
*/
long loadInfo(FileSystem fs) throws IOException {
Path p = isReference()?
getInfoFilePath(this.reference.getRegionName(),
this.reference.getFileId()):
getInfoFilePath();
Path p = isReference() ?
getInfoFilePath(this.reference.getEncodedRegionName(),
this.reference.getFileId()) :
getInfoFilePath();
DataInputStream in = new DataInputStream(fs.open(p));
try {
byte flag = in.readByte();
@ -930,7 +930,7 @@ public class HStoreFile implements HConstants, WritableComparable {
throws IOException {
return isReference()?
new HStoreFile.HalfMapFileReader(fs,
getMapFilePath(getReference().getRegionName(),
getMapFilePath(getReference().getEncodedRegionName(),
getReference().getFileId()).toString(),
this.conf, getReference().getFileRegion(), getReference().getMidkey(),
bloomFilter):
@ -975,7 +975,7 @@ public class HStoreFile implements HConstants, WritableComparable {
/** {@inheritDoc} */
@Override
public String toString() {
return this.regionName.toString() + "/" + this.colFamily.toString() +
return this.encodedRegionName.toString() + "/" + this.colFamily.toString() +
"/" + this.fileId +
(isReference()? "/" + this.reference.toString(): "");
}
@ -990,7 +990,7 @@ public class HStoreFile implements HConstants, WritableComparable {
@Override
public int hashCode() {
int result = this.dir.hashCode();
result ^= this.regionName.hashCode();
result ^= this.encodedRegionName.hashCode();
result ^= this.colFamily.hashCode();
result ^= Long.valueOf(this.fileId).hashCode();
return result;
@ -1001,7 +1001,7 @@ public class HStoreFile implements HConstants, WritableComparable {
/** {@inheritDoc} */
public void write(DataOutput out) throws IOException {
out.writeUTF(dir.toString());
this.regionName.write(out);
out.writeUTF(this.encodedRegionName);
this.colFamily.write(out);
out.writeLong(fileId);
out.writeBoolean(isReference());
@ -1013,7 +1013,7 @@ public class HStoreFile implements HConstants, WritableComparable {
/** {@inheritDoc} */
public void readFields(DataInput in) throws IOException {
this.dir = new Path(in.readUTF());
this.regionName.readFields(in);
this.encodedRegionName = in.readUTF();
this.colFamily.readFields(in);
this.fileId = in.readLong();
this.reference = null;
@ -1031,7 +1031,7 @@ public class HStoreFile implements HConstants, WritableComparable {
HStoreFile other = (HStoreFile) o;
int result = this.dir.compareTo(other.dir);
if(result == 0) {
this.regionName.compareTo(other.regionName);
this.encodedRegionName.compareTo(other.encodedRegionName);
}
if(result == 0) {
result = this.colFamily.compareTo(other.colFamily);

View File

@ -38,6 +38,21 @@ import org.apache.hadoop.io.WritableComparable;
* column families.
*/
public class HTableDescriptor implements WritableComparable {
/** table descriptor for root table */
public static final HTableDescriptor rootTableDesc =
new HTableDescriptor(HConstants.ROOT_TABLE_NAME,
new HColumnDescriptor(HConstants.COLUMN_FAMILY, 1,
HColumnDescriptor.CompressionType.NONE, false, Integer.MAX_VALUE,
null));
/** table descriptor for meta table */
public static final HTableDescriptor metaTableDesc =
new HTableDescriptor(HConstants.META_TABLE_NAME,
new HColumnDescriptor(HConstants.COLUMN_FAMILY, 1,
HColumnDescriptor.CompressionType.NONE, false, Integer.MAX_VALUE,
null));
private Text name;
// TODO: Does this need to be a treemap? Can it be a HashMap?
private final TreeMap<Text, HColumnDescriptor> families;
@ -52,6 +67,13 @@ public class HTableDescriptor implements WritableComparable {
private static final Pattern LEGAL_TABLE_NAME =
Pattern.compile("^[\\w-.]+$");
/** Used to construct the table descriptors for root and meta tables */
private HTableDescriptor(Text name, HColumnDescriptor family) {
this.name = new Text(name);
this.families = new TreeMap<Text, HColumnDescriptor>();
families.put(family.getName(), family);
}
/** Constructs an empty object */
public HTableDescriptor() {
this.name = new Text();

View File

@ -33,9 +33,10 @@ import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HGlobals;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.hbase.HLogEdit;
/**
* Implementation of RowFilterInterface that can filter by rowkey regular
* expression and/or individual column values (equals comparison only).
@ -176,7 +177,7 @@ public class RegExpRowFilter implements RowFilterInterface {
}
}
if (nullColumns.contains(colKey)) {
if (data != null && !Arrays.equals(HGlobals.deleteBytes.get(), data)) {
if (data != null && !HLogEdit.isDeleted(data)) {
if (LOG.isDebugEnabled()) {
LOG.debug("filter returning true for rowKey: " + rowKey +
" colKey: " + colKey);
@ -198,7 +199,7 @@ public class RegExpRowFilter implements RowFilterInterface {
public boolean filterNotNull(final TreeMap<Text, byte[]> columns) {
for (Entry<Text, byte[]> col : columns.entrySet()) {
if (nullColumns.contains(col.getKey())
&& !Arrays.equals(HGlobals.deleteBytes.get(), col.getValue())) {
&& !HLogEdit.isDeleted(col.getValue())) {
if (LOG.isDebugEnabled()) {
LOG.debug("filterNotNull returning true for colKey: " + col.getKey()
+ ", column should be null.");

View File

@ -88,12 +88,8 @@ public abstract class AbstractMergeTestBase extends HBaseTestCase {
// Now create the root and meta regions and insert the data regions
// created above into the meta
HRegion root =
createNewHRegion(dir, conf, HGlobals.rootTableDesc, 0L, null, null);
HRegion meta =
createNewHRegion(dir, conf, HGlobals.metaTableDesc, 1L, null, null);
HRegion root = createNewHRegion(dir, conf, HRegionInfo.rootRegionInfo);
HRegion meta = createNewHRegion(dir, conf, HRegionInfo.firstMetaRegionInfo);
HRegion.addRegionToMETA(root, meta);
for(int i = 0; i < regions.length; i++) {
@ -134,8 +130,7 @@ public abstract class AbstractMergeTestBase extends HBaseTestCase {
private HRegion createAregion(Text startKey, Text endKey, int firstRow,
int nrows) throws IOException {
HRegion region =
createNewHRegion(dir, conf, desc, rand.nextLong(), startKey, endKey);
HRegion region = createNewHRegion(dir, conf, desc, startKey, endKey);
System.out.println("created region " + region.getRegionName());
@ -155,7 +150,7 @@ public abstract class AbstractMergeTestBase extends HBaseTestCase {
region.compactStores();
region.close();
region.getLog().closeAndDelete();
region.getRegionInfo().offLine = true;
region.getRegionInfo().setOffline(true);
return region;
}
}

View File

@ -20,6 +20,7 @@
package org.apache.hadoop.hbase;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import junit.framework.TestCase;
@ -40,8 +41,10 @@ public abstract class HBaseTestCase extends TestCase {
protected FileSystem localFs = null;
protected static final char FIRST_CHAR = 'a';
protected static final char LAST_CHAR = 'z';
protected static final String PUNCTUATION = "~`@#$%^&*()-_+=:;',.<>/?[]{}|";
protected static final byte [] START_KEY_BYTES =
{FIRST_CHAR, FIRST_CHAR, FIRST_CHAR};
protected String START_KEY;
protected static final int MAXVERSIONS = 3;
static {
@ -50,10 +53,18 @@ public abstract class HBaseTestCase extends TestCase {
protected volatile Configuration conf;
/** constructor */
/**
* constructor
*/
public HBaseTestCase() {
super();
conf = new HBaseConfiguration();
try {
START_KEY =
new String(START_KEY_BYTES, HConstants.UTF8_ENCODING) + PUNCTUATION;
} catch (UnsupportedEncodingException e) {
fail();
}
}
/**
@ -62,6 +73,12 @@ public abstract class HBaseTestCase extends TestCase {
public HBaseTestCase(String name) {
super(name);
conf = new HBaseConfiguration();
try {
START_KEY =
new String(START_KEY_BYTES, HConstants.UTF8_ENCODING) + PUNCTUATION;
} catch (UnsupportedEncodingException e) {
fail();
}
}
/** {@inheritDoc} */
@ -90,10 +107,13 @@ public abstract class HBaseTestCase extends TestCase {
}
protected HRegion createNewHRegion(Path dir, Configuration c,
HTableDescriptor desc, long regionId, Text startKey, Text endKey)
throws IOException {
HRegionInfo info = new HRegionInfo(regionId, desc, startKey, endKey);
Path regionDir = HRegion.getRegionDir(dir, info.regionName);
HTableDescriptor desc, Text startKey, Text endKey) throws IOException {
return createNewHRegion(dir, c, new HRegionInfo(desc, startKey, endKey));
}
protected HRegion createNewHRegion(Path dir, Configuration c,
HRegionInfo info) throws IOException {
Path regionDir = HRegion.getRegionDir(dir, info.getEncodedName());
FileSystem fs = dir.getFileSystem(c);
fs.mkdirs(regionDir);
return new HRegion(dir,
@ -189,7 +209,9 @@ public abstract class HBaseTestCase extends TestCase {
for (char d = secondCharStart; d <= LAST_CHAR; d++) {
for (char e = thirdCharStart; e <= LAST_CHAR; e++) {
byte [] bytes = new byte [] {(byte)c, (byte)d, (byte)e};
Text t = new Text(new String(bytes, HConstants.UTF8_ENCODING));
String s = new String(bytes, HConstants.UTF8_ENCODING) + PUNCTUATION;
bytes = s.getBytes(HConstants.UTF8_ENCODING);
Text t = new Text(s);
if (endKey != null && endKey.getLength() > 0
&& endKey.compareTo(t) <= 0) {
break EXIT;

View File

@ -130,7 +130,7 @@ public class MultiRegionTable extends HBaseTestCase {
Writables.getHRegionInfoOrNull(data.get(HConstants.COL_SPLITA));
HRegionInfo splitB =
Writables.getHRegionInfoOrNull(data.get(HConstants.COL_SPLITB));
Path parentDir = HRegion.getRegionDir(d, parent.getRegionName());
Path parentDir = HRegion.getRegionDir(d, parent.getEncodedName());
assertTrue(fs.exists(parentDir));
LOG.info("Split happened. Parent is " + parent.getRegionName() +
" and daughters are " + splitA.getRegionName() + ", " +

View File

@ -35,21 +35,28 @@ public class TestCompaction extends HBaseTestCase {
private HLog hlog = null;
private HRegion r = null;
private static final String COLUMN_FAMILY = COLFAMILY_NAME1;
private static final Text STARTROW = new Text(START_KEY_BYTES);
private final Text STARTROW;
private static final Text COLUMN_FAMILY_TEXT = new Text(COLUMN_FAMILY);
private static final Text COLUMN_FAMILY_TEXT_MINUS_COLON =
new Text(COLUMN_FAMILY.substring(0, COLUMN_FAMILY.length() - 1));
private static final int COMPACTION_THRESHOLD = MAXVERSIONS;
/** constructor */
public TestCompaction() {
STARTROW = new Text(START_KEY);
}
/** {@inheritDoc} */
@Override
public void setUp() throws Exception {
super.setUp();
this.hlog = new HLog(this.localFs, this.testDir, this.conf);
HTableDescriptor htd = createTableDescriptor(getName());
HRegionInfo hri = new HRegionInfo(1, htd, null, null);
HRegionInfo hri = new HRegionInfo(htd, null, null);
this.r = new HRegion(testDir, hlog, this.localFs, this.conf, hri, null);
}
/** {@inheritDoc} */
@Override
public void tearDown() throws Exception {
this.r.close();
@ -111,9 +118,7 @@ public class TestCompaction extends HBaseTestCase {
// we added when we flushed. But could be 3 only if the flush happened
// before the compaction started though we tried to have the threads run
// concurrently (On hudson this happens).
byte [] secondRowBytes = new byte[START_KEY_BYTES.length];
System.arraycopy(START_KEY_BYTES, 0, secondRowBytes, 0,
START_KEY_BYTES.length);
byte [] secondRowBytes = START_KEY.getBytes(HConstants.UTF8_ENCODING);
// Increment the least significant character so we get to next row.
secondRowBytes[START_KEY_BYTES.length - 1]++;
Text secondRow = new Text(secondRowBytes);

View File

@ -48,25 +48,25 @@ public class TestCompare extends TestCase {
* Sort of HRegionInfo.
*/
public void testHRegionInfo() {
HRegionInfo a = new HRegionInfo(1, new HTableDescriptor("a"), null, null);
HRegionInfo b = new HRegionInfo(2, new HTableDescriptor("b"), null, null);
HRegionInfo a = new HRegionInfo(new HTableDescriptor("a"), null, null);
HRegionInfo b = new HRegionInfo(new HTableDescriptor("b"), null, null);
assertTrue(a.compareTo(b) != 0);
HTableDescriptor t = new HTableDescriptor("t");
Text midway = new Text("midway");
a = new HRegionInfo(1, t, null, midway);
b = new HRegionInfo(2, t, midway, null);
a = new HRegionInfo(t, null, midway);
b = new HRegionInfo(t, midway, null);
assertTrue(a.compareTo(b) < 0);
assertTrue(b.compareTo(a) > 0);
assertEquals(a, a);
assertTrue(a.compareTo(a) == 0);
a = new HRegionInfo(1, t, new Text("a"), new Text("d"));
b = new HRegionInfo(2, t, new Text("e"), new Text("g"));
a = new HRegionInfo(t, new Text("a"), new Text("d"));
b = new HRegionInfo(t, new Text("e"), new Text("g"));
assertTrue(a.compareTo(b) < 0);
a = new HRegionInfo(1, t, new Text("aaaa"), new Text("dddd"));
b = new HRegionInfo(2, t, new Text("e"), new Text("g"));
a = new HRegionInfo(t, new Text("aaaa"), new Text("dddd"));
b = new HRegionInfo(t, new Text("e"), new Text("g"));
assertTrue(a.compareTo(b) < 0);
a = new HRegionInfo(1, t, new Text("aaaa"), new Text("dddd"));
b = new HRegionInfo(2, t, new Text("aaaa"), new Text("eeee"));
a = new HRegionInfo(t, new Text("aaaa"), new Text("dddd"));
b = new HRegionInfo(t, new Text("aaaa"), new Text("eeee"));
assertTrue(a.compareTo(b) < 0);
}
}

View File

@ -40,7 +40,7 @@ public class TestGet extends HBaseTestCase {
private static final Text CONTENTS = new Text("contents:");
private static final Text ROW_KEY =
new Text(HGlobals.rootRegionInfo.regionName);
new Text(HRegionInfo.rootRegionInfo.getRegionName());
private static final String SERVER_ADDRESS = "foo.bar.com:1234";
@ -88,8 +88,8 @@ public class TestGet extends HBaseTestCase {
desc.addFamily(new HColumnDescriptor(CONTENTS.toString()));
desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString()));
HRegionInfo info = new HRegionInfo(0L, desc, null, null);
Path regionDir = HRegion.getRegionDir(dir, info.regionName);
HRegionInfo info = new HRegionInfo(desc, null, null);
Path regionDir = HRegion.getRegionDir(dir, info.getEncodedName());
fs.mkdirs(regionDir);
HLog log = new HLog(fs, new Path(regionDir, "log"), conf);
@ -105,10 +105,10 @@ public class TestGet extends HBaseTestCase {
r.put(lockid, CONTENTS, bytes.toByteArray());
bytes.reset();
HGlobals.rootRegionInfo.write(s);
HRegionInfo.rootRegionInfo.write(s);
r.put(lockid, HConstants.COL_REGIONINFO,
Writables.getBytes(HGlobals.rootRegionInfo));
Writables.getBytes(HRegionInfo.rootRegionInfo));
r.commit(lockid, System.currentTimeMillis());

View File

@ -79,7 +79,7 @@ public class TestHLog extends HBaseTestCase implements HConstants {
assertEquals(tableName, key.getTablename());
assertEquals(HLog.METAROW, key.getRow());
assertEquals(HLog.METACOLUMN, val.getColumn());
assertEquals(0, HGlobals.completeCacheFlush.compareTo(val.getVal()));
assertEquals(0, HLogEdit.completeCacheFlush.compareTo(val.getVal()));
System.out.println(key + " " + val);
}
} finally {

View File

@ -104,7 +104,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe
desc.addFamily(new HColumnDescriptor("contents:"));
desc.addFamily(new HColumnDescriptor("anchor:"));
region = new HRegion(parentdir, log, fs, conf,
new HRegionInfo(1, desc, null, null), null);
new HRegionInfo(desc, null, null), null);
}
// Test basic functionality. Writes to contents:basic and anchor:anchornum-*

View File

@ -20,7 +20,6 @@
package org.apache.hadoop.hbase;
import java.io.IOException;
import java.io.PrintWriter;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -32,7 +31,6 @@ import org.apache.hadoop.io.MapFile;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.util.ReflectionUtils;
/**
* Test HStoreFile
@ -125,7 +123,7 @@ public class TestHStoreFile extends HBaseTestCase {
public void testReference()
throws IOException {
// Make a store file and write data to it.
HStoreFile hsf = new HStoreFile(this.conf, this.dir, new Text(getName()),
HStoreFile hsf = new HStoreFile(this.conf, this.dir, getName(),
new Text("colfamily"), 1234567890L);
MapFile.Writer writer =
hsf.getWriter(this.fs, SequenceFile.CompressionType.NONE, null);
@ -140,10 +138,10 @@ public class TestHStoreFile extends HBaseTestCase {
Text finalKey = hsk.getRow();
// Make a reference for the bottom half of the just written file.
HStoreFile.Reference reference =
new HStoreFile.Reference(hsf.getRegionName(), hsf.getFileId(), midkey,
HStoreFile.Range.top);
new HStoreFile.Reference(hsf.getEncodedRegionName(), hsf.getFileId(),
midkey, HStoreFile.Range.top);
HStoreFile refHsf = new HStoreFile(this.conf, new Path(DIR, getName()),
new Text(getName() + "_reference"), hsf.getColFamily(), 456,
getName() + "_reference", hsf.getColFamily(), 456,
reference);
// Assert that reference files are written and that we can write and
// read the info reference file at least.
@ -152,8 +150,8 @@ public class TestHStoreFile extends HBaseTestCase {
assertTrue(this.fs.exists(refHsf.getInfoFilePath()));
HStoreFile.Reference otherReference =
HStoreFile.readSplitInfo(refHsf.getInfoFilePath(), this.fs);
assertEquals(reference.getRegionName().toString(),
otherReference.getRegionName().toString());
assertEquals(reference.getEncodedRegionName(),
otherReference.getEncodedRegionName());
assertEquals(reference.getFileId(),
otherReference.getFileId());
assertEquals(reference.getMidkey().toString(),

View File

@ -46,9 +46,9 @@ public class TestScanner extends HBaseTestCase {
HConstants.COL_STARTCODE
};
private static final Text ROW_KEY = new Text(HGlobals.rootRegionInfo.regionName);
private static final HRegionInfo REGION_INFO =
new HRegionInfo(0L, HGlobals.rootTableDesc, null, null);
private static final Text ROW_KEY =
new Text(HRegionInfo.rootRegionInfo.getRegionName());
private static final HRegionInfo REGION_INFO = HRegionInfo.rootRegionInfo;
private static final long START_CODE = Long.MAX_VALUE;
@ -59,11 +59,11 @@ public class TestScanner extends HBaseTestCase {
HRegionInfo info =
(HRegionInfo) Writables.getWritable(regionBytes, new HRegionInfo());
assertEquals(REGION_INFO.regionId, info.regionId);
assertEquals(0, info.startKey.getLength());
assertEquals(0, info.endKey.getLength());
assertEquals(0, info.regionName.compareTo(REGION_INFO.regionName));
assertEquals(0, info.tableDesc.compareTo(REGION_INFO.tableDesc));
assertEquals(REGION_INFO.getRegionId(), info.getRegionId());
assertEquals(0, info.getStartKey().getLength());
assertEquals(0, info.getEndKey().getLength());
assertEquals(0, info.getRegionName().compareTo(REGION_INFO.getRegionName()));
assertEquals(0, info.getTableDesc().compareTo(REGION_INFO.getTableDesc()));
}
/** Use a scanner to get the region info and then validate the results */
@ -109,9 +109,10 @@ public class TestScanner extends HBaseTestCase {
}
} finally {
if(scanner != null) {
scanner.close();
scanner = null;
HInternalScannerInterface s = scanner;
scanner = null;
if(s != null) {
s.close();
}
}
}
@ -140,7 +141,7 @@ public class TestScanner extends HBaseTestCase {
Path dir = new Path("/hbase");
fs.mkdirs(dir);
Path regionDir = HRegion.getRegionDir(dir, REGION_INFO.regionName);
Path regionDir = HRegion.getRegionDir(dir, REGION_INFO.getEncodedName());
fs.mkdirs(regionDir);
HLog log = new HLog(fs, new Path(regionDir, "log"), conf);
@ -153,7 +154,7 @@ public class TestScanner extends HBaseTestCase {
ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
DataOutputStream s = new DataOutputStream(byteStream);
HGlobals.rootRegionInfo.write(s);
HRegionInfo.rootRegionInfo.write(s);
region.put(lockid, HConstants.COL_REGIONINFO, byteStream.toByteArray());
region.commit(lockid, System.currentTimeMillis());

View File

@ -244,19 +244,19 @@ public class TestScanner2 extends HBaseClusterTestCase {
assertEquals("Expected one region", regions.size(), 1);
HRegionInfo region = regions.get(0);
assertTrue("Expected region named for test",
region.regionName.toString().startsWith(getName()));
region.getRegionName().toString().startsWith(getName()));
// Now do what happens at split time; remove old region and then add two
// new ones in its place.
removeRegionFromMETA(new HTable(conf, HConstants.META_TABLE_NAME),
region.regionName);
HTableDescriptor desc = region.tableDesc;
region.getRegionName());
HTableDescriptor desc = region.getTableDesc();
Path homedir = new Path(getName());
List<HRegion> newRegions = new ArrayList<HRegion>(2);
newRegions.add(HRegion.createHRegion(
new HRegionInfo(2L, desc, null, new Text("midway")),
new HRegionInfo(desc, null, new Text("midway")),
homedir, this.conf, null));
newRegions.add(HRegion.createHRegion(
new HRegionInfo(3L, desc, new Text("midway"), null),
new HRegionInfo(desc, new Text("midway"), null),
homedir, this.conf, null));
try {
for (HRegion r : newRegions) {

View File

@ -70,7 +70,7 @@ public class TestSplit extends MultiRegionTable {
HLog hlog = new HLog(this.localFs, this.testDir, this.conf);
try {
HTableDescriptor htd = createTableDescriptor(getName());
HRegionInfo hri = new HRegionInfo(1, htd, null, null);
HRegionInfo hri = new HRegionInfo(htd, null, null);
region = new HRegion(testDir, hlog, this.localFs, this.conf, hri, null);
basicSplit(region);
} finally {
@ -89,11 +89,10 @@ public class TestSplit extends MultiRegionTable {
HRegion [] regions = split(region);
// Assert can get rows out of new regions. Should be able to get first
// row from first region and the midkey from second region.
byte [] b = new byte [] {FIRST_CHAR, FIRST_CHAR, FIRST_CHAR};
assertGet(regions[0], COLFAMILY_NAME3, new Text(b));
assertGet(regions[0], COLFAMILY_NAME3, new Text(START_KEY));
assertGet(regions[1], COLFAMILY_NAME3, midkey);
// Test I can get scanner and that it starts at right place.
assertScan(regions[0], COLFAMILY_NAME3, new Text(b));
assertScan(regions[0], COLFAMILY_NAME3, new Text(START_KEY));
assertScan(regions[1], COLFAMILY_NAME3, midkey);
// Now prove can't split regions that have references.
Text [] midkeys = new Text[regions.length];
@ -143,6 +142,7 @@ public class TestSplit extends MultiRegionTable {
// The splits should have been even. Test I can get some arbitrary row out
// of each.
int interval = (LAST_CHAR - FIRST_CHAR) / 3;
byte[] b = START_KEY.getBytes(HConstants.UTF8_ENCODING);
for (HRegion r: sortedMap.values()) {
assertGet(r, COLFAMILY_NAME3,
new Text(new String(b, HConstants.UTF8_ENCODING)));

View File

@ -33,23 +33,23 @@ public class TestTable extends HBaseClusterTestCase {
final HBaseAdmin admin = new HBaseAdmin(conf);
String msg = null;
try {
admin.createTable(HGlobals.rootTableDesc);
admin.createTable(HTableDescriptor.rootTableDesc);
} catch (IllegalArgumentException e) {
msg = e.toString();
}
assertTrue("Unexcepted exception message " + msg, msg != null &&
msg.startsWith(IllegalArgumentException.class.getName()) &&
msg.contains(HGlobals.rootTableDesc.getName().toString()));
msg.contains(HTableDescriptor.rootTableDesc.getName().toString()));
msg = null;
try {
admin.createTable(HGlobals.metaTableDesc);
admin.createTable(HTableDescriptor.metaTableDesc);
} catch(IllegalArgumentException e) {
msg = e.toString();
}
assertTrue("Unexcepted exception message " + msg, msg != null &&
msg.startsWith(IllegalArgumentException.class.getName()) &&
msg.contains(HGlobals.metaTableDesc.getName().toString()));
msg.contains(HTableDescriptor.metaTableDesc.getName().toString()));
// Try doing a duplicate database create.
msg = null;

View File

@ -320,7 +320,7 @@ public class TestTimestamp extends HBaseTestCase {
HTableDescriptor htd = createTableDescriptor(getName());
htd.addFamily(new HColumnDescriptor(COLUMN, VERSIONS,
CompressionType.NONE, false, Integer.MAX_VALUE, null));
HRegionInfo hri = new HRegionInfo(1, htd, null, null);
HRegionInfo hri = new HRegionInfo(htd, null, null);
return new HRegion(testDir, hlog, this.localFs, this.conf, hri, null);
}
}

View File

@ -18,8 +18,6 @@
package org.apache.hadoop.hbase;
import org.apache.hadoop.io.Text;
import junit.framework.TestCase;
/**
@ -38,32 +36,28 @@ public class TestToString extends TestCase {
assertEquals("HServerInfo", "address: " + hostport + ", startcode: -1" +
", load: (requests: 0 regions: 0)", info.toString());
}
/**
* Test HTableDescriptor.toString();
*/
public void testHTableDescriptor() {
HTableDescriptor htd = HTableDescriptor.rootTableDesc;
System. out.println(htd.toString());
assertEquals("Table descriptor", "name: -ROOT-, families: {info:={name: " +
"info, max versions: 1, compression: NONE, in memory: false, max " +
"length: 2147483647, bloom filter: none}}", htd.toString());
}
/**
* Tests toString method on HRegionInfo
* @throws Exception
* Tests HRegionInfo.toString()
*/
public void testHRegionInfo() throws Exception {
HTableDescriptor htd = new HTableDescriptor("hank");
htd.addFamily(new HColumnDescriptor("hankfamily:"));
htd.addFamily(new HColumnDescriptor(new Text("hankotherfamily:"), 10,
HColumnDescriptor.CompressionType.BLOCK, true, 1000, null));
System. out.println(htd.toString());
assertEquals("Table descriptor", "name: hank, families: " +
"{hankfamily:={name: hankfamily, max versions: 3, compression: NONE, " +
"in memory: false, max length: 2147483647, bloom filter: none}, " +
"hankotherfamily:={name: hankotherfamily, max versions: 10, " +
"compression: BLOCK, in memory: true, max length: 1000, " +
"bloom filter: none}}", htd.toString());
HRegionInfo hri = new HRegionInfo(-1, htd, new Text(), new Text("10"));
public void testHRegionInfo() {
HRegionInfo hri = HRegionInfo.rootRegionInfo;
System.out.println(hri.toString());
assertEquals("HRegionInfo",
"regionname: hank,,-1, startKey: <>, tableDesc: {name: hank, " +
"families: {hankfamily:={name: hankfamily, max versions: 3, " +
"compression: NONE, in memory: false, max length: 2147483647, " +
"bloom filter: none}, hankotherfamily:={name: hankotherfamily, " +
"max versions: 10, compression: BLOCK, in memory: true, " +
"max length: 1000, bloom filter: none}}}",
"regionname: -ROOT-,,0, startKey: <>, tableDesc: {name: -ROOT-, " +
"families: {info:={name: info, max versions: 1, compression: NONE, " +
"in memory: false, max length: 2147483647, bloom filter: none}}}",
hri.toString());
}
}

View File

@ -30,7 +30,7 @@ import java.util.TreeMap;
import junit.framework.TestCase;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HGlobals;
import org.apache.hadoop.hbase.HLogEdit;
import org.apache.hadoop.io.Text;
/**
@ -186,7 +186,7 @@ public class TestRegExpRowFilter extends TestCase {
// that maps to a null value.
// Testing row with columnKeys: a-e, e maps to null
colvalues.put(new Text(new String(new char[] { LAST_CHAR })),
HGlobals.deleteBytes.get());
HLogEdit.deleteBytes.get());
assertFalse("Failed with last columnKey " + LAST_CHAR + " mapping to null.",
filter.filterNotNull(colvalues));
}