HBASE-2531 32-bit encoding of regionnames waaaaaaayyyyy too susceptible to hash clashes

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@949919 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2010-06-01 03:50:57 +00:00
parent 12855081cc
commit 677738d3ed
17 changed files with 263 additions and 81 deletions

View File

@ -1032,6 +1032,8 @@ Release 0.20.0 - Tue Sep 8 12:53:05 PDT 2009
HBASE-2352 Small values for hbase.client.retries.number and
ipc.client.connect.max.retries breaks long ops in hbase shell
(Alexey Kovyrin via Stack)
HBASE-2531 32-bit encoding of regionnames waaaaaaayyyyy too susceptible to
hash clashes (Kannan Muthukkaruppan via Stack)
IMPROVEMENTS
HBASE-1089 Add count of regions on filesystem to master UI; add percentage

View File

@ -19,14 +19,15 @@
#
# Script adds a table back to a running hbase.
# Currently only works on if table data is in place.
#
# To see usage for this script, run:
#
# To see usage for this script, run:
#
# ${HBASE_HOME}/bin/hbase org.jruby.Main addtable.rb
#
include Java
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.HConstants
import org.apache.hadoop.hbase.regionserver.HRegion
import org.apache.hadoop.hbase.HRegionInfo
import org.apache.hadoop.hbase.client.HTable
import org.apache.hadoop.hbase.client.Delete
@ -84,7 +85,7 @@ elsif
end
HTableDescriptor.isLegalTableName(tableName.to_java_bytes)
# Figure locations under hbase.rootdir
# Figure locations under hbase.rootdir
# Move directories into place; be careful not to overwrite.
rootdir = FSUtils.getRootDir(c)
tableDir = fs.makeQualified(Path.new(rootdir, tableName))
@ -129,15 +130,15 @@ statuses = fs.listStatus(srcdir)
for status in statuses
next unless status.isDir()
next if status.getPath().getName() == "compaction.dir"
regioninfofile = Path.new(status.getPath(), ".regioninfo")
regioninfofile = Path.new(status.getPath(), HRegion::REGIONINFO_FILE)
unless fs.exists(regioninfofile)
LOG.warn("Missing .regioninfo: " + regioninfofile.toString())
next
end
is = fs.open(regioninfofile)
is = fs.open(regioninfofile)
hri = HRegionInfo.new()
hri.readFields(is)
is.close()
is.close()
# TODO: Need to redo table descriptor with passed table name and then recalculate the region encoded names.
p = Put.new(hri.getRegionName())
p.add(HConstants::CATALOG_FAMILY, HConstants::REGIONINFO_QUALIFIER, Writables.getBytes(hri))

View File

@ -24,9 +24,12 @@ import java.io.DataOutput;
import java.io.IOException;
import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JenkinsHash;
import org.apache.hadoop.hbase.util.MD5Hash;
import org.apache.hadoop.io.VersionedWritable;
import org.apache.hadoop.io.WritableComparable;
@ -37,13 +40,74 @@ import org.apache.hadoop.io.WritableComparable;
*/
public class HRegionInfo extends VersionedWritable implements WritableComparable<HRegionInfo>{
private static final byte VERSION = 0;
private static final Log LOG = LogFactory.getLog(HRegionInfo.class);
/**
* The new format for a region name contains its encodedName at the end.
* The encoded name also serves as the directory name for the region
* in the filesystem.
*
* New region name format:
* &lt;tablename>,,&lt;startkey>,&lt;regionIdTimestamp>.&lt;encodedName>.
* where,
* &lt;encodedName> is a hex version of the MD5 hash of
* &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
*
* The old region name format:
* &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
* For region names in the old format, the encoded name is a 32-bit
* JenkinsHash integer value (in its decimal notation, string form).
*<p>
* **NOTE**
*
* ROOT, the first META region, and regions created by an older
* version of HBase (0.20 or prior) will continue to use the
* old region name format.
*/
/** Separator used to demarcate the encodedName in a region name
* in the new format. See description on new format above.
*/
private static final int ENC_SEPARATOR = '.';
public static final int MD5_HEX_LENGTH = 32;
/**
* Does region name contain its encoded name?
* @param regionName region name
* @return boolean indicating if this a new format region
* name which contains its encoded name.
*/
private static boolean hasEncodedName(final byte[] regionName) {
// check if region name ends in ENC_SEPARATOR
if ((regionName.length >= 1)
&& (regionName[regionName.length - 1] == ENC_SEPARATOR)) {
// region name is new format. it contains the encoded name.
return true;
}
return false;
}
/**
* @param regionName
* @return the encodedName
*/
public static int encodeRegionName(final byte [] regionName) {
return Math.abs(JenkinsHash.getInstance().hash(regionName, regionName.length, 0));
public static String encodeRegionName(final byte [] regionName) {
String encodedName;
if (hasEncodedName(regionName)) {
// region is in new format:
// <tableName>,<startKey>,<regionIdTimeStamp>/encodedName/
encodedName = Bytes.toString(regionName,
regionName.length - MD5_HEX_LENGTH - 1,
MD5_HEX_LENGTH);
} else {
// old format region name. ROOT and first META region also
// use this format.EncodedName is the JenkinsHash value.
int hashVal = Math.abs(JenkinsHash.getInstance().hash(regionName,
regionName.length,
0));
encodedName = String.valueOf(hashVal);
}
return encodedName;
}
/** delimiter used between portions of a region name */
@ -67,8 +131,8 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
protected HTableDescriptor tableDesc = null;
private int hashCode = -1;
//TODO: Move NO_HASH to HStoreFile which is really the only place it is used.
public static final int NO_HASH = -1;
private volatile int encodedName = NO_HASH;
public static final String NO_HASH = null;
private volatile String encodedName = NO_HASH;
private void setHashCode() {
int result = Arrays.hashCode(this.regionName);
@ -88,7 +152,10 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
super();
this.regionId = regionId;
this.tableDesc = tableDesc;
this.regionName = createRegionName(tableDesc.getName(), null, regionId);
// Note: Root & First Meta regions names are still in old format
this.regionName = createRegionName(tableDesc.getName(), null,
regionId, false);
this.regionNameStr = Bytes.toStringBinary(this.regionName);
setHashCode();
}
@ -149,7 +216,7 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
}
this.offLine = false;
this.regionId = regionid;
this.regionName = createRegionName(tableDesc.getName(), startKey, regionId);
this.regionName = createRegionName(tableDesc.getName(), startKey, regionId, true);
this.regionNameStr = Bytes.toStringBinary(this.regionName);
this.split = split;
this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone();
@ -179,8 +246,8 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
}
private static byte [] createRegionName(final byte [] tableName,
final byte [] startKey, final long regionid) {
return createRegionName(tableName, startKey, Long.toString(regionid));
final byte [] startKey, final long regionid, boolean newFormat) {
return createRegionName(tableName, startKey, Long.toString(regionid), newFormat);
}
/**
@ -188,23 +255,29 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
* @param tableName
* @param startKey Can be null
* @param id Region id.
* @param newFormat should we create the region name in the new format
* (such that it contains its encoded name?).
* @return Region name made of passed tableName, startKey and id
*/
public static byte [] createRegionName(final byte [] tableName,
final byte [] startKey, final String id) {
return createRegionName(tableName, startKey, Bytes.toBytes(id));
final byte [] startKey, final String id, boolean newFormat) {
return createRegionName(tableName, startKey, Bytes.toBytes(id), newFormat);
}
/**
* Make a region name of passed parameters.
* @param tableName
* @param startKey Can be null
* @param id Region id
* @param newFormat should we create the region name in the new format
* (such that it contains its encoded name?).
* @return Region name made of passed tableName, startKey and id
*/
public static byte [] createRegionName(final byte [] tableName,
final byte [] startKey, final byte [] id) {
final byte [] startKey, final byte [] id, boolean newFormat) {
byte [] b = new byte [tableName.length + 2 + id.length +
(startKey == null? 0: startKey.length)];
(startKey == null? 0: startKey.length) +
(newFormat ? (MD5_HEX_LENGTH + 2) : 0)];
int offset = tableName.length;
System.arraycopy(tableName, 0, b, 0, offset);
b[offset++] = DELIMITER;
@ -214,6 +287,31 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
}
b[offset++] = DELIMITER;
System.arraycopy(id, 0, b, offset, id.length);
offset += id.length;
if (newFormat) {
//
// Encoded name should be built into the region name.
//
// Use the region name thus far (namely, <tablename>,<startKey>,<id>)
// to compute a MD5 hash to be used as the encoded name, and append
// it to the byte buffer.
//
String md5Hash = MD5Hash.getMD5AsHex(b, 0, offset);
byte [] md5HashBytes = Bytes.toBytes(md5Hash);
if (md5HashBytes.length != MD5_HEX_LENGTH) {
LOG.error("MD5-hash length mismatch: Expected=" + MD5_HEX_LENGTH +
"; Got=" + md5HashBytes.length);
}
// now append the bytes '.<encodedName>.' to the end
b[offset++] = ENC_SEPARATOR;
System.arraycopy(md5HashBytes, 0, b, offset, MD5_HEX_LENGTH);
offset += MD5_HEX_LENGTH;
b[offset++] = ENC_SEPARATOR;
}
return b;
}
@ -281,11 +379,19 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
* @return Region name as a String for use in logging, etc.
*/
public String getRegionNameAsString() {
return this.regionNameStr;
if (hasEncodedName(this.regionName)) {
// new format region names already have their encoded name.
return this.regionNameStr;
}
// old format. regionNameStr doesn't have the region name.
//
//
return this.regionNameStr + "." + this.getEncodedName();
}
/** @return the encoded region name */
public synchronized int getEncodedName() {
public synchronized String getEncodedName() {
if (this.encodedName == NO_HASH) {
this.encodedName = encodeRegionName(this.regionName);
}

View File

@ -915,7 +915,7 @@ public class HBaseAdmin {
private HRegionLocation getFirstMetaServerForTable(final byte [] tableName)
throws IOException {
return connection.locateRegion(HConstants.META_TABLE_NAME,
HRegionInfo.createRegionName(tableName, null, HConstants.NINES));
HRegionInfo.createRegionName(tableName, null, HConstants.NINES, false));
}
/**

View File

@ -515,7 +515,7 @@ public class HConnectionManager implements HConstants {
int rowsScanned = 0;
int rowsOffline = 0;
byte[] startKey =
HRegionInfo.createRegionName(tableName, null, HConstants.ZEROES);
HRegionInfo.createRegionName(tableName, null, HConstants.ZEROES, false);
byte[] endKey;
HRegionInfo currentRegion;
Scan scan = new Scan(startKey);
@ -663,7 +663,7 @@ public class HConnectionManager implements HConstants {
// the extra 9's on the end are necessary to allow "exact" matches
// without knowing the precise region names.
byte [] metaKey = HRegionInfo.createRegionName(tableName, row,
HConstants.NINES);
HConstants.NINES, false);
for (int tries = 0; true; tries++) {
if (tries >= numRetries) {
throw new NoServerForRegionException("Unable to find region for "

View File

@ -63,7 +63,7 @@ class MetaScanner implements HConstants {
HConnection connection = HConnectionManager.getConnection(configuration);
byte [] startRow = tableName == null || tableName.length == 0 ?
HConstants.EMPTY_START_ROW :
HRegionInfo.createRegionName(tableName, null, ZEROES);
HRegionInfo.createRegionName(tableName, null, ZEROES, false);
// Scan over each meta region
ScannerCallable callable;

View File

@ -800,7 +800,7 @@ public class HFile {
* See {@link Writer#appendFileInfo(byte[], byte[])}.
* @throws IOException
*/
public Map<byte [], byte []> loadFileInfo()
public Map<byte [], byte []> loadFileInfo()
throws IOException {
this.trailer = readTrailer();
@ -895,7 +895,7 @@ public class HFile {
* @return Block wrapped in a ByteBuffer
* @throws IOException
*/
public ByteBuffer getMetaBlock(String metaBlockName, boolean cacheBlock)
public ByteBuffer getMetaBlock(String metaBlockName, boolean cacheBlock)
throws IOException {
if (trailer.metaIndexCount == 0) {
return null; // there are no meta blocks
@ -903,7 +903,7 @@ public class HFile {
if (metaIndex == null) {
throw new IOException("Meta index not loaded");
}
byte [] mbname = Bytes.toBytes(metaBlockName);
int block = metaIndex.blockContainingKey(mbname, 0, mbname.length);
if (block == -1)
@ -924,34 +924,34 @@ public class HFile {
if (cache != null) {
ByteBuffer cachedBuf = cache.getBlock(name + "meta" + block);
if (cachedBuf != null) {
// Return a distinct 'shallow copy' of the block,
// Return a distinct 'shallow copy' of the block,
// so pos doesnt get messed by the scanner
cacheHits++;
return cachedBuf.duplicate();
}
// Cache Miss, please load.
}
ByteBuffer buf = decompress(metaIndex.blockOffsets[block],
longToInt(blockSize), metaIndex.blockDataSizes[block], true);
byte [] magic = new byte[METABLOCKMAGIC.length];
buf.get(magic, 0, magic.length);
if (! Arrays.equals(magic, METABLOCKMAGIC)) {
throw new IOException("Meta magic is bad in block " + block);
}
// Create a new ByteBuffer 'shallow copy' to hide the magic header
buf = buf.slice();
readTime += System.currentTimeMillis() - now;
readOps++;
// Cache the block
if(cacheBlock && cache != null) {
cache.cacheBlock(name + "meta" + block, buf.duplicate(), inMemory);
}
return buf;
}
}
@ -983,7 +983,7 @@ public class HFile {
if (cache != null) {
ByteBuffer cachedBuf = cache.getBlock(name + block);
if (cachedBuf != null) {
// Return a distinct 'shallow copy' of the block,
// Return a distinct 'shallow copy' of the block,
// so pos doesnt get messed by the scanner
cacheHits++;
return cachedBuf.duplicate();
@ -1015,10 +1015,10 @@ public class HFile {
}
// 'shallow copy' to hide the header
// NOTE: you WILL GET BIT if you call buf.array() but don't start
// NOTE: you WILL GET BIT if you call buf.array() but don't start
// reading at buf.arrayOffset()
buf = buf.slice();
readTime += System.currentTimeMillis() - now;
readOps++;
@ -1096,7 +1096,7 @@ public class HFile {
}
return this.blockIndex.isEmpty()? null: this.lastkey;
}
/**
* @return number of K entries in this HFile's filter. Returns KV count if no filter.
*/
@ -1222,7 +1222,7 @@ public class HFile {
return true;
}
public boolean shouldSeek(final byte[] row,
public boolean shouldSeek(final byte[] row,
final SortedSet<byte[]> columns) {
return true;
}
@ -1773,8 +1773,8 @@ public class HFile {
byte[][] hri = HRegionInfo.parseRegionName(rn);
Path rootDir = FSUtils.getRootDir(conf);
Path tableDir = new Path(rootDir, Bytes.toString(hri[0]));
int enc = HRegionInfo.encodeRegionName(rn);
Path regionDir = new Path(tableDir, Integer.toString(enc));
String enc = HRegionInfo.encodeRegionName(rn);
Path regionDir = new Path(tableDir, enc);
if (verbose) System.out.println("region dir -> " + regionDir);
List<Path> regionFiles = getStoreFiles(fs, regionDir);
if (verbose) System.out.println("Number of region files found -> " +

View File

@ -224,12 +224,6 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
*/
public final static String REGIONINFO_FILE = ".regioninfo";
/**
* REGIONINFO_FILE as byte array.
*/
public final static byte [] REGIONINFO_FILE_BYTES =
Bytes.toBytes(REGIONINFO_FILE);
/**
* Should only be used for testing purposes
*/
@ -283,12 +277,11 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
this.regionInfo = regionInfo;
this.flushListener = flushListener;
this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000);
String encodedNameStr = Integer.toString(this.regionInfo.getEncodedName());
String encodedNameStr = this.regionInfo.getEncodedName();
this.regiondir = new Path(basedir, encodedNameStr);
if (LOG.isDebugEnabled()) {
// Write out region name as string and its encoded name.
LOG.debug("Creating region " + this + ", encoded=" +
this.regionInfo.getEncodedName());
LOG.debug("Creating region " + this);
}
this.regionCompactionDir =
new Path(getCompactionDir(basedir), encodedNameStr);
@ -363,8 +356,8 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
// HRegion is ready to go!
this.writestate.compacting = false;
this.lastFlushTime = System.currentTimeMillis();
LOG.info("region " + this + "/" + this.regionInfo.getEncodedName() +
" available; sequence id is " + this.minSequenceId);
LOG.info("region " + this +
" available; sequence id is " + this.minSequenceId);
}
/*
@ -728,7 +721,7 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
private Path getSplitDirForDaughter(final Path splits, final HRegionInfo hri)
throws IOException {
Path d =
new Path(splits, Integer.toString(hri.getEncodedName()));
new Path(splits, hri.getEncodedName());
if (fs.exists(d)) {
// This should never happen; the splits dir will be newly made when we
// come in here. Even if we crashed midway through a split, the reopen
@ -2322,8 +2315,8 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
* @param name ENCODED region name
* @return Path of HRegion directory
*/
public static Path getRegionDir(final Path tabledir, final int name) {
return new Path(tabledir, Integer.toString(name));
public static Path getRegionDir(final Path tabledir, final String name) {
return new Path(tabledir, name);
}
/**
@ -2336,7 +2329,7 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
return new Path(
HTableDescriptor.getTableDir(rootdir, info.getTableDesc().getName()),
Integer.toString(info.getEncodedName()));
info.getEncodedName());
}
/**
@ -2464,7 +2457,7 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey);
LOG.info("Creating new region " + newRegionInfo.toString());
int encodedName = newRegionInfo.getEncodedName();
String encodedName = newRegionInfo.getEncodedName();
Path newRegionDir = HRegion.getRegionDir(a.getBaseDir(), encodedName);
if(fs.exists(newRegionDir)) {
throw new IOException("Cannot merge; target file collision at " +

View File

@ -191,7 +191,7 @@ public class Store implements HConstants, HeapSize {
}
this.memstore = new MemStore(this.comparator);
this.regionCompactionDir = new Path(HRegion.getCompactionDir(basedir),
Integer.toString(info.getEncodedName()));
info.getEncodedName());
this.storeName = this.family.getName();
this.storeNameStr = Bytes.toString(this.storeName);
@ -252,8 +252,8 @@ public class Store implements HConstants, HeapSize {
* @return Path to family/Store home directory.
*/
public static Path getStoreHomedir(final Path tabledir,
final int encodedName, final byte [] family) {
return new Path(tabledir, new Path(Integer.toString(encodedName),
final String encodedName, final byte [] family) {
return new Path(tabledir, new Path(encodedName,
new Path(Bytes.toString(family))));
}
@ -413,7 +413,7 @@ public class Store implements HConstants, HeapSize {
}
StoreFile curfile = null;
try {
curfile = new StoreFile(fs, p, blockcache, this.conf,
curfile = new StoreFile(fs, p, blockcache, this.conf,
this.family.getBloomFilterType(), this.inMemory);
curfile.createReader();
} catch (IOException ioe) {
@ -582,10 +582,10 @@ public class Store implements HConstants, HeapSize {
* @param basedir Directory to put writer in.
* @throws IOException
*/
private StoreFile.Writer createWriter(final Path basedir, int maxKeyCount)
private StoreFile.Writer createWriter(final Path basedir, int maxKeyCount)
throws IOException {
return StoreFile.createWriter(this.fs, basedir, this.blocksize,
this.compression, this.comparator, this.conf,
this.compression, this.comparator, this.conf,
this.family.getBloomFilterType(), maxKeyCount);
}
@ -879,9 +879,9 @@ public class Store implements HConstants, HeapSize {
for (StoreFile file : filesToCompact) {
StoreFile.Reader r = file.getReader();
if (r != null) {
// NOTE: getFilterEntries could cause under-sized blooms if the user
// NOTE: getFilterEntries could cause under-sized blooms if the user
// switches bloom type (e.g. from ROW to ROWCOL)
maxKeyCount += (r.getBloomFilterType() == family.getBloomFilterType())
maxKeyCount += (r.getBloomFilterType() == family.getBloomFilterType())
? r.getFilterEntries() : r.getEntries();
}
}
@ -977,7 +977,7 @@ public class Store implements HConstants, HeapSize {
LOG.error("Failed move of compacted file " + compactedFile.getPath(), e);
return null;
}
result = new StoreFile(this.fs, p, blockcache, this.conf,
result = new StoreFile(this.fs, p, blockcache, this.conf,
this.family.getBloomFilterType(), this.inMemory);
result.createReader();
}
@ -1004,7 +1004,7 @@ public class Store implements HConstants, HeapSize {
// WARN ugly hack here, but necessary sadly.
// TODO why is this necessary? need a comment here if it's unintuitive!
ReadWriteConsistencyControl.resetThreadReadPoint(region.getRWCC());
// Tell observers that list of StoreFiles has changed.
notifyChangedReadersObservers();
// Finally, delete old store files.
@ -1475,7 +1475,7 @@ public class Store implements HConstants, HeapSize {
* This function will always be seen as atomic by other readers
* because it only puts a single KV to memstore. Thus no
* read/write control necessary.
*
*
* @param row
* @param f
* @param qualifier
@ -1563,7 +1563,7 @@ public class Store implements HConstants, HeapSize {
if (storeFile == null) {
return false;
}
// Add new file to store files. Clear snapshot too while we have
// Add new file to store files. Clear snapshot too while we have
// the Store write lock.
return Store.this.updateStorefiles(cacheFlushId, storeFile, snapshot);
}

View File

@ -0,0 +1,67 @@
/**
* Copyright 2010 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.util;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import org.apache.commons.codec.binary.Hex;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
/**
* Utility class for MD5
* MD5 hash produces a 128-bit digest.
*/
public class MD5Hash {
private static final Log LOG = LogFactory.getLog(MD5Hash.class);
/**
* Given a byte array, returns in MD5 hash as a hex string.
* @param key
* @return SHA1 hash as a 32 character hex string.
*/
public static String getMD5AsHex(byte[] key) {
return getMD5AsHex(key, 0, key.length);
}
/**
* Given a byte array, returns its MD5 hash as a hex string.
* Only "length" number of bytes starting at "offset" within the
* byte array are used.
*
* @param key the key to hash (variable length byte array)
* @param offset
* @param length
* @return MD5 hash as a 32 character hex string.
*/
public static String getMD5AsHex(byte[] key, int offset, int length) {
try {
MessageDigest md = MessageDigest.getInstance("MD5");
md.update(key, offset, length);
byte[] digest = md.digest();
return new String(Hex.encodeHex(digest));
} catch (NoSuchAlgorithmException e) {
// this should never happen unless the JDK is messed up.
throw new RuntimeException("Error computing MD5 hash", e);
}
}
}

View File

@ -18,7 +18,7 @@
HBaseAdmin hbadmin = new HBaseAdmin(conf);
String tableName = request.getParameter("name");
HTable table = new HTable(conf, tableName);
String tableHeader = "<h2>Table Regions</h2><table><tr><th>Name</th><th>Region Server</th><th>Encoded Name</th><th>Start Key</th><th>End Key</th></tr>";
String tableHeader = "<h2>Table Regions</h2><table><tr><th>Name</th><th>Region Server</th><th>Start Key</th><th>End Key</th></tr>";
HServerAddress rootLocation = master.getRegionManager().getRootRegionLocation();
boolean showFragmentation = conf.getBoolean("hbase.master.ui.fragmentation.enabled", false);
Map<String, Integer> frags = null;
@ -146,7 +146,7 @@
<tr>
<td><%= Bytes.toStringBinary(hriEntry.getKey().getRegionName())%></td>
<td><a href="<%= urlRegionServer %>"><%= hriEntry.getValue().getHostname().toString() + ":" + infoPort %></a></td>
<td><%= hriEntry.getKey().getEncodedName()%></td> <td><%= Bytes.toStringBinary(hriEntry.getKey().getStartKey())%></td>
<td><%= Bytes.toStringBinary(hriEntry.getKey().getStartKey())%></td>
<td><%= Bytes.toStringBinary(hriEntry.getKey().getEndKey())%></td>
</tr>
<% } %>

View File

@ -48,11 +48,11 @@
<h2>Online Regions</h2>
<% if (onlineRegions != null && onlineRegions.size() > 0) { %>
<table>
<tr><th>Region Name</th><th>Encoded Name</th><th>Start Key</th><th>End Key</th><th>Metrics</th></tr>
<tr><th>Region Name</th><th>Start Key</th><th>End Key</th><th>Metrics</th></tr>
<% for (HRegionInfo r: onlineRegions) {
HServerLoad.RegionLoad load = regionServer.createRegionLoad(r.getRegionName());
%>
<tr><td><%= r.getRegionNameAsString() %></td><td><%= r.getEncodedName() %></td>
<tr><td><%= r.getRegionNameAsString() %></td>
<td><%= Bytes.toStringBinary(r.getStartKey()) %></td><td><%= Bytes.toStringBinary(r.getEndKey()) %></td>
<td><%= load.toString() %></td>
</tr>

View File

@ -44,7 +44,7 @@ public class TestEmptyMetaInfo extends HBaseClusterTestCase {
for (int i = 0; i < COUNT; i++) {
byte [] regionName = HRegionInfo.createRegionName(tableName,
Bytes.toBytes(i == 0? "": Integer.toString(i)),
Long.toString(System.currentTimeMillis()));
Long.toString(System.currentTimeMillis()), true);
Put put = new Put(regionName);
put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes("localhost:1234"));

View File

@ -76,7 +76,7 @@ public class TestCompaction extends HBaseTestCase {
this.r = createNewHRegion(htd, null, null);
this.compactionDir = HRegion.getCompactionDir(this.r.getBaseDir());
this.regionCompactionDir = new Path(this.compactionDir,
Integer.toString(this.r.getRegionInfo().getEncodedName()));
this.r.getRegionInfo().getEncodedName());
}
@Override

View File

@ -111,7 +111,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase implements HConstant
findRow(mr, 'C', 43, 42);
// Now delete 'C' and make sure I don't get entries from 'B'.
byte [] firstRowInC = HRegionInfo.createRegionName(Bytes.toBytes("" + 'C'),
HConstants.EMPTY_BYTE_ARRAY, HConstants.ZEROES);
HConstants.EMPTY_BYTE_ARRAY, HConstants.ZEROES, false);
Scan scan = new Scan(firstRowInC);
s = mr.getScanner(scan);
try {
@ -150,7 +150,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase implements HConstant
// Find the row.
byte [] tofindBytes = Bytes.toBytes((short)rowToFind);
byte [] metaKey = HRegionInfo.createRegionName(tableb, tofindBytes,
HConstants.NINES);
HConstants.NINES, false);
LOG.info("find=" + new String(metaKey));
Result r = mr.getClosestRowBefore(metaKey);
if (answer == -1) {

View File

@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.MD5Hash;
public class TestHRegionInfo extends HBaseTestCase {
public void testCreateHRegionInfoName() throws Exception {
@ -30,8 +31,20 @@ public class TestHRegionInfo extends HBaseTestCase {
String startKey = "startkey";
final byte [] sk = Bytes.toBytes(startKey);
String id = "id";
byte [] name = HRegionInfo.createRegionName(tn, sk, id);
// old format region name
byte [] name = HRegionInfo.createRegionName(tn, sk, id, false);
String nameStr = Bytes.toString(name);
assertEquals(nameStr, tableName + "," + startKey + "," + id);
assertEquals(tableName + "," + startKey + "," + id, nameStr);
// new format region name.
String md5HashInHex = MD5Hash.getMD5AsHex(name);
assertEquals(HRegionInfo.MD5_HEX_LENGTH, md5HashInHex.length());
name = HRegionInfo.createRegionName(tn, sk, id, true);
nameStr = Bytes.toString(name);
assertEquals(tableName + "," + startKey + ","
+ id + "." + md5HashInHex + ".",
nameStr);
}
}

View File

@ -149,14 +149,14 @@ public class TestStoreFile extends HBaseTestCase {
KeyValue midKV = KeyValue.createKeyValueFromKey(midkey);
byte [] midRow = midKV.getRow();
// Create top split.
Path topDir = Store.getStoreHomedir(this.testDir, 1,
Path topDir = Store.getStoreHomedir(this.testDir, "1",
Bytes.toBytes(f.getPath().getParent().getName()));
if (this.fs.exists(topDir)) {
this.fs.delete(topDir, true);
}
Path topPath = StoreFile.split(this.fs, topDir, f, midRow, Range.top);
// Create bottom split.
Path bottomDir = Store.getStoreHomedir(this.testDir, 2,
Path bottomDir = Store.getStoreHomedir(this.testDir, "2",
Bytes.toBytes(f.getPath().getParent().getName()));
if (this.fs.exists(bottomDir)) {
this.fs.delete(bottomDir, true);